Merge branch 'master' into storage-merge-aliases

This commit is contained in:
Kseniia Sumarokova 2021-06-25 17:33:31 +03:00 committed by GitHub
commit 1eb005850c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
44 changed files with 472 additions and 101 deletions

View File

@ -2446,6 +2446,8 @@ public:
{
/// param_name value
++arg_num;
if (arg_num >= argc)
throw Exception("Parameter requires value", ErrorCodes::BAD_ARGUMENTS);
arg = argv[arg_num];
query_parameters.emplace(String(param_continuation), String(arg));
}

View File

@ -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(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, 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(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) \

View File

@ -3,9 +3,10 @@
#include <city.h>
#include <farmhash.h>
#include <metrohash.h>
#include <MurmurHash2.h>
#include <MurmurHash3.h>
#if !defined(ARCADIA_BUILD)
# include <MurmurHash2.h>
# include <MurmurHash3.h>
# include "config_functions.h"
# include "config_core.h"
#endif

View File

@ -15,6 +15,7 @@ ADDINCL(
contrib/libs/libdivide
contrib/libs/rapidjson/include
contrib/libs/xxhash
contrib/restricted/murmurhash
)
PEERDIR(
@ -30,6 +31,7 @@ PEERDIR(
contrib/libs/metrohash
contrib/libs/rapidjson
contrib/libs/xxhash
contrib/restricted/murmurhash
library/cpp/consistent_hashing
)

View File

@ -14,6 +14,7 @@ ADDINCL(
contrib/libs/libdivide
contrib/libs/rapidjson/include
contrib/libs/xxhash
contrib/restricted/murmurhash
)
PEERDIR(
@ -29,6 +30,7 @@ PEERDIR(
contrib/libs/metrohash
contrib/libs/rapidjson
contrib/libs/xxhash
contrib/restricted/murmurhash
library/cpp/consistent_hashing
)

View File

@ -212,6 +212,7 @@ public:
/// Conversion should be possible with only usage of CAST function and renames.
/// @param ignore_constant_values - Do not check that constants are same. Use value from result_header.
/// @param add_casted_columns - Create new columns with converted values instead of replacing original.
/// @param new_names - Output parameter for new column names when add_casted_columns is used.
static ActionsDAGPtr makeConvertingActions(
const ColumnsWithTypeAndName & source,
const ColumnsWithTypeAndName & result,

View File

@ -348,7 +348,7 @@ SetPtr makeExplicitSet(
const ASTPtr & left_arg = args.children.at(0);
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 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.
// 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>();
if (as_literal)
{
@ -703,7 +703,7 @@ ASTs ActionsMatcher::doUntuple(const ASTFunction * function, ActionsMatcher::Dat
func->setAlias(data.getUniqueName("_ut_" + name));
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));
}
@ -740,6 +740,7 @@ void ActionsMatcher::visit(ASTExpressionList & expression_list, const ASTPtr &,
void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr &, Data & data)
{
auto column_name = identifier.getColumnName();
if (data.hasColumn(column_name))
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)
{
auto column_name = ast->getColumnName();
auto column_name = ast->getColumnName(data.getContext()->getSettingsRef());
if (data.hasColumn(column_name))
return;
@ -782,7 +783,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
ASTPtr arg = node.arguments->children.at(0);
visit(arg, data);
if (!data.only_consts)
data.addArrayJoin(arg->getColumnName(), column_name);
data.addArrayJoin(arg->getColumnName(data.getContext()->getSettingsRef()), column_name);
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.
/// 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(
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())
column.name = data.getUniqueName("__set");
else
column.name = child->getColumnName();
column.name = child->getColumnName(data.getContext()->getSettingsRef());
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);
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));
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())
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).
String lambda_name = data.getUniqueName("__lambda");
@ -1057,7 +1058,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
if (arguments_present)
{
/// 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.
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 * 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.
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];
@ -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 & 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.
return makeExplicitSet(&node, last_actions, false, data.getContext(), data.set_size_limit, data.prepared_sets);
else

View File

