mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-11 17:02:25 +00:00
Add prefer_column_name_to_alias settings (#22044)
This commit is contained in:
parent
1f5052a7c5
commit
69204e1d21
@ -240,6 +240,7 @@ class IColumn;
|
|||||||
M(Bool, metrics_perf_events_enabled, false, "If enabled, some of the perf events will be measured throughout queries' execution.", 0) \
|
M(Bool, metrics_perf_events_enabled, false, "If enabled, some of the perf events will be measured throughout queries' execution.", 0) \
|
||||||
M(String, metrics_perf_events_list, "", "Comma separated list of perf metrics that will be measured throughout queries' execution. Empty means all events. See PerfEventInfo in sources for the available events.", 0) \
|
M(String, metrics_perf_events_list, "", "Comma separated list of perf metrics that will be measured throughout queries' execution. Empty means all events. See PerfEventInfo in sources for the available events.", 0) \
|
||||||
M(Float, opentelemetry_start_trace_probability, 0., "Probability to start an OpenTelemetry trace for an incoming query.", 0) \
|
M(Float, opentelemetry_start_trace_probability, 0., "Probability to start an OpenTelemetry trace for an incoming query.", 0) \
|
||||||
|
M(Bool, prefer_column_name_to_alias, false, "Prefer using column names instead of aliases if possible.", 0) \
|
||||||
\
|
\
|
||||||
\
|
\
|
||||||
/** Limits during query execution are part of the settings. \
|
/** Limits during query execution are part of the settings. \
|
||||||
|
@ -72,6 +72,12 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data)
|
|||||||
if (!IdentifierSemantic::getColumnName(node))
|
if (!IdentifierSemantic::getColumnName(node))
|
||||||
return;
|
return;
|
||||||
|
|
||||||
|
if (data.settings.prefer_column_name_to_alias)
|
||||||
|
{
|
||||||
|
if (data.source_columns_set.find(node.name()) != data.source_columns_set.end())
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
/// If it is an alias, but not a parent alias (for constructs like "SELECT column + 1 AS column").
|
/// If it is an alias, but not a parent alias (for constructs like "SELECT column + 1 AS column").
|
||||||
auto it_alias = data.aliases.find(node.name());
|
auto it_alias = data.aliases.find(node.name());
|
||||||
if (it_alias != data.aliases.end() && current_alias != node.name())
|
if (it_alias != data.aliases.end() && current_alias != node.name())
|
||||||
@ -131,8 +137,20 @@ static bool needVisitChild(const ASTPtr & child)
|
|||||||
void QueryNormalizer::visit(ASTSelectQuery & select, const ASTPtr &, Data & data)
|
void QueryNormalizer::visit(ASTSelectQuery & select, const ASTPtr &, Data & data)
|
||||||
{
|
{
|
||||||
for (auto & child : select.children)
|
for (auto & child : select.children)
|
||||||
if (needVisitChild(child))
|
{
|
||||||
|
if (child == select.groupBy() || child == select.orderBy() || child == select.having())
|
||||||
|
{
|
||||||
|
bool old_setting = data.settings.prefer_column_name_to_alias;
|
||||||
|
data.settings.prefer_column_name_to_alias = false;
|
||||||
visit(child, data);
|
visit(child, data);
|
||||||
|
data.settings.prefer_column_name_to_alias = old_setting;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
if (needVisitChild(child))
|
||||||
|
visit(child, data);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/// If the WHERE clause or HAVING consists of a single alias, the reference must be replaced not only in children,
|
/// If the WHERE clause or HAVING consists of a single alias, the reference must be replaced not only in children,
|
||||||
/// but also in where_expression and having_expression.
|
/// but also in where_expression and having_expression.
|
||||||
|
@ -4,6 +4,7 @@
|
|||||||
|
|
||||||
#include <Parsers/IAST.h>
|
#include <Parsers/IAST.h>
|
||||||
#include <Interpreters/Aliases.h>
|
#include <Interpreters/Aliases.h>
|
||||||
|
#include <Core/Names.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -21,12 +22,15 @@ class QueryNormalizer
|
|||||||
{
|
{
|
||||||
const UInt64 max_ast_depth;
|
const UInt64 max_ast_depth;
|
||||||
const UInt64 max_expanded_ast_elements;
|
const UInt64 max_expanded_ast_elements;
|
||||||
|
bool prefer_column_name_to_alias;
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
ExtractedSettings(const T & settings)
|
ExtractedSettings(const T & settings)
|
||||||
: max_ast_depth(settings.max_ast_depth),
|
: max_ast_depth(settings.max_ast_depth)
|
||||||
max_expanded_ast_elements(settings.max_expanded_ast_elements)
|
, max_expanded_ast_elements(settings.max_expanded_ast_elements)
|
||||||
{}
|
, prefer_column_name_to_alias(settings.prefer_column_name_to_alias)
|
||||||
|
{
|
||||||
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
public:
|
public:
|
||||||
@ -36,7 +40,8 @@ public:
|
|||||||
using MapOfASTs = std::map<ASTPtr, ASTPtr>;
|
using MapOfASTs = std::map<ASTPtr, ASTPtr>;
|
||||||
|
|
||||||
const Aliases & aliases;
|
const Aliases & aliases;
|
||||||
const ExtractedSettings settings;
|
const NameSet & source_columns_set;
|
||||||
|
ExtractedSettings settings;
|
||||||
|
|
||||||
/// tmp data
|
/// tmp data
|
||||||
size_t level;
|
size_t level;
|
||||||
@ -44,8 +49,9 @@ public:
|
|||||||
SetOfASTs current_asts; /// vertices in the current call stack of this method
|
SetOfASTs current_asts; /// vertices in the current call stack of this method
|
||||||
std::string current_alias; /// the alias referencing to the ancestor of ast (the deepest ancestor with aliases)
|
std::string current_alias; /// the alias referencing to the ancestor of ast (the deepest ancestor with aliases)
|
||||||
|
|
||||||
Data(const Aliases & aliases_, ExtractedSettings && settings_)
|
Data(const Aliases & aliases_, const NameSet & source_columns_set_, ExtractedSettings && settings_)
|
||||||
: aliases(aliases_)
|
: aliases(aliases_)
|
||||||
|
, source_columns_set(source_columns_set_)
|
||||||
, settings(settings_)
|
, settings(settings_)
|
||||||
, level(0)
|
, level(0)
|
||||||
{}
|
{}
|
||||||
|
@ -813,7 +813,14 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
|
|||||||
/// Optimizes logical expressions.
|
/// Optimizes logical expressions.
|
||||||
LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length.value).perform();
|
LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length.value).perform();
|
||||||
|
|
||||||
normalize(query, result.aliases, settings);
|
NameSet all_source_columns_set = source_columns_set;
|
||||||
|
if (table_join)
|
||||||
|
{
|
||||||
|
for (const auto & [name, _] : table_join->columns_from_joined_table)
|
||||||
|
all_source_columns_set.insert(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
normalize(query, result.aliases, all_source_columns_set, settings);
|
||||||
|
|
||||||
/// Remove unneeded columns according to 'required_result_columns'.
|
/// Remove unneeded columns according to 'required_result_columns'.
|
||||||
/// Leave all selected columns in case of DISTINCT; columns that contain arrayJoin function inside.
|
/// Leave all selected columns in case of DISTINCT; columns that contain arrayJoin function inside.
|
||||||
@ -871,7 +878,7 @@ TreeRewriterResultPtr TreeRewriter::analyze(
|
|||||||
|
|
||||||
TreeRewriterResult result(source_columns, storage, metadata_snapshot, false);
|
TreeRewriterResult result(source_columns, storage, metadata_snapshot, false);
|
||||||
|
|
||||||
normalize(query, result.aliases, settings);
|
normalize(query, result.aliases, result.source_columns_set, settings);
|
||||||
|
|
||||||
/// Executing scalar subqueries. Column defaults could be a scalar subquery.
|
/// Executing scalar subqueries. Column defaults could be a scalar subquery.
|
||||||
executeScalarSubqueries(query, context, 0, result.scalars, false);
|
executeScalarSubqueries(query, context, 0, result.scalars, false);
|
||||||
@ -896,7 +903,7 @@ TreeRewriterResultPtr TreeRewriter::analyze(
|
|||||||
return std::make_shared<const TreeRewriterResult>(result);
|
return std::make_shared<const TreeRewriterResult>(result);
|
||||||
}
|
}
|
||||||
|
|
||||||
void TreeRewriter::normalize(ASTPtr & query, Aliases & aliases, const Settings & settings)
|
void TreeRewriter::normalize(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, const Settings & settings)
|
||||||
{
|
{
|
||||||
CustomizeCountDistinctVisitor::Data data_count_distinct{settings.count_distinct_implementation};
|
CustomizeCountDistinctVisitor::Data data_count_distinct{settings.count_distinct_implementation};
|
||||||
CustomizeCountDistinctVisitor(data_count_distinct).visit(query);
|
CustomizeCountDistinctVisitor(data_count_distinct).visit(query);
|
||||||
@ -945,7 +952,7 @@ void TreeRewriter::normalize(ASTPtr & query, Aliases & aliases, const Settings &
|
|||||||
FunctionNameNormalizer().visit(query.get());
|
FunctionNameNormalizer().visit(query.get());
|
||||||
|
|
||||||
/// Common subexpression elimination. Rewrite rules.
|
/// Common subexpression elimination. Rewrite rules.
|
||||||
QueryNormalizer::Data normalizer_data(aliases, settings);
|
QueryNormalizer::Data normalizer_data(aliases, source_columns_set, settings);
|
||||||
QueryNormalizer(normalizer_data).visit(query);
|
QueryNormalizer(normalizer_data).visit(query);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -119,7 +119,7 @@ public:
|
|||||||
private:
|
private:
|
||||||
const Context & context;
|
const Context & context;
|
||||||
|
|
||||||
static void normalize(ASTPtr & query, Aliases & aliases, const Settings & settings);
|
static void normalize(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, const Settings & settings);
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -20,6 +20,6 @@ TEST(QueryNormalizer, SimpleCycleAlias)
|
|||||||
aliases["b"] = parseQuery(parser, "a as b", 0, 0)->children[0];
|
aliases["b"] = parseQuery(parser, "a as b", 0, 0)->children[0];
|
||||||
|
|
||||||
Settings settings;
|
Settings settings;
|
||||||
QueryNormalizer::Data normalizer_data(aliases, settings);
|
QueryNormalizer::Data normalizer_data(aliases, {}, settings);
|
||||||
EXPECT_THROW(QueryNormalizer(normalizer_data).visit(ast), Exception);
|
EXPECT_THROW(QueryNormalizer(normalizer_data).visit(ast), Exception);
|
||||||
}
|
}
|
||||||
|
@ -0,0 +1,3 @@
|
|||||||
|
4.5 9
|
||||||
|
3 2
|
||||||
|
3 3
|
@ -0,0 +1,8 @@
|
|||||||
|
SELECT avg(number) AS number, max(number) FROM numbers(10); -- { serverError 184 }
|
||||||
|
SELECT sum(x) AS x, max(x) FROM (SELECT 1 AS x UNION ALL SELECT 2 AS x) t; -- { serverError 184 }
|
||||||
|
select sum(C1) as C1, count(C1) as C2 from (select number as C1 from numbers(3)) as ITBL; -- { serverError 184 }
|
||||||
|
|
||||||
|
set prefer_column_name_to_alias = 1;
|
||||||
|
SELECT avg(number) AS number, max(number) FROM numbers(10);
|
||||||
|
SELECT sum(x) AS x, max(x) FROM (SELECT 1 AS x UNION ALL SELECT 2 AS x) t settings prefer_column_name_to_alias = 1;
|
||||||
|
select sum(C1) as C1, count(C1) as C2 from (select number as C1 from numbers(3)) as ITBL settings prefer_column_name_to_alias = 1;
|
Loading…
Reference in New Issue
Block a user