未验证 提交 9b383240 编写于 作者: A Artem Zuikov 提交者: GitHub

fix JOIN with PREWHERE with second analyze() (#11051)

上级 2673c985
......@@ -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);
......
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;
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册