mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #25371 from CurtizJ/add-compatibility-setting
Add compatibility setting for name of tuple literals
This commit is contained in:
commit
4a250dc14e
@ -469,6 +469,7 @@ class IColumn;
|
|||||||
M(UnionMode, union_default_mode, UnionMode::Unspecified, "Set default Union Mode in SelectWithUnion query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without Union Mode will throw exception.", 0) \
|
M(UnionMode, union_default_mode, UnionMode::Unspecified, "Set default Union Mode in SelectWithUnion query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without Union Mode will throw exception.", 0) \
|
||||||
M(Bool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \
|
M(Bool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \
|
||||||
M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \
|
M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \
|
||||||
|
M(Bool, legacy_column_name_of_tuple_literal, false, "List all names of element of large tuple literals in their column names instead of hash. This settings exists only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher.", 0) \
|
||||||
\
|
\
|
||||||
M(Bool, query_plan_enable_optimizations, true, "Apply optimizations to query plan", 0) \
|
M(Bool, query_plan_enable_optimizations, true, "Apply optimizations to query plan", 0) \
|
||||||
M(UInt64, query_plan_max_optimizations_to_apply, 10000, "Limit the total number of optimizations applied to query plan. If zero, ignored. If limit reached, throw exception", 0) \
|
M(UInt64, query_plan_max_optimizations_to_apply, 10000, "Limit the total number of optimizations applied to query plan. If zero, ignored. If limit reached, throw exception", 0) \
|
||||||
|
@ -348,7 +348,7 @@ SetPtr makeExplicitSet(
|
|||||||
const ASTPtr & left_arg = args.children.at(0);
|
const ASTPtr & left_arg = args.children.at(0);
|
||||||
const ASTPtr & right_arg = args.children.at(1);
|
const ASTPtr & right_arg = args.children.at(1);
|
||||||
|
|
||||||
auto column_name = left_arg->getColumnName();
|
auto column_name = left_arg->getColumnName(context->getSettingsRef());
|
||||||
const auto & dag_node = actions.findInIndex(column_name);
|
const auto & dag_node = actions.findInIndex(column_name);
|
||||||
const DataTypePtr & left_arg_type = dag_node.result_type;
|
const DataTypePtr & left_arg_type = dag_node.result_type;
|
||||||
|
|
||||||
@ -641,7 +641,7 @@ std::optional<NameAndTypePair> ActionsMatcher::getNameAndTypeFromAST(const ASTPt
|
|||||||
{
|
{
|
||||||
// If the argument is a literal, we generated a unique column name for it.
|
// If the argument is a literal, we generated a unique column name for it.
|
||||||
// Use it instead of a generic display name.
|
// Use it instead of a generic display name.
|
||||||
auto child_column_name = ast->getColumnName();
|
auto child_column_name = ast->getColumnName(data.getContext()->getSettingsRef());
|
||||||
const auto * as_literal = ast->as<ASTLiteral>();
|
const auto * as_literal = ast->as<ASTLiteral>();
|
||||||
if (as_literal)
|
if (as_literal)
|
||||||
{
|
{
|
||||||
@ -703,7 +703,7 @@ ASTs ActionsMatcher::doUntuple(const ASTFunction * function, ActionsMatcher::Dat
|
|||||||
func->setAlias(data.getUniqueName("_ut_" + name));
|
func->setAlias(data.getUniqueName("_ut_" + name));
|
||||||
|
|
||||||
auto function_builder = FunctionFactory::instance().get(func->name, data.getContext());
|
auto function_builder = FunctionFactory::instance().get(func->name, data.getContext());
|
||||||
data.addFunction(function_builder, {tuple_name_type->name, literal->getColumnName()}, func->getColumnName());
|
data.addFunction(function_builder, {tuple_name_type->name, literal->getColumnName(data.getContext()->getSettingsRef())}, func->getColumnName(data.getContext()->getSettingsRef()));
|
||||||
|
|
||||||
columns.push_back(std::move(func));
|
columns.push_back(std::move(func));
|
||||||
}
|
}
|
||||||
@ -740,6 +740,7 @@ void ActionsMatcher::visit(ASTExpressionList & expression_list, const ASTPtr &,
|
|||||||
|
|
||||||
void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr &, Data & data)
|
void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr &, Data & data)
|
||||||
{
|
{
|
||||||
|
|
||||||
auto column_name = identifier.getColumnName();
|
auto column_name = identifier.getColumnName();
|
||||||
if (data.hasColumn(column_name))
|
if (data.hasColumn(column_name))
|
||||||
return;
|
return;
|
||||||
@ -766,7 +767,7 @@ void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr &, Dat
|
|||||||
|
|
||||||
void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data)
|
void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data)
|
||||||
{
|
{
|
||||||
auto column_name = ast->getColumnName();
|
auto column_name = ast->getColumnName(data.getContext()->getSettingsRef());
|
||||||
if (data.hasColumn(column_name))
|
if (data.hasColumn(column_name))
|
||||||
return;
|
return;
|
||||||
|
|
||||||
@ -782,7 +783,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
|||||||
ASTPtr arg = node.arguments->children.at(0);
|
ASTPtr arg = node.arguments->children.at(0);
|
||||||
visit(arg, data);
|
visit(arg, data);
|
||||||
if (!data.only_consts)
|
if (!data.only_consts)
|
||||||
data.addArrayJoin(arg->getColumnName(), column_name);
|
data.addArrayJoin(arg->getColumnName(data.getContext()->getSettingsRef()), column_name);
|
||||||
|
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
@ -804,7 +805,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
|||||||
/// We are in the part of the tree that we are not going to compute. You just need to define types.
|
/// We are in the part of the tree that we are not going to compute. You just need to define types.
|
||||||
/// Do not subquery and create sets. We replace "in*" function to "in*IgnoreSet".
|
/// Do not subquery and create sets. We replace "in*" function to "in*IgnoreSet".
|
||||||
|
|
||||||
auto argument_name = node.arguments->children.at(0)->getColumnName();
|
auto argument_name = node.arguments->children.at(0)->getColumnName(data.getContext()->getSettingsRef());
|
||||||
|
|
||||||
data.addFunction(
|
data.addFunction(
|
||||||
FunctionFactory::instance().get(node.name + "IgnoreSet", data.getContext()),
|
FunctionFactory::instance().get(node.name + "IgnoreSet", data.getContext()),
|
||||||
@ -933,7 +934,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
|||||||
if (!prepared_set->empty())
|
if (!prepared_set->empty())
|
||||||
column.name = data.getUniqueName("__set");
|
column.name = data.getUniqueName("__set");
|
||||||
else
|
else
|
||||||
column.name = child->getColumnName();
|
column.name = child->getColumnName(data.getContext()->getSettingsRef());
|
||||||
|
|
||||||
if (!data.hasColumn(column.name))
|
if (!data.hasColumn(column.name))
|
||||||
{
|
{
|
||||||
@ -1012,7 +1013,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
|||||||
visit(lambda->arguments->children.at(1), data);
|
visit(lambda->arguments->children.at(1), data);
|
||||||
auto lambda_dag = data.actions_stack.popLevel();
|
auto lambda_dag = data.actions_stack.popLevel();
|
||||||
|
|
||||||
String result_name = lambda->arguments->children.at(1)->getColumnName();
|
String result_name = lambda->arguments->children.at(1)->getColumnName(data.getContext()->getSettingsRef());
|
||||||
lambda_dag->removeUnusedActions(Names(1, result_name));
|
lambda_dag->removeUnusedActions(Names(1, result_name));
|
||||||
|
|
||||||
auto lambda_actions = std::make_shared<ExpressionActions>(
|
auto lambda_actions = std::make_shared<ExpressionActions>(
|
||||||
@ -1027,7 +1028,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
|||||||
if (findColumn(required_arg, lambda_arguments) == lambda_arguments.end())
|
if (findColumn(required_arg, lambda_arguments) == lambda_arguments.end())
|
||||||
captured.push_back(required_arg);
|
captured.push_back(required_arg);
|
||||||
|
|
||||||
/// We can not name `getColumnName()`,
|
/// We can not name `getColumnName(data.getContext()->getSettingsRef())`,
|
||||||
/// because it does not uniquely define the expression (the types of arguments can be different).
|
/// because it does not uniquely define the expression (the types of arguments can be different).
|
||||||
String lambda_name = data.getUniqueName("__lambda");
|
String lambda_name = data.getUniqueName("__lambda");
|
||||||
|
|
||||||
@ -1057,7 +1058,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
|||||||
if (arguments_present)
|
if (arguments_present)
|
||||||
{
|
{
|
||||||
/// Calculate column name here again, because AST may be changed here (in case of untuple).
|
/// Calculate column name here again, because AST may be changed here (in case of untuple).
|
||||||
data.addFunction(function_builder, argument_names, ast->getColumnName());
|
data.addFunction(function_builder, argument_names, ast->getColumnName(data.getContext()->getSettingsRef()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1071,7 +1072,7 @@ void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & /* ast */,
|
|||||||
// AST here? Anyway, do not modify the column name if it is set already.
|
// AST here? Anyway, do not modify the column name if it is set already.
|
||||||
if (literal.unique_column_name.empty())
|
if (literal.unique_column_name.empty())
|
||||||
{
|
{
|
||||||
const auto default_name = literal.getColumnName();
|
const auto default_name = literal.getColumnName(data.getContext()->getSettingsRef());
|
||||||
const auto & index = data.actions_stack.getLastActionsIndex();
|
const auto & index = data.actions_stack.getLastActionsIndex();
|
||||||
const auto * existing_column = index.tryGetNode(default_name);
|
const auto * existing_column = index.tryGetNode(default_name);
|
||||||
|
|
||||||
@ -1151,7 +1152,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery.
|
/// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery.
|
||||||
String set_id = right_in_operand->getColumnName();
|
String set_id = right_in_operand->getColumnName(data.getContext()->getSettingsRef());
|
||||||
|
|
||||||
SubqueryForSet & subquery_for_set = data.subqueries_for_sets[set_id];
|
SubqueryForSet & subquery_for_set = data.subqueries_for_sets[set_id];
|
||||||
|
|
||||||
@ -1187,7 +1188,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su
|
|||||||
{
|
{
|
||||||
const auto & last_actions = data.actions_stack.getLastActions();
|
const auto & last_actions = data.actions_stack.getLastActions();
|
||||||
const auto & index = data.actions_stack.getLastActionsIndex();
|
const auto & index = data.actions_stack.getLastActionsIndex();
|
||||||
if (index.contains(left_in_operand->getColumnName()))
|
if (index.contains(left_in_operand->getColumnName(data.getContext()->getSettingsRef())))
|
||||||
/// An explicit enumeration of values in parentheses.
|
/// An explicit enumeration of values in parentheses.
|
||||||
return makeExplicitSet(&node, last_actions, false, data.getContext(), data.set_size_limit, data.prepared_sets);
|
return makeExplicitSet(&node, last_actions, false, data.getContext(), data.set_size_limit, data.prepared_sets);
|
||||||
else
|
else
|
||||||
|
@ -244,7 +244,7 @@ void ExpressionAnalyzer::analyzeAggregation()
|
|||||||
ssize_t size = group_asts.size();
|
ssize_t size = group_asts.size();
|
||||||
getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false);
|
getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false);
|
||||||
|
|
||||||
const auto & column_name = group_asts[i]->getColumnName();
|
const auto & column_name = group_asts[i]->getColumnName(getContext()->getSettingsRef());
|
||||||
const auto * node = temp_actions->tryFindInIndex(column_name);
|
const auto * node = temp_actions->tryFindInIndex(column_name);
|
||||||
if (!node)
|
if (!node)
|
||||||
throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER);
|
throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||||
@ -398,7 +398,7 @@ void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node)
|
|||||||
auto temp_actions = std::make_shared<ActionsDAG>(columns_after_join);
|
auto temp_actions = std::make_shared<ActionsDAG>(columns_after_join);
|
||||||
getRootActions(left_in_operand, true, temp_actions);
|
getRootActions(left_in_operand, true, temp_actions);
|
||||||
|
|
||||||
if (temp_actions->tryFindInIndex(left_in_operand->getColumnName()))
|
if (temp_actions->tryFindInIndex(left_in_operand->getColumnName(getContext()->getSettingsRef())))
|
||||||
makeExplicitSet(func, *temp_actions, true, getContext(), settings.size_limits_for_set, prepared_sets);
|
makeExplicitSet(func, *temp_actions, true, getContext(), settings.size_limits_for_set, prepared_sets);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -446,7 +446,7 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions)
|
|||||||
if (node->arguments)
|
if (node->arguments)
|
||||||
getRootActionsNoMakeSet(node->arguments, true, actions);
|
getRootActionsNoMakeSet(node->arguments, true, actions);
|
||||||
|
|
||||||
aggregate.column_name = node->getColumnName();
|
aggregate.column_name = node->getColumnName(getContext()->getSettingsRef());
|
||||||
|
|
||||||
const ASTs & arguments = node->arguments ? node->arguments->children : ASTs();
|
const ASTs & arguments = node->arguments ? node->arguments->children : ASTs();
|
||||||
aggregate.argument_names.resize(arguments.size());
|
aggregate.argument_names.resize(arguments.size());
|
||||||
@ -454,7 +454,7 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions)
|
|||||||
|
|
||||||
for (size_t i = 0; i < arguments.size(); ++i)
|
for (size_t i = 0; i < arguments.size(); ++i)
|
||||||
{
|
{
|
||||||
const std::string & name = arguments[i]->getColumnName();
|
const std::string & name = arguments[i]->getColumnName(getContext()->getSettingsRef());
|
||||||
const auto * dag_node = actions->tryFindInIndex(name);
|
const auto * dag_node = actions->tryFindInIndex(name);
|
||||||
if (!dag_node)
|
if (!dag_node)
|
||||||
{
|
{
|
||||||
@ -647,7 +647,7 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
|
|||||||
WindowFunctionDescription window_function;
|
WindowFunctionDescription window_function;
|
||||||
window_function.function_node = function_node;
|
window_function.function_node = function_node;
|
||||||
window_function.column_name
|
window_function.column_name
|
||||||
= window_function.function_node->getColumnName();
|
= window_function.function_node->getColumnName(getContext()->getSettingsRef());
|
||||||
window_function.function_parameters
|
window_function.function_parameters
|
||||||
= window_function.function_node->parameters
|
= window_function.function_node->parameters
|
||||||
? getAggregateFunctionParametersArray(
|
? getAggregateFunctionParametersArray(
|
||||||
@ -666,7 +666,7 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
|
|||||||
window_function.argument_names.resize(arguments.size());
|
window_function.argument_names.resize(arguments.size());
|
||||||
for (size_t i = 0; i < arguments.size(); ++i)
|
for (size_t i = 0; i < arguments.size(); ++i)
|
||||||
{
|
{
|
||||||
const std::string & name = arguments[i]->getColumnName();
|
const std::string & name = arguments[i]->getColumnName(getContext()->getSettingsRef());
|
||||||
const auto * node = actions->tryFindInIndex(name);
|
const auto * node = actions->tryFindInIndex(name);
|
||||||
|
|
||||||
if (!node)
|
if (!node)
|
||||||
@ -964,7 +964,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere(
|
|||||||
|
|
||||||
auto & step = chain.lastStep(sourceColumns());
|
auto & step = chain.lastStep(sourceColumns());
|
||||||
getRootActions(select_query->prewhere(), only_types, step.actions());
|
getRootActions(select_query->prewhere(), only_types, step.actions());
|
||||||
String prewhere_column_name = select_query->prewhere()->getColumnName();
|
String prewhere_column_name = select_query->prewhere()->getColumnName(getContext()->getSettingsRef());
|
||||||
step.addRequiredOutput(prewhere_column_name);
|
step.addRequiredOutput(prewhere_column_name);
|
||||||
|
|
||||||
const auto & node = step.actions()->findInIndex(prewhere_column_name);
|
const auto & node = step.actions()->findInIndex(prewhere_column_name);
|
||||||
@ -1061,7 +1061,7 @@ bool SelectQueryExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain,
|
|||||||
|
|
||||||
getRootActions(select_query->where(), only_types, step.actions());
|
getRootActions(select_query->where(), only_types, step.actions());
|
||||||
|
|
||||||
auto where_column_name = select_query->where()->getColumnName();
|
auto where_column_name = select_query->where()->getColumnName(getContext()->getSettingsRef());
|
||||||
step.addRequiredOutput(where_column_name);
|
step.addRequiredOutput(where_column_name);
|
||||||
|
|
||||||
const auto & node = step.actions()->findInIndex(where_column_name);
|
const auto & node = step.actions()->findInIndex(where_column_name);
|
||||||
@ -1086,7 +1086,7 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain
|
|||||||
ASTs asts = select_query->groupBy()->children;
|
ASTs asts = select_query->groupBy()->children;
|
||||||
for (const auto & ast : asts)
|
for (const auto & ast : asts)
|
||||||
{
|
{
|
||||||
step.addRequiredOutput(ast->getColumnName());
|
step.addRequiredOutput(ast->getColumnName(getContext()->getSettingsRef()));
|
||||||
getRootActions(ast, only_types, step.actions());
|
getRootActions(ast, only_types, step.actions());
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1114,7 +1114,7 @@ void SelectQueryExpressionAnalyzer::appendAggregateFunctionsArguments(Expression
|
|||||||
for (const auto & name : desc.argument_names)
|
for (const auto & name : desc.argument_names)
|
||||||
step.addRequiredOutput(name);
|
step.addRequiredOutput(name);
|
||||||
|
|
||||||
/// Collect aggregates removing duplicates by node.getColumnName()
|
/// Collect aggregates removing duplicates by node.getColumnName(getContext()->getSettingsRef())
|
||||||
/// It's not clear why we recollect aggregates (for query parts) while we're able to use previously collected ones (for entire query)
|
/// It's not clear why we recollect aggregates (for query parts) while we're able to use previously collected ones (for entire query)
|
||||||
/// @note The original recollection logic didn't remove duplicates.
|
/// @note The original recollection logic didn't remove duplicates.
|
||||||
GetAggregatesVisitor::Data data;
|
GetAggregatesVisitor::Data data;
|
||||||
@ -1169,7 +1169,7 @@ void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments(
|
|||||||
// (2b) Required function argument columns.
|
// (2b) Required function argument columns.
|
||||||
for (const auto & a : f.function_node->arguments->children)
|
for (const auto & a : f.function_node->arguments->children)
|
||||||
{
|
{
|
||||||
step.addRequiredOutput(a->getColumnName());
|
step.addRequiredOutput(a->getColumnName(getContext()->getSettingsRef()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1191,7 +1191,7 @@ bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain,
|
|||||||
ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns);
|
ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns);
|
||||||
|
|
||||||
getRootActionsForHaving(select_query->having(), only_types, step.actions());
|
getRootActionsForHaving(select_query->having(), only_types, step.actions());
|
||||||
step.addRequiredOutput(select_query->having()->getColumnName());
|
step.addRequiredOutput(select_query->having()->getColumnName(getContext()->getSettingsRef()));
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
@ -1215,7 +1215,7 @@ void SelectQueryExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain,
|
|||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
step.addRequiredOutput(child->getColumnName());
|
step.addRequiredOutput(child->getColumnName(getContext()->getSettingsRef()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1243,7 +1243,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai
|
|||||||
if (!ast || ast->children.empty())
|
if (!ast || ast->children.empty())
|
||||||
throw Exception("Bad order expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE);
|
throw Exception("Bad order expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE);
|
||||||
ASTPtr order_expression = ast->children.at(0);
|
ASTPtr order_expression = ast->children.at(0);
|
||||||
step.addRequiredOutput(order_expression->getColumnName());
|
step.addRequiredOutput(order_expression->getColumnName(getContext()->getSettingsRef()));
|
||||||
|
|
||||||
if (ast->with_fill)
|
if (ast->with_fill)
|
||||||
with_fill = true;
|
with_fill = true;
|
||||||
@ -1293,7 +1293,7 @@ bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain
|
|||||||
|
|
||||||
for (const auto & child : select_query->limitBy()->children)
|
for (const auto & child : select_query->limitBy()->children)
|
||||||
{
|
{
|
||||||
auto child_name = child->getColumnName();
|
auto child_name = child->getColumnName(getContext()->getSettingsRef());
|
||||||
if (!aggregated_names.count(child_name))
|
if (!aggregated_names.count(child_name))
|
||||||
step.addRequiredOutput(std::move(child_name));
|
step.addRequiredOutput(std::move(child_name));
|
||||||
}
|
}
|
||||||
@ -1309,13 +1309,15 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActio
|
|||||||
|
|
||||||
NamesWithAliases result_columns;
|
NamesWithAliases result_columns;
|
||||||
|
|
||||||
|
const auto & settings = getContext()->getSettingsRef();
|
||||||
|
|
||||||
ASTs asts = select_query->select()->children;
|
ASTs asts = select_query->select()->children;
|
||||||
for (const auto & ast : asts)
|
for (const auto & ast : asts)
|
||||||
{
|
{
|
||||||
String result_name = ast->getAliasOrColumnName();
|
String result_name = ast->getAliasOrColumnName(settings);
|
||||||
if (required_result_columns.empty() || required_result_columns.count(result_name))
|
if (required_result_columns.empty() || required_result_columns.count(result_name))
|
||||||
{
|
{
|
||||||
std::string source_name = ast->getColumnName();
|
std::string source_name = ast->getColumnName(settings);
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* For temporary columns created by ExpressionAnalyzer for literals,
|
* For temporary columns created by ExpressionAnalyzer for literals,
|
||||||
@ -1357,7 +1359,7 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const
|
|||||||
{
|
{
|
||||||
ExpressionActionsChain::Step & step = chain.lastStep(sourceColumns());
|
ExpressionActionsChain::Step & step = chain.lastStep(sourceColumns());
|
||||||
getRootActions(expr, only_types, step.actions());
|
getRootActions(expr, only_types, step.actions());
|
||||||
step.addRequiredOutput(expr->getColumnName());
|
step.addRequiredOutput(expr->getColumnName(getContext()->getSettingsRef()));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -1374,12 +1376,13 @@ ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool project_r
|
|||||||
else
|
else
|
||||||
asts = ASTs(1, query);
|
asts = ASTs(1, query);
|
||||||
|
|
||||||
|
const auto & settings = getContext()->getSettingsRef();
|
||||||
for (const auto & ast : asts)
|
for (const auto & ast : asts)
|
||||||
{
|
{
|
||||||
std::string name = ast->getColumnName();
|
std::string name = ast->getColumnName(settings);
|
||||||
std::string alias;
|
std::string alias;
|
||||||
if (add_aliases)
|
if (add_aliases)
|
||||||
alias = ast->getAliasOrColumnName();
|
alias = ast->getAliasOrColumnName(settings);
|
||||||
else
|
else
|
||||||
alias = name;
|
alias = name;
|
||||||
result_columns.emplace_back(name, alias);
|
result_columns.emplace_back(name, alias);
|
||||||
@ -1514,7 +1517,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
|
|||||||
|
|
||||||
if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage, additional_required_columns_after_prewhere))
|
if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage, additional_required_columns_after_prewhere))
|
||||||
{
|
{
|
||||||
prewhere_info = std::make_shared<PrewhereDAGInfo>(actions, query.prewhere()->getColumnName());
|
prewhere_info = std::make_shared<PrewhereDAGInfo>(actions, query.prewhere()->getColumnName(settings));
|
||||||
|
|
||||||
if (allowEarlyConstantFolding(*prewhere_info->prewhere_actions, settings))
|
if (allowEarlyConstantFolding(*prewhere_info->prewhere_actions, settings))
|
||||||
{
|
{
|
||||||
@ -1524,7 +1527,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
|
|||||||
ExpressionActions(
|
ExpressionActions(
|
||||||
prewhere_info->prewhere_actions,
|
prewhere_info->prewhere_actions,
|
||||||
ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample);
|
ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample);
|
||||||
auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName());
|
auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName(settings));
|
||||||
/// If the filter column is a constant, record it.
|
/// If the filter column is a constant, record it.
|
||||||
if (column_elem.column)
|
if (column_elem.column)
|
||||||
prewhere_constant_filter_description = ConstantFilterDescription(*column_elem.column);
|
prewhere_constant_filter_description = ConstantFilterDescription(*column_elem.column);
|
||||||
@ -1559,7 +1562,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
|
|||||||
ExpressionActions(
|
ExpressionActions(
|
||||||
before_where,
|
before_where,
|
||||||
ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample);
|
ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample);
|
||||||
auto & column_elem = before_where_sample.getByName(query.where()->getColumnName());
|
auto & column_elem = before_where_sample.getByName(query.where()->getColumnName(settings));
|
||||||
/// If the filter column is a constant, record it.
|
/// If the filter column is a constant, record it.
|
||||||
if (column_elem.column)
|
if (column_elem.column)
|
||||||
where_constant_filter_description = ConstantFilterDescription(*column_elem.column);
|
where_constant_filter_description = ConstantFilterDescription(*column_elem.column);
|
||||||
@ -1650,7 +1653,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
|
|||||||
const auto * select_query = query_analyzer.getSelectQuery();
|
const auto * select_query = query_analyzer.getSelectQuery();
|
||||||
for (const auto & child : select_query->select()->children)
|
for (const auto & child : select_query->select()->children)
|
||||||
{
|
{
|
||||||
step.addRequiredOutput(child->getColumnName());
|
step.addRequiredOutput(child->getColumnName(settings));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1706,7 +1709,8 @@ void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, si
|
|||||||
|
|
||||||
if (hasWhere())
|
if (hasWhere())
|
||||||
{
|
{
|
||||||
where_column_name = query.where()->getColumnName();
|
const auto & settings = chain.getContext()->getSettingsRef();
|
||||||
|
where_column_name = query.where()->getColumnName(settings);
|
||||||
remove_where_filter = chain.steps.at(where_step_num)->required_output.find(where_column_name)->second;
|
remove_where_filter = chain.steps.at(where_step_num)->required_output.find(where_column_name)->second;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -143,7 +143,7 @@ String InterpreterSelectQuery::generateFilterActions(ActionsDAGPtr & actions, co
|
|||||||
SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context, metadata_snapshot);
|
SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context, metadata_snapshot);
|
||||||
actions = analyzer.simpleSelectActions();
|
actions = analyzer.simpleSelectActions();
|
||||||
|
|
||||||
auto column_name = expr_list->children.at(0)->getColumnName();
|
auto column_name = expr_list->children.at(0)->getColumnName(context->getSettingsRef());
|
||||||
actions->removeUnusedActions(NameSet{column_name});
|
actions->removeUnusedActions(NameSet{column_name});
|
||||||
actions->projectInput(false);
|
actions->projectInput(false);
|
||||||
|
|
||||||
@ -779,7 +779,7 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP
|
|||||||
order_descr.reserve(query.orderBy()->children.size());
|
order_descr.reserve(query.orderBy()->children.size());
|
||||||
for (const auto & elem : query.orderBy()->children)
|
for (const auto & elem : query.orderBy()->children)
|
||||||
{
|
{
|
||||||
String name = elem->children.front()->getColumnName();
|
String name = elem->children.front()->getColumnName(context->getSettingsRef());
|
||||||
const auto & order_by_elem = elem->as<ASTOrderByElement &>();
|
const auto & order_by_elem = elem->as<ASTOrderByElement &>();
|
||||||
|
|
||||||
std::shared_ptr<Collator> collator;
|
std::shared_ptr<Collator> collator;
|
||||||
@ -798,14 +798,14 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP
|
|||||||
return order_descr;
|
return order_descr;
|
||||||
}
|
}
|
||||||
|
|
||||||
static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & query)
|
static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & query, ContextPtr context)
|
||||||
{
|
{
|
||||||
SortDescription order_descr;
|
SortDescription order_descr;
|
||||||
order_descr.reserve(query.groupBy()->children.size());
|
order_descr.reserve(query.groupBy()->children.size());
|
||||||
|
|
||||||
for (const auto & elem : query.groupBy()->children)
|
for (const auto & elem : query.groupBy()->children)
|
||||||
{
|
{
|
||||||
String name = elem->getColumnName();
|
String name = elem->getColumnName(context->getSettingsRef());
|
||||||
order_descr.emplace_back(name, 1, 1);
|
order_descr.emplace_back(name, 1, 1);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1948,13 +1948,13 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
|
|||||||
{
|
{
|
||||||
query_info.projection->order_optimizer = std::make_shared<ReadInOrderOptimizer>(
|
query_info.projection->order_optimizer = std::make_shared<ReadInOrderOptimizer>(
|
||||||
query_info.projection->group_by_elements_actions,
|
query_info.projection->group_by_elements_actions,
|
||||||
getSortDescriptionFromGroupBy(query),
|
getSortDescriptionFromGroupBy(query, context),
|
||||||
query_info.syntax_analyzer_result);
|
query_info.syntax_analyzer_result);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
query_info.order_optimizer = std::make_shared<ReadInOrderOptimizer>(
|
query_info.order_optimizer = std::make_shared<ReadInOrderOptimizer>(
|
||||||
analysis_result.group_by_elements_actions, getSortDescriptionFromGroupBy(query), query_info.syntax_analyzer_result);
|
analysis_result.group_by_elements_actions, getSortDescriptionFromGroupBy(query, context), query_info.syntax_analyzer_result);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -2033,7 +2033,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
|
|||||||
void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter)
|
void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter)
|
||||||
{
|
{
|
||||||
auto where_step = std::make_unique<FilterStep>(
|
auto where_step = std::make_unique<FilterStep>(
|
||||||
query_plan.getCurrentDataStream(), expression, getSelectQuery().where()->getColumnName(), remove_filter);
|
query_plan.getCurrentDataStream(), expression, getSelectQuery().where()->getColumnName(context->getSettingsRef()), remove_filter);
|
||||||
|
|
||||||
where_step->setStepDescription("WHERE");
|
where_step->setStepDescription("WHERE");
|
||||||
query_plan.addStep(std::move(where_step));
|
query_plan.addStep(std::move(where_step));
|
||||||
@ -2080,7 +2080,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac
|
|||||||
SortDescription group_by_sort_description;
|
SortDescription group_by_sort_description;
|
||||||
|
|
||||||
if (group_by_info && settings.optimize_aggregation_in_order)
|
if (group_by_info && settings.optimize_aggregation_in_order)
|
||||||
group_by_sort_description = getSortDescriptionFromGroupBy(getSelectQuery());
|
group_by_sort_description = getSortDescriptionFromGroupBy(getSelectQuery(), context);
|
||||||
else
|
else
|
||||||
group_by_info = nullptr;
|
group_by_info = nullptr;
|
||||||
|
|
||||||
@ -2128,7 +2128,7 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool
|
|||||||
void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression)
|
void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression)
|
||||||
{
|
{
|
||||||
auto having_step
|
auto having_step
|
||||||
= std::make_unique<FilterStep>(query_plan.getCurrentDataStream(), expression, getSelectQuery().having()->getColumnName(), false);
|
= std::make_unique<FilterStep>(query_plan.getCurrentDataStream(), expression, getSelectQuery().having()->getColumnName(context->getSettingsRef()), false);
|
||||||
|
|
||||||
having_step->setStepDescription("HAVING");
|
having_step->setStepDescription("HAVING");
|
||||||
query_plan.addStep(std::move(having_step));
|
query_plan.addStep(std::move(having_step));
|
||||||
@ -2144,7 +2144,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving(
|
|||||||
query_plan.getCurrentDataStream(),
|
query_plan.getCurrentDataStream(),
|
||||||
overflow_row,
|
overflow_row,
|
||||||
expression,
|
expression,
|
||||||
has_having ? getSelectQuery().having()->getColumnName() : "",
|
has_having ? getSelectQuery().having()->getColumnName(context->getSettingsRef()) : "",
|
||||||
settings.totals_mode,
|
settings.totals_mode,
|
||||||
settings.totals_auto_threshold,
|
settings.totals_auto_threshold,
|
||||||
final);
|
final);
|
||||||
@ -2461,7 +2461,7 @@ void InterpreterSelectQuery::executeLimitBy(QueryPlan & query_plan)
|
|||||||
|
|
||||||
Names columns;
|
Names columns;
|
||||||
for (const auto & elem : query.limitBy()->children)
|
for (const auto & elem : query.limitBy()->children)
|
||||||
columns.emplace_back(elem->getColumnName());
|
columns.emplace_back(elem->getColumnName(context->getSettingsRef()));
|
||||||
|
|
||||||
UInt64 length = getLimitUIntValue(query.limitByLength(), context, "LIMIT");
|
UInt64 length = getLimitUIntValue(query.limitByLength(), context, "LIMIT");
|
||||||
UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), context, "OFFSET") : 0);
|
UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), context, "OFFSET") : 0);
|
||||||
|
@ -39,7 +39,7 @@ std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(co
|
|||||||
if (context->getSettingsRef().normalize_function_names)
|
if (context->getSettingsRef().normalize_function_names)
|
||||||
FunctionNameNormalizer().visit(ast.get());
|
FunctionNameNormalizer().visit(ast.get());
|
||||||
|
|
||||||
String name = ast->getColumnName();
|
String name = ast->getColumnName(context->getSettingsRef());
|
||||||
auto syntax_result = TreeRewriter(context).analyze(ast, source_columns);
|
auto syntax_result = TreeRewriter(context).analyze(ast, source_columns);
|
||||||
ExpressionActionsPtr expr_for_constant_folding = ExpressionAnalyzer(ast, syntax_result, context).getConstActions();
|
ExpressionActionsPtr expr_for_constant_folding = ExpressionAnalyzer(ast, syntax_result, context).getConstActions();
|
||||||
|
|
||||||
|
@ -24,6 +24,16 @@ namespace ErrorCodes
|
|||||||
}
|
}
|
||||||
|
|
||||||
void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const
|
void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const
|
||||||
|
{
|
||||||
|
appendColumnNameImpl(ostr, nullptr);
|
||||||
|
}
|
||||||
|
|
||||||
|
void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr, const Settings & settings) const
|
||||||
|
{
|
||||||
|
appendColumnNameImpl(ostr, &settings);
|
||||||
|
}
|
||||||
|
|
||||||
|
void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr, const Settings * settings) const
|
||||||
{
|
{
|
||||||
if (name == "view")
|
if (name == "view")
|
||||||
throw Exception("Table function view cannot be used as an expression", ErrorCodes::UNEXPECTED_EXPRESSION);
|
throw Exception("Table function view cannot be used as an expression", ErrorCodes::UNEXPECTED_EXPRESSION);
|
||||||
@ -37,19 +47,30 @@ void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const
|
|||||||
{
|
{
|
||||||
if (it != parameters->children.begin())
|
if (it != parameters->children.begin())
|
||||||
writeCString(", ", ostr);
|
writeCString(", ", ostr);
|
||||||
(*it)->appendColumnName(ostr);
|
|
||||||
|
if (settings)
|
||||||
|
(*it)->appendColumnName(ostr, *settings);
|
||||||
|
else
|
||||||
|
(*it)->appendColumnName(ostr);
|
||||||
}
|
}
|
||||||
writeChar(')', ostr);
|
writeChar(')', ostr);
|
||||||
}
|
}
|
||||||
|
|
||||||
writeChar('(', ostr);
|
writeChar('(', ostr);
|
||||||
if (arguments)
|
if (arguments)
|
||||||
|
{
|
||||||
for (auto it = arguments->children.begin(); it != arguments->children.end(); ++it)
|
for (auto it = arguments->children.begin(); it != arguments->children.end(); ++it)
|
||||||
{
|
{
|
||||||
if (it != arguments->children.begin())
|
if (it != arguments->children.begin())
|
||||||
writeCString(", ", ostr);
|
writeCString(", ", ostr);
|
||||||
(*it)->appendColumnName(ostr);
|
|
||||||
|
if (settings)
|
||||||
|
(*it)->appendColumnName(ostr, *settings);
|
||||||
|
else
|
||||||
|
(*it)->appendColumnName(ostr);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
writeChar(')', ostr);
|
writeChar(')', ostr);
|
||||||
|
|
||||||
if (is_window_function)
|
if (is_window_function)
|
||||||
@ -61,11 +82,11 @@ void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
FormatSettings settings{ostr, true /* one_line */};
|
FormatSettings format_settings{ostr, true /* one_line */};
|
||||||
FormatState state;
|
FormatState state;
|
||||||
FormatStateStacked frame;
|
FormatStateStacked frame;
|
||||||
writeCString("(", ostr);
|
writeCString("(", ostr);
|
||||||
window_definition->formatImpl(settings, state, frame);
|
window_definition->formatImpl(format_settings, state, frame);
|
||||||
writeCString(")", ostr);
|
writeCString(")", ostr);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -54,6 +54,10 @@ public:
|
|||||||
protected:
|
protected:
|
||||||
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||||
void appendColumnNameImpl(WriteBuffer & ostr) const override;
|
void appendColumnNameImpl(WriteBuffer & ostr) const override;
|
||||||
|
void appendColumnNameImpl(WriteBuffer & ostr, const Settings & settings) const override;
|
||||||
|
|
||||||
|
private:
|
||||||
|
void appendColumnNameImpl(WriteBuffer & ostr, const Settings * settings) const;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
@ -17,8 +17,10 @@ void ASTLiteral::updateTreeHashImpl(SipHash & hash_state) const
|
|||||||
applyVisitor(FieldVisitorHash(hash_state), value);
|
applyVisitor(FieldVisitorHash(hash_state), value);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
/// Writes 'tuple' word before tuple literals for backward compatibility reasons.
|
/// Writes 'tuple' word before tuple literals for backward compatibility reasons.
|
||||||
/// TODO: remove, when versions lower than 20.3 will be rarely used.
|
|
||||||
class FieldVisitorToColumnName : public StaticVisitor<String>
|
class FieldVisitorToColumnName : public StaticVisitor<String>
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
@ -46,14 +48,51 @@ String FieldVisitorToColumnName::operator() (const Tuple & x) const
|
|||||||
return wb.str();
|
return wb.str();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr, const Settings & settings) const
|
||||||
|
{
|
||||||
|
if (settings.legacy_column_name_of_tuple_literal)
|
||||||
|
appendColumnNameImplLegacy(ostr);
|
||||||
|
else
|
||||||
|
appendColumnNameImpl(ostr);
|
||||||
|
}
|
||||||
|
|
||||||
void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const
|
void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const
|
||||||
{
|
{
|
||||||
/// 100 - just arbitrary value.
|
/// 100 - just arbitrary value.
|
||||||
constexpr auto min_elements_for_hashing = 100;
|
constexpr auto min_elements_for_hashing = 100;
|
||||||
|
|
||||||
|
/// Special case for very large arrays and tuples. Instead of listing all elements, will use hash of them.
|
||||||
|
/// (Otherwise column name will be too long, that will lead to significant slowdown of expression analysis.)
|
||||||
|
auto type = value.getType();
|
||||||
|
if ((type == Field::Types::Array && value.get<const Array &>().size() > min_elements_for_hashing)
|
||||||
|
|| (type == Field::Types::Tuple && value.get<const Tuple &>().size() > min_elements_for_hashing))
|
||||||
|
{
|
||||||
|
SipHash hash;
|
||||||
|
applyVisitor(FieldVisitorHash(hash), value);
|
||||||
|
UInt64 low, high;
|
||||||
|
hash.get128(low, high);
|
||||||
|
|
||||||
|
writeCString(type == Field::Types::Array ? "__array_" : "__tuple_", ostr);
|
||||||
|
writeText(low, ostr);
|
||||||
|
ostr.write('_');
|
||||||
|
writeText(high, ostr);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
String column_name = applyVisitor(FieldVisitorToString(), value);
|
||||||
|
writeString(column_name, ostr);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void ASTLiteral::appendColumnNameImplLegacy(WriteBuffer & ostr) const
|
||||||
|
{
|
||||||
|
/// 100 - just arbitrary value.
|
||||||
|
constexpr auto min_elements_for_hashing = 100;
|
||||||
|
|
||||||
/// Special case for very large arrays. Instead of listing all elements, will use hash of them.
|
/// Special case for very large arrays. Instead of listing all elements, will use hash of them.
|
||||||
/// (Otherwise column name will be too long, that will lead to significant slowdown of expression analysis.)
|
/// (Otherwise column name will be too long, that will lead to significant slowdown of expression analysis.)
|
||||||
/// TODO: Also do hashing for large tuples, when versions lower than 20.3 will be rarely used, because it breaks backward compatibility.
|
|
||||||
auto type = value.getType();
|
auto type = value.getType();
|
||||||
if ((type == Field::Types::Array && value.get<const Array &>().size() > min_elements_for_hashing))
|
if ((type == Field::Types::Array && value.get<const Array &>().size() > min_elements_for_hashing))
|
||||||
{
|
{
|
||||||
|
@ -44,6 +44,13 @@ protected:
|
|||||||
void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
|
void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
|
||||||
|
|
||||||
void appendColumnNameImpl(WriteBuffer & ostr) const override;
|
void appendColumnNameImpl(WriteBuffer & ostr) const override;
|
||||||
|
void appendColumnNameImpl(WriteBuffer & ostr, const Settings & settings) const override;
|
||||||
|
|
||||||
|
private:
|
||||||
|
/// Legacy version of 'appendColumnNameImpl'. It differs only with tuple literals.
|
||||||
|
/// It's only needed to continue working of queries with tuple literals
|
||||||
|
/// in distributed tables while rolling update.
|
||||||
|
void appendColumnNameImplLegacy(WriteBuffer & ostr) const;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -48,6 +48,14 @@ void ASTWithAlias::appendColumnName(WriteBuffer & ostr) const
|
|||||||
appendColumnNameImpl(ostr);
|
appendColumnNameImpl(ostr);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void ASTWithAlias::appendColumnName(WriteBuffer & ostr, const Settings & settings) const
|
||||||
|
{
|
||||||
|
if (prefer_alias_to_column_name && !alias.empty())
|
||||||
|
writeString(alias, ostr);
|
||||||
|
else
|
||||||
|
appendColumnNameImpl(ostr, settings);
|
||||||
|
}
|
||||||
|
|
||||||
void ASTWithAlias::appendColumnNameWithoutAlias(WriteBuffer & ostr) const
|
void ASTWithAlias::appendColumnNameWithoutAlias(WriteBuffer & ostr) const
|
||||||
{
|
{
|
||||||
appendColumnNameImpl(ostr);
|
appendColumnNameImpl(ostr);
|
||||||
|
@ -21,8 +21,10 @@ public:
|
|||||||
using IAST::IAST;
|
using IAST::IAST;
|
||||||
|
|
||||||
void appendColumnName(WriteBuffer & ostr) const final;
|
void appendColumnName(WriteBuffer & ostr) const final;
|
||||||
|
void appendColumnName(WriteBuffer & ostr, const Settings & settings) const final;
|
||||||
void appendColumnNameWithoutAlias(WriteBuffer & ostr) const final;
|
void appendColumnNameWithoutAlias(WriteBuffer & ostr) const final;
|
||||||
String getAliasOrColumnName() const override { return alias.empty() ? getColumnName() : alias; }
|
String getAliasOrColumnName() const override { return alias.empty() ? getColumnName() : alias; }
|
||||||
|
String getAliasOrColumnName(const Settings & settings) const override { return alias.empty() ? getColumnName(settings) : alias; }
|
||||||
String tryGetAlias() const override { return alias; }
|
String tryGetAlias() const override { return alias; }
|
||||||
void setAlias(const String & to) override { alias = to; }
|
void setAlias(const String & to) override { alias = to; }
|
||||||
|
|
||||||
@ -33,6 +35,7 @@ public:
|
|||||||
|
|
||||||
protected:
|
protected:
|
||||||
virtual void appendColumnNameImpl(WriteBuffer & ostr) const = 0;
|
virtual void appendColumnNameImpl(WriteBuffer & ostr) const = 0;
|
||||||
|
virtual void appendColumnNameImpl(WriteBuffer & ostr, const Settings &) const { appendColumnNameImpl(ostr); }
|
||||||
};
|
};
|
||||||
|
|
||||||
/// helper for setting aliases and chaining result to other functions
|
/// helper for setting aliases and chaining result to other functions
|
||||||
|
@ -109,6 +109,14 @@ String IAST::getColumnName() const
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
String IAST::getColumnName(const Settings & settings) const
|
||||||
|
{
|
||||||
|
WriteBufferFromOwnString write_buffer;
|
||||||
|
appendColumnName(write_buffer, settings);
|
||||||
|
return write_buffer.str();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
String IAST::getColumnNameWithoutAlias() const
|
String IAST::getColumnNameWithoutAlias() const
|
||||||
{
|
{
|
||||||
WriteBufferFromOwnString write_buffer;
|
WriteBufferFromOwnString write_buffer;
|
||||||
|
@ -5,6 +5,7 @@
|
|||||||
#include <Parsers/IdentifierQuotingStyle.h>
|
#include <Parsers/IdentifierQuotingStyle.h>
|
||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
#include <Common/TypePromotion.h>
|
#include <Common/TypePromotion.h>
|
||||||
|
#include <Core/Settings.h>
|
||||||
#include <IO/WriteBufferFromString.h>
|
#include <IO/WriteBufferFromString.h>
|
||||||
|
|
||||||
#include <algorithm>
|
#include <algorithm>
|
||||||
@ -41,13 +42,18 @@ public:
|
|||||||
|
|
||||||
/** Get the canonical name of the column if the element is a column */
|
/** Get the canonical name of the column if the element is a column */
|
||||||
String getColumnName() const;
|
String getColumnName() const;
|
||||||
|
String getColumnName(const Settings & settings) const;
|
||||||
|
|
||||||
/** Same as the above but ensure no alias names are used. This is for index analysis */
|
/** Same as the above but ensure no alias names are used. This is for index analysis */
|
||||||
String getColumnNameWithoutAlias() const;
|
String getColumnNameWithoutAlias() const;
|
||||||
|
|
||||||
virtual void appendColumnName(WriteBuffer &) const
|
virtual void appendColumnName(WriteBuffer &) const
|
||||||
{
|
{
|
||||||
throw Exception("Trying to get name of not a column: " + getID(), ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Trying to get name of not a column: " + getID(), ErrorCodes::LOGICAL_ERROR);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
virtual void appendColumnName(WriteBuffer & ostr, const Settings &) const { appendColumnName(ostr); }
|
||||||
|
|
||||||
virtual void appendColumnNameWithoutAlias(WriteBuffer &) const
|
virtual void appendColumnNameWithoutAlias(WriteBuffer &) const
|
||||||
{
|
{
|
||||||
throw Exception("Trying to get name of not a column: " + getID(), ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Trying to get name of not a column: " + getID(), ErrorCodes::LOGICAL_ERROR);
|
||||||
@ -55,6 +61,7 @@ public:
|
|||||||
|
|
||||||
/** Get the alias, if any, or the canonical name of the column, if it is not. */
|
/** Get the alias, if any, or the canonical name of the column, if it is not. */
|
||||||
virtual String getAliasOrColumnName() const { return getColumnName(); }
|
virtual String getAliasOrColumnName() const { return getColumnName(); }
|
||||||
|
virtual String getAliasOrColumnName(const Settings & settings) const { return getColumnName(settings); }
|
||||||
|
|
||||||
/** Get the alias, if any, or an empty string if it does not exist, or if the element does not support aliases. */
|
/** Get the alias, if any, or an empty string if it does not exist, or if the element does not support aliases. */
|
||||||
virtual String tryGetAlias() const { return String(); }
|
virtual String tryGetAlias() const { return String(); }
|
||||||
|
@ -0,0 +1,7 @@
|
|||||||
|
<yandex>
|
||||||
|
<profiles>
|
||||||
|
<default>
|
||||||
|
<legacy_column_name_of_tuple_literal>1</legacy_column_name_of_tuple_literal>
|
||||||
|
</default>
|
||||||
|
</profiles>
|
||||||
|
</yandex>
|
@ -5,8 +5,8 @@ from helpers.cluster import ClickHouseCluster
|
|||||||
cluster = ClickHouseCluster(__file__)
|
cluster = ClickHouseCluster(__file__)
|
||||||
|
|
||||||
node_old = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], image='yandex/clickhouse-server',
|
node_old = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], image='yandex/clickhouse-server',
|
||||||
tag='19.17.8.54', stay_alive=True, with_installed_binary=True)
|
tag='20.8.9.6', stay_alive=True, with_installed_binary=True)
|
||||||
node_new = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'])
|
node_new = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'], user_configs=['configs/legacy.xml'])
|
||||||
|
|
||||||
|
|
||||||
@pytest.fixture(scope="module")
|
@pytest.fixture(scope="module")
|
||||||
|
@ -14,7 +14,7 @@
|
|||||||
"type": "Array(UInt8)"
|
"type": "Array(UInt8)"
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"name": "tuple(1, 'a')",
|
"name": "(1, 'a')",
|
||||||
"type": "Tuple(UInt8, String)"
|
"type": "Tuple(UInt8, String)"
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
@ -33,7 +33,7 @@
|
|||||||
"1": "1",
|
"1": "1",
|
||||||
"'a'": "a",
|
"'a'": "a",
|
||||||
"[1, 2, 3]": "[1,2,3]",
|
"[1, 2, 3]": "[1,2,3]",
|
||||||
"tuple(1, 'a')": "(1,'a')",
|
"(1, 'a')": "(1,'a')",
|
||||||
"NULL": "ᴺᵁᴸᴸ",
|
"NULL": "ᴺᵁᴸᴸ",
|
||||||
"nan": "nan"
|
"nan": "nan"
|
||||||
}
|
}
|
||||||
|
@ -14,7 +14,7 @@
|
|||||||
"type": "Array(UInt8)"
|
"type": "Array(UInt8)"
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"name": "tuple(1, 'a')",
|
"name": "(1, 'a')",
|
||||||
"type": "Tuple(UInt8, String)"
|
"type": "Tuple(UInt8, String)"
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
|
@ -0,0 +1,4 @@
|
|||||||
|
((1, 2), (2, 3), (3, 4))
|
||||||
|
((1,2),(2,3),(3,4))
|
||||||
|
tuple(tuple(1, 2), tuple(2, 3), tuple(3, 4))
|
||||||
|
((1,2),(2,3),(3,4))
|
@ -0,0 +1,2 @@
|
|||||||
|
SELECT ((1, 2), (2, 3), (3, 4)) FORMAT TSVWithNames;
|
||||||
|
SELECT ((1, 2), (2, 3), (3, 4)) FORMAT TSVWithNames SETTINGS legacy_column_name_of_tuple_literal = 1;
|
@ -653,7 +653,7 @@ a
|
|||||||
"""
|
"""
|
||||||
|
|
||||||
Inline___Int128___arrayReduceInRanges__sum_____1__5____ = r"""
|
Inline___Int128___arrayReduceInRanges__sum_____1__5____ = r"""
|
||||||
arrayReduceInRanges(\'sum\', array(tuple(1, 5)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')))
|
arrayReduceInRanges(\'sum\', array((1, 5)), array(toInt128(\'3\'), toInt128(\'2\'), toInt128(\'1\')))
|
||||||
[6]
|
[6]
|
||||||
"""
|
"""
|
||||||
|
|
||||||
@ -1181,7 +1181,7 @@ a
|
|||||||
"""
|
"""
|
||||||
|
|
||||||
Inline___Int256___arrayReduceInRanges__sum_____1__5____ = r"""
|
Inline___Int256___arrayReduceInRanges__sum_____1__5____ = r"""
|
||||||
arrayReduceInRanges(\'sum\', array(tuple(1, 5)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')))
|
arrayReduceInRanges(\'sum\', array((1, 5)), array(toInt256(\'3\'), toInt256(\'2\'), toInt256(\'1\')))
|
||||||
[6]
|
[6]
|
||||||
"""
|
"""
|
||||||
|
|
||||||
@ -1709,7 +1709,7 @@ a
|
|||||||
"""
|
"""
|
||||||
|
|
||||||
Inline___UInt128___arrayReduceInRanges__sum_____1__5____ = r"""
|
Inline___UInt128___arrayReduceInRanges__sum_____1__5____ = r"""
|
||||||
arrayReduceInRanges(\'sum\', array(tuple(1, 5)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')))
|
arrayReduceInRanges(\'sum\', array((1, 5)), array(toUInt128(\'3\'), toUInt128(\'2\'), toUInt128(\'1\')))
|
||||||
[6]
|
[6]
|
||||||
"""
|
"""
|
||||||
|
|
||||||
@ -2237,7 +2237,7 @@ a
|
|||||||
"""
|
"""
|
||||||
|
|
||||||
Inline___UInt256___arrayReduceInRanges__sum_____1__5____ = r"""
|
Inline___UInt256___arrayReduceInRanges__sum_____1__5____ = r"""
|
||||||
arrayReduceInRanges(\'sum\', array(tuple(1, 5)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')))
|
arrayReduceInRanges(\'sum\', array((1, 5)), array(toUInt256(\'3\'), toUInt256(\'2\'), toUInt256(\'1\')))
|
||||||
[6]
|
[6]
|
||||||
"""
|
"""
|
||||||
|
|
||||||
@ -2765,7 +2765,7 @@ a
|
|||||||
"""
|
"""
|
||||||
|
|
||||||
Inline___Decimal256_0____arrayReduceInRanges__sum_____1__5____ = r"""
|
Inline___Decimal256_0____arrayReduceInRanges__sum_____1__5____ = r"""
|
||||||
arrayReduceInRanges(\'sum\', array(tuple(1, 5)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)))
|
arrayReduceInRanges(\'sum\', array((1, 5)), array(toDecimal256(\'3\', 0), toDecimal256(\'2\', 0), toDecimal256(\'1\', 0)))
|
||||||
[6]
|
[6]
|
||||||
"""
|
"""
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user