@ -244,7 +244,7 @@ void ExpressionAnalyzer::analyzeAggregation()
ssize_t size = group_asts.size();
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);
if (!node)
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);
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);
}
}
@ -446,7 +446,7 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions)
if (node->arguments)
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();
aggregate.argument_names.resize(arguments.size());
@ -454,7 +454,7 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions)
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);
if (!dag_node)
{
@ -647,7 +647,7 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
WindowFunctionDescription window_function;
window_function.function_node = function_node;
window_function.column_name
= window_function.function_node->getColumnName();
= window_function.function_node->getColumnName(getContext()->getSettingsRef());
window_function.function_parameters
= window_function.function_node->parameters
? getAggregateFunctionParametersArray(
@ -666,7 +666,7 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
window_function.argument_names.resize(arguments.size());
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);
if (!node)
@ -964,7 +964,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere(
auto & step = chain.lastStep(sourceColumns());
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);
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());
auto where_column_name = select_query->where()->getColumnName();
auto where_column_name = select_query->where()->getColumnName(getContext()->getSettingsRef());
step.addRequiredOutput(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;
for (const auto & ast : asts)
{
step.addRequiredOutput(ast->getColumnName());
step.addRequiredOutput(ast->getColumnName(getContext()->getSettingsRef()));
getRootActions(ast, only_types, step.actions());
}
@ -1114,7 +1114,7 @@ void SelectQueryExpressionAnalyzer::appendAggregateFunctionsArguments(Expression
for (const auto & name : desc.argument_names)
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)
/// @note The original recollection logic didn't remove duplicates.
GetAggregatesVisitor::Data data;
@ -1169,7 +1169,7 @@ void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments(
// (2b) Required function argument columns.
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);
getRootActionsForHaving(select_query->having(), only_types, step.actions());
step.addRequiredOutput(select_query->having()->getColumnName());
step.addRequiredOutput(select_query->having()->getColumnName(getContext()->getSettingsRef()));
return true;
}
@ -1215,7 +1215,7 @@ void SelectQueryExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain,
continue;
}
step.addRequiredOutput(child->getColumnName());
step.addRequiredOutput(child->getColumnName(getContext()->getSettingsRef()));
}
}
@ -1243,7 +1243,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai
if (!ast || ast->children.empty())
throw Exception("Bad order expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE);
ASTPtr order_expression = ast->children.at(0);
step.addRequiredOutput(order_expression->getColumnName());
step.addRequiredOutput(order_expression->getColumnName(getContext()->getSettingsRef()));
if (ast->with_fill)
with_fill = true;
@ -1293,7 +1293,7 @@ bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain
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))
step.addRequiredOutput(std::move(child_name));
}
@ -1309,13 +1309,15 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActio
NamesWithAliases result_columns;
const auto & settings = getContext()->getSettingsRef();
ASTs asts = select_query->select()->children;
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))
{
std::string source_name = ast->getColumnName();
std::string source_name = ast->getColumnName(settings);
/*
* For temporary columns created by ExpressionAnalyzer for literals,
@ -1357,7 +1359,7 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const
{
ExpressionActionsChain::Step & step = chain.lastStep(sourceColumns());
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
asts = ASTs(1, query);
const auto & settings = getContext()->getSettingsRef();
for (const auto & ast : asts)
{
std::string name = ast->getColumnName();
std::string name = ast->getColumnName(settings);
std::string alias;
if (add_aliases)
alias = ast->getAliasOrColumnName();
alias = ast->getAliasOrColumnName(settings);
else
alias = name;
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))
{
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))
{
@ -1524,7 +1527,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
ExpressionActions(
prewhere_info->prewhere_actions,
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 (column_elem.column)
prewhere_constant_filter_description = ConstantFilterDescription(*column_elem.column);
@ -1559,7 +1562,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
ExpressionActions(
before_where,
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 (column_elem.column)
where_constant_filter_description = ConstantFilterDescription(*column_elem.column);
@ -1650,7 +1653,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
const auto * select_query = query_analyzer.getSelectQuery();
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())
{
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;
}
}

View File

@ -143,7 +143,7 @@ String InterpreterSelectQuery::generateFilterActions(ActionsDAGPtr & actions, co
SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context, metadata_snapshot);
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->projectInput(false);
@ -779,7 +779,7 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP
order_descr.reserve(query.orderBy()->children.size());
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 &>();
std::shared_ptr<Collator> collator;
@ -798,14 +798,14 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, ContextP
return order_descr;
}
static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & query)
static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & query, ContextPtr context)
{
SortDescription order_descr;
order_descr.reserve(query.groupBy()->children.size());
for (const auto & elem : query.groupBy()->children)
{
String name = elem->getColumnName();
String name = elem->getColumnName(context->getSettingsRef());
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->group_by_elements_actions,
getSortDescriptionFromGroupBy(query),
getSortDescriptionFromGroupBy(query, context),
query_info.syntax_analyzer_result);
}
else
{
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)
{
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");
query_plan.addStep(std::move(where_step));
@ -2080,7 +2080,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac
SortDescription group_by_sort_description;
if (group_by_info && settings.optimize_aggregation_in_order)
group_by_sort_description = getSortDescriptionFromGroupBy(getSelectQuery());
group_by_sort_description = getSortDescriptionFromGroupBy(getSelectQuery(), context);
else
group_by_info = nullptr;
@ -2128,7 +2128,7 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool
void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression)
{
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");
query_plan.addStep(std::move(having_step));
@ -2144,7 +2144,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving(
query_plan.getCurrentDataStream(),
overflow_row,
expression,
has_having ? getSelectQuery().having()->getColumnName() : "",
has_having ? getSelectQuery().having()->getColumnName(context->getSettingsRef()) : "",
settings.totals_mode,
settings.totals_auto_threshold,
final);
@ -2461,7 +2461,7 @@ void InterpreterSelectQuery::executeLimitBy(QueryPlan & query_plan)
Names columns;
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 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), context, "OFFSET") : 0);

View File

@ -39,7 +39,7 @@ std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(co
if (context->getSettingsRef().normalize_function_names)
FunctionNameNormalizer().visit(ast.get());
String name = ast->getColumnName();
String name = ast->getColumnName(context->getSettingsRef());
auto syntax_result = TreeRewriter(context).analyze(ast, source_columns);
ExpressionActionsPtr expr_for_constant_folding = ExpressionAnalyzer(ast, syntax_result, context).getConstActions();

View File

@ -24,6 +24,16 @@ namespace ErrorCodes
}
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")
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())
writeCString(", ", ostr);
(*it)->appendColumnName(ostr);
if (settings)
(*it)->appendColumnName(ostr, *settings);
else
(*it)->appendColumnName(ostr);
}
writeChar(')', ostr);
}
writeChar('(', ostr);
if (arguments)
{
for (auto it = arguments->children.begin(); it != arguments->children.end(); ++it)
{
if (it != arguments->children.begin())
writeCString(", ", ostr);
(*it)->appendColumnName(ostr);
if (settings)
(*it)->appendColumnName(ostr, *settings);
else
(*it)->appendColumnName(ostr);
}
}
writeChar(')', ostr);
if (is_window_function)
@ -61,11 +82,11 @@ void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const
}
else
{
FormatSettings settings{ostr, true /* one_line */};
FormatSettings format_settings{ostr, true /* one_line */};
FormatState state;
FormatStateStacked frame;
writeCString("(", ostr);
window_definition->formatImpl(settings, state, frame);
window_definition->formatImpl(format_settings, state, frame);
writeCString(")", ostr);
}
}

