mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-14 02:12:21 +00:00
Start cleaning up things
This commit is contained in:
parent
97717e0e88
commit
d5c332327d
@ -852,37 +852,32 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
|||||||
else if (auto storage_merge_tree = std::dynamic_pointer_cast<MergeTreeData>(storage);
|
else if (auto storage_merge_tree = std::dynamic_pointer_cast<MergeTreeData>(storage);
|
||||||
storage_merge_tree && settings.parallel_replicas_min_number_of_rows_per_replica)
|
storage_merge_tree && settings.parallel_replicas_min_number_of_rows_per_replica)
|
||||||
{
|
{
|
||||||
|
auto query_info_copy = query_info;
|
||||||
/// TODO: Improve this block as this should only happen once, right? vvvvvvvvvvvvvvvvvvvvvv
|
/// TODO: Improve this block as this should only happen once, right? vvvvvvvvvvvvvvvvvvvvvv
|
||||||
addPrewhereAliasActions();
|
addPrewhereAliasActions();
|
||||||
auto & prewhere_info = analysis_result.prewhere_info;
|
auto & prewhere_info = analysis_result.prewhere_info;
|
||||||
if (prewhere_info)
|
if (prewhere_info)
|
||||||
{
|
{
|
||||||
query_info.prewhere_info = prewhere_info;
|
query_info_copy.prewhere_info = prewhere_info;
|
||||||
if (query.prewhere() && !query.where())
|
if (query.prewhere() && !query.where())
|
||||||
query_info.prewhere_info->need_filter = true;
|
query_info_copy.prewhere_info->need_filter = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
ActionDAGNodes added_filter_nodes;
|
ActionDAGNodes added_filter_nodes = MergeTreeData::getFiltersForPrimaryKeyAnalysis(*this);
|
||||||
if (additional_filter_info)
|
|
||||||
added_filter_nodes.nodes.push_back(&additional_filter_info->actions->findInOutputs(additional_filter_info->column_name));
|
|
||||||
|
|
||||||
if (analysis_result.before_where)
|
|
||||||
added_filter_nodes.nodes.push_back(&analysis_result.before_where->findInOutputs(analysis_result.where_column_name));
|
|
||||||
|
|
||||||
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context);
|
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context);
|
||||||
|
|
||||||
auto local_limits = getStorageLimits(*context, options);
|
auto local_limits = getStorageLimits(*context, options);
|
||||||
|
if (!query.distinct && !query.limit_with_ties && !query.prewhere() && !query.where() && query_info_copy.filter_asts.empty()
|
||||||
if (!query.distinct && !query.limit_with_ties && !query.prewhere() && !query.where() && query_info.filter_asts.empty()
|
|
||||||
&& !query.groupBy() && !query.having() && !query.orderBy() && !query.limitBy() && !query.join()
|
&& !query.groupBy() && !query.having() && !query.orderBy() && !query.limitBy() && !query.join()
|
||||||
&& !query_analyzer->hasAggregation() && !query_analyzer->hasWindow() && query.limitLength()
|
&& !query_analyzer->hasAggregation() && !query_analyzer->hasWindow() && query.limitLength()
|
||||||
&& limit_length <= std::numeric_limits<UInt64>::max() - limit_offset)
|
&& limit_length <= std::numeric_limits<UInt64>::max() - limit_offset)
|
||||||
{
|
{
|
||||||
query_info.limit = limit_length + limit_offset;
|
query_info_copy.limit = limit_length + limit_offset;
|
||||||
}
|
}
|
||||||
/// END OF TODO BLOCK ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
/// END OF TODO BLOCK ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||||
|
|
||||||
UInt64 rows_to_read = storage_merge_tree->estimateNumberOfRowsToRead(context, storage_snapshot, query_info, added_filter_nodes);
|
UInt64 rows_to_read
|
||||||
|
= storage_merge_tree->estimateNumberOfRowsToRead(context, storage_snapshot, query_info_copy, added_filter_nodes);
|
||||||
/// Note that we treat an estimation of 0 rows as a real estimation of no data to be read
|
/// Note that we treat an estimation of 0 rows as a real estimation of no data to be read
|
||||||
size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica;
|
size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica;
|
||||||
LOG_TRACE(
|
LOG_TRACE(
|
||||||
|
@ -6697,6 +6697,22 @@ Block MergeTreeData::getMinMaxCountProjectionBlock(
|
|||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ActionDAGNodes MergeTreeData::getFiltersForPrimaryKeyAnalysis(const InterpreterSelectQuery & select)
|
||||||
|
{
|
||||||
|
const auto & analysis_result = select.getAnalysisResult();
|
||||||
|
const auto & before_where = analysis_result.before_where;
|
||||||
|
const auto & where_column_name = analysis_result.where_column_name;
|
||||||
|
|
||||||
|
ActionDAGNodes filter_nodes;
|
||||||
|
if (auto additional_filter_info = select.getAdditionalQueryInfo())
|
||||||
|
filter_nodes.nodes.push_back(&additional_filter_info->actions->findInOutputs(additional_filter_info->column_name));
|
||||||
|
|
||||||
|
if (before_where)
|
||||||
|
filter_nodes.nodes.push_back(&before_where->findInOutputs(where_column_name));
|
||||||
|
|
||||||
|
return filter_nodes;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
std::optional<ProjectionCandidate> MergeTreeData::getQueryProcessingStageWithAggregateProjection(
|
std::optional<ProjectionCandidate> MergeTreeData::getQueryProcessingStageWithAggregateProjection(
|
||||||
ContextPtr query_context, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info) const
|
ContextPtr query_context, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info) const
|
||||||
@ -6778,19 +6794,10 @@ std::optional<ProjectionCandidate> MergeTreeData::getQueryProcessingStageWithAgg
|
|||||||
query_info.prepared_sets);
|
query_info.prepared_sets);
|
||||||
|
|
||||||
const auto & analysis_result = select.getAnalysisResult();
|
const auto & analysis_result = select.getAnalysisResult();
|
||||||
|
|
||||||
query_info.prepared_sets = select.getQueryAnalyzer()->getPreparedSets();
|
query_info.prepared_sets = select.getQueryAnalyzer()->getPreparedSets();
|
||||||
|
|
||||||
const auto & before_where = analysis_result.before_where;
|
|
||||||
const auto & where_column_name = analysis_result.where_column_name;
|
|
||||||
|
|
||||||
/// For PK analysis
|
/// For PK analysis
|
||||||
ActionDAGNodes added_filter_nodes;
|
ActionDAGNodes added_filter_nodes = MergeTreeData::getFiltersForPrimaryKeyAnalysis(select);
|
||||||
if (auto additional_filter_info = select.getAdditionalQueryInfo())
|
|
||||||
added_filter_nodes.nodes.push_back(&additional_filter_info->actions->findInOutputs(additional_filter_info->column_name));
|
|
||||||
|
|
||||||
if (before_where)
|
|
||||||
added_filter_nodes.nodes.push_back(&before_where->findInOutputs(where_column_name));
|
|
||||||
|
|
||||||
bool can_use_aggregate_projection = true;
|
bool can_use_aggregate_projection = true;
|
||||||
/// If the first stage of the query pipeline is more complex than Aggregating - Expression - Filter - ReadFromStorage,
|
/// If the first stage of the query pipeline is more complex than Aggregating - Expression - Filter - ReadFromStorage,
|
||||||
|
@ -48,6 +48,7 @@ namespace DB
|
|||||||
const size_t DEFAULT_DELAYED_STREAMS_FOR_PARALLEL_WRITE = 1000;
|
const size_t DEFAULT_DELAYED_STREAMS_FOR_PARALLEL_WRITE = 1000;
|
||||||
|
|
||||||
class AlterCommands;
|
class AlterCommands;
|
||||||
|
class InterpreterSelectQuery;
|
||||||
class MergeTreePartsMover;
|
class MergeTreePartsMover;
|
||||||
class MergeTreeDataMergerMutator;
|
class MergeTreeDataMergerMutator;
|
||||||
class MutationCommands;
|
class MutationCommands;
|
||||||
@ -1062,6 +1063,9 @@ public:
|
|||||||
/// TODO: make enabled by default in the next release if no problems found.
|
/// TODO: make enabled by default in the next release if no problems found.
|
||||||
bool allowRemoveStaleMovingParts() const;
|
bool allowRemoveStaleMovingParts() const;
|
||||||
|
|
||||||
|
/// Generate DAG filters based on query info (for PK analysis)
|
||||||
|
static struct ActionDAGNodes getFiltersForPrimaryKeyAnalysis(const InterpreterSelectQuery & select);
|
||||||
|
|
||||||
/// Estimate the number of rows to read based on primary key analysis (which could be very rough)
|
/// Estimate the number of rows to read based on primary key analysis (which could be very rough)
|
||||||
/// It is used to make a decision whether to enable parallel replicas (distributed processing) or not and how
|
/// It is used to make a decision whether to enable parallel replicas (distributed processing) or not and how
|
||||||
/// many to replicas to use
|
/// many to replicas to use
|
||||||
|
Loading…
Reference in New Issue
Block a user