This commit is contained in:
Nikolai Kochetov 2024-07-03 14:44:47 +00:00
parent 6a06024983
commit 41c62ca663
5 changed files with 75 additions and 78 deletions

View File

@ -10,7 +10,6 @@
#include <Functions/FunctionFactory.h>
#include <Functions/CastOverloadResolver.h>
#include <Functions/indexHint.h>
#include <QueryPipeline/Pipe.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
@ -331,14 +330,14 @@ public:
void addExpressionStep(QueryPlan & query_plan,
const ActionsAndProjectInputsFlagPtr & expression_actions,
const std::string & step_description,
std::vector<const ActionsDAG *> & result_actions_to_execute)
UsefulSets & useful_sets)
{
auto actions = ActionsDAG::clone(&expression_actions->dag);
if (expression_actions->project_input)
actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
auto expression_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), actions);
result_actions_to_execute.push_back(expression_step->getExpression().get());
appendSetsFromActionsDAG(*expression_step->getExpression(), useful_sets);
expression_step->setStepDescription(step_description);
query_plan.addStep(std::move(expression_step));
}
@ -346,7 +345,7 @@ void addExpressionStep(QueryPlan & query_plan,
void addFilterStep(QueryPlan & query_plan,
const FilterAnalysisResult & filter_analysis_result,
const std::string & step_description,
std::vector<const ActionsDAG *> & result_actions_to_execute)
UsefulSets & useful_sets)
{
auto actions = ActionsDAG::clone(&filter_analysis_result.filter_actions->dag);
if (filter_analysis_result.filter_actions->project_input)
@ -356,7 +355,7 @@ void addFilterStep(QueryPlan & query_plan,
actions,
filter_analysis_result.filter_column_name,
filter_analysis_result.remove_filter_column);
result_actions_to_execute.push_back(where_step->getExpression().get());
appendSetsFromActionsDAG(*where_step->getExpression(), useful_sets);
where_step->setStepDescription(step_description);
query_plan.addStep(std::move(where_step));
}
@ -544,7 +543,7 @@ void addTotalsHavingStep(QueryPlan & query_plan,
const QueryAnalysisResult & query_analysis_result,
const PlannerContextPtr & planner_context,
const QueryNode & query_node,
std::vector<const ActionsDAG *> & result_actions_to_execute)
UsefulSets & useful_sets)
{
const auto & query_context = planner_context->getQueryContext();
const auto & settings = query_context->getSettingsRef();
@ -573,7 +572,7 @@ void addTotalsHavingStep(QueryPlan & query_plan,
need_finalize);
if (having_analysis_result.filter_actions)
result_actions_to_execute.push_back(totals_having_step->getActions().get());
appendSetsFromActionsDAG(*totals_having_step->getActions(), useful_sets);
query_plan.addStep(std::move(totals_having_step));
}
@ -887,7 +886,7 @@ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan,
const PlannerContextPtr & planner_context,
const PlannerQueryProcessingInfo & query_processing_info,
const QueryTreeNodePtr & query_tree,
std::vector<const ActionsDAG *> & result_actions_to_execute)
UsefulSets & useful_sets)
{
const auto & query_node = query_tree->as<QueryNode &>();
@ -919,7 +918,7 @@ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan,
if (expressions_analysis_result.hasLimitBy())
{
const auto & limit_by_analysis_result = expressions_analysis_result.getLimitBy();
addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", result_actions_to_execute);
addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", useful_sets);
addLimitByStep(query_plan, limit_by_analysis_result, query_node);
}
@ -1057,47 +1056,15 @@ void addOffsetStep(QueryPlan & query_plan, const QueryAnalysisResult & query_ana
}
}
void collectSetsFromActionsDAG(const ActionsDAG & dag, std::unordered_set<const FutureSet *> & useful_sets)
{
for (const auto & node : dag.getNodes())
{
if (node.column)
{
const IColumn * column = node.column.get();
if (const auto * column_const = typeid_cast<const ColumnConst *>(column))
column = &column_const->getDataColumn();
if (const auto * column_set = typeid_cast<const ColumnSet *>(column))
useful_sets.insert(column_set->getData().get());
}
if (node.type == ActionsDAG::ActionType::FUNCTION && node.function_base->getName() == "indexHint")
{
ActionsDAG::NodeRawConstPtrs children;
if (const auto * adaptor = typeid_cast<const FunctionToFunctionBaseAdaptor *>(node.function_base.get()))
{
if (const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get()))
{
collectSetsFromActionsDAG(*index_hint->getActions(), useful_sets);
}
}
}
}
}
void addBuildSubqueriesForSetsStepIfNeeded(
QueryPlan & query_plan,
const SelectQueryOptions & select_query_options,
const PlannerContextPtr & planner_context,
const std::vector<const ActionsDAG *> & result_actions_to_execute)
const UsefulSets & useful_sets)
{
auto subqueries = planner_context->getPreparedSets().getSubqueries();
std::unordered_set<const FutureSet *> useful_sets;
for (const auto * actions_to_execute : result_actions_to_execute)
collectSetsFromActionsDAG(*actions_to_execute, useful_sets);
auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.get()); };
auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set); };
auto it = std::remove_if(subqueries.begin(), subqueries.end(), std::move(predicate));
subqueries.erase(it, subqueries.end());
@ -1542,15 +1509,15 @@ void Planner::buildPlanForQueryNode()
planner_context,
query_processing_info);
std::vector<const ActionsDAG *> result_actions_to_execute = std::move(join_tree_query_plan.actions_dags);
auto useful_sets = std::move(join_tree_query_plan.useful_sets);
for (auto & [_, table_expression_data] : planner_context->getTableExpressionNodeToData())
{
if (table_expression_data.getPrewhereFilterActions())
result_actions_to_execute.push_back(table_expression_data.getPrewhereFilterActions().get());
appendSetsFromActionsDAG(*table_expression_data.getPrewhereFilterActions(), useful_sets);
if (table_expression_data.getRowLevelFilterActions())
result_actions_to_execute.push_back(table_expression_data.getRowLevelFilterActions().get());
appendSetsFromActionsDAG(*table_expression_data.getRowLevelFilterActions(), useful_sets);
}
if (query_processing_info.isIntermediateStage())
@ -1561,7 +1528,7 @@ void Planner::buildPlanForQueryNode()
planner_context,
query_processing_info,
query_tree,
result_actions_to_execute);
useful_sets);
if (expression_analysis_result.hasAggregation())
{
@ -1573,13 +1540,13 @@ void Planner::buildPlanForQueryNode()
if (query_processing_info.isFirstStage())
{
if (expression_analysis_result.hasWhere())
addFilterStep(query_plan, expression_analysis_result.getWhere(), "WHERE", result_actions_to_execute);
addFilterStep(query_plan, expression_analysis_result.getWhere(), "WHERE", useful_sets);
if (expression_analysis_result.hasAggregation())
{
const auto & aggregation_analysis_result = expression_analysis_result.getAggregation();
if (aggregation_analysis_result.before_aggregation_actions)
addExpressionStep(query_plan, aggregation_analysis_result.before_aggregation_actions, "Before GROUP BY", result_actions_to_execute);
addExpressionStep(query_plan, aggregation_analysis_result.before_aggregation_actions, "Before GROUP BY", useful_sets);
addAggregationStep(query_plan, aggregation_analysis_result, query_analysis_result, planner_context, select_query_info);
}
@ -1598,7 +1565,7 @@ void Planner::buildPlanForQueryNode()
*/
const auto & window_analysis_result = expression_analysis_result.getWindow();
if (window_analysis_result.before_window_actions)
addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before WINDOW", result_actions_to_execute);
addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before WINDOW", useful_sets);
}
else
{
@ -1607,7 +1574,7 @@ void Planner::buildPlanForQueryNode()
* now, on shards (first_stage).
*/
const auto & projection_analysis_result = expression_analysis_result.getProjection();
addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", result_actions_to_execute);
addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", useful_sets);
if (query_node.isDistinct())
{
@ -1623,7 +1590,7 @@ void Planner::buildPlanForQueryNode()
if (expression_analysis_result.hasSort())
{
const auto & sort_analysis_result = expression_analysis_result.getSort();
addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", result_actions_to_execute);
addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", useful_sets);
}
}
}
@ -1634,7 +1601,7 @@ void Planner::buildPlanForQueryNode()
planner_context,
query_processing_info,
query_tree,
result_actions_to_execute);
useful_sets);
}
if (query_processing_info.isSecondStage() || query_processing_info.isFromAggregationState())
@ -1656,14 +1623,14 @@ void Planner::buildPlanForQueryNode()
if (query_node.isGroupByWithTotals())
{
addTotalsHavingStep(query_plan, expression_analysis_result, query_analysis_result, planner_context, query_node, result_actions_to_execute);
addTotalsHavingStep(query_plan, expression_analysis_result, query_analysis_result, planner_context, query_node, useful_sets);
having_executed = true;
}
addCubeOrRollupStepIfNeeded(query_plan, aggregation_analysis_result, query_analysis_result, planner_context, select_query_info, query_node);
if (!having_executed && expression_analysis_result.hasHaving())
addFilterStep(query_plan, expression_analysis_result.getHaving(), "HAVING", result_actions_to_execute);
addFilterStep(query_plan, expression_analysis_result.getHaving(), "HAVING", useful_sets);
}
if (query_processing_info.isFromAggregationState())
@ -1678,16 +1645,16 @@ void Planner::buildPlanForQueryNode()
{
const auto & window_analysis_result = expression_analysis_result.getWindow();
if (expression_analysis_result.hasAggregation())
addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before window functions", result_actions_to_execute);
addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before window functions", useful_sets);
addWindowSteps(query_plan, planner_context, window_analysis_result);
}
if (expression_analysis_result.hasQualify())
addFilterStep(query_plan, expression_analysis_result.getQualify(), "QUALIFY", result_actions_to_execute);
addFilterStep(query_plan, expression_analysis_result.getQualify(), "QUALIFY", useful_sets);
const auto & projection_analysis_result = expression_analysis_result.getProjection();
addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", result_actions_to_execute);
addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", useful_sets);
if (query_node.isDistinct())
{
@ -1703,7 +1670,7 @@ void Planner::buildPlanForQueryNode()
if (expression_analysis_result.hasSort())
{
const auto & sort_analysis_result = expression_analysis_result.getSort();
addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", result_actions_to_execute);
addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", useful_sets);
}
}
else
@ -1756,7 +1723,7 @@ void Planner::buildPlanForQueryNode()
if (!query_processing_info.isFromAggregationState() && expression_analysis_result.hasLimitBy())
{
const auto & limit_by_analysis_result = expression_analysis_result.getLimitBy();
addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", result_actions_to_execute);
addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", useful_sets);
addLimitByStep(query_plan, limit_by_analysis_result, query_node);
}
@ -1788,7 +1755,7 @@ void Planner::buildPlanForQueryNode()
if (!query_processing_info.isToAggregationState())
{
const auto & projection_analysis_result = expression_analysis_result.getProjection();
addExpressionStep(query_plan, projection_analysis_result.project_names_actions, "Project names", result_actions_to_execute);
addExpressionStep(query_plan, projection_analysis_result.project_names_actions, "Project names", useful_sets);
}
// For additional_result_filter setting
@ -1796,7 +1763,7 @@ void Planner::buildPlanForQueryNode()
}
if (!select_query_options.only_analyze)
addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute);
addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, useful_sets);
query_node_to_plan_step_mapping[&query_node] = query_plan.getRootNode();
}