View File

@ -54,6 +54,10 @@ public:
protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) 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;
};

View File

@ -17,8 +17,10 @@ void ASTLiteral::updateTreeHashImpl(SipHash & hash_state) const
applyVisitor(FieldVisitorHash(hash_state), value);
}
namespace
{
/// 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>
{
public:
@ -46,14 +48,51 @@ String FieldVisitorToColumnName::operator() (const Tuple & x) const
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
{
/// 100 - just arbitrary value.
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.
/// (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();
if ((type == Field::Types::Array && value.get<const Array &>().size() > min_elements_for_hashing))
{

View File

@ -44,6 +44,13 @@ protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) 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;
};
}

View File

@ -48,6 +48,14 @@ void ASTWithAlias::appendColumnName(WriteBuffer & ostr) const
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
{
appendColumnNameImpl(ostr);

View File

@ -21,8 +21,10 @@ public:
using IAST::IAST;
void appendColumnName(WriteBuffer & ostr) const final;
void appendColumnName(WriteBuffer & ostr, const Settings & settings) const final;
void appendColumnNameWithoutAlias(WriteBuffer & ostr) const final;
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; }
void setAlias(const String & to) override { alias = to; }
@ -33,6 +35,7 @@ public:
protected:
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

View File

@ -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
{
WriteBufferFromOwnString write_buffer;

View File

@ -5,6 +5,7 @@
#include <Parsers/IdentifierQuotingStyle.h>
#include <Common/Exception.h>
#include <Common/TypePromotion.h>
#include <Core/Settings.h>
#include <IO/WriteBufferFromString.h>
#include <algorithm>
@ -41,13 +42,18 @@ public:
/** Get the canonical name of the column if the element is a column */
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 */
String getColumnNameWithoutAlias() const;
virtual void appendColumnName(WriteBuffer &) const
{
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
{
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. */
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. */
virtual String tryGetAlias() const { return String(); }

View File

@ -2331,7 +2331,14 @@ MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet(c
if (part->info.partition_id != drop_range.partition_id)
throw Exception("Unexpected partition_id of part " + part->name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR);
if (part->info.min_block < drop_range.min_block) /// NOTE Always false, because drop_range.min_block == 0
/// It's a DROP PART and it's already executed by fetching some covering part
if (part->info != drop_range && part->info.contains(drop_range))
{
LOG_INFO(log, "Skipping drop range for part {} because covering part {} already exists", drop_range.getPartName(), part->name);
return {};
}
if (part->info.min_block < drop_range.min_block)
{
if (drop_range.min_block <= part->info.max_block)
{
@ -2658,7 +2665,6 @@ void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until) const
std::this_thread::sleep_for(std::chrono::milliseconds(static_cast<size_t>(delay_milliseconds)));
}
MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(
const MergeTreePartInfo & part_info, MergeTreeData::DataPartState state, DataPartsLock & /*lock*/) const
{
@ -2760,7 +2766,6 @@ MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVectorInPartition(Merg
data_parts_by_state_and_info.upper_bound(state_with_partition));
}
MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const MergeTreePartInfo & part_info, const MergeTreeData::DataPartStates & valid_states)
{
auto lock = lockParts();

View File

@ -264,6 +264,10 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge(
if (!can_merge_callback(nullptr, part, nullptr))
continue;
/// This part can be merged only with next parts (no prev part exists), so start
/// new interval if previous was not empty.
if (!parts_ranges.back().empty())
parts_ranges.emplace_back();
}
else
{
@ -271,12 +275,21 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge(
/// interval (in the same partition)
if (!can_merge_callback(*prev_part, part, nullptr))
{
/// Starting new interval in the same partition
assert(!parts_ranges.back().empty());
parts_ranges.emplace_back();
/// Now we have no previous part, but it affects only logging
/// Now we have no previous part
prev_part = nullptr;
/// Mustn't be empty
assert(!parts_ranges.back().empty());
/// Some parts cannot be merged with previous parts and also cannot be merged with themselves,
/// for example, merge is already assigned for such parts, or they participate in quorum inserts
/// and so on.
/// Also we don't start new interval here (maybe all next parts cannot be merged and we don't want to have empty interval)
if (!can_merge_callback(nullptr, part, nullptr))
continue;
/// Starting new interval in the same partition
parts_ranges.emplace_back();
}
}

View File

@ -137,14 +137,21 @@ void ReplicatedMergeTreeQueue::insertUnlocked(
for (const String & virtual_part_name : entry->getVirtualPartNames(format_version))
{
virtual_parts.add(virtual_part_name, nullptr, log);
addPartToMutations(virtual_part_name);
/// Don't add drop range parts to mutations
/// they don't produce any useful parts
if (entry->type != LogEntry::DROP_RANGE)
addPartToMutations(virtual_part_name);
}
/// Put 'DROP PARTITION' entries at the beginning of the queue not to make superfluous fetches of parts that will be eventually deleted
if (entry->type != LogEntry::DROP_RANGE)
{
queue.push_back(entry);
}
else
{
queue.push_front(entry);
}
if (entry->type == LogEntry::GET_PART || entry->type == LogEntry::ATTACH_PART)
{
@ -891,6 +898,10 @@ bool ReplicatedMergeTreeQueue::checkReplaceRangeCanBeRemoved(const MergeTreePart
if (entry_ptr->replace_range_entry == current.replace_range_entry) /// same partition, don't want to drop ourselves
return false;
if (!part_info.contains(MergeTreePartInfo::fromPartName(entry_ptr->replace_range_entry->drop_range_part_name, format_version)))
return false;
size_t number_of_covered_parts = 0;
for (const String & new_part_name : entry_ptr->replace_range_entry->new_part_names)
{

View File

@ -1155,6 +1155,7 @@ void StorageDistributed::renameOnDisk(const String & new_path_to_table_data)
{
for (const DiskPtr & disk : data_volume->getDisks())
{
disk->createDirectories(new_path_to_table_data);
disk->moveDirectory(relative_data_path, new_path_to_table_data);
auto new_path = disk->getPath() + new_path_to_table_data;

View File

@ -2196,11 +2196,6 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry)
auto drop_range_info = MergeTreePartInfo::fromPartName(entry.new_part_name, format_version);
queue.removePartProducingOpsInRange(getZooKeeper(), drop_range_info, entry);
if (entry.detach)
LOG_DEBUG(log, "Detaching parts.");
else
LOG_DEBUG(log, "Removing parts.");
/// Delete the parts contained in the range to be deleted.
/// It's important that no old parts remain (after the merge), because otherwise,
/// after adding a new replica, this new replica downloads them, but does not delete them.
@ -2212,8 +2207,15 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry)
{
auto data_parts_lock = lockParts();
parts_to_remove = removePartsInRangeFromWorkingSet(drop_range_info, true, data_parts_lock);
if (parts_to_remove.empty())
return;
}
if (entry.detach)
LOG_DEBUG(log, "Detaching parts.");
else
LOG_DEBUG(log, "Removing parts.");
if (entry.detach)
{
/// If DETACH clone parts to detached/ directory
@ -6992,15 +6994,16 @@ bool StorageReplicatedMergeTree::dropPartImpl(
getClearBlocksInPartitionOps(ops, *zookeeper, part_info.partition_id, part_info.min_block, part_info.max_block);
size_t clear_block_ops_size = ops.size();
/// Set fake level to treat this part as virtual in queue.
auto drop_part_info = part->info;
drop_part_info.level = MergeTreePartInfo::MAX_LEVEL;
/// If `part_name` is result of a recent merge and source parts are still available then
/// DROP_RANGE with detach will move this part together with source parts to `detached/` dir.
entry.type = LogEntry::DROP_RANGE;
entry.source_replica = replica_name;
entry.new_part_name = getPartNamePossiblyFake(format_version, drop_part_info);
/// We don't set fake drop level (999999999) for the single part DROP_RANGE.
/// First of all we don't guarantee anything other than the part will not be
/// active after DROP PART, but covering part (without data of dropped part) can exist.
/// If we add part with 9999999 level than we can break invariant in virtual_parts of
/// the queue.
entry.new_part_name = getPartNamePossiblyFake(format_version, part->info);
entry.detach = detach;
entry.create_time = time(nullptr);

View File

@ -31,7 +31,10 @@ namespace ErrorCodes
StorageView::StorageView(
const StorageID & table_id_, const ASTCreateQuery & query, const ColumnsDescription & columns_, const String & comment)
const StorageID & table_id_,
const ASTCreateQuery & query,
const ColumnsDescription & columns_,
const String & comment)
: IStorage(table_id_)
{
StorageInMemoryMetadata storage_metadata;
@ -40,7 +43,6 @@ StorageView::StorageView(
if (!query.select)
throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY);
SelectQueryDescription description;
description.inner_query = query.select->ptr();
@ -84,7 +86,12 @@ void StorageView::read(
current_inner_query = query_info.view_query->clone();
}
InterpreterSelectWithUnionQuery interpreter(current_inner_query, context, {}, column_names);
auto modified_context = Context::createCopy(context);
/// Use settings from global context,
/// because difference between settings set on VIEW creation and query execution can break queries
modified_context->setSettings(context->getGlobalContext()->getSettingsRef());
InterpreterSelectWithUnionQuery interpreter(current_inner_query, modified_context, {}, column_names);
interpreter.buildQueryPlan(query_plan);
/// It's expected that the columns read from storage are not constant.

View File

@ -49,7 +49,11 @@ public:
static ASTPtr restoreViewName(ASTSelectQuery & select_query, const ASTPtr & view_name);
protected:
StorageView(const StorageID & table_id_, const ASTCreateQuery & query, const ColumnsDescription & columns_, const String & comment);
StorageView(
const StorageID & table_id_,
const ASTCreateQuery & query,
const ColumnsDescription & columns_,
const String & comment);
};
}

View File

@ -42,7 +42,7 @@ StoragePtr TableFunctionView::executeImpl(
const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const
{
auto columns = getActualTableStructure(context);
auto res = StorageView::create(StorageID(getDatabaseName(), table_name), create, columns, String{});
auto res = StorageView::create(StorageID(getDatabaseName(), table_name), create, columns, "");
res->startup();
return res;
}

View File

@ -0,0 +1,7 @@
<yandex>
<profiles>
<default>
<legacy_column_name_of_tuple_literal>1</legacy_column_name_of_tuple_literal>
</default>
</profiles>
</yandex>

View File

@ -5,8 +5,8 @@ from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
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)
node_new = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'])
tag='20.8.9.6', stay_alive=True, with_installed_binary=True)
node_new = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'], user_configs=['configs/legacy.xml'])
@pytest.fixture(scope="module")

View File

@ -3,7 +3,9 @@ SET max_block_size = 100001;
SET group_by_overflow_mode = 'any';
DROP TABLE IF EXISTS numbers500k;
CREATE VIEW numbers500k AS SELECT number FROM system.numbers LIMIT 500000;
CREATE TABLE numbers500k (number UInt32) ENGINE = TinyLog;
INSERT INTO numbers500k SELECT number FROM system.numbers LIMIT 500000;
SET totals_mode = 'after_having_auto';
SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM remote('127.0.0.{2,3}', currentDatabase(), numbers500k) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10;

View File

@ -1,10 +1,12 @@
1 .inner.mv1 before moving tablesmv1
1 dist before moving tables
1 dst before moving tablesmv2
1 mv1 before moving tablesmv1
1 mv2 before moving tablesmv2
1 src before moving tables
ordinary:
.inner.mv1
dist
dst
mv1
mv2
@ -12,12 +14,16 @@ src
ordinary after rename:
atomic after rename:
.inner_id.
dist
dst
mv1
mv2
src
3 .inner_id. after renaming databasemv1
3 .inner_id. before moving tablesmv1
3 dist after moving tables
3 dist after renaming database
3 dist before moving tables
3 dst after renaming databasemv2
3 dst before moving tablesmv2
3 mv1 after renaming databasemv1
@ -28,6 +34,7 @@ src
3 src after renaming database
3 src before moving tables
.inner_id.
dist
dst
mv1
mv2
@ -36,6 +43,10 @@ CREATE DATABASE test_01155_atomic\nENGINE = Atomic
4 .inner.mv1 after renaming databasemv1
4 .inner.mv1 after renaming tablesmv1
4 .inner.mv1 before moving tablesmv1
4 dist after moving tables
4 dist after renaming database
4 dist after renaming tables
4 dist before moving tables
4 dst after renaming databasemv2
4 dst after renaming tablesmv2
4 dst before moving tablesmv2
@ -51,6 +62,7 @@ CREATE DATABASE test_01155_atomic\nENGINE = Atomic
4 src before moving tables
test_01155_ordinary:
.inner.mv1
dist
dst
mv1
mv2

View File

@ -9,8 +9,11 @@ CREATE TABLE src (s String) ENGINE=MergeTree() PARTITION BY tuple() ORDER BY s;
CREATE MATERIALIZED VIEW mv1 (s String) ENGINE=MergeTree() PARTITION BY tuple() ORDER BY s AS SELECT (*,).1 || 'mv1' as s FROM src;
CREATE TABLE dst (s String) ENGINE=MergeTree() PARTITION BY tuple() ORDER BY s;
CREATE MATERIALIZED VIEW mv2 TO dst (s String) AS SELECT (*,).1 || 'mv2' as s FROM src;
INSERT INTO src VALUES ('before moving tables');
SELECT 1, substr(_table, 1, 10), s FROM merge('test_01155_ordinary', '') ORDER BY _table, s;
CREATE TABLE dist (s String) Engine=Distributed(test_shard_localhost, test_01155_ordinary, src);
INSERT INTO dist VALUES ('before moving tables');
SYSTEM FLUSH DISTRIBUTED dist;
-- FIXME Cannot convert column `1` because it is non constant in source stream but must be constant in result
SELECT materialize(1), substr(_table, 1, 10), s FROM merge('test_01155_ordinary', '') ORDER BY _table, s;
-- Move tables with materialized views from Ordinary to Atomic
SELECT 'ordinary:';
@ -19,6 +22,7 @@ RENAME TABLE test_01155_ordinary.mv1 TO test_01155_atomic.mv1;
RENAME TABLE test_01155_ordinary.mv2 TO test_01155_atomic.mv2;
RENAME TABLE test_01155_ordinary.dst TO test_01155_atomic.dst;
RENAME TABLE test_01155_ordinary.src TO test_01155_atomic.src;
RENAME TABLE test_01155_ordinary.dist TO test_01155_atomic.dist;
SELECT 'ordinary after rename:';
SELECT substr(name, 1, 10) FROM system.tables WHERE database='test_01155_ordinary';
SELECT 'atomic after rename:';
@ -27,13 +31,14 @@ DROP DATABASE test_01155_ordinary;
USE default;
INSERT INTO test_01155_atomic.src VALUES ('after moving tables');
SELECT 2, substr(_table, 1, 10), s FROM merge('test_01155_atomic', '') ORDER BY _table, s; -- { serverError 81 }
SELECT materialize(2), substr(_table, 1, 10), s FROM merge('test_01155_atomic', '') ORDER BY _table, s; -- { serverError 81 }
RENAME DATABASE test_01155_atomic TO test_01155_ordinary;
USE test_01155_ordinary;
INSERT INTO src VALUES ('after renaming database');
SELECT 3, substr(_table, 1, 10), s FROM merge('test_01155_ordinary', '') ORDER BY _table, s;
INSERT INTO dist VALUES ('after renaming database');
SYSTEM FLUSH DISTRIBUTED dist;
SELECT materialize(3), substr(_table, 1, 10), s FROM merge('test_01155_ordinary', '') ORDER BY _table, s;
SELECT substr(name, 1, 10) FROM system.tables WHERE database='test_01155_ordinary';
@ -47,9 +52,11 @@ RENAME TABLE test_01155_atomic.mv1 TO test_01155_ordinary.mv1;
RENAME TABLE test_01155_atomic.mv2 TO test_01155_ordinary.mv2;
RENAME TABLE test_01155_atomic.dst TO test_01155_ordinary.dst;
RENAME TABLE test_01155_atomic.src TO test_01155_ordinary.src;
RENAME TABLE test_01155_atomic.dist TO test_01155_ordinary.dist;
INSERT INTO src VALUES ('after renaming tables');
SELECT 4, substr(_table, 1, 10), s FROM merge('test_01155_ordinary', '') ORDER BY _table, s;
INSERT INTO dist VALUES ('after renaming tables');
SYSTEM FLUSH DISTRIBUTED dist;
SELECT materialize(4), substr(_table, 1, 10), s FROM merge('test_01155_ordinary', '') ORDER BY _table, s;
SELECT 'test_01155_ordinary:';
SHOW TABLES FROM test_01155_ordinary;
SELECT 'test_01155_atomic:';

View File

@ -14,7 +14,7 @@
"type": "Array(UInt8)"
},
{
"name": "tuple(1, 'a')",
"name": "(1, 'a')",
"type": "Tuple(UInt8, String)"
},
{
@ -33,7 +33,7 @@
"1": "1",
"'a'": "a",
"[1, 2, 3]": "[1,2,3]",
"tuple(1, 'a')": "(1,'a')",
"(1, 'a')": "(1,'a')",
"NULL": "ᴺᵁᴸᴸ",
"nan": "nan"
}

View File

@ -14,7 +14,7 @@
"type": "Array(UInt8)"
},
{
"name": "tuple(1, 'a')",
"name": "(1, 'a')",
"type": "Tuple(UInt8, String)"
},
{

View File

@ -0,0 +1,34 @@
join_use_nulls = 1
-
1 11 0
2 12 22
3 0 23
-
1 11 0
2 12 22
3 0 23
-
1 11 \N
2 12 22
3 \N 23
-
1 11 0
2 12 22
3 0 23
join_use_nulls = 0
-
1 11 0
2 12 22
3 0 23
-
1 11 0
2 12 22
3 0 23
-
1 11 \N
2 12 22
3 \N 23
-
1 11 0
2 12 22
3 0 23

View File

@ -0,0 +1,61 @@
DROP TABLE IF EXISTS view_no_nulls;
DROP TABLE IF EXISTS view_no_nulls_set;
DROP TABLE IF EXISTS view_nulls_set;
DROP TABLE IF EXISTS view_nulls;
SET join_use_nulls = 0;
CREATE OR REPLACE VIEW view_no_nulls AS
SELECT * FROM ( SELECT number + 1 AS a, number + 11 AS b FROM numbers(2) ) AS t1
FULL JOIN ( SELECT number + 2 AS a, number + 22 AS c FROM numbers(2) ) AS t2
USING a ORDER BY a;
CREATE OR REPLACE VIEW view_nulls_set AS
SELECT * FROM ( SELECT number + 1 AS a, number + 11 AS b FROM numbers(2) ) AS t1
FULL JOIN ( SELECT number + 2 AS a, number + 22 AS c FROM numbers(2) ) AS t2
USING a ORDER BY a
SETTINGS join_use_nulls = 1;
SET join_use_nulls = 1;
CREATE OR REPLACE VIEW view_nulls AS
SELECT * FROM ( SELECT number + 1 AS a, number + 11 AS b FROM numbers(2) ) AS t1
FULL JOIN ( SELECT number + 2 AS a, number + 22 AS c FROM numbers(2) ) AS t2
USING a ORDER BY a;
CREATE OR REPLACE VIEW view_no_nulls_set AS
SELECT * FROM ( SELECT number + 1 AS a, number + 11 AS b FROM numbers(2) ) AS t1
FULL JOIN ( SELECT number + 2 AS a, number + 22 AS c FROM numbers(2) ) AS t2
USING a ORDER BY a
SETTINGS join_use_nulls = 0;
SET join_use_nulls = 1;
SELECT 'join_use_nulls = 1';
SELECT '-';
SELECT * FROM view_no_nulls;
SELECT '-';
SELECT * FROM view_no_nulls_set;
SELECT '-';
SELECT * FROM view_nulls_set;
SELECT '-';
SELECT * FROM view_nulls;
SET join_use_nulls = 0;
SELECT 'join_use_nulls = 0';
SELECT '-';
SELECT * FROM view_no_nulls;
SELECT '-';
SELECT * FROM view_no_nulls_set;
SELECT '-';
SELECT * FROM view_nulls_set;
SELECT '-';
SELECT * FROM view_nulls;
DROP TABLE IF EXISTS view_no_nulls;
DROP TABLE IF EXISTS view_no_nulls_set;
DROP TABLE IF EXISTS view_nulls_set;
DROP TABLE IF EXISTS view_nulls;

View File

@ -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))

View File

@ -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;

View File

@ -0,0 +1,73 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
NUM_REPLICAS=5
for i in $(seq 1 $NUM_REPLICAS); do
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS ttl_table$i" &
done
wait
for i in $(seq 1 $NUM_REPLICAS); do
$CLICKHOUSE_CLIENT -n --query "CREATE TABLE ttl_table$i(
key DateTime
)
ENGINE ReplicatedMergeTree('/test/01921_concurrent_ttl_and_normal_merges/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/ttl_table', '$i')
ORDER BY tuple()
TTL key + INTERVAL 1 SECOND
SETTINGS merge_with_ttl_timeout=1, max_replicated_merges_with_ttl_in_queue=100, max_number_of_merges_with_ttl_in_pool=100, cleanup_delay_period=1, cleanup_delay_period_random_add=0;"
done
function optimize_thread
{
while true; do
REPLICA=$(($RANDOM % 5 + 1))
$CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE ttl_table$REPLICA FINAl"
done
}
function insert_thread
{
while true; do
REPLICA=$(($RANDOM % 5 + 1))
$CLICKHOUSE_CLIENT --optimize_on_insert=0 --query "INSERT INTO ttl_table$REPLICA SELECT now() + rand() % 5 - rand() % 3 FROM numbers(5)"
$CLICKHOUSE_CLIENT --optimize_on_insert=0 --query "INSERT INTO ttl_table$REPLICA SELECT now() + rand() % 5 - rand() % 3 FROM numbers(5)"
$CLICKHOUSE_CLIENT --optimize_on_insert=0 --query "INSERT INTO ttl_table$REPLICA SELECT now() + rand() % 5 - rand() % 3 FROM numbers(5)"
done
}
export -f insert_thread;
export -f optimize_thread;
TIMEOUT=30
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c optimize_thread 2> /dev/null &
timeout $TIMEOUT bash -c optimize_thread 2> /dev/null &
timeout $TIMEOUT bash -c optimize_thread 2> /dev/null &
timeout $TIMEOUT bash -c optimize_thread 2> /dev/null &
timeout $TIMEOUT bash -c optimize_thread 2> /dev/null &
wait
for i in $(seq 1 $NUM_REPLICAS); do
$CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA ttl_table$i"
done
$CLICKHOUSE_CLIENT --query "SELECT * FROM system.replication_queue where table like 'ttl_table%' and database = '${CLICKHOUSE_DATABASE}' and type='MERGE_PARTS' and last_exception != '' FORMAT Vertical"
$CLICKHOUSE_CLIENT --query "SELECT COUNT() > 0 FROM system.part_log where table like 'ttl_table%' and database = '${CLICKHOUSE_DATABASE}'"
for i in $(seq 1 $NUM_REPLICAS); do
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS ttl_table$i" &
done
wait

View File

@ -0,0 +1,2 @@
foo
foo

View File

@ -0,0 +1,9 @@
#!/usr/bin/env bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
$CLICKHOUSE_CLIENT --param_foo |& grep -q -x 'Code: 36. DB::Exception: Parameter requires value'
$CLICKHOUSE_CLIENT --param_foo foo -q 'select {foo:String}'
$CLICKHOUSE_CLIENT -q 'select {foo:String}' --param_foo foo

View File

@ -92,6 +92,7 @@
01129_dict_get_join_lose_constness
01138_join_on_distributed_and_tmp
01153_attach_mv_uuid
01155_rename_move_materialized_view
01191_rename_dictionary
01200_mutations_memory_consumption
01211_optimize_skip_unused_shards_type_mismatch

View File

@ -843,6 +843,7 @@
"01870_buffer_flush", // creates database
"01889_postgresql_protocol_null_fields",
"01889_check_row_policy_defined_using_user_function",
"01925_test_storage_merge_aliases"
"01925_test_storage_merge_aliases",
"01921_concurrent_ttl_and_normal_merges_zookeeper_long" // heavy test, better to run sequentially
]
}

View File

@ -653,7 +653,7 @@ a
"""
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]
"""
@ -1181,7 +1181,7 @@ a
"""
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]
"""
@ -1709,7 +1709,7 @@ a
"""
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]
"""
@ -2237,7 +2237,7 @@ a
"""
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]
"""
@ -2765,7 +2765,7 @@ a
"""
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]
"""