Merge pull request #29008 from vdimir/issue-26980-join-id-alias

Do not replaceAliasColumnsInQuery for JOIN OR/USING sections
This commit is contained in:
Kruglov Pavel 2021-09-15 16:27:46 +03:00 committed by GitHub
commit 8d1bf1b675
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 80 additions and 28 deletions

View File

@ -3,7 +3,6 @@
#include <Interpreters/RequiredSourceColumnsVisitor.h> #include <Interpreters/RequiredSourceColumnsVisitor.h>
#include <Interpreters/addTypeConversionToAST.h> #include <Interpreters/addTypeConversionToAST.h>
#include <Parsers/ASTTablesInSelectQuery.h> #include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSelectQuery.h> #include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSubquery.h> #include <Parsers/ASTSubquery.h>
#include <Parsers/ASTAlterQuery.h> #include <Parsers/ASTAlterQuery.h>
@ -14,8 +13,11 @@
namespace DB namespace DB
{ {
bool ColumnAliasesMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &) bool ColumnAliasesMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &, const Data & data)
{ {
if (data.excluded_nodes.contains(node.get()))
return false;
if (const auto * f = node->as<ASTFunction>()) if (const auto * f = node->as<ASTFunction>())
{ {
/// "lambda" visits children itself. /// "lambda" visits children itself.

View File

@ -46,7 +46,7 @@ using DataTypePtr = std::shared_ptr<const IDataType>;
class ColumnAliasesMatcher class ColumnAliasesMatcher
{ {
public: public:
using Visitor = InDepthNodeVisitor<ColumnAliasesMatcher, false>; using Visitor = InDepthNodeVisitor<ColumnAliasesMatcher, false, true>;
struct Data struct Data
{ {
@ -57,14 +57,16 @@ public:
NameSet array_join_source_columns; NameSet array_join_source_columns;
ContextPtr context; ContextPtr context;
const std::unordered_set<IAST *> & excluded_nodes;
/// private_aliases are from lambda, so these are local names. /// private_aliases are from lambda, so these are local names.
NameSet private_aliases; NameSet private_aliases;
/// Check if query is changed by this visitor. /// Check if query is changed by this visitor.
bool changed = false; bool changed = false;
Data(const ColumnsDescription & columns_, const NameToNameMap & array_join_result_columns_, ContextPtr context_) Data(const ColumnsDescription & columns_, const NameToNameMap & array_join_result_columns_, ContextPtr context_, const std::unordered_set<IAST *> & excluded_nodes_)
: columns(columns_), context(context_) : columns(columns_), context(context_), excluded_nodes(excluded_nodes_)
{ {
for (const auto & [result, source] : array_join_result_columns_) for (const auto & [result, source] : array_join_result_columns_)
{ {
@ -75,7 +77,7 @@ public:
}; };
static void visit(ASTPtr & ast, Data & data); static void visit(ASTPtr & ast, Data & data);
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child); static bool needChildVisit(const ASTPtr & node, const ASTPtr & child, const Data & data);
private: private:
static void visit(ASTIdentifier & node, ASTPtr & ast, Data & data); static void visit(ASTIdentifier & node, ASTPtr & ast, Data & data);

View File

@ -10,7 +10,7 @@ namespace DB
/// Visits AST tree in depth, call functions for nodes according to Matcher type data. /// Visits AST tree in depth, call functions for nodes according to Matcher type data.
/// You need to define Data, visit() and needChildVisit() in Matcher class. /// You need to define Data, visit() and needChildVisit() in Matcher class.
template <typename Matcher, bool _top_to_bottom, typename T = ASTPtr> template <typename Matcher, bool _top_to_bottom, bool need_child_accept_data = false, typename T = ASTPtr>
class InDepthNodeVisitor class InDepthNodeVisitor
{ {
public: public:
@ -51,13 +51,21 @@ private:
void visitChildren(T & ast) void visitChildren(T & ast)
{ {
for (auto & child : ast->children) for (auto & child : ast->children)
if (Matcher::needChildVisit(ast, child)) {
bool need_visit_child = false;
if constexpr (need_child_accept_data)
need_visit_child = Matcher::needChildVisit(ast, child, data);
else
need_visit_child = Matcher::needChildVisit(ast, child);
if (need_visit_child)
visit(child); visit(child);
} }
}
}; };
template <typename Matcher, bool top_to_bottom> template <typename Matcher, bool top_to_bottom, bool need_child_accept_data = false>
using ConstInDepthNodeVisitor = InDepthNodeVisitor<Matcher, top_to_bottom, const ASTPtr>; using ConstInDepthNodeVisitor = InDepthNodeVisitor<Matcher, top_to_bottom, need_child_accept_data, const ASTPtr>;
struct NeedChild struct NeedChild
{ {

View File

@ -951,16 +951,9 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
setJoinStrictness( setJoinStrictness(
*select_query, settings.join_default_strictness, settings.any_join_distinct_right_table_keys, result.analyzed_join->table_join); *select_query, settings.join_default_strictness, settings.any_join_distinct_right_table_keys, result.analyzed_join->table_join);
if (const auto * join_ast = select_query->join(); join_ast && tables_with_columns.size() >= 2) auto * table_join_ast = select_query->join() ? select_query->join()->table_join->as<ASTTableJoin>() : nullptr;
{ if (table_join_ast && tables_with_columns.size() >= 2)
auto & table_join_ast = join_ast->table_join->as<ASTTableJoin &>(); collectJoinedColumns(*result.analyzed_join, *table_join_ast, tables_with_columns, result.aliases);
if (table_join_ast.using_expression_list && result.metadata_snapshot)
replaceAliasColumnsInQuery(table_join_ast.using_expression_list, result.metadata_snapshot->getColumns(), result.array_join_result_to_source, getContext());
if (table_join_ast.on_expression && result.metadata_snapshot)
replaceAliasColumnsInQuery(table_join_ast.on_expression, result.metadata_snapshot->getColumns(), result.array_join_result_to_source, getContext());
collectJoinedColumns(*result.analyzed_join, table_join_ast, tables_with_columns, result.aliases);
}
result.aggregates = getAggregates(query, *select_query); result.aggregates = getAggregates(query, *select_query);
result.window_function_asts = getWindowFunctions(query, *select_query); result.window_function_asts = getWindowFunctions(query, *select_query);
@ -971,8 +964,19 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
bool is_initiator = getContext()->getClientInfo().distributed_depth == 0; bool is_initiator = getContext()->getClientInfo().distributed_depth == 0;
if (settings.optimize_respect_aliases && result.metadata_snapshot && is_initiator) if (settings.optimize_respect_aliases && result.metadata_snapshot && is_initiator)
{ {
std::unordered_set<IAST *> excluded_nodes;
{
/// Do not replace ALIASed columns in JOIN ON/USING sections
if (table_join_ast && table_join_ast->on_expression)
excluded_nodes.insert(table_join_ast->on_expression.get());
if (table_join_ast && table_join_ast->using_expression_list)
excluded_nodes.insert(table_join_ast->using_expression_list.get());
}
bool is_changed = replaceAliasColumnsInQuery(query, result.metadata_snapshot->getColumns(),
result.array_join_result_to_source, getContext(), excluded_nodes);
/// If query is changed, we need to redo some work to correct name resolution. /// If query is changed, we need to redo some work to correct name resolution.
if (replaceAliasColumnsInQuery(query, result.metadata_snapshot->getColumns(), result.array_join_result_to_source, getContext())) if (is_changed)
{ {
result.aggregates = getAggregates(query, *select_query); result.aggregates = getAggregates(query, *select_query);
result.window_function_asts = getWindowFunctions(query, *select_query); result.window_function_asts = getWindowFunctions(query, *select_query);

View File

@ -7,9 +7,13 @@ namespace DB
{ {
bool replaceAliasColumnsInQuery( bool replaceAliasColumnsInQuery(
ASTPtr & ast, const ColumnsDescription & columns, const NameToNameMap & array_join_result_to_source, ContextPtr context) ASTPtr & ast,
const ColumnsDescription & columns,
const NameToNameMap & array_join_result_to_source,
ContextPtr context,
const std::unordered_set<IAST *> & excluded_nodes)
{ {
ColumnAliasesVisitor::Data aliases_column_data(columns, array_join_result_to_source, context); ColumnAliasesVisitor::Data aliases_column_data(columns, array_join_result_to_source, context, excluded_nodes);
ColumnAliasesVisitor aliases_column_visitor(aliases_column_data); ColumnAliasesVisitor aliases_column_visitor(aliases_column_data);
aliases_column_visitor.visit(ast); aliases_column_visitor.visit(ast);
return aliases_column_data.changed; return aliases_column_data.changed;

View File

@ -12,6 +12,10 @@ class ColumnsDescription;
/// Replace storage alias columns in select query if possible. Return true if the query is changed. /// Replace storage alias columns in select query if possible. Return true if the query is changed.
bool replaceAliasColumnsInQuery( bool replaceAliasColumnsInQuery(
ASTPtr & ast, const ColumnsDescription & columns, const NameToNameMap & array_join_result_to_source, ContextPtr context); ASTPtr & ast,
const ColumnsDescription & columns,
const NameToNameMap & array_join_result_to_source,
ContextPtr context,
const std::unordered_set<IAST *> & excluded_nodes = {});
} }

View File

@ -22,3 +22,13 @@ fact1t1_val1 fact1t2_val2
fact2t1_val2 fact2t1_val2 fact2t1_val2 fact2t1_val2
- -
2020-02-02 13:00:00 2020-02-05 13:00:00 2020-02-02 13:00:00 2020-02-05 13:00:00
-
1
1
1
1
-
2020-01-01 12:00:00
2020-01-01 12:00:00
2020-01-01 12:00:00
2020-01-01 12:00:00

View File

@ -2,17 +2,23 @@ DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2; DROP TABLE IF EXISTS t2;
CREATE TABLE t1 ( CREATE TABLE t1 (
time DateTime, foo String, dimension_1 String, time DateTime,
foo String,
dimension_1 String,
dt Date MATERIALIZED toDate(time), dt Date MATERIALIZED toDate(time),
dt1 Date MATERIALIZED toDayOfYear(time), dt1 Date MATERIALIZED toDayOfYear(time),
aliascol1 ALIAS foo || dimension_1 aliascol1 ALIAS foo || dimension_1,
time_alias DateTime ALIAS time
) ENGINE = MergeTree() PARTITION BY toYYYYMM(dt) ORDER BY (dt, foo); ) ENGINE = MergeTree() PARTITION BY toYYYYMM(dt) ORDER BY (dt, foo);
CREATE TABLE t2 ( CREATE TABLE t2 (
time DateTime, bar String, dimension_2 String, time DateTime,
bar String,
dimension_2 String,
dt Date MATERIALIZED toDate(time), dt Date MATERIALIZED toDate(time),
dt2 Date MATERIALIZED toDayOfYear(time), dt2 Date MATERIALIZED toDayOfYear(time),
aliascol2 ALIAS bar || dimension_2 aliascol2 ALIAS bar || dimension_2,
time_alias DateTime ALIAS time
) ENGINE = MergeTree() PARTITION BY toYYYYMM(dt) ORDER BY (dt, bar); ) ENGINE = MergeTree() PARTITION BY toYYYYMM(dt) ORDER BY (dt, bar);
INSERT INTO t1 VALUES ('2020-01-01 12:00:00', 'fact1', 't1_val1'), ('2020-02-02 13:00:00', 'fact2', 't1_val2'), ('2020-01-01 13:00:00', 'fact3', 't1_val3'); INSERT INTO t1 VALUES ('2020-01-01 12:00:00', 'fact1', 't1_val1'), ('2020-02-02 13:00:00', 'fact2', 't1_val2'), ('2020-01-01 13:00:00', 'fact3', 't1_val3');
@ -35,3 +41,15 @@ SELECT '-';
SELECT t1.aliascol1, t2.aliascol2 FROM t1 JOIN t2 ON t1.foo = t2.bar ORDER BY t1.time, t2.time; SELECT t1.aliascol1, t2.aliascol2 FROM t1 JOIN t2 ON t1.foo = t2.bar ORDER BY t1.time, t2.time;
SELECT '-'; SELECT '-';
SELECT t1.time, t2.time FROM t1 JOIN t2 ON t1.aliascol1 = t2.aliascol2 ORDER BY t1.time, t2.time; SELECT t1.time, t2.time FROM t1 JOIN t2 ON t1.aliascol1 = t2.aliascol2 ORDER BY t1.time, t2.time;
SELECT '-';
SELECT count() FROM t1 JOIN t2 ON t1.time_alias = t2.time;
SELECT count() FROM t1 JOIN t2 ON t1.time = t2.time_alias;
SELECT count() FROM t1 JOIN t2 ON t1.time_alias = t2.time_alias;
SELECT count() FROM t1 JOIN t2 USING (time_alias);
SELECT '-';
SELECT t1.time as talias FROM t1 JOIN t2 ON talias = t2.time;
SELECT t1.time as talias FROM t1 JOIN t2 ON talias = t2.time_alias;
SELECT t2.time as talias FROM t1 JOIN t2 ON t1.time = talias;
SELECT t2.time as talias FROM t1 JOIN t2 ON t1.time_alias = talias;
SELECT time as talias FROM t1 JOIN t2 ON t1.time = talias; -- { serverError AMBIGUOUS_COLUMN_NAME }
SELECT time as talias FROM t1 JOIN t2 ON talias = t2.time; -- { serverError AMBIGUOUS_COLUMN_NAME }