mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
fix JOIN with PREWHERE with second analyze() (#11051)
This commit is contained in:
parent
2673c985bd
commit
9b38324028
@ -1025,6 +1025,12 @@ 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;
|
||||
|
@ -243,8 +243,6 @@ public:
|
||||
const NamesAndTypesList & aggregationKeys() const { return aggregation_keys; }
|
||||
const AggregateDescriptions & aggregates() const { return aggregate_descriptions; }
|
||||
|
||||
/// Create Set-s that we make from IN section to use index on them.
|
||||
void makeSetsForIndex(const ASTPtr & node);
|
||||
const PreparedSets & getPreparedSets() const { return prepared_sets; }
|
||||
|
||||
/// Tables that will need to be sent to remote servers for distributed query processing.
|
||||
@ -275,6 +273,9 @@ private:
|
||||
*/
|
||||
SetPtr isPlainStorageSetInSubquery(const ASTPtr & subquery_or_table_name);
|
||||
|
||||
/// 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 ASTSelectQuery * getAggregatingQuery() const;
|
||||
|
@ -309,12 +309,29 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
ASTSelectQuery & query = getSelectQuery();
|
||||
std::shared_ptr<TableJoin> table_join = joined_tables.makeTableJoin(query);
|
||||
|
||||
auto analyze = [&] (bool try_move_to_prewhere = true)
|
||||
ASTPtr row_policy_filter;
|
||||
if (storage)
|
||||
row_policy_filter = context->getRowPolicyCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER);
|
||||
|
||||
auto analyze = [&] (bool try_move_to_prewhere)
|
||||
{
|
||||
syntax_analyzer_result = SyntaxAnalyzer(*context).analyzeSelect(
|
||||
query_ptr, SyntaxAnalyzerResult(source_header.getNamesAndTypesList(), storage),
|
||||
options, joined_tables.tablesWithColumns(), required_result_column_names, table_join);
|
||||
|
||||
if (try_move_to_prewhere && storage && !row_policy_filter && query.where() && !query.prewhere() && !query.final())
|
||||
{
|
||||
/// PREWHERE optimization: transfer some condition from WHERE to PREWHERE if enabled and viable
|
||||
if (const auto * merge_tree = dynamic_cast<const MergeTreeData *>(storage.get()))
|
||||
{
|
||||
SelectQueryInfo current_info;
|
||||
current_info.query = query_ptr;
|
||||
current_info.syntax_analyzer_result = syntax_analyzer_result;
|
||||
|
||||
MergeTreeWhereOptimizer{current_info, *context, *merge_tree, syntax_analyzer_result->requiredSourceColumns(), log};
|
||||
}
|
||||
}
|
||||
|
||||
/// Save scalar sub queries's results in the query context
|
||||
if (!options.only_analyze && context->hasQueryContext())
|
||||
for (const auto & it : syntax_analyzer_result->getScalars())
|
||||
@ -365,7 +382,6 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
source_header = storage->getSampleBlockForColumns(required_columns);
|
||||
|
||||
/// Fix source_header for filter actions.
|
||||
auto row_policy_filter = context->getRowPolicyCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER);
|
||||
if (row_policy_filter)
|
||||
{
|
||||
filter_info = std::make_shared<FilterInfo>();
|
||||
@ -378,10 +394,10 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
throw Exception("PREWHERE is not supported if the table is filtered by row-level security expression", ErrorCodes::ILLEGAL_PREWHERE);
|
||||
|
||||
/// Calculate structure of the result.
|
||||
result_header = getSampleBlockImpl(try_move_to_prewhere);
|
||||
result_header = getSampleBlockImpl();
|
||||
};
|
||||
|
||||
analyze();
|
||||
analyze(settings.optimize_move_to_prewhere);
|
||||
|
||||
bool need_analyze_again = false;
|
||||
if (analysis_result.prewhere_constant_filter_description.always_false || analysis_result.prewhere_constant_filter_description.always_true)
|
||||
@ -481,40 +497,8 @@ QueryPipeline InterpreterSelectQuery::executeWithProcessors()
|
||||
}
|
||||
|
||||
|
||||
Block InterpreterSelectQuery::getSampleBlockImpl(bool try_move_to_prewhere)
|
||||
Block InterpreterSelectQuery::getSampleBlockImpl()
|
||||
{
|
||||
auto & query = getSelectQuery();
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
/// Do all AST changes here, because actions from analysis_result will be used later in readImpl.
|
||||
|
||||
if (storage)
|
||||
{
|
||||
query_analyzer->makeSetsForIndex(query.where());
|
||||
query_analyzer->makeSetsForIndex(query.prewhere());
|
||||
|
||||
/// PREWHERE optimization.
|
||||
/// Turn off, if the table filter (row-level security) is applied.
|
||||
if (!context->getRowPolicyCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER))
|
||||
{
|
||||
auto optimize_prewhere = [&](auto & merge_tree)
|
||||
{
|
||||
SelectQueryInfo current_info;
|
||||
current_info.query = query_ptr;
|
||||
current_info.syntax_analyzer_result = syntax_analyzer_result;
|
||||
current_info.sets = query_analyzer->getPreparedSets();
|
||||
|
||||
/// Try transferring some condition from WHERE to PREWHERE if enabled and viable
|
||||
if (settings.optimize_move_to_prewhere && try_move_to_prewhere && query.where() && !query.prewhere() && !query.final())
|
||||
MergeTreeWhereOptimizer{current_info, *context, merge_tree,
|
||||
syntax_analyzer_result->requiredSourceColumns(), log};
|
||||
};
|
||||
|
||||
if (const auto * merge_tree_data = dynamic_cast<const MergeTreeData *>(storage.get()))
|
||||
optimize_prewhere(*merge_tree_data);
|
||||
}
|
||||
}
|
||||
|
||||
if (storage && !options.only_analyze)
|
||||
from_stage = storage->getQueryProcessingStage(*context, options.to_stage, query_ptr);
|
||||
|
||||
|
@ -106,7 +106,7 @@ private:
|
||||
|
||||
ASTSelectQuery & getSelectQuery() { return query_ptr->as<ASTSelectQuery &>(); }
|
||||
|
||||
Block getSampleBlockImpl(bool try_move_to_prewhere);
|
||||
Block getSampleBlockImpl();
|
||||
|
||||
struct Pipeline
|
||||
{
|
||||
|
@ -767,8 +767,13 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyzeSelect(
|
||||
const auto & settings = context.getSettingsRef();
|
||||
|
||||
const NameSet & source_columns_set = result.source_columns_set;
|
||||
result.analyzed_join = table_join;
|
||||
if (!result.analyzed_join) /// ExpressionAnalyzer expects some not empty object here
|
||||
|
||||
if (table_join)
|
||||
{
|
||||
result.analyzed_join = table_join;
|
||||
result.analyzed_join->resetCollected();
|
||||
}
|
||||
else /// TODO: remove. For now ExpressionAnalyzer expects some not empty object here
|
||||
result.analyzed_join = std::make_shared<TableJoin>();
|
||||
|
||||
if (remove_duplicates)
|
||||
|
@ -29,6 +29,18 @@ TableJoin::TableJoin(const Settings & settings, VolumeJBODPtr tmp_volume_)
|
||||
join_algorithm = JoinAlgorithm::PREFER_PARTIAL_MERGE;
|
||||
}
|
||||
|
||||
void TableJoin::resetCollected()
|
||||
{
|
||||
key_names_left.clear();
|
||||
key_names_right.clear();
|
||||
key_asts_left.clear();
|
||||
key_asts_right.clear();
|
||||
columns_from_joined_table.clear();
|
||||
columns_added_by_join.clear();
|
||||
original_names.clear();
|
||||
renames.clear();
|
||||
}
|
||||
|
||||
void TableJoin::addUsingKey(const ASTPtr & ast)
|
||||
{
|
||||
key_names_left.push_back(ast->getColumnName());
|
||||
|
@ -112,6 +112,7 @@ public:
|
||||
const String & temporaryFilesCodec() const { return temporary_files_codec; }
|
||||
bool enablePartialMergeJoinOptimizations() const { return partial_merge_join_optimizations; }
|
||||
|
||||
void resetCollected();
|
||||
void addUsingKey(const ASTPtr & ast);
|
||||
void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast);
|
||||
|
||||
|
@ -0,0 +1,8 @@
|
||||
1
|
||||
2
|
||||
3
|
||||
4
|
||||
5
|
||||
6
|
||||
7
|
||||
8
|
21
tests/queries/0_stateless/01281_join_with_prewhere_fix.sql
Normal file
21
tests/queries/0_stateless/01281_join_with_prewhere_fix.sql
Normal file
@ -0,0 +1,21 @@
|
||||
drop table if exists t;
|
||||
|
||||
create table t (x UInt8, id UInt8) ENGINE = MergeTree() order by (id);
|
||||
insert into t values (1, 1);
|
||||
|
||||
set enable_optimize_predicate_expression = 0;
|
||||
|
||||
select 1 from t as l join t as r on l.id = r.id prewhere l.x;
|
||||
select 2 from t as l join t as r on l.id = r.id where r.x;
|
||||
select 3 from t as l join t as r on l.id = r.id prewhere l.x where r.x;
|
||||
select 4 from t as l join t as r using id prewhere l.x where r.x;
|
||||
|
||||
select 5 from t as l join t as r on l.id = r.id where l.x and r.x;
|
||||
select 6 from t as l join t as r using id where l.x and r.x;
|
||||
|
||||
set optimize_move_to_prewhere = 0;
|
||||
|
||||
select 7 from t as l join t as r on l.id = r.id where l.x and r.x;
|
||||
select 8 from t as l join t as r using id where l.x and r.x;
|
||||
|
||||
drop table t;
|
Loading…
Reference in New Issue
Block a user