View File

@ -1181,13 +1181,13 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
join_clauses_and_actions.left_join_expressions_actions->appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header);
auto left_join_expressions_actions_step = std::make_unique<ExpressionStep>(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions);
left_join_expressions_actions_step->setStepDescription("JOIN actions");
left_join_tree_query_plan.actions_dags.emplace_back(left_join_expressions_actions_step->getExpression().get());
appendSetsFromActionsDAG(*left_join_expressions_actions_step->getExpression(), left_join_tree_query_plan.useful_sets);
left_plan.addStep(std::move(left_join_expressions_actions_step));
join_clauses_and_actions.right_join_expressions_actions->appendInputsForUnusedColumns(right_plan.getCurrentDataStream().header);
auto right_join_expressions_actions_step = std::make_unique<ExpressionStep>(right_plan.getCurrentDataStream(), join_clauses_and_actions.right_join_expressions_actions);
right_join_expressions_actions_step->setStepDescription("JOIN actions");
right_join_tree_query_plan.actions_dags.emplace_back(right_join_expressions_actions_step->getExpression().get());
appendSetsFromActionsDAG(*right_join_expressions_actions_step->getExpression(), right_join_tree_query_plan.useful_sets);
right_plan.addStep(std::move(right_join_expressions_actions_step));
}
@ -1387,7 +1387,8 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
mixed_join_expression = std::make_shared<ExpressionActions>(
std::move(join_clauses_and_actions.mixed_join_expressions_actions),
ExpressionActionsSettings::fromContext(planner_context->getQueryContext()));
left_join_tree_query_plan.actions_dags.push_back(&mixed_join_expression->getActionsDAG());
appendSetsFromActionsDAG(mixed_join_expression->getActionsDAG(), left_join_tree_query_plan.useful_sets);
}
}
else if (join_node.isUsingJoinExpression())
@ -1585,16 +1586,10 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
for (const auto & right_join_tree_query_plan_row_policy : right_join_tree_query_plan.used_row_policies)
left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy);
/// Collect all required actions dags in `left_join_tree_query_plan.actions_dags`
/// Collect all required actions sets in `left_join_tree_query_plan.useful_sets`
if (!is_filled_join)
for (const auto * action_dag : right_join_tree_query_plan.actions_dags)
left_join_tree_query_plan.actions_dags.emplace_back(action_dag);
// if (join_clauses_and_actions.left_join_expressions_actions)
// left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.left_join_expressions_actions.get());
// if (join_clauses_and_actions.right_join_expressions_actions)
// left_join_tree_query_plan.actions_dags.emplace_back(join_clauses_and_actions.right_join_expressions_actions.get());
// if (join_clauses_and_actions.mixed_join_expressions_actions)
// left_join_tree_query_plan.actions_dags.push_back(join_clauses_and_actions.mixed_join_expressions_actions.get());
for (const auto & useful_set : right_join_tree_query_plan.useful_sets)
left_join_tree_query_plan.useful_sets.insert(useful_set);
auto mapping = std::move(left_join_tree_query_plan.query_node_to_plan_step_mapping);
auto & r_mapping = right_join_tree_query_plan.query_node_to_plan_step_mapping;
@ -1604,7 +1599,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
.query_plan = std::move(result_plan),
.from_stage = QueryProcessingStage::FetchColumns,
.used_row_policies = std::move(left_join_tree_query_plan.used_row_policies),
.actions_dags = std::move(left_join_tree_query_plan.actions_dags),
.useful_sets = std::move(left_join_tree_query_plan.useful_sets),
.query_node_to_plan_step_mapping = std::move(mapping),
};
}
@ -1649,7 +1644,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
auto array_join_actions = std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), std::move(array_join_action_dag));
array_join_actions->setStepDescription("ARRAY JOIN actions");
join_tree_query_plan.actions_dags.push_back(array_join_actions->getExpression().get());
appendSetsFromActionsDAG(*array_join_actions->getExpression(), join_tree_query_plan.useful_sets);
plan.addStep(std::move(array_join_actions));
auto drop_unused_columns_before_array_join_actions_dag = std::make_unique<ActionsDAG>(plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
@ -1690,7 +1685,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
.query_plan = std::move(plan),
.from_stage = QueryProcessingStage::FetchColumns,
.used_row_policies = std::move(join_tree_query_plan.used_row_policies),
.actions_dags = std::move(join_tree_query_plan.actions_dags),
.useful_sets = std::move(join_tree_query_plan.useful_sets),
.query_node_to_plan_step_mapping = std::move(join_tree_query_plan.query_node_to_plan_step_mapping),
};
}

