mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #59595 from ClickHouse/vdimir/analyzer/comute_alias_columns
Analyzer: compute ALIAS columns right after reading
This commit is contained in:
commit
5dd81f38fb
@ -6651,7 +6651,6 @@ void QueryAnalyzer::initializeTableExpressionData(const QueryTreeNodePtr & table
|
||||
if (column_default && column_default->kind == ColumnDefaultKind::Alias)
|
||||
{
|
||||
auto alias_expression = buildQueryTree(column_default->expression, scope.context);
|
||||
alias_expression = buildCastFunction(alias_expression, column_name_and_type.type, scope.context, false /*resolve*/);
|
||||
auto column_node = std::make_shared<ColumnNode>(column_name_and_type, std::move(alias_expression), table_expression_node);
|
||||
column_name_to_column_node.emplace(column_name_and_type.name, column_node);
|
||||
alias_columns_to_resolve.emplace_back(column_name_and_type.name, column_node);
|
||||
@ -6684,7 +6683,9 @@ void QueryAnalyzer::initializeTableExpressionData(const QueryTreeNodePtr & table
|
||||
alias_column_resolve_scope,
|
||||
false /*allow_lambda_expression*/,
|
||||
false /*allow_table_expression*/);
|
||||
|
||||
auto & resolved_expression = alias_column_to_resolve->getExpression();
|
||||
if (!resolved_expression->getResultType()->equals(*alias_column_to_resolve->getResultType()))
|
||||
resolved_expression = buildCastFunction(resolved_expression, alias_column_to_resolve->getResultType(), scope.context, true);
|
||||
column_name_to_column_node = std::move(alias_column_resolve_scope.column_name_to_column_node);
|
||||
column_name_to_column_node[alias_column_to_resolve_name] = alias_column_to_resolve;
|
||||
}
|
||||
|
@ -121,7 +121,12 @@ Block getHeaderForProcessingStage(
|
||||
|
||||
auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(left_table_expression);
|
||||
const auto & query_context = query_info.planner_context->getQueryContext();
|
||||
auto columns = table_expression_data.getColumns();
|
||||
|
||||
NamesAndTypes columns;
|
||||
const auto & column_name_to_column = table_expression_data.getColumnNameToColumn();
|
||||
for (const auto & column_name : table_expression_data.getSelectedColumnsNames())
|
||||
columns.push_back(column_name_to_column.at(column_name));
|
||||
|
||||
auto new_query_node = buildSubqueryToReadColumnsFromTableExpression(columns, left_table_expression, query_context);
|
||||
query = new_query_node->toAST();
|
||||
}
|
||||
|
@ -29,34 +29,13 @@ namespace
|
||||
class CollectSourceColumnsVisitor : public InDepthQueryTreeVisitor<CollectSourceColumnsVisitor>
|
||||
{
|
||||
public:
|
||||
explicit CollectSourceColumnsVisitor(PlannerContext & planner_context_)
|
||||
explicit CollectSourceColumnsVisitor(PlannerContextPtr & planner_context_, bool keep_alias_columns_ = true)
|
||||
: planner_context(planner_context_)
|
||||
, keep_alias_columns(keep_alias_columns_)
|
||||
{}
|
||||
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
/// Special case for USING clause which contains references to ALIAS columns.
|
||||
/// We can not modify such ColumnNode.
|
||||
if (auto * join_node = node->as<JoinNode>())
|
||||
{
|
||||
if (!join_node->isUsingJoinExpression())
|
||||
return;
|
||||
|
||||
auto & using_list = join_node->getJoinExpression()->as<ListNode&>();
|
||||
for (auto & using_element : using_list)
|
||||
{
|
||||
auto & column_node = using_element->as<ColumnNode&>();
|
||||
/// This list contains column nodes from left and right tables.
|
||||
auto & columns_from_subtrees = column_node.getExpressionOrThrow()->as<ListNode&>().getNodes();
|
||||
|
||||
/// Visit left table column node.
|
||||
visitUsingColumn(columns_from_subtrees[0]);
|
||||
/// Visit right table column node.
|
||||
visitUsingColumn(columns_from_subtrees[1]);
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
auto * column_node = node->as<ColumnNode>();
|
||||
if (!column_node)
|
||||
return;
|
||||
@ -72,22 +51,55 @@ public:
|
||||
|
||||
/// JOIN using expression
|
||||
if (column_node->hasExpression() && column_source_node_type == QueryTreeNodeType::JOIN)
|
||||
return;
|
||||
|
||||
auto & table_expression_data = planner_context.getOrCreateTableExpressionData(column_source_node);
|
||||
|
||||
if (column_node->hasExpression() && column_source_node_type != QueryTreeNodeType::ARRAY_JOIN)
|
||||
{
|
||||
/// Replace ALIAS column with expression
|
||||
auto & columns_from_subtrees = column_node->getExpression()->as<ListNode &>().getNodes();
|
||||
if (columns_from_subtrees.size() != 2)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Expected two columns in JOIN using expression for column {}", column_node->dumpTree());
|
||||
|
||||
visit(columns_from_subtrees[0]);
|
||||
visit(columns_from_subtrees[1]);
|
||||
return;
|
||||
}
|
||||
|
||||
auto & table_expression_data = planner_context->getOrCreateTableExpressionData(column_source_node);
|
||||
|
||||
if (isAliasColumn(node))
|
||||
{
|
||||
/// Column is an ALIAS column with expression
|
||||
bool column_already_exists = table_expression_data.hasColumn(column_node->getColumnName());
|
||||
if (!column_already_exists)
|
||||
{
|
||||
auto column_identifier = planner_context.getGlobalPlannerContext()->createColumnIdentifier(node);
|
||||
table_expression_data.addAliasColumnName(column_node->getColumnName(), column_identifier);
|
||||
CollectSourceColumnsVisitor visitor_for_alias_column(planner_context);
|
||||
/// While we are processing expression of ALIAS columns we should not add source columns to selected.
|
||||
/// See also comment for `select_added_columns`
|
||||
visitor_for_alias_column.select_added_columns = false;
|
||||
visitor_for_alias_column.keep_alias_columns = keep_alias_columns;
|
||||
visitor_for_alias_column.visit(column_node->getExpression());
|
||||
|
||||
if (!keep_alias_columns)
|
||||
{
|
||||
/// For PREWHERE we can just replace ALIAS column with it's expression,
|
||||
/// because ActionsDAG for PREWHERE applied right on top of table expression
|
||||
/// and cannot affect subqueries or other table expressions.
|
||||
node = column_node->getExpression();
|
||||
return;
|
||||
}
|
||||
|
||||
auto column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(node);
|
||||
|
||||
ActionsDAGPtr alias_column_actions_dag = std::make_shared<ActionsDAG>();
|
||||
PlannerActionsVisitor actions_visitor(planner_context, false);
|
||||
auto outputs = actions_visitor.visit(alias_column_actions_dag, column_node->getExpression());
|
||||
if (outputs.size() != 1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Expected single output in actions dag for alias column {}. Actual {}", column_node->dumpTree(), outputs.size());
|
||||
const auto & column_name = column_node->getColumnName();
|
||||
const auto & alias_node = alias_column_actions_dag->addAlias(*outputs[0], column_name);
|
||||
alias_column_actions_dag->addOrReplaceInOutputs(alias_node);
|
||||
table_expression_data.addAliasColumn(column_node->getColumn(), column_identifier, alias_column_actions_dag, select_added_columns);
|
||||
}
|
||||
|
||||
node = column_node->getExpression();
|
||||
visitImpl(node);
|
||||
return;
|
||||
}
|
||||
|
||||
@ -102,45 +114,58 @@ public:
|
||||
|
||||
bool column_already_exists = table_expression_data.hasColumn(column_node->getColumnName());
|
||||
if (column_already_exists)
|
||||
{
|
||||
/// Column may be added when we collected data for ALIAS column
|
||||
/// But now we see it directly in the query, so make sure it's marked as selected
|
||||
if (select_added_columns)
|
||||
table_expression_data.markSelectedColumn(column_node->getColumnName());
|
||||
return;
|
||||
}
|
||||
|
||||
auto column_identifier = planner_context.getGlobalPlannerContext()->createColumnIdentifier(node);
|
||||
table_expression_data.addColumn(column_node->getColumn(), column_identifier);
|
||||
auto column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(node);
|
||||
table_expression_data.addColumn(column_node->getColumn(), column_identifier, select_added_columns);
|
||||
}
|
||||
|
||||
static bool needChildVisit(const QueryTreeNodePtr & parent, const QueryTreeNodePtr & child_node)
|
||||
static bool isAliasColumn(const QueryTreeNodePtr & node)
|
||||
{
|
||||
const auto * column_node = node->as<ColumnNode>();
|
||||
if (!column_node || !column_node->hasExpression())
|
||||
return false;
|
||||
const auto & column_source = column_node->getColumnSourceOrNull();
|
||||
if (!column_source)
|
||||
return false;
|
||||
return column_source->getNodeType() != QueryTreeNodeType::JOIN &&
|
||||
column_source->getNodeType() != QueryTreeNodeType::ARRAY_JOIN;
|
||||
}
|
||||
|
||||
static bool needChildVisit(const QueryTreeNodePtr & parent_node, const QueryTreeNodePtr & child_node)
|
||||
{
|
||||
if (auto * join_node = parent->as<JoinNode>())
|
||||
{
|
||||
if (join_node->getJoinExpression() == child_node && join_node->isUsingJoinExpression())
|
||||
return false;
|
||||
}
|
||||
auto child_node_type = child_node->getNodeType();
|
||||
return !(child_node_type == QueryTreeNodeType::QUERY || child_node_type == QueryTreeNodeType::UNION);
|
||||
return !(child_node_type == QueryTreeNodeType::QUERY ||
|
||||
child_node_type == QueryTreeNodeType::UNION ||
|
||||
isAliasColumn(parent_node));
|
||||
}
|
||||
|
||||
void setKeepAliasColumns(bool keep_alias_columns_)
|
||||
{
|
||||
keep_alias_columns = keep_alias_columns_;
|
||||
}
|
||||
|
||||
private:
|
||||
PlannerContextPtr & planner_context;
|
||||
|
||||
void visitUsingColumn(QueryTreeNodePtr & node)
|
||||
{
|
||||
auto & column_node = node->as<ColumnNode&>();
|
||||
if (column_node.hasExpression())
|
||||
{
|
||||
auto & table_expression_data = planner_context.getOrCreateTableExpressionData(column_node.getColumnSource());
|
||||
bool column_already_exists = table_expression_data.hasColumn(column_node.getColumnName());
|
||||
if (column_already_exists)
|
||||
return;
|
||||
/// Replace ALIAS columns with their expressions or register them in table expression data.
|
||||
/// Usually we can replace them when we build some "local" actions DAG
|
||||
/// (for example Row Policy or PREWHERE) that is applied on top of the table expression.
|
||||
/// In other cases, we keep ALIAS columns as ColumnNode with an expression child node,
|
||||
/// and handle them in the Planner by inserting ActionsDAG to compute them after reading from storage.
|
||||
bool keep_alias_columns = true;
|
||||
|
||||
auto column_identifier = planner_context.getGlobalPlannerContext()->createColumnIdentifier(node);
|
||||
table_expression_data.addAliasColumnName(column_node.getColumnName(), column_identifier);
|
||||
|
||||
visitImpl(column_node.getExpressionOrThrow());
|
||||
}
|
||||
else
|
||||
visitImpl(node);
|
||||
}
|
||||
|
||||
PlannerContext & planner_context;
|
||||
/// Flag `select_added_columns` indicates if we should mark column as explicitly selected.
|
||||
/// For example, for table with columns (a Int32, b ALIAS a+1) and query SELECT b FROM table
|
||||
/// Column `b` is selected explicitly by user, but not `a` (that is also read though).
|
||||
/// Distinguishing such columns is important for checking access rights for ALIAS columns.
|
||||
bool select_added_columns = true;
|
||||
};
|
||||
|
||||
class CollectPrewhereTableExpressionVisitor : public ConstInDepthQueryTreeVisitor<CollectPrewhereTableExpressionVisitor>
|
||||
@ -274,7 +299,7 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr
|
||||
}
|
||||
}
|
||||
|
||||
CollectSourceColumnsVisitor collect_source_columns_visitor(*planner_context);
|
||||
CollectSourceColumnsVisitor collect_source_columns_visitor(planner_context);
|
||||
for (auto & node : query_node_typed.getChildren())
|
||||
{
|
||||
if (!node || node == query_node_typed.getPrewhere())
|
||||
@ -300,21 +325,26 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr
|
||||
}
|
||||
|
||||
auto & table_expression_data = planner_context->getOrCreateTableExpressionData(prewhere_table_expression);
|
||||
const auto & column_names = table_expression_data.getColumnNames();
|
||||
NameSet required_column_names_without_prewhere(column_names.begin(), column_names.end());
|
||||
const auto & read_column_names = table_expression_data.getColumnNames();
|
||||
NameSet required_column_names_without_prewhere(read_column_names.begin(), read_column_names.end());
|
||||
const auto & selected_column_names = table_expression_data.getSelectedColumnsNames();
|
||||
required_column_names_without_prewhere.insert(selected_column_names.begin(), selected_column_names.end());
|
||||
|
||||
collect_source_columns_visitor.setKeepAliasColumns(false);
|
||||
collect_source_columns_visitor.visit(query_node_typed.getPrewhere());
|
||||
|
||||
auto prewhere_actions_dag = std::make_shared<ActionsDAG>();
|
||||
|
||||
QueryTreeNodePtr query_tree_node = query_node_typed.getPrewhere();
|
||||
|
||||
PlannerActionsVisitor visitor(planner_context, false /*use_column_identifier_as_action_node_name*/);
|
||||
auto expression_nodes = visitor.visit(prewhere_actions_dag, query_node_typed.getPrewhere());
|
||||
auto expression_nodes = visitor.visit(prewhere_actions_dag, query_tree_node);
|
||||
if (expression_nodes.size() != 1)
|
||||
throw Exception(ErrorCodes::ILLEGAL_PREWHERE,
|
||||
"Invalid PREWHERE. Expected single boolean expression. In query {}",
|
||||
query_node->formatASTForErrorMessage());
|
||||
|
||||
prewhere_actions_dag->getOutputs().push_back(expression_nodes[0]);
|
||||
prewhere_actions_dag->getOutputs().push_back(expression_nodes.back());
|
||||
|
||||
for (const auto & prewhere_input_node : prewhere_actions_dag->getInputs())
|
||||
if (required_column_names_without_prewhere.contains(prewhere_input_node->result_name))
|
||||
@ -324,9 +354,9 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr
|
||||
}
|
||||
}
|
||||
|
||||
void collectSourceColumns(QueryTreeNodePtr & expression_node, PlannerContextPtr & planner_context)
|
||||
void collectSourceColumns(QueryTreeNodePtr & expression_node, PlannerContextPtr & planner_context, bool keep_alias_columns)
|
||||
{
|
||||
CollectSourceColumnsVisitor collect_source_columns_visitor(*planner_context);
|
||||
CollectSourceColumnsVisitor collect_source_columns_visitor(planner_context, keep_alias_columns);
|
||||
collect_source_columns_visitor.visit(expression_node);
|
||||
}
|
||||
|
||||
|
@ -19,6 +19,6 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr
|
||||
*
|
||||
* ALIAS table column nodes are registered in table expression data and replaced in query tree with inner alias expression.
|
||||
*/
|
||||
void collectSourceColumns(QueryTreeNodePtr & expression_node, PlannerContextPtr & planner_context);
|
||||
void collectSourceColumns(QueryTreeNodePtr & expression_node, PlannerContextPtr & planner_context, bool keep_alias_columns = true);
|
||||
|
||||
}
|
||||
|
@ -451,6 +451,7 @@ private:
|
||||
std::unordered_map<QueryTreeNodePtr, std::string> node_to_node_name;
|
||||
const PlannerContextPtr planner_context;
|
||||
ActionNodeNameHelper action_node_name_helper;
|
||||
bool use_column_identifier_as_action_node_name;
|
||||
};
|
||||
|
||||
PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(ActionsDAGPtr actions_dag,
|
||||
@ -458,6 +459,7 @@ PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(ActionsDAGPtr actions_dag,
|
||||
bool use_column_identifier_as_action_node_name_)
|
||||
: planner_context(planner_context_)
|
||||
, action_node_name_helper(node_to_node_name, *planner_context, use_column_identifier_as_action_node_name_)
|
||||
, use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_)
|
||||
{
|
||||
actions_stack.emplace_back(std::move(actions_dag), nullptr);
|
||||
}
|
||||
@ -503,7 +505,8 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
|
||||
{
|
||||
auto column_node_name = action_node_name_helper.calculateActionNodeName(node);
|
||||
const auto & column_node = node->as<ColumnNode &>();
|
||||
|
||||
if (column_node.hasExpression() && !use_column_identifier_as_action_node_name)
|
||||
return visitImpl(column_node.getExpression());
|
||||
Int64 actions_stack_size = static_cast<Int64>(actions_stack.size() - 1);
|
||||
for (Int64 i = actions_stack_size; i >= 0; --i)
|
||||
{
|
||||
|
@ -86,7 +86,7 @@ namespace
|
||||
/// Check if current user has privileges to SELECT columns from table
|
||||
/// Throws an exception if access to any column from `column_names` is not granted
|
||||
/// If `column_names` is empty, check access to any columns and return names of accessible columns
|
||||
NameSet checkAccessRights(const TableNode & table_node, Names & column_names, const ContextPtr & query_context)
|
||||
NameSet checkAccessRights(const TableNode & table_node, const Names & column_names, const ContextPtr & query_context)
|
||||
{
|
||||
/// StorageDummy is created on preliminary stage, ignore access check for it.
|
||||
if (typeid_cast<const StorageDummy *>(table_node.getStorage().get()))
|
||||
@ -353,9 +353,7 @@ void prepareBuildQueryPlanForTableExpression(const QueryTreeNodePtr & table_expr
|
||||
NameSet columns_names_allowed_to_select;
|
||||
if (table_node)
|
||||
{
|
||||
auto column_names_with_aliases = columns_names;
|
||||
const auto & alias_columns_names = table_expression_data.getAliasColumnsNames();
|
||||
column_names_with_aliases.insert(column_names_with_aliases.end(), alias_columns_names.begin(), alias_columns_names.end());
|
||||
const auto & column_names_with_aliases = table_expression_data.getSelectedColumnsNames();
|
||||
columns_names_allowed_to_select = checkAccessRights(*table_node, column_names_with_aliases, query_context);
|
||||
}
|
||||
|
||||
@ -864,6 +862,28 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
max_block_size,
|
||||
max_streams);
|
||||
|
||||
const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions();
|
||||
if (!alias_column_expressions.empty() && query_plan.isInitialized() && from_stage == QueryProcessingStage::FetchColumns)
|
||||
{
|
||||
ActionsDAGPtr merged_alias_columns_actions_dag = std::make_shared<ActionsDAG>(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
|
||||
ActionsDAG::NodeRawConstPtrs action_dag_outputs = merged_alias_columns_actions_dag->getInputs();
|
||||
|
||||
for (const auto & [column_name, alias_column_actions_dag] : alias_column_expressions)
|
||||
{
|
||||
const auto & current_outputs = alias_column_actions_dag->getOutputs();
|
||||
action_dag_outputs.insert(action_dag_outputs.end(), current_outputs.begin(), current_outputs.end());
|
||||
merged_alias_columns_actions_dag->mergeNodes(std::move(*alias_column_actions_dag));
|
||||
}
|
||||
|
||||
for (const auto * output_node : action_dag_outputs)
|
||||
merged_alias_columns_actions_dag->addOrReplaceInOutputs(*output_node);
|
||||
merged_alias_columns_actions_dag->removeUnusedActions(false);
|
||||
|
||||
auto alias_column_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(merged_alias_columns_actions_dag));
|
||||
alias_column_step->setStepDescription("Compute alias columns");
|
||||
query_plan.addStep(std::move(alias_column_step));
|
||||
}
|
||||
|
||||
for (const auto & filter_info_and_description : where_filters)
|
||||
{
|
||||
const auto & [filter_info, description] = filter_info_and_description;
|
||||
@ -907,7 +927,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
else
|
||||
{
|
||||
/// Create step which reads from empty source if storage has no data.
|
||||
auto source_header = storage_snapshot->getSampleBlockForColumns(table_expression_data.getColumnNames());
|
||||
const auto & column_names = table_expression_data.getSelectedColumnsNames();
|
||||
auto source_header = storage_snapshot->getSampleBlockForColumns(column_names);
|
||||
Pipe pipe(std::make_shared<NullSource>(source_header));
|
||||
auto read_from_pipe = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
|
||||
read_from_pipe->setStepDescription("Read from NullSource");
|
||||
@ -1024,57 +1045,6 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP
|
||||
plan_to_add_cast.addStep(std::move(cast_join_columns_step));
|
||||
}
|
||||
|
||||
/// Actions to calculate table columns that have a functional representation (ALIASes and subcolumns)
|
||||
/// and used in USING clause of JOIN expression.
|
||||
struct UsingAliasKeyActions
|
||||
{
|
||||
UsingAliasKeyActions(
|
||||
const ColumnsWithTypeAndName & left_plan_output_columns,
|
||||
const ColumnsWithTypeAndName & right_plan_output_columns
|
||||
)
|
||||
: left_alias_columns_keys(std::make_shared<ActionsDAG>(left_plan_output_columns))
|
||||
, right_alias_columns_keys(std::make_shared<ActionsDAG>(right_plan_output_columns))
|
||||
{}
|
||||
|
||||
void addLeftColumn(QueryTreeNodePtr & node, const ColumnsWithTypeAndName & plan_output_columns, const PlannerContextPtr & planner_context)
|
||||
{
|
||||
addColumnImpl(left_alias_columns_keys, node, plan_output_columns, planner_context);
|
||||
}
|
||||
|
||||
void addRightColumn(QueryTreeNodePtr & node, const ColumnsWithTypeAndName & plan_output_columns, const PlannerContextPtr & planner_context)
|
||||
{
|
||||
addColumnImpl(right_alias_columns_keys, node, plan_output_columns, planner_context);
|
||||
}
|
||||
|
||||
ActionsDAGPtr getLeftActions()
|
||||
{
|
||||
left_alias_columns_keys->projectInput();
|
||||
return std::move(left_alias_columns_keys);
|
||||
}
|
||||
|
||||
ActionsDAGPtr getRightActions()
|
||||
{
|
||||
right_alias_columns_keys->projectInput();
|
||||
return std::move(right_alias_columns_keys);
|
||||
}
|
||||
|
||||
private:
|
||||
void addColumnImpl(ActionsDAGPtr & alias_columns_keys, QueryTreeNodePtr & node, const ColumnsWithTypeAndName & plan_output_columns, const PlannerContextPtr & planner_context)
|
||||
{
|
||||
auto & column_node = node->as<ColumnNode&>();
|
||||
if (column_node.hasExpression())
|
||||
{
|
||||
auto dag = buildActionsDAGFromExpressionNode(column_node.getExpressionOrThrow(), plan_output_columns, planner_context);
|
||||
const auto & left_inner_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(node);
|
||||
dag->addOrReplaceInOutputs(dag->addAlias(*dag->getOutputs().front(), left_inner_column_identifier));
|
||||
alias_columns_keys->mergeInplace(std::move(*dag));
|
||||
}
|
||||
}
|
||||
|
||||
ActionsDAGPtr left_alias_columns_keys;
|
||||
ActionsDAGPtr right_alias_columns_keys;
|
||||
};
|
||||
|
||||
JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_expression,
|
||||
JoinTreeQueryPlan left_join_tree_query_plan,
|
||||
JoinTreeQueryPlan right_join_tree_query_plan,
|
||||
@ -1143,8 +1113,6 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
|
||||
|
||||
if (join_node.isUsingJoinExpression())
|
||||
{
|
||||
UsingAliasKeyActions using_alias_key_actions{left_plan_output_columns, right_plan_output_columns};
|
||||
|
||||
auto & join_node_using_columns_list = join_node.getJoinExpression()->as<ListNode &>();
|
||||
for (auto & join_node_using_node : join_node_using_columns_list.getNodes())
|
||||
{
|
||||
@ -1154,13 +1122,9 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
|
||||
auto & left_inner_column_node = inner_columns_list.getNodes().at(0);
|
||||
auto & left_inner_column = left_inner_column_node->as<ColumnNode &>();
|
||||
|
||||
using_alias_key_actions.addLeftColumn(left_inner_column_node, left_plan_output_columns, planner_context);
|
||||
|
||||
auto & right_inner_column_node = inner_columns_list.getNodes().at(1);
|
||||
auto & right_inner_column = right_inner_column_node->as<ColumnNode &>();
|
||||
|
||||
using_alias_key_actions.addRightColumn(right_inner_column_node, right_plan_output_columns, planner_context);
|
||||
|
||||
const auto & join_node_using_column_node_type = join_node_using_column_node.getColumnType();
|
||||
if (!left_inner_column.getColumnType()->equals(*join_node_using_column_node_type))
|
||||
{
|
||||
@ -1174,14 +1138,6 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
|
||||
right_plan_column_name_to_cast_type.emplace(right_inner_column_identifier, join_node_using_column_node_type);
|
||||
}
|
||||
}
|
||||
|
||||
auto left_alias_columns_keys_step = std::make_unique<ExpressionStep>(left_plan.getCurrentDataStream(), using_alias_key_actions.getLeftActions());
|
||||
left_alias_columns_keys_step->setStepDescription("Actions for left table alias column keys");
|
||||
left_plan.addStep(std::move(left_alias_columns_keys_step));
|
||||
|
||||
auto right_alias_columns_keys_step = std::make_unique<ExpressionStep>(right_plan.getCurrentDataStream(), using_alias_key_actions.getRightActions());
|
||||
right_alias_columns_keys_step->setStepDescription("Actions for right table alias column keys");
|
||||
right_plan.addStep(std::move(right_alias_columns_keys_step));
|
||||
}
|
||||
|
||||
auto join_cast_plan_output_nodes = [&](QueryPlan & plan_to_add_cast, std::unordered_map<std::string, DataTypePtr> & plan_column_name_to_cast_type)
|
||||
|
@ -55,7 +55,7 @@ public:
|
||||
/// Return true if column with name exists, false otherwise
|
||||
bool hasColumn(const std::string & column_name) const
|
||||
{
|
||||
return alias_columns_names.contains(column_name) || column_name_to_column.contains(column_name);
|
||||
return column_name_to_column.contains(column_name);
|
||||
}
|
||||
|
||||
/** Add column in table expression data.
|
||||
@ -63,37 +63,40 @@ public:
|
||||
*
|
||||
* Logical error exception is thrown if column already exists.
|
||||
*/
|
||||
void addColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier)
|
||||
void addColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier, bool is_selected_column = true)
|
||||
{
|
||||
if (hasColumn(column.name))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Column with name {} already exists", column.name);
|
||||
|
||||
addColumnImpl(column, column_identifier);
|
||||
column_names.push_back(column.name);
|
||||
addColumnImpl(column, column_identifier, is_selected_column);
|
||||
}
|
||||
|
||||
/** Add column if it does not exists in table expression data.
|
||||
* Column identifier must be created using global planner context.
|
||||
*/
|
||||
void addColumnIfNotExists(const NameAndTypePair & column, const ColumnIdentifier & column_identifier)
|
||||
/// Add alias column
|
||||
void addAliasColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier, ActionsDAGPtr actions_dag, bool is_selected_column = true)
|
||||
{
|
||||
if (hasColumn(column.name))
|
||||
return;
|
||||
|
||||
addColumnImpl(column, column_identifier);
|
||||
alias_column_expressions.emplace(column.name, std::move(actions_dag));
|
||||
addColumnImpl(column, column_identifier, is_selected_column);
|
||||
}
|
||||
|
||||
/// Add alias column name
|
||||
void addAliasColumnName(const std::string & column_name, const ColumnIdentifier & column_identifier)
|
||||
/// Mark existing column as selected
|
||||
void markSelectedColumn(const std::string & column_name)
|
||||
{
|
||||
alias_columns_names.insert(column_name);
|
||||
|
||||
column_name_to_column_identifier.emplace(column_name, column_identifier);
|
||||
auto [_, inserted] = selected_column_names_set.emplace(column_name);
|
||||
if (inserted)
|
||||
selected_column_names.push_back(column_name);
|
||||
}
|
||||
|
||||
/// Get alias columns names
|
||||
const NameSet & getAliasColumnsNames() const
|
||||
/// Get columns that are requested from table expression, including ALIAS columns
|
||||
const Names & getSelectedColumnsNames() const
|
||||
{
|
||||
return alias_columns_names;
|
||||
return selected_column_names;
|
||||
}
|
||||
|
||||
/// Get ALIAS columns names mapped to expressions
|
||||
const std::unordered_map<std::string, ActionsDAGPtr> & getAliasColumnExpressions() const
|
||||
{
|
||||
return alias_column_expressions;
|
||||
}
|
||||
|
||||
/// Get column name to column map
|
||||
@ -102,7 +105,7 @@ public:
|
||||
return column_name_to_column;
|
||||
}
|
||||
|
||||
/// Get column names
|
||||
/// Get column names that are read from table expression
|
||||
const Names & getColumnNames() const
|
||||
{
|
||||
return column_names;
|
||||
@ -119,23 +122,6 @@ public:
|
||||
return result;
|
||||
}
|
||||
|
||||
ColumnIdentifiers getColumnIdentifiers() const
|
||||
{
|
||||
ColumnIdentifiers result;
|
||||
result.reserve(column_identifier_to_column_name.size());
|
||||
|
||||
for (const auto & [column_identifier, _] : column_identifier_to_column_name)
|
||||
result.push_back(column_identifier);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
/// Get column name to column identifier map
|
||||
const ColumnNameToColumnIdentifier & getColumnNameToIdentifier() const
|
||||
{
|
||||
return column_name_to_column_identifier;
|
||||
}
|
||||
|
||||
/// Get column identifier to column name map
|
||||
const ColumnNameToColumnIdentifier & getColumnIdentifierToColumnName() const
|
||||
{
|
||||
@ -159,18 +145,6 @@ public:
|
||||
return it->second;
|
||||
}
|
||||
|
||||
/** Get column for column name.
|
||||
* Null is returned if there are no column for column name.
|
||||
*/
|
||||
const NameAndTypePair * getColumnOrNull(const std::string & column_name) const
|
||||
{
|
||||
auto it = column_name_to_column.find(column_name);
|
||||
if (it == column_name_to_column.end())
|
||||
return nullptr;
|
||||
|
||||
return &it->second;
|
||||
}
|
||||
|
||||
/** Get column identifier for column name.
|
||||
* Exception is thrown if there are no column identifier for column name.
|
||||
*/
|
||||
@ -200,24 +174,6 @@ public:
|
||||
return &it->second;
|
||||
}
|
||||
|
||||
/** Get column name for column identifier.
|
||||
* Exception is thrown if there are no column name for column identifier.
|
||||
*/
|
||||
const std::string & getColumnNameOrThrow(const ColumnIdentifier & column_identifier) const
|
||||
{
|
||||
auto it = column_identifier_to_column_name.find(column_identifier);
|
||||
if (it == column_identifier_to_column_name.end())
|
||||
{
|
||||
auto column_identifiers = getColumnIdentifiers();
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Column name for column identifier {} does not exists. There are only column identifiers: {}",
|
||||
column_identifier,
|
||||
fmt::join(column_identifiers.begin(), column_identifiers.end(), ", "));
|
||||
}
|
||||
|
||||
return it->second;
|
||||
}
|
||||
|
||||
/** Get column name for column identifier.
|
||||
* Null is returned if there are no column name for column identifier.
|
||||
*/
|
||||
@ -296,23 +252,36 @@ public:
|
||||
}
|
||||
|
||||
private:
|
||||
void addColumnImpl(const NameAndTypePair & column, const ColumnIdentifier & column_identifier)
|
||||
void addColumnImpl(const NameAndTypePair & column, const ColumnIdentifier & column_identifier, bool add_to_selected_columns)
|
||||
{
|
||||
column_names.push_back(column.name);
|
||||
if (add_to_selected_columns)
|
||||
markSelectedColumn(column.name);
|
||||
|
||||
column_name_to_column.emplace(column.name, column);
|
||||
column_name_to_column_identifier.emplace(column.name, column_identifier);
|
||||
column_identifier_to_column_name.emplace(column_identifier, column.name);
|
||||
}
|
||||
|
||||
/// Valid for table, table function, array join, query, union nodes
|
||||
/// Set of columns that are physically read from table expression
|
||||
/// In case of ALIAS columns it contains source column names that are used to calculate alias
|
||||
/// This source column may be not used by user
|
||||
Names column_names;
|
||||
|
||||
/// Set of columns that are SELECTed from table expression
|
||||
/// It may contain ALIAS columns.
|
||||
/// Mainly it's used to determine access to which columns to check
|
||||
/// For example user may have an access to column `a ALIAS x + y` but not to `x` and `y`
|
||||
/// In that case we can read `x` and `y` and calculate `a`, but not return `x` and `y` to user
|
||||
Names selected_column_names;
|
||||
/// To deduplicate columns in `selected_column_names`
|
||||
NameSet selected_column_names_set;
|
||||
|
||||
/// Expression to calculate ALIAS columns
|
||||
std::unordered_map<std::string, ActionsDAGPtr> alias_column_expressions;
|
||||
|
||||
/// Valid for table, table function, array join, query, union nodes
|
||||
ColumnNameToColumn column_name_to_column;
|
||||
|
||||
/// Valid only for table node
|
||||
NameSet alias_columns_names;
|
||||
|
||||
/// Valid for table, table function, array join, query, union nodes
|
||||
ColumnNameToColumnIdentifier column_name_to_column_identifier;
|
||||
|
||||
|
@ -469,12 +469,19 @@ FilterDAGInfo buildFilterInfo(ASTPtr filter_expression,
|
||||
NameSet table_expression_required_names_without_filter)
|
||||
{
|
||||
const auto & query_context = planner_context->getQueryContext();
|
||||
|
||||
auto filter_query_tree = buildQueryTree(filter_expression, query_context);
|
||||
|
||||
QueryAnalysisPass query_analysis_pass(table_expression);
|
||||
query_analysis_pass.run(filter_query_tree, query_context);
|
||||
|
||||
return buildFilterInfo(std::move(filter_query_tree), table_expression, planner_context, std::move(table_expression_required_names_without_filter));
|
||||
}
|
||||
|
||||
FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree,
|
||||
const QueryTreeNodePtr & table_expression,
|
||||
PlannerContextPtr & planner_context,
|
||||
NameSet table_expression_required_names_without_filter)
|
||||
{
|
||||
if (table_expression_required_names_without_filter.empty())
|
||||
{
|
||||
auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression);
|
||||
@ -482,7 +489,7 @@ FilterDAGInfo buildFilterInfo(ASTPtr filter_expression,
|
||||
table_expression_required_names_without_filter.insert(table_expression_names.begin(), table_expression_names.end());
|
||||
}
|
||||
|
||||
collectSourceColumns(filter_query_tree, planner_context);
|
||||
collectSourceColumns(filter_query_tree, planner_context, false /*keep_alias_columns*/);
|
||||
collectSets(filter_query_tree, *planner_context);
|
||||
|
||||
auto filter_actions_dag = std::make_shared<ActionsDAG>();
|
||||
|
@ -89,6 +89,11 @@ FilterDAGInfo buildFilterInfo(ASTPtr filter_expression,
|
||||
PlannerContextPtr & planner_context,
|
||||
NameSet table_expression_required_names_without_filter = {});
|
||||
|
||||
FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree,
|
||||
const QueryTreeNodePtr & table_expression,
|
||||
PlannerContextPtr & planner_context,
|
||||
NameSet table_expression_required_names_without_filter = {});
|
||||
|
||||
ASTPtr parseAdditionalResultFilter(const Settings & settings);
|
||||
|
||||
}
|
||||
|
@ -1434,8 +1434,13 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes)
|
||||
if (query_info.planner_context)
|
||||
{
|
||||
const auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(query_info.table_expression);
|
||||
const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions();
|
||||
for (const auto & [column_identifier, column_name] : table_expression_data.getColumnIdentifierToColumnName())
|
||||
{
|
||||
/// ALIAS columns cannot be used in the filter expression without being calculated in ActionsDAG,
|
||||
/// so they should not be added to the input nodes.
|
||||
if (alias_column_expressions.contains(column_name))
|
||||
continue;
|
||||
const auto & column = table_expression_data.getColumnOrThrow(column_name);
|
||||
node_name_to_input_node_column.emplace(column_identifier, ColumnWithTypeAndName(column.type, column_name));
|
||||
}
|
||||
|
@ -744,6 +744,32 @@ StorageSnapshotPtr StorageDistributed::getStorageSnapshotForQuery(
|
||||
namespace
|
||||
{
|
||||
|
||||
class ReplaseAliasColumnsVisitor : public InDepthQueryTreeVisitor<ReplaseAliasColumnsVisitor>
|
||||
{
|
||||
static QueryTreeNodePtr getColumnNodeAliasExpression(const QueryTreeNodePtr & node)
|
||||
{
|
||||
const auto * column_node = node->as<ColumnNode>();
|
||||
if (!column_node || !column_node->hasExpression())
|
||||
return nullptr;
|
||||
|
||||
const auto & column_source = column_node->getColumnSourceOrNull();
|
||||
if (!column_source || column_source->getNodeType() == QueryTreeNodeType::JOIN
|
||||
|| column_source->getNodeType() == QueryTreeNodeType::ARRAY_JOIN)
|
||||
return nullptr;
|
||||
|
||||
auto column_expression = column_node->getExpression();
|
||||
column_expression->setAlias(column_node->getColumnName());
|
||||
return column_expression;
|
||||
}
|
||||
|
||||
public:
|
||||
void visitImpl(QueryTreeNodePtr & node)
|
||||
{
|
||||
if (auto column_expression = getColumnNodeAliasExpression(node))
|
||||
node = column_expression;
|
||||
}
|
||||
};
|
||||
|
||||
QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info,
|
||||
const StorageSnapshotPtr & distributed_storage_snapshot,
|
||||
const StorageID & remote_storage_id,
|
||||
@ -796,6 +822,8 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info,
|
||||
replacement_table_expression->setAlias(query_info.table_expression->getAlias());
|
||||
|
||||
auto query_tree_to_modify = query_info.query_tree->cloneAndReplace(query_info.table_expression, std::move(replacement_table_expression));
|
||||
ReplaseAliasColumnsVisitor replase_alias_columns_visitor;
|
||||
replase_alias_columns_visitor.visit(query_tree_to_modify);
|
||||
|
||||
return buildQueryTreeForShard(query_info.planner_context, query_tree_to_modify);
|
||||
}
|
||||
|
@ -4,4 +4,3 @@ test_distributed_type_object/test.py::test_distributed_type_object
|
||||
test_merge_table_over_distributed/test.py::test_global_in
|
||||
test_merge_table_over_distributed/test.py::test_select_table_name_from_merge_over_distributed
|
||||
test_passing_max_partitions_to_read_remotely/test.py::test_default_database_on_cluster
|
||||
test_select_access_rights/test_main.py::test_alias_columns
|
||||
|
@ -41,7 +41,7 @@ def started_cluster():
|
||||
CREATE TABLE mydb.filtered_table2 (a UInt8, b UInt8, c UInt8, d UInt8) ENGINE MergeTree ORDER BY a;
|
||||
INSERT INTO mydb.filtered_table2 values (0, 0, 0, 0), (1, 2, 3, 4), (4, 3, 2, 1), (0, 0, 6, 0);
|
||||
|
||||
CREATE TABLE mydb.filtered_table3 (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a;
|
||||
CREATE TABLE mydb.filtered_table3 (a UInt8, b UInt8, bb ALIAS b + 1, c UInt16 ALIAS a + bb - 1) ENGINE MergeTree ORDER BY a;
|
||||
INSERT INTO mydb.filtered_table3 values (0, 0), (0, 1), (1, 0), (1, 1);
|
||||
|
||||
CREATE TABLE mydb.`.filtered_table4` (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a;
|
||||
|
@ -60,7 +60,7 @@ def started_cluster():
|
||||
CREATE TABLE mydb.filtered_table2 (a UInt8, b UInt8, c UInt8, d UInt8) ENGINE MergeTree ORDER BY a;
|
||||
INSERT INTO mydb.filtered_table2 values (0, 0, 0, 0), (1, 2, 3, 4), (4, 3, 2, 1), (0, 0, 6, 0);
|
||||
|
||||
CREATE TABLE mydb.filtered_table3 (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a;
|
||||
CREATE TABLE mydb.filtered_table3 (a UInt8, b UInt8, bb ALIAS b + 1, c UInt16 ALIAS a + bb - 1) ENGINE MergeTree ORDER BY a;
|
||||
INSERT INTO mydb.filtered_table3 values (0, 0), (0, 1), (1, 0), (1, 1);
|
||||
|
||||
CREATE TABLE mydb.`.filtered_table4` (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a;
|
||||
@ -113,6 +113,7 @@ def test_smoke():
|
||||
|
||||
assert node.query("SELECT a FROM mydb.filtered_table3") == TSV([[0], [1]])
|
||||
assert node.query("SELECT b FROM mydb.filtered_table3") == TSV([[1], [0]])
|
||||
assert node.query("SELECT bb FROM mydb.filtered_table3") == TSV([[2], [1]])
|
||||
assert node.query("SELECT c FROM mydb.filtered_table3") == TSV([[1], [1]])
|
||||
assert node.query("SELECT a + b FROM mydb.filtered_table3") == TSV([[1], [1]])
|
||||
assert node.query("SELECT a FROM mydb.filtered_table3 WHERE c = 1") == TSV(
|
||||
|
@ -55,33 +55,33 @@ Header: a2 String
|
||||
Header: __table1.a2 String
|
||||
__table1.k UInt64
|
||||
__table4.d2 String
|
||||
Expression ((Actions for left table alias column keys + DROP unused columns after JOIN))
|
||||
Expression (DROP unused columns after JOIN)
|
||||
Header: __table1.a2 String
|
||||
__table1.k UInt64
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.a2 String
|
||||
__table1.k UInt64
|
||||
Expression ((Actions for left table alias column keys + DROP unused columns after JOIN))
|
||||
Expression (DROP unused columns after JOIN)
|
||||
Header: __table1.a2 String
|
||||
__table1.k UInt64
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: __table1.a2 String
|
||||
__table1.k UInt64
|
||||
Expression ((Actions for left table alias column keys + Change column names to column identifiers))
|
||||
Expression (Change column names to column identifiers)
|
||||
Header: __table1.a2 String
|
||||
__table1.k UInt64
|
||||
ReadFromMemoryStorage
|
||||
Header: a2 String
|
||||
k UInt64
|
||||
Expression ((Actions for right table alias column keys + Change column names to column identifiers))
|
||||
Expression (Change column names to column identifiers)
|
||||
Header: __table2.k UInt64
|
||||
ReadFromMemoryStorage
|
||||
Header: k UInt64
|
||||
Expression ((Actions for right table alias column keys + Change column names to column identifiers))
|
||||
Expression (Change column names to column identifiers)
|
||||
Header: __table3.k UInt64
|
||||
ReadFromMemoryStorage
|
||||
Header: k UInt64
|
||||
Expression ((Actions for right table alias column keys + Change column names to column identifiers))
|
||||
Expression (Change column names to column identifiers)
|
||||
Header: __table4.d2 String
|
||||
__table4.k UInt64
|
||||
ReadFromMemoryStorage
|
||||
|
@ -14,13 +14,13 @@ Expression ((Projection + Before ORDER BY))
|
||||
Parts: 1/1
|
||||
Granules: 1/1
|
||||
Expression ((Project names + Projection))
|
||||
Filter ((WHERE + Change column names to column identifiers))
|
||||
Filter ((WHERE + (Change column names to column identifiers + Compute alias columns)))
|
||||
ReadFromMergeTree (02911_support_alias_column_in_indices.test1)
|
||||
Indexes:
|
||||
PrimaryKey
|
||||
Keys:
|
||||
c
|
||||
Condition: (_CAST(plus(c, \'UInt64\'), 1) in [11, +Inf))
|
||||
Condition: (plus(c, 1) in [11, +Inf))
|
||||
Parts: 1/2
|
||||
Granules: 1/2
|
||||
Skip
|
||||
@ -44,12 +44,17 @@ Expression ((Projection + Before ORDER BY))
|
||||
Parts: 1/1
|
||||
Granules: 1/1
|
||||
Expression ((Project names + Projection))
|
||||
Filter ((WHERE + Change column names to column identifiers))
|
||||
Filter ((WHERE + (Change column names to column identifiers + Compute alias columns)))
|
||||
ReadFromMergeTree (02911_support_alias_column_in_indices.test2)
|
||||
Indexes:
|
||||
PrimaryKey
|
||||
Keys:
|
||||
c
|
||||
Condition: (_CAST(plus(_CAST(plus(c, \'UInt64\'), 1), \'UInt64\'), 1) in [16, +Inf))
|
||||
Condition: (plus(plus(c, 1), 1) in [16, +Inf))
|
||||
Parts: 1/2
|
||||
Granules: 1/2
|
||||
Skip
|
||||
Name: i
|
||||
Description: minmax GRANULARITY 1
|
||||
Parts: 1/1
|
||||
Granules: 1/1
|
||||
|
Loading…
Reference in New Issue
Block a user