mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #30310 from amosbird/projection-fix20
Fix IN set filtering when projection is used.
This commit is contained in:
commit
34009ea4a3
@ -1951,7 +1951,7 @@ void Context::shutdownKeeperDispatcher() const
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void Context::updateKeeperConfiguration(const Poco::Util::AbstractConfiguration & config)
|
void Context::updateKeeperConfiguration([[maybe_unused]] const Poco::Util::AbstractConfiguration & config)
|
||||||
{
|
{
|
||||||
#if USE_NURAFT
|
#if USE_NURAFT
|
||||||
std::lock_guard lock(shared->keeper_dispatcher_mutex);
|
std::lock_guard lock(shared->keeper_dispatcher_mutex);
|
||||||
|
@ -156,6 +156,15 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
|||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
|
InterpreterSelectQuery::InterpreterSelectQuery(
|
||||||
|
const ASTPtr & query_ptr_,
|
||||||
|
ContextPtr context_,
|
||||||
|
const SelectQueryOptions & options_,
|
||||||
|
PreparedSets prepared_sets_)
|
||||||
|
: InterpreterSelectQuery(query_ptr_, context_, std::nullopt, nullptr, options_, {}, {}, std::move(prepared_sets_))
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
InterpreterSelectQuery::InterpreterSelectQuery(
|
InterpreterSelectQuery::InterpreterSelectQuery(
|
||||||
const ASTPtr & query_ptr_,
|
const ASTPtr & query_ptr_,
|
||||||
ContextPtr context_,
|
ContextPtr context_,
|
||||||
@ -258,13 +267,15 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
|||||||
const StoragePtr & storage_,
|
const StoragePtr & storage_,
|
||||||
const SelectQueryOptions & options_,
|
const SelectQueryOptions & options_,
|
||||||
const Names & required_result_column_names,
|
const Names & required_result_column_names,
|
||||||
const StorageMetadataPtr & metadata_snapshot_)
|
const StorageMetadataPtr & metadata_snapshot_,
|
||||||
|
PreparedSets prepared_sets_)
|
||||||
/// NOTE: the query almost always should be cloned because it will be modified during analysis.
|
/// NOTE: the query almost always should be cloned because it will be modified during analysis.
|
||||||
: IInterpreterUnionOrSelectQuery(options_.modify_inplace ? query_ptr_ : query_ptr_->clone(), context_, options_)
|
: IInterpreterUnionOrSelectQuery(options_.modify_inplace ? query_ptr_ : query_ptr_->clone(), context_, options_)
|
||||||
, storage(storage_)
|
, storage(storage_)
|
||||||
, input_pipe(std::move(input_pipe_))
|
, input_pipe(std::move(input_pipe_))
|
||||||
, log(&Poco::Logger::get("InterpreterSelectQuery"))
|
, log(&Poco::Logger::get("InterpreterSelectQuery"))
|
||||||
, metadata_snapshot(metadata_snapshot_)
|
, metadata_snapshot(metadata_snapshot_)
|
||||||
|
, prepared_sets(std::move(prepared_sets_))
|
||||||
{
|
{
|
||||||
checkStackSize();
|
checkStackSize();
|
||||||
|
|
||||||
@ -354,7 +365,6 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
|||||||
|
|
||||||
/// Reuse already built sets for multiple passes of analysis
|
/// Reuse already built sets for multiple passes of analysis
|
||||||
SubqueriesForSets subquery_for_sets;
|
SubqueriesForSets subquery_for_sets;
|
||||||
PreparedSets prepared_sets;
|
|
||||||
|
|
||||||
auto analyze = [&] (bool try_move_to_prewhere)
|
auto analyze = [&] (bool try_move_to_prewhere)
|
||||||
{
|
{
|
||||||
@ -517,7 +527,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
|||||||
|
|
||||||
/// Reuse already built sets for multiple passes of analysis
|
/// Reuse already built sets for multiple passes of analysis
|
||||||
subquery_for_sets = std::move(query_analyzer->getSubqueriesForSets());
|
subquery_for_sets = std::move(query_analyzer->getSubqueriesForSets());
|
||||||
prepared_sets = std::move(query_analyzer->getPreparedSets());
|
prepared_sets = query_info.sets.empty() ? std::move(query_analyzer->getPreparedSets()) : std::move(query_info.sets);
|
||||||
|
|
||||||
/// Do not try move conditions to PREWHERE for the second time.
|
/// Do not try move conditions to PREWHERE for the second time.
|
||||||
/// Otherwise, we won't be able to fallback from inefficient PREWHERE to WHERE later.
|
/// Otherwise, we won't be able to fallback from inefficient PREWHERE to WHERE later.
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
#include <Interpreters/ExpressionActions.h>
|
#include <Interpreters/ExpressionActions.h>
|
||||||
#include <Interpreters/ExpressionAnalyzer.h>
|
#include <Interpreters/ExpressionAnalyzer.h>
|
||||||
#include <Interpreters/IInterpreterUnionOrSelectQuery.h>
|
#include <Interpreters/IInterpreterUnionOrSelectQuery.h>
|
||||||
|
#include <Interpreters/PreparedSets.h>
|
||||||
#include <Interpreters/StorageID.h>
|
#include <Interpreters/StorageID.h>
|
||||||
#include <Parsers/ASTSelectQuery.h>
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
#include <Storages/ReadInOrderOptimizer.h>
|
#include <Storages/ReadInOrderOptimizer.h>
|
||||||
@ -66,6 +67,13 @@ public:
|
|||||||
const StorageMetadataPtr & metadata_snapshot_ = nullptr,
|
const StorageMetadataPtr & metadata_snapshot_ = nullptr,
|
||||||
const SelectQueryOptions & = {});
|
const SelectQueryOptions & = {});
|
||||||
|
|
||||||
|
/// Read data not from the table specified in the query, but from the specified `storage_`.
|
||||||
|
InterpreterSelectQuery(
|
||||||
|
const ASTPtr & query_ptr_,
|
||||||
|
ContextPtr context_,
|
||||||
|
const SelectQueryOptions &,
|
||||||
|
PreparedSets prepared_sets_);
|
||||||
|
|
||||||
~InterpreterSelectQuery() override;
|
~InterpreterSelectQuery() override;
|
||||||
|
|
||||||
/// Execute a query. Get the stream of blocks to read.
|
/// Execute a query. Get the stream of blocks to read.
|
||||||
@ -83,7 +91,7 @@ public:
|
|||||||
|
|
||||||
const SelectQueryInfo & getQueryInfo() const { return query_info; }
|
const SelectQueryInfo & getQueryInfo() const { return query_info; }
|
||||||
|
|
||||||
const SelectQueryExpressionAnalyzer * getQueryAnalyzer() const { return query_analyzer.get(); }
|
SelectQueryExpressionAnalyzer * getQueryAnalyzer() const { return query_analyzer.get(); }
|
||||||
|
|
||||||
const ExpressionAnalysisResult & getAnalysisResult() const { return analysis_result; }
|
const ExpressionAnalysisResult & getAnalysisResult() const { return analysis_result; }
|
||||||
|
|
||||||
@ -104,7 +112,8 @@ private:
|
|||||||
const StoragePtr & storage_,
|
const StoragePtr & storage_,
|
||||||
const SelectQueryOptions &,
|
const SelectQueryOptions &,
|
||||||
const Names & required_result_column_names = {},
|
const Names & required_result_column_names = {},
|
||||||
const StorageMetadataPtr & metadata_snapshot_ = nullptr);
|
const StorageMetadataPtr & metadata_snapshot_ = nullptr,
|
||||||
|
PreparedSets prepared_sets_ = {});
|
||||||
|
|
||||||
ASTSelectQuery & getSelectQuery() { return query_ptr->as<ASTSelectQuery &>(); }
|
ASTSelectQuery & getSelectQuery() { return query_ptr->as<ASTSelectQuery &>(); }
|
||||||
|
|
||||||
@ -193,6 +202,9 @@ private:
|
|||||||
|
|
||||||
Poco::Logger * log;
|
Poco::Logger * log;
|
||||||
StorageMetadataPtr metadata_snapshot;
|
StorageMetadataPtr metadata_snapshot;
|
||||||
|
|
||||||
|
/// Reuse already built sets for multiple passes of analysis, possibly across interpreters.
|
||||||
|
PreparedSets prepared_sets;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -4552,8 +4552,12 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection(
|
|||||||
return false;
|
return false;
|
||||||
|
|
||||||
InterpreterSelectQuery select(
|
InterpreterSelectQuery select(
|
||||||
query_ptr, query_context, SelectQueryOptions{QueryProcessingStage::WithMergeableState}.ignoreProjections().ignoreAlias());
|
query_ptr,
|
||||||
|
query_context,
|
||||||
|
SelectQueryOptions{QueryProcessingStage::WithMergeableState}.ignoreProjections().ignoreAlias(),
|
||||||
|
query_info.sets /* prepared_sets */);
|
||||||
const auto & analysis_result = select.getAnalysisResult();
|
const auto & analysis_result = select.getAnalysisResult();
|
||||||
|
query_info.sets = std::move(select.getQueryAnalyzer()->getPreparedSets());
|
||||||
|
|
||||||
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,
|
||||||
@ -4897,6 +4901,8 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection(
|
|||||||
{
|
{
|
||||||
selected_candidate->aggregation_keys = select.getQueryAnalyzer()->aggregationKeys();
|
selected_candidate->aggregation_keys = select.getQueryAnalyzer()->aggregationKeys();
|
||||||
selected_candidate->aggregate_descriptions = select.getQueryAnalyzer()->aggregates();
|
selected_candidate->aggregate_descriptions = select.getQueryAnalyzer()->aggregates();
|
||||||
|
selected_candidate->subqueries_for_sets
|
||||||
|
= std::make_shared<SubqueriesForSets>(std::move(select.getQueryAnalyzer()->getSubqueriesForSets()));
|
||||||
}
|
}
|
||||||
|
|
||||||
query_info.projection = std::move(*selected_candidate);
|
query_info.projection = std::move(*selected_candidate);
|
||||||
|
@ -19,6 +19,7 @@
|
|||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Processors/ConcatProcessor.h>
|
#include <Processors/ConcatProcessor.h>
|
||||||
#include <Processors/QueryPlan/QueryPlan.h>
|
#include <Processors/QueryPlan/QueryPlan.h>
|
||||||
|
#include <Processors/QueryPlan/CreatingSetsStep.h>
|
||||||
#include <Processors/QueryPlan/FilterStep.h>
|
#include <Processors/QueryPlan/FilterStep.h>
|
||||||
#include <Processors/QueryPlan/ExpressionStep.h>
|
#include <Processors/QueryPlan/ExpressionStep.h>
|
||||||
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
|
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
|
||||||
@ -374,6 +375,12 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read(
|
|||||||
std::move(pipe),
|
std::move(pipe),
|
||||||
fmt::format("MergeTree(with {} projection {})", query_info.projection->desc->type, query_info.projection->desc->name));
|
fmt::format("MergeTree(with {} projection {})", query_info.projection->desc->type, query_info.projection->desc->name));
|
||||||
plan->addStep(std::move(step));
|
plan->addStep(std::move(step));
|
||||||
|
|
||||||
|
if (query_info.projection->subqueries_for_sets && !query_info.projection->subqueries_for_sets->empty())
|
||||||
|
{
|
||||||
|
SizeLimits limits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode);
|
||||||
|
addCreatingSetsStep(*plan, std::move(*query_info.projection->subqueries_for_sets), limits, context);
|
||||||
|
}
|
||||||
return plan;
|
return plan;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -42,6 +42,9 @@ using ClusterPtr = std::shared_ptr<Cluster>;
|
|||||||
struct MergeTreeDataSelectAnalysisResult;
|
struct MergeTreeDataSelectAnalysisResult;
|
||||||
using MergeTreeDataSelectAnalysisResultPtr = std::shared_ptr<MergeTreeDataSelectAnalysisResult>;
|
using MergeTreeDataSelectAnalysisResultPtr = std::shared_ptr<MergeTreeDataSelectAnalysisResult>;
|
||||||
|
|
||||||
|
struct SubqueryForSet;
|
||||||
|
using SubqueriesForSets = std::unordered_map<String, SubqueryForSet>;
|
||||||
|
|
||||||
struct PrewhereInfo
|
struct PrewhereInfo
|
||||||
{
|
{
|
||||||
/// Actions which are executed in order to alias columns are used for prewhere actions.
|
/// Actions which are executed in order to alias columns are used for prewhere actions.
|
||||||
@ -121,6 +124,7 @@ struct ProjectionCandidate
|
|||||||
ReadInOrderOptimizerPtr order_optimizer;
|
ReadInOrderOptimizerPtr order_optimizer;
|
||||||
InputOrderInfoPtr input_order_info;
|
InputOrderInfoPtr input_order_info;
|
||||||
ManyExpressionActions group_by_elements_actions;
|
ManyExpressionActions group_by_elements_actions;
|
||||||
|
std::shared_ptr<SubqueriesForSets> subqueries_for_sets;
|
||||||
MergeTreeDataSelectAnalysisResultPtr merge_tree_projection_select_result_ptr;
|
MergeTreeDataSelectAnalysisResultPtr merge_tree_projection_select_result_ptr;
|
||||||
MergeTreeDataSelectAnalysisResultPtr merge_tree_normal_select_result_ptr;
|
MergeTreeDataSelectAnalysisResultPtr merge_tree_normal_select_result_ptr;
|
||||||
};
|
};
|
||||||
|
@ -0,0 +1,2 @@
|
|||||||
|
2 3
|
||||||
|
2 3
|
12
tests/queries/0_stateless/01710_projection_in_set.sql
Normal file
12
tests/queries/0_stateless/01710_projection_in_set.sql
Normal file
@ -0,0 +1,12 @@
|
|||||||
|
drop table if exists x;
|
||||||
|
create table x (i UInt64, j UInt64, k UInt64, projection agg (select sum(j), avg(k) group by i), projection norm (select j, k order by i)) engine MergeTree order by tuple();
|
||||||
|
|
||||||
|
insert into x values (1, 2, 3);
|
||||||
|
|
||||||
|
set allow_experimental_projection_optimization = 1, use_index_for_in_with_subqueries = 0;
|
||||||
|
|
||||||
|
select sum(j), avg(k) from x where i in (select number from numbers(4));
|
||||||
|
|
||||||
|
select j, k from x where i in (select number from numbers(4));
|
||||||
|
|
||||||
|
drop table x;
|
Loading…
Reference in New Issue
Block a user