View File

@ -11,12 +11,14 @@
namespace DB
{
using UsefulSets = std::unordered_set<FutureSetPtr>;
struct JoinTreeQueryPlan
{
QueryPlan query_plan;
QueryProcessingStage::Enum from_stage;
std::set<std::string> used_row_policies{};
std::vector<const ActionsDAG *> actions_dags{};
UsefulSets useful_sets;
std::unordered_map<const QueryNode *, const QueryPlan::Node *> query_node_to_plan_step_mapping{};
};

View File

@ -11,10 +11,12 @@
#include <DataTypes/DataTypeNullable.h>
#include <Columns/getLeastSuperColumn.h>
#include <Columns/ColumnSet.h>
#include <IO/WriteBufferFromString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/indexHint.h>
#include <Storages/StorageDummy.h>
@ -475,4 +477,32 @@ ASTPtr parseAdditionalResultFilter(const Settings & settings)
return additional_result_filter_ast;
}
void appendSetsFromActionsDAG(const ActionsDAG & dag, UsefulSets & useful_sets)
{
for (const auto & node : dag.getNodes())
{
if (node.column)
{
const IColumn * column = node.column.get();
if (const auto * column_const = typeid_cast<const ColumnConst *>(column))
column = &column_const->getDataColumn();
if (const auto * column_set = typeid_cast<const ColumnSet *>(column))
useful_sets.insert(column_set->getData());
}
if (node.type == ActionsDAG::ActionType::FUNCTION && node.function_base->getName() == "indexHint")
{
ActionsDAG::NodeRawConstPtrs children;
if (const auto * adaptor = typeid_cast<const FunctionToFunctionBaseAdaptor *>(node.function_base.get()))
{
if (const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get()))
{
appendSetsFromActionsDAG(*index_hint->getActions(), useful_sets);
}
}
}
}
}
}

View File

@ -88,4 +88,7 @@ FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree,
ASTPtr parseAdditionalResultFilter(const Settings & settings);
using UsefulSets = std::unordered_set<FutureSetPtr>;
void appendSetsFromActionsDAG(const ActionsDAG & dag, UsefulSets & useful_sets);
}