Merge pull request #52146 from ClickHouse/more-changes

Fix ORDER BY tuple of WINDOW functions (and slightly more changes)
This commit is contained in:
Alexey Milovidov 2023-07-24 03:00:35 +03:00 committed by GitHub
commit ffc3028438
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 51 additions and 48 deletions

View File

@ -548,15 +548,17 @@ void ExpressionAnalyzer::getRootActionsForWindowFunctions(const ASTPtr & ast, bo
void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions, AggregateDescriptions & descriptions)
{
for (const ASTFunction * node : aggregates())
for (const ASTPtr & ast : aggregates())
{
const ASTFunction & node = typeid_cast<const ASTFunction &>(*ast);
AggregateDescription aggregate;
if (node->arguments)
getRootActionsNoMakeSet(node->arguments, actions);
if (node.arguments)
getRootActionsNoMakeSet(node.arguments, actions);
aggregate.column_name = node->getColumnName();
aggregate.column_name = node.getColumnName();
const ASTs & arguments = node->arguments ? node->arguments->children : ASTs();
const ASTs & arguments = node.arguments ? node.arguments->children : ASTs();
aggregate.argument_names.resize(arguments.size());
DataTypes types(arguments.size());
@ -568,7 +570,7 @@ void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions, Aggr
{
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER,
"Unknown identifier '{}' in aggregate function '{}'",
name, node->formatForErrorMessage());
name, node.formatForErrorMessage());
}
types[i] = dag_node->result_type;
@ -576,8 +578,8 @@ void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions, Aggr
}
AggregateFunctionProperties properties;
aggregate.parameters = (node->parameters) ? getAggregateFunctionParametersArray(node->parameters, "", getContext()) : Array();
aggregate.function = AggregateFunctionFactory::instance().get(node->name, types, aggregate.parameters, properties);
aggregate.parameters = (node.parameters) ? getAggregateFunctionParametersArray(node.parameters, "", getContext()) : Array();
aggregate.function = AggregateFunctionFactory::instance().get(node.name, types, aggregate.parameters, properties);
descriptions.push_back(aggregate);
}
@ -744,12 +746,13 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
}
// Window functions
for (const ASTFunction * function_node : syntax->window_function_asts)
for (const ASTPtr & ast : syntax->window_function_asts)
{
assert(function_node->is_window_function);
const ASTFunction & function_node = typeid_cast<const ASTFunction &>(*ast);
assert(function_node.is_window_function);
WindowFunctionDescription window_function;
window_function.function_node = function_node;
window_function.function_node = &function_node;
window_function.column_name
= window_function.function_node->getColumnName();
window_function.function_parameters
@ -760,7 +763,7 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
// Requiring a constant reference to a shared pointer to non-const AST
// doesn't really look sane, but the visitor does indeed require it.
// Hence we clone the node (not very sane either, I know).
// Hence, we clone the node (not very sane either, I know).
getRootActionsNoMakeSet(window_function.function_node->clone(), actions);
const ASTs & arguments
@ -793,22 +796,22 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
// Find the window corresponding to this function. It may be either
// referenced by name and previously defined in WINDOW clause, or it
// may be defined inline.
if (!function_node->window_name.empty())
if (!function_node.window_name.empty())
{
auto it = window_descriptions.find(function_node->window_name);
auto it = window_descriptions.find(function_node.window_name);
if (it == std::end(window_descriptions))
{
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER,
"Window '{}' is not defined (referenced by '{}')",
function_node->window_name,
function_node->formatForErrorMessage());
function_node.window_name,
function_node.formatForErrorMessage());
}
it->second.window_functions.push_back(window_function);
}
else
{
const auto & definition = function_node->window_definition->as<
const auto & definition = function_node.window_definition->as<
const ASTWindowDefinition &>();
WindowDescription desc;
desc.window_name = definition.getDefaultWindowName();
@ -1323,10 +1326,13 @@ void SelectQueryExpressionAnalyzer::appendAggregateFunctionsArguments(Expression
GetAggregatesVisitor(data).visit(select_query->orderBy());
/// TODO: data.aggregates -> aggregates()
for (const ASTFunction * node : data.aggregates)
if (node->arguments)
for (auto & argument : node->arguments->children)
for (const ASTPtr & ast : data.aggregates)
{
const ASTFunction & node = typeid_cast<const ASTFunction &>(*ast);
if (node.arguments)
for (auto & argument : node.arguments->children)
getRootActions(argument, only_types, step.actions());
}
}
void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments(

View File

@ -168,7 +168,7 @@ protected:
const ConstStoragePtr & storage() const { return syntax->storage; } /// The main table in FROM clause, if exists.
const TableJoin & analyzedJoin() const { return *syntax->analyzed_join; }
const NamesAndTypesList & sourceColumns() const { return syntax->required_source_columns; }
const std::vector<const ASTFunction *> & aggregates() const { return syntax->aggregates; }
const ASTs & aggregates() const { return syntax->aggregates; }
/// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables.
void initGlobalSubqueriesAndExternalTables(bool do_global, bool is_explain);

View File

@ -26,8 +26,8 @@ public:
// Explicit empty initializers are needed to make designated initializers
// work on GCC 10.
std::unordered_set<String> uniq_names {};
std::vector<const ASTFunction *> aggregates {};
std::vector<const ASTFunction *> window_functions {};
ASTs aggregates;
ASTs window_functions;
};
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child)
@ -61,7 +61,7 @@ public:
}
private:
static void visit(const ASTFunction & node, const ASTPtr &, Data & data)
static void visit(const ASTFunction & node, const ASTPtr & ast, Data & data)
{
if (isAggregateFunction(node))
{
@ -74,7 +74,7 @@ private:
return;
data.uniq_names.insert(column_name);
data.aggregates.push_back(&node);
data.aggregates.push_back(ast);
}
else if (node.is_window_function)
{
@ -87,7 +87,7 @@ private:
return;
data.uniq_names.insert(column_name);
data.window_functions.push_back(&node);
data.window_functions.push_back(ast);
}
}

