Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 3 additions & 2 deletions src/Planner/PlannerJoinTree.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -654,13 +654,14 @@ UInt64 mainQueryNodeBlockSizeByLimit(const SelectQueryInfo & select_query_info)
}

std::unique_ptr<ExpressionStep> createComputeAliasColumnsStep(
std::unordered_map<std::string, ActionsDAG> & alias_column_expressions, const SharedHeader & current_header)
AliasColumnExpressions & alias_column_expressions, const SharedHeader & current_header)
{
ActionsDAG merged_alias_columns_actions_dag(current_header->getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs action_dag_outputs = merged_alias_columns_actions_dag.getInputs();

for (auto & [column_name, alias_column_actions_dag] : alias_column_expressions)
for (auto & alias_column_expression : alias_column_expressions)
{
auto & alias_column_actions_dag = alias_column_expression.second;
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));
Expand Down
22 changes: 19 additions & 3 deletions src/Planner/TableExpressionData.h
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@ namespace ErrorCodes
using ColumnIdentifier = std::string;
using ColumnIdentifiers = std::vector<ColumnIdentifier>;
using ColumnIdentifierSet = std::unordered_set<ColumnIdentifier>;
using AliasColumnExpression = std::pair<std::string, ActionsDAG>;
using AliasColumnExpressions = std::vector<AliasColumnExpression>;

struct PrewhereInfo;
using PrewhereInfoPtr = std::shared_ptr<PrewhereInfo>;
Expand Down Expand Up @@ -77,7 +79,8 @@ class TableExpressionData
/// Add alias column
void addAliasColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier, ActionsDAG actions_dag, bool is_selected_column = true)
{
alias_column_expressions.emplace(column.name, std::move(actions_dag));
alias_column_expressions.emplace_back(column.name, std::move(actions_dag));
alias_column_names_set.emplace(column.name);
addColumnImpl(column, column_identifier, is_selected_column);
}

Expand All @@ -96,11 +99,21 @@ class TableExpressionData
}

/// Get ALIAS columns names mapped to expressions
std::unordered_map<std::string, ActionsDAG> & getAliasColumnExpressions()
AliasColumnExpressions & getAliasColumnExpressions()
{
return alias_column_expressions;
}

const AliasColumnExpressions & getAliasColumnExpressions() const
{
return alias_column_expressions;
}

bool hasAliasColumn(const std::string & column_name) const
{
return alias_column_names_set.contains(column_name);
}

/// Get column name to column map
const ColumnNameToColumn & getColumnNameToColumn() const
{
Expand Down Expand Up @@ -279,7 +292,10 @@ class TableExpressionData
NameSet selected_column_names_set;

/// Expression to calculate ALIAS columns
std::unordered_map<std::string, ActionsDAG> alias_column_expressions;
/// Keep alias name (String) + expression (ActionsDAG) pairs; vector preserves insertion order.
AliasColumnExpressions alias_column_expressions;
/// Fast membership checks for alias column names.
NameSet alias_column_names_set;

/// Valid for table, table function, array join, query, union nodes
ColumnNameToColumn column_name_to_column;
Expand Down
3 changes: 1 addition & 2 deletions src/Storages/SelectQueryInfo.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -25,12 +25,11 @@ std::unordered_map<std::string, ColumnWithTypeAndName> SelectQueryInfo::buildNod
if (planner_context)
{
auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(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))
if (table_expression_data.hasAliasColumn(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));
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,2 @@
e f
e f
17 changes: 17 additions & 0 deletions tests/queries/0_stateless/03726_distributed_alias_column_order.sql
Original file line number Diff line number Diff line change
@@ -0,0 +1,17 @@
DROP TABLE IF EXISTS test_local;

CREATE TABLE test_local
(
`a` UInt64,
`e` String ALIAS 'e',
`f` String ALIAS 'f'
)
ENGINE = Memory;

INSERT INTO test_local VALUES (1);

SELECT e, f
FROM remote('127.0.0.{1,2}', currentDatabase(), test_local)
ORDER BY a;

DROP TABLE IF EXISTS test_local;
Loading