mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Skip JOIN OR/USING sections in replaceAliasColumnsInQuery
This commit is contained in:
parent
a7f770ecb7
commit
3d2d994c5d
@ -3,7 +3,6 @@
|
||||
#include <Interpreters/RequiredSourceColumnsVisitor.h>
|
||||
#include <Interpreters/addTypeConversionToAST.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
@ -14,8 +13,11 @@
|
||||
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>())
|
||||
{
|
||||
/// "lambda" visits children itself.
|
||||
|
@ -46,7 +46,7 @@ using DataTypePtr = std::shared_ptr<const IDataType>;
|
||||
class ColumnAliasesMatcher
|
||||
{
|
||||
public:
|
||||
using Visitor = InDepthNodeVisitor<ColumnAliasesMatcher, false>;
|
||||
using Visitor = InDepthNodeVisitor<ColumnAliasesMatcher, false, true>;
|
||||
|
||||
struct Data
|
||||
{
|
||||
@ -57,14 +57,16 @@ public:
|
||||
NameSet array_join_source_columns;
|
||||
ContextPtr context;
|
||||
|
||||
const std::unordered_set<IAST *> & excluded_nodes;
|
||||
|
||||
/// private_aliases are from lambda, so these are local names.
|
||||
NameSet private_aliases;
|
||||
|
||||
/// Check if query is changed by this visitor.
|
||||
bool changed = false;
|
||||
|
||||
Data(const ColumnsDescription & columns_, const NameToNameMap & array_join_result_columns_, ContextPtr context_)
|
||||
: columns(columns_), context(context_)
|
||||
Data(const ColumnsDescription & columns_, const NameToNameMap & array_join_result_columns_, ContextPtr context_, const std::unordered_set<IAST *> & excluded_nodes_)
|
||||
: columns(columns_), context(context_), excluded_nodes(excluded_nodes_)
|
||||
{
|
||||
for (const auto & [result, source] : array_join_result_columns_)
|
||||
{
|
||||
@ -75,7 +77,7 @@ public:
|
||||
};
|
||||
|
||||
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:
|
||||
static void visit(ASTIdentifier & node, ASTPtr & ast, Data & data);
|
||||
|
@ -10,7 +10,7 @@ namespace DB
|
||||
|
||||
/// 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.
|
||||
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
|
||||
{
|
||||
public:
|
||||
@ -51,13 +51,21 @@ private:
|
||||
void visitChildren(T & ast)
|
||||
{
|
||||
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);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Matcher, bool top_to_bottom>
|
||||
using ConstInDepthNodeVisitor = InDepthNodeVisitor<Matcher, top_to_bottom, const ASTPtr>;
|
||||
template <typename Matcher, bool top_to_bottom, bool need_child_accept_data = false>
|
||||
using ConstInDepthNodeVisitor = InDepthNodeVisitor<Matcher, top_to_bottom, need_child_accept_data, const ASTPtr>;
|
||||
|
||||
struct NeedChild
|
||||
{
|
||||
|
@ -951,12 +951,9 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
|
||||
setJoinStrictness(
|
||||
*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 = join_ast->table_join->as<ASTTableJoin &>();
|
||||
|
||||
collectJoinedColumns(*result.analyzed_join, table_join_ast, tables_with_columns, result.aliases);
|
||||
}
|
||||
auto * table_join_ast = select_query->join() ? select_query->join()->table_join->as<ASTTableJoin>() : nullptr;
|
||||
if (table_join_ast && tables_with_columns.size() >= 2)
|
||||
collectJoinedColumns(*result.analyzed_join, *table_join_ast, tables_with_columns, result.aliases);
|
||||
|
||||
result.aggregates = getAggregates(query, *select_query);
|
||||
result.window_function_asts = getWindowFunctions(query, *select_query);
|
||||
@ -967,8 +964,19 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
|
||||
bool is_initiator = getContext()->getClientInfo().distributed_depth == 0;
|
||||
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 (replaceAliasColumnsInQuery(query, result.metadata_snapshot->getColumns(), result.array_join_result_to_source, getContext()))
|
||||
if (is_changed)
|
||||
{
|
||||
result.aggregates = getAggregates(query, *select_query);
|
||||
result.window_function_asts = getWindowFunctions(query, *select_query);
|
||||
|
@ -7,9 +7,13 @@ namespace DB
|
||||
{
|
||||
|
||||
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);
|
||||
aliases_column_visitor.visit(ast);
|
||||
return aliases_column_data.changed;
|
||||
|
@ -12,6 +12,10 @@ class ColumnsDescription;
|
||||
|
||||
/// Replace storage alias columns in select query if possible. Return true if the query is changed.
|
||||
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 = {});
|
||||
|
||||
}
|
||||
|
@ -22,3 +22,13 @@ fact1t1_val1 fact1t2_val2
|
||||
fact2t1_val2 fact2t1_val2
|
||||
-
|
||||
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
|
||||
|
@ -2,17 +2,23 @@ DROP TABLE IF EXISTS t1;
|
||||
DROP TABLE IF EXISTS t2;
|
||||
|
||||
CREATE TABLE t1 (
|
||||
time DateTime, foo String, dimension_1 String,
|
||||
time DateTime,
|
||||
foo String,
|
||||
dimension_1 String,
|
||||
dt Date MATERIALIZED toDate(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);
|
||||
|
||||
CREATE TABLE t2 (
|
||||
time DateTime, bar String, dimension_2 String,
|
||||
time DateTime,
|
||||
bar String,
|
||||
dimension_2 String,
|
||||
dt Date MATERIALIZED toDate(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);
|
||||
|
||||
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 '-';
|
||||
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 }
|
||||
|
Loading…
Reference in New Issue
Block a user