View File

@ -731,7 +731,7 @@ void expandGroupByAll(ASTSelectQuery * select_query)
select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, group_expression_list);
}
std::vector<const ASTFunction *> getAggregates(ASTPtr & query, const ASTSelectQuery & select_query)
ASTs getAggregates(ASTPtr & query, const ASTSelectQuery & select_query)
{
/// There can not be aggregate functions inside the WHERE and PREWHERE.
if (select_query.where())
@ -743,11 +743,12 @@ std::vector<const ASTFunction *> getAggregates(ASTPtr & query, const ASTSelectQu
GetAggregatesVisitor(data).visit(query);
/// There can not be other aggregate functions within the aggregate functions.
for (const ASTFunction * node : data.aggregates)
for (const ASTPtr & ast : data.aggregates)
{
if (node->arguments)
const ASTFunction & node = typeid_cast<const ASTFunction &>(*ast);
if (node.arguments)
{
for (auto & arg : node->arguments->children)
for (auto & arg : node.arguments->children)
{
assertNoAggregates(arg, "inside another aggregate function");
// We also can't have window functions inside aggregate functions,
@ -759,7 +760,7 @@ std::vector<const ASTFunction *> getAggregates(ASTPtr & query, const ASTSelectQu
return data.aggregates;
}
std::vector<const ASTFunction *> getWindowFunctions(ASTPtr & query, const ASTSelectQuery & select_query)
ASTs getWindowFunctions(ASTPtr & query, const ASTSelectQuery & select_query)
{
/// There can not be window functions inside the WHERE, PREWHERE and HAVING
if (select_query.having())
@ -777,20 +778,16 @@ std::vector<const ASTFunction *> getWindowFunctions(ASTPtr & query, const ASTSel
/// Window functions cannot be inside aggregates or other window functions.
/// Aggregate functions can be inside window functions because they are
/// calculated earlier.
for (const ASTFunction * node : data.window_functions)
for (const ASTPtr & ast : data.window_functions)
{
if (node->arguments)
{
for (auto & arg : node->arguments->children)
{
assertNoWindows(arg, "inside another window function");
}
}
const ASTFunction & node = typeid_cast<const ASTFunction &>(*ast);
if (node->window_definition)
{
assertNoWindows(node->window_definition, "inside window definition");
}
if (node.arguments)
for (auto & arg : node.arguments->children)
assertNoWindows(arg, "inside another window function");
if (node.window_definition)
assertNoWindows(node.window_definition, "inside window definition");
}
return data.window_functions;
@ -1357,8 +1354,8 @@ TreeRewriterResultPtr TreeRewriter::analyze(
GetAggregatesVisitor(data).visit(query);
/// There can not be other aggregate functions within the aggregate functions.
for (const ASTFunction * node : data.aggregates)
for (auto & arg : node->arguments->children)
for (const ASTPtr & node : data.aggregates)
for (auto & arg : typeid_cast<const ASTFunction &>(*node).arguments->children)
assertNoAggregates(arg, "inside another aggregate function");
result.aggregates = data.aggregates;
}

View File

@ -41,8 +41,8 @@ struct TreeRewriterResult
Aliases aliases;
std::vector<const ASTFunction *> aggregates;
std::vector<const ASTFunction *> window_function_asts;
ASTs aggregates;
ASTs window_function_asts;
ASTs expressions_with_window_function;
/// Which column is needed to be ARRAY-JOIN'ed to get the specified.

View File

@ -12,6 +12,6 @@ opts=(
${CLICKHOUSE_CLIENT} "${opts[@]}" --multiquery --multiline --query """
DROP TABLE IF EXISTS mt ON CLUSTER test_shard_localhost;
DROP TABLE IF EXISTS wv ON CLUSTER test_shard_localhost;
CREATE TABLE mt ON CLUSTER test_shard_localhost (a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple();
CREATE TABLE mt ON CLUSTER test_shard_localhost (a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple();
CREATE WINDOW VIEW wv ON CLUSTER test_shard_localhost TO input_deduplicated INNER ENGINE Memory WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), hopStart(wid) AS w_start, hopEnd(wid) AS w_end FROM mt GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid;
""" 2>&1 | grep -q -e "Code: 344" -e "Code: 60" && echo 'ok' || echo 'fail' ||: