mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
better
This commit is contained in:
parent
6a06024983
commit
41c62ca663
@ -10,7 +10,6 @@
|
|||||||
|
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
#include <Functions/CastOverloadResolver.h>
|
#include <Functions/CastOverloadResolver.h>
|
||||||
#include <Functions/indexHint.h>
|
|
||||||
|
|
||||||
#include <QueryPipeline/Pipe.h>
|
#include <QueryPipeline/Pipe.h>
|
||||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||||
@ -331,14 +330,14 @@ public:
|
|||||||
void addExpressionStep(QueryPlan & query_plan,
|
void addExpressionStep(QueryPlan & query_plan,
|
||||||
const ActionsAndProjectInputsFlagPtr & expression_actions,
|
const ActionsAndProjectInputsFlagPtr & expression_actions,
|
||||||
const std::string & step_description,
|
const std::string & step_description,
|
||||||
std::vector<const ActionsDAG *> & result_actions_to_execute)
|
UsefulSets & useful_sets)
|
||||||
{
|
{
|
||||||
auto actions = ActionsDAG::clone(&expression_actions->dag);
|
auto actions = ActionsDAG::clone(&expression_actions->dag);
|
||||||
if (expression_actions->project_input)
|
if (expression_actions->project_input)
|
||||||
actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
|
actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
|
||||||
|
|
||||||
auto expression_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), actions);
|
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);
|
expression_step->setStepDescription(step_description);
|
||||||
query_plan.addStep(std::move(expression_step));
|
query_plan.addStep(std::move(expression_step));
|
||||||
}
|
}
|
||||||
@ -346,7 +345,7 @@ void addExpressionStep(QueryPlan & query_plan,
|
|||||||
void addFilterStep(QueryPlan & query_plan,
|
void addFilterStep(QueryPlan & query_plan,
|
||||||
const FilterAnalysisResult & filter_analysis_result,
|
const FilterAnalysisResult & filter_analysis_result,
|
||||||
const std::string & step_description,
|
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);
|
auto actions = ActionsDAG::clone(&filter_analysis_result.filter_actions->dag);
|
||||||
if (filter_analysis_result.filter_actions->project_input)
|
if (filter_analysis_result.filter_actions->project_input)
|
||||||
@ -356,7 +355,7 @@ void addFilterStep(QueryPlan & query_plan,
|
|||||||
actions,
|
actions,
|
||||||
filter_analysis_result.filter_column_name,
|
filter_analysis_result.filter_column_name,
|
||||||
filter_analysis_result.remove_filter_column);
|
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);
|
where_step->setStepDescription(step_description);
|
||||||
query_plan.addStep(std::move(where_step));
|
query_plan.addStep(std::move(where_step));
|
||||||
}
|
}
|
||||||
@ -544,7 +543,7 @@ void addTotalsHavingStep(QueryPlan & query_plan,
|
|||||||
const QueryAnalysisResult & query_analysis_result,
|
const QueryAnalysisResult & query_analysis_result,
|
||||||
const PlannerContextPtr & planner_context,
|
const PlannerContextPtr & planner_context,
|
||||||
const QueryNode & query_node,
|
const QueryNode & query_node,
|
||||||
std::vector<const ActionsDAG *> & result_actions_to_execute)
|
UsefulSets & useful_sets)
|
||||||
{
|
{
|
||||||
const auto & query_context = planner_context->getQueryContext();
|
const auto & query_context = planner_context->getQueryContext();
|
||||||
const auto & settings = query_context->getSettingsRef();
|
const auto & settings = query_context->getSettingsRef();
|
||||||
@ -573,7 +572,7 @@ void addTotalsHavingStep(QueryPlan & query_plan,
|
|||||||
need_finalize);
|
need_finalize);
|
||||||
|
|
||||||
if (having_analysis_result.filter_actions)
|
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));
|
query_plan.addStep(std::move(totals_having_step));
|
||||||
}
|
}
|
||||||
@ -887,7 +886,7 @@ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan,
|
|||||||
const PlannerContextPtr & planner_context,
|
const PlannerContextPtr & planner_context,
|
||||||
const PlannerQueryProcessingInfo & query_processing_info,
|
const PlannerQueryProcessingInfo & query_processing_info,
|
||||||
const QueryTreeNodePtr & query_tree,
|
const QueryTreeNodePtr & query_tree,
|
||||||
std::vector<const ActionsDAG *> & result_actions_to_execute)
|
UsefulSets & useful_sets)
|
||||||
{
|
{
|
||||||
const auto & query_node = query_tree->as<QueryNode &>();
|
const auto & query_node = query_tree->as<QueryNode &>();
|
||||||
|
|
||||||
@ -919,7 +918,7 @@ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan,
|
|||||||
if (expressions_analysis_result.hasLimitBy())
|
if (expressions_analysis_result.hasLimitBy())
|
||||||
{
|
{
|
||||||
const auto & limit_by_analysis_result = expressions_analysis_result.getLimitBy();
|
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);
|
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(
|
void addBuildSubqueriesForSetsStepIfNeeded(
|
||||||
QueryPlan & query_plan,
|
QueryPlan & query_plan,
|
||||||
const SelectQueryOptions & select_query_options,
|
const SelectQueryOptions & select_query_options,
|
||||||
const PlannerContextPtr & planner_context,
|
const PlannerContextPtr & planner_context,
|
||||||
const std::vector<const ActionsDAG *> & result_actions_to_execute)
|
const UsefulSets & useful_sets)
|
||||||
{
|
{
|
||||||
auto subqueries = planner_context->getPreparedSets().getSubqueries();
|
auto subqueries = planner_context->getPreparedSets().getSubqueries();
|
||||||
std::unordered_set<const FutureSet *> useful_sets;
|
|
||||||
|
|
||||||
for (const auto * actions_to_execute : result_actions_to_execute)
|
auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set); };
|
||||||
collectSetsFromActionsDAG(*actions_to_execute, useful_sets);
|
|
||||||
|
|
||||||
auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.get()); };
|
|
||||||
auto it = std::remove_if(subqueries.begin(), subqueries.end(), std::move(predicate));
|
auto it = std::remove_if(subqueries.begin(), subqueries.end(), std::move(predicate));
|
||||||
subqueries.erase(it, subqueries.end());
|
subqueries.erase(it, subqueries.end());
|
||||||
|
|
||||||
@ -1542,15 +1509,15 @@ void Planner::buildPlanForQueryNode()
|
|||||||
planner_context,
|
planner_context,
|
||||||
query_processing_info);
|
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())
|
for (auto & [_, table_expression_data] : planner_context->getTableExpressionNodeToData())
|
||||||
{
|
{
|
||||||
if (table_expression_data.getPrewhereFilterActions())
|
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())
|
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())
|
if (query_processing_info.isIntermediateStage())
|
||||||
@ -1561,7 +1528,7 @@ void Planner::buildPlanForQueryNode()
|
|||||||
planner_context,
|
planner_context,
|
||||||
query_processing_info,
|
query_processing_info,
|
||||||
query_tree,
|
query_tree,
|
||||||
result_actions_to_execute);
|
useful_sets);
|
||||||
|
|
||||||
if (expression_analysis_result.hasAggregation())
|
if (expression_analysis_result.hasAggregation())
|
||||||
{
|
{
|
||||||
@ -1573,13 +1540,13 @@ void Planner::buildPlanForQueryNode()
|
|||||||
if (query_processing_info.isFirstStage())
|
if (query_processing_info.isFirstStage())
|
||||||
{
|
{
|
||||||
if (expression_analysis_result.hasWhere())
|
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())
|
if (expression_analysis_result.hasAggregation())
|
||||||
{
|
{
|
||||||
const auto & aggregation_analysis_result = expression_analysis_result.getAggregation();
|
const auto & aggregation_analysis_result = expression_analysis_result.getAggregation();
|
||||||
if (aggregation_analysis_result.before_aggregation_actions)
|
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);
|
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();
|
const auto & window_analysis_result = expression_analysis_result.getWindow();
|
||||||
if (window_analysis_result.before_window_actions)
|
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
|
else
|
||||||
{
|
{
|
||||||
@ -1607,7 +1574,7 @@ void Planner::buildPlanForQueryNode()
|
|||||||
* now, on shards (first_stage).
|
* now, on shards (first_stage).
|
||||||
*/
|
*/
|
||||||
const auto & projection_analysis_result = expression_analysis_result.getProjection();
|
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())
|
if (query_node.isDistinct())
|
||||||
{
|
{
|
||||||
@ -1623,7 +1590,7 @@ void Planner::buildPlanForQueryNode()
|
|||||||
if (expression_analysis_result.hasSort())
|
if (expression_analysis_result.hasSort())
|
||||||
{
|
{
|
||||||
const auto & sort_analysis_result = expression_analysis_result.getSort();
|
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,
|
planner_context,
|
||||||
query_processing_info,
|
query_processing_info,
|
||||||
query_tree,
|
query_tree,
|
||||||
result_actions_to_execute);
|
useful_sets);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (query_processing_info.isSecondStage() || query_processing_info.isFromAggregationState())
|
if (query_processing_info.isSecondStage() || query_processing_info.isFromAggregationState())
|
||||||
@ -1656,14 +1623,14 @@ void Planner::buildPlanForQueryNode()
|
|||||||
|
|
||||||
if (query_node.isGroupByWithTotals())
|
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;
|
having_executed = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
addCubeOrRollupStepIfNeeded(query_plan, aggregation_analysis_result, query_analysis_result, planner_context, select_query_info, query_node);
|
addCubeOrRollupStepIfNeeded(query_plan, aggregation_analysis_result, query_analysis_result, planner_context, select_query_info, query_node);
|
||||||
|
|
||||||
if (!having_executed && expression_analysis_result.hasHaving())
|
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())
|
if (query_processing_info.isFromAggregationState())
|
||||||
@ -1678,16 +1645,16 @@ void Planner::buildPlanForQueryNode()
|
|||||||
{
|
{
|
||||||
const auto & window_analysis_result = expression_analysis_result.getWindow();
|
const auto & window_analysis_result = expression_analysis_result.getWindow();
|
||||||
if (expression_analysis_result.hasAggregation())
|
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);
|
addWindowSteps(query_plan, planner_context, window_analysis_result);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (expression_analysis_result.hasQualify())
|
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();
|
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())
|
if (query_node.isDistinct())
|
||||||
{
|
{
|
||||||
@ -1703,7 +1670,7 @@ void Planner::buildPlanForQueryNode()
|
|||||||
if (expression_analysis_result.hasSort())
|
if (expression_analysis_result.hasSort())
|
||||||
{
|
{
|
||||||
const auto & sort_analysis_result = expression_analysis_result.getSort();
|
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
|
else
|
||||||
@ -1756,7 +1723,7 @@ void Planner::buildPlanForQueryNode()
|
|||||||
if (!query_processing_info.isFromAggregationState() && expression_analysis_result.hasLimitBy())
|
if (!query_processing_info.isFromAggregationState() && expression_analysis_result.hasLimitBy())
|
||||||
{
|
{
|
||||||
const auto & limit_by_analysis_result = expression_analysis_result.getLimitBy();
|
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);
|
addLimitByStep(query_plan, limit_by_analysis_result, query_node);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1788,7 +1755,7 @@ void Planner::buildPlanForQueryNode()
|
|||||||
if (!query_processing_info.isToAggregationState())
|
if (!query_processing_info.isToAggregationState())
|
||||||
{
|
{
|
||||||
const auto & projection_analysis_result = expression_analysis_result.getProjection();
|
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
|
// For additional_result_filter setting
|
||||||
@ -1796,7 +1763,7 @@ void Planner::buildPlanForQueryNode()
|
|||||||
}
|
}
|
||||||
|
|
||||||
if (!select_query_options.only_analyze)
|
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();
|
query_node_to_plan_step_mapping[&query_node] = query_plan.getRootNode();
|
||||||
}
|
}
|
||||||
|
@ -1181,13 +1181,13 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
|
|||||||
join_clauses_and_actions.left_join_expressions_actions->appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header);
|
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);
|
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_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));
|
left_plan.addStep(std::move(left_join_expressions_actions_step));
|
||||||
|
|
||||||
join_clauses_and_actions.right_join_expressions_actions->appendInputsForUnusedColumns(right_plan.getCurrentDataStream().header);
|
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);
|
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_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));
|
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>(
|
mixed_join_expression = std::make_shared<ExpressionActions>(
|
||||||
std::move(join_clauses_and_actions.mixed_join_expressions_actions),
|
std::move(join_clauses_and_actions.mixed_join_expressions_actions),
|
||||||
ExpressionActionsSettings::fromContext(planner_context->getQueryContext()));
|
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())
|
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)
|
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);
|
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)
|
if (!is_filled_join)
|
||||||
for (const auto * action_dag : right_join_tree_query_plan.actions_dags)
|
for (const auto & useful_set : right_join_tree_query_plan.useful_sets)
|
||||||
left_join_tree_query_plan.actions_dags.emplace_back(action_dag);
|
left_join_tree_query_plan.useful_sets.insert(useful_set);
|
||||||
// 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());
|
|
||||||
|
|
||||||
auto mapping = std::move(left_join_tree_query_plan.query_node_to_plan_step_mapping);
|
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;
|
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),
|
.query_plan = std::move(result_plan),
|
||||||
.from_stage = QueryProcessingStage::FetchColumns,
|
.from_stage = QueryProcessingStage::FetchColumns,
|
||||||
.used_row_policies = std::move(left_join_tree_query_plan.used_row_policies),
|
.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),
|
.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));
|
auto array_join_actions = std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), std::move(array_join_action_dag));
|
||||||
array_join_actions->setStepDescription("ARRAY JOIN actions");
|
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));
|
plan.addStep(std::move(array_join_actions));
|
||||||
|
|
||||||
auto drop_unused_columns_before_array_join_actions_dag = std::make_unique<ActionsDAG>(plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
|
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),
|
.query_plan = std::move(plan),
|
||||||
.from_stage = QueryProcessingStage::FetchColumns,
|
.from_stage = QueryProcessingStage::FetchColumns,
|
||||||
.used_row_policies = std::move(join_tree_query_plan.used_row_policies),
|
.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),
|
.query_node_to_plan_step_mapping = std::move(join_tree_query_plan.query_node_to_plan_step_mapping),
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
@ -11,12 +11,14 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
using UsefulSets = std::unordered_set<FutureSetPtr>;
|
||||||
|
|
||||||
struct JoinTreeQueryPlan
|
struct JoinTreeQueryPlan
|
||||||
{
|
{
|
||||||
QueryPlan query_plan;
|
QueryPlan query_plan;
|
||||||
QueryProcessingStage::Enum from_stage;
|
QueryProcessingStage::Enum from_stage;
|
||||||
std::set<std::string> used_row_policies{};
|
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{};
|
std::unordered_map<const QueryNode *, const QueryPlan::Node *> query_node_to_plan_step_mapping{};
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -11,10 +11,12 @@
|
|||||||
#include <DataTypes/DataTypeNullable.h>
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
|
|
||||||
#include <Columns/getLeastSuperColumn.h>
|
#include <Columns/getLeastSuperColumn.h>
|
||||||
|
#include <Columns/ColumnSet.h>
|
||||||
|
|
||||||
#include <IO/WriteBufferFromString.h>
|
#include <IO/WriteBufferFromString.h>
|
||||||
|
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
#include <Functions/indexHint.h>
|
||||||
|
|
||||||
#include <Storages/StorageDummy.h>
|
#include <Storages/StorageDummy.h>
|
||||||
|
|
||||||
@ -475,4 +477,32 @@ ASTPtr parseAdditionalResultFilter(const Settings & settings)
|
|||||||
return additional_result_filter_ast;
|
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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -88,4 +88,7 @@ FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree,
|
|||||||
|
|
||||||
ASTPtr parseAdditionalResultFilter(const Settings & settings);
|
ASTPtr parseAdditionalResultFilter(const Settings & settings);
|
||||||
|
|
||||||
|
using UsefulSets = std::unordered_set<FutureSetPtr>;
|
||||||
|
void appendSetsFromActionsDAG(const ActionsDAG & dag, UsefulSets & useful_sets);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user