mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #26218 from amosbird/projection-improve1
Aggressive IN index analysis for projections.
This commit is contained in:
commit
0893b9ff8e
@ -1478,12 +1478,6 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
|
||||
chain.clear();
|
||||
};
|
||||
|
||||
if (storage)
|
||||
{
|
||||
query_analyzer.makeSetsForIndex(query.where());
|
||||
query_analyzer.makeSetsForIndex(query.prewhere());
|
||||
}
|
||||
|
||||
{
|
||||
ExpressionActionsChain chain(context);
|
||||
Names additional_required_columns_after_prewhere;
|
||||
|
@ -326,15 +326,15 @@ public:
|
||||
/// Deletes all columns except mentioned by SELECT, arranges the remaining columns and renames them to aliases.
|
||||
ActionsDAGPtr appendProjectResult(ExpressionActionsChain & chain) const;
|
||||
|
||||
/// Create Set-s that we make from IN section to use index on them.
|
||||
void makeSetsForIndex(const ASTPtr & node);
|
||||
|
||||
private:
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
/// If non-empty, ignore all expressions not from this list.
|
||||
NameSet required_result_columns;
|
||||
SelectQueryOptions query_options;
|
||||
|
||||
/// Create Set-s that we make from IN section to use index on them.
|
||||
void makeSetsForIndex(const ASTPtr & node);
|
||||
|
||||
JoinPtr makeTableJoin(
|
||||
const ASTTablesInSelectQueryElement & join_element,
|
||||
const ColumnsWithTypeAndName & left_sample_columns);
|
||||
|
@ -609,17 +609,17 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
|
||||
|
||||
query_info.query = query_ptr;
|
||||
query_info.has_window = query_analyzer->hasWindow();
|
||||
if (storage)
|
||||
{
|
||||
auto & query = getSelectQuery();
|
||||
query_analyzer->makeSetsForIndex(query.where());
|
||||
query_analyzer->makeSetsForIndex(query.prewhere());
|
||||
query_info.sets = query_analyzer->getPreparedSets();
|
||||
}
|
||||
|
||||
if (storage && !options.only_analyze)
|
||||
{
|
||||
from_stage = storage->getQueryProcessingStage(context, options.to_stage, metadata_snapshot, query_info);
|
||||
|
||||
/// TODO how can we make IN index work if we cache parts before selecting a projection?
|
||||
/// XXX Used for IN set index analysis. Is this a proper way?
|
||||
if (query_info.projection)
|
||||
metadata_snapshot->selected_projection = query_info.projection->desc;
|
||||
}
|
||||
|
||||
/// Do I need to perform the first part of the pipeline?
|
||||
/// Running on remote servers during distributed processing or if query is not distributed.
|
||||
///
|
||||
@ -1882,8 +1882,6 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
|
||||
if (max_streams > 1 && !is_remote)
|
||||
max_streams *= settings.max_streams_to_max_threads_ratio;
|
||||
|
||||
// TODO figure out how to make set for projections
|
||||
query_info.sets = query_analyzer->getPreparedSets();
|
||||
auto & prewhere_info = analysis_result.prewhere_info;
|
||||
|
||||
if (prewhere_info)
|
||||
|
@ -829,7 +829,8 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre
|
||||
log,
|
||||
requested_num_streams,
|
||||
result.index_stats,
|
||||
true);
|
||||
true /* use_skip_indexes */,
|
||||
true /* check_limits */);
|
||||
|
||||
size_t sum_marks_pk = total_marks_pk;
|
||||
for (const auto & stat : result.index_stats)
|
||||
|
@ -3854,16 +3854,20 @@ bool MergeTreeData::mayBenefitFromIndexForIn(
|
||||
for (const auto & index : metadata_snapshot->getSecondaryIndices())
|
||||
if (index_wrapper_factory.get(index)->mayBenefitFromIndexForIn(item))
|
||||
return true;
|
||||
if (metadata_snapshot->selected_projection
|
||||
&& metadata_snapshot->selected_projection->isPrimaryKeyColumnPossiblyWrappedInFunctions(item))
|
||||
return true;
|
||||
for (const auto & projection : metadata_snapshot->getProjections())
|
||||
{
|
||||
if (projection.isPrimaryKeyColumnPossiblyWrappedInFunctions(item))
|
||||
return true;
|
||||
}
|
||||
}
|
||||
/// The tuple itself may be part of the primary key, so check that as a last resort.
|
||||
if (isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(left_in_operand, metadata_snapshot))
|
||||
return true;
|
||||
if (metadata_snapshot->selected_projection
|
||||
&& metadata_snapshot->selected_projection->isPrimaryKeyColumnPossiblyWrappedInFunctions(left_in_operand))
|
||||
return true;
|
||||
for (const auto & projection : metadata_snapshot->getProjections())
|
||||
{
|
||||
if (projection.isPrimaryKeyColumnPossiblyWrappedInFunctions(left_in_operand))
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
else
|
||||
@ -3872,10 +3876,11 @@ bool MergeTreeData::mayBenefitFromIndexForIn(
|
||||
if (index_wrapper_factory.get(index)->mayBenefitFromIndexForIn(left_in_operand))
|
||||
return true;
|
||||
|
||||
if (metadata_snapshot->selected_projection
|
||||
&& metadata_snapshot->selected_projection->isPrimaryKeyColumnPossiblyWrappedInFunctions(left_in_operand))
|
||||
return true;
|
||||
|
||||
for (const auto & projection : metadata_snapshot->getProjections())
|
||||
{
|
||||
if (projection.isPrimaryKeyColumnPossiblyWrappedInFunctions(left_in_operand))
|
||||
return true;
|
||||
}
|
||||
return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(left_in_operand, metadata_snapshot);
|
||||
}
|
||||
}
|
||||
@ -3915,7 +3920,7 @@ static void selectBestProjection(
|
||||
candidate.required_columns,
|
||||
metadata_snapshot,
|
||||
candidate.desc->metadata,
|
||||
query_info, // TODO syntax_analysis_result set in index
|
||||
query_info,
|
||||
query_context,
|
||||
settings.max_threads,
|
||||
max_added_blocks);
|
||||
@ -3933,7 +3938,7 @@ static void selectBestProjection(
|
||||
required_columns,
|
||||
metadata_snapshot,
|
||||
metadata_snapshot,
|
||||
query_info, // TODO syntax_analysis_result set in index
|
||||
query_info,
|
||||
query_context,
|
||||
settings.max_threads,
|
||||
max_added_blocks);
|
||||
@ -4191,7 +4196,7 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection(
|
||||
analysis_result.required_columns,
|
||||
metadata_snapshot,
|
||||
metadata_snapshot,
|
||||
query_info, // TODO syntax_analysis_result set in index
|
||||
query_info,
|
||||
query_context,
|
||||
settings.max_threads,
|
||||
max_added_blocks);
|
||||
|
@ -762,7 +762,8 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
|
||||
Poco::Logger * log,
|
||||
size_t num_streams,
|
||||
ReadFromMergeTree::IndexStats & index_stats,
|
||||
bool use_skip_indexes)
|
||||
bool use_skip_indexes,
|
||||
bool check_limits)
|
||||
{
|
||||
RangesInDataParts parts_with_ranges(parts.size());
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
@ -890,7 +891,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
|
||||
|
||||
if (!ranges.ranges.empty())
|
||||
{
|
||||
if (limits.max_rows || leaf_limits.max_rows)
|
||||
if (check_limits && (limits.max_rows || leaf_limits.max_rows))
|
||||
{
|
||||
/// Fail fast if estimated number of rows to read exceeds the limit
|
||||
auto current_rows_estimate = ranges.getRowsCount();
|
||||
@ -1155,7 +1156,8 @@ size_t MergeTreeDataSelectExecutor::estimateNumMarksToRead(
|
||||
log,
|
||||
num_streams,
|
||||
index_stats,
|
||||
false);
|
||||
true /* use_skip_indexes */,
|
||||
false /* check_limits */);
|
||||
|
||||
return index_stats.back().num_granules_after;
|
||||
}
|
||||
|
@ -174,6 +174,7 @@ public:
|
||||
|
||||
/// Filter parts using primary key and secondary indexes.
|
||||
/// For every part, select mark ranges to read.
|
||||
/// If 'check_limits = true' it will throw exception if the amount of data exceed the limits from settings.
|
||||
static RangesInDataParts filterPartsByPrimaryKeyAndSkipIndexes(
|
||||
MergeTreeData::DataPartsVector && parts,
|
||||
StorageMetadataPtr metadata_snapshot,
|
||||
@ -184,7 +185,8 @@ public:
|
||||
Poco::Logger * log,
|
||||
size_t num_streams,
|
||||
ReadFromMergeTree::IndexStats & index_stats,
|
||||
bool use_skip_indexes);
|
||||
bool use_skip_indexes,
|
||||
bool check_limits);
|
||||
|
||||
/// Create expression for sampling.
|
||||
/// Also, calculate _sample_factor if needed.
|
||||
|
@ -28,7 +28,6 @@ struct StorageInMemoryMetadata
|
||||
ConstraintsDescription constraints;
|
||||
/// Table projections. Currently supported for MergeTree only.
|
||||
ProjectionsDescription projections;
|
||||
mutable const ProjectionDescription * selected_projection{};
|
||||
/// PARTITION BY expression. Currently supported for MergeTree only.
|
||||
KeyDescription partition_key;
|
||||
/// PRIMARY KEY expression. If absent, than equal to order_by_ast.
|
||||
|
@ -0,0 +1,2 @@
|
||||
1 1 1
|
||||
2 2 2
|
11
tests/queries/0_stateless/01710_projection_in_index.sql
Normal file
11
tests/queries/0_stateless/01710_projection_in_index.sql
Normal file
@ -0,0 +1,11 @@
|
||||
drop table if exists t;
|
||||
|
||||
create table t (i int, j int, k int, projection p (select * order by j)) engine MergeTree order by i settings index_granularity = 1;
|
||||
|
||||
insert into t select number, number, number from numbers(10);
|
||||
|
||||
set allow_experimental_projection_optimization = 1, max_rows_to_read = 3;
|
||||
|
||||
select * from t where i < 5 and j in (1, 2);
|
||||
|
||||
drop table t;
|
Loading…
Reference in New Issue
Block a user