some grammar

This commit is contained in:
Alexander Kuzmenkov 2020-12-17 00:44:05 +03:00
parent e335074c39
commit fef24c81e3
17 changed files with 234 additions and 166 deletions

View File

@ -735,17 +735,25 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
}
}
if (AggregateFunctionFactory::instance().isAggregateFunctionName(node.name))
return;
/// FIXME need proper grammar for window functions. For now, ignore it --
/// the resulting column is added in ExpressionAnalyzer, similar to the
/// aggregate functions.
if (node.name == "window")
if (node.is_window_function)
{
// Also add columns from PARTITION BY and ORDER BY of window functions.
// Requiring a constant reference to a shared pointer to non-const AST
// doesn't really look sane, but the visitor does indeed require it.
visit(node.window_partition_by->clone(), data);
visit(node.window_order_by->clone(), data);
// Don't need to do anything more for window functions here -- the
// resulting column is added in ExpressionAnalyzer, similar to the
// aggregate functions.
return;
}
// An aggregate function can also be calculated as a window function, but we
// checked for it above, so no need to do anything more.
if (AggregateFunctionFactory::instance().isAggregateFunctionName(node.name))
return;
FunctionOverloadResolverPtr function_builder;
try
{

View File

@ -31,7 +31,6 @@ struct WindowFunctionDescription
{
std::string window_name;
std::string column_name;
const ASTFunction * wrapper_node;
const ASTFunction * function_node;
AggregateFunctionPtr aggregate_function;
Array function_parameters;

View File

@ -469,63 +469,14 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions)
return !aggregates().empty();
}
// Parses order by & partition by from window() wrapper function.
// Remove this when we have proper grammar.
static SortDescription windowArgumentToSortDescription(IAST* ast, const WindowDescription & w)
{
SortDescription result;
if (const auto * as_tuple = ast->as<ASTFunction>();
as_tuple
&& as_tuple->name == "tuple"
&& as_tuple->arguments)
{
// untuple it
for (const auto & element_ast
: as_tuple->arguments->children)
{
const auto * with_alias = dynamic_cast<
const ASTWithAlias *>(element_ast.get());
if (!with_alias)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"(1) Expected column in PARTITION BY"
" for window '{}', got '{}'",
w.window_name,
element_ast->formatForErrorMessage());
}
result.push_back(
SortColumnDescription(
with_alias->getColumnName(),
1 /* direction */,
1 /* nulls_direction */));
}
}
else if (const auto * with_alias
= dynamic_cast<const ASTWithAlias *>(ast))
{
result.push_back(
SortColumnDescription(
with_alias->getColumnName(),
1 /* direction */,
1 /* nulls_direction */));
}
else
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"(2) Expected tuple or column in PARTITION BY"
" for window '{}', got '{}'",
w.window_name,
ast->formatForErrorMessage());
}
return result;
}
bool ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr & actions)
{
for (const ASTFunction * wrapper_node : windowFunctions())
for (const ASTFunction * function_node : windowFunctions())
{
fmt::print(stderr, "window function ast: {}\n", wrapper_node->dumpTree());
fmt::print(stderr, "window function ast: {}\n", function_node->dumpTree());
assert(function_node->is_window_function);
// FIXME not thread-safe, should use a per-query counter.
static int window_index = 1;
@ -533,38 +484,44 @@ bool ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr & actions)
WindowDescription window_description;
window_description.window_name = fmt::format("window_{}", window_index++);
const auto * elist = wrapper_node->arguments
? wrapper_node->arguments->as<const ASTExpressionList>()
: nullptr;
if (elist)
if (function_node->window_partition_by)
{
if (elist->children.size() >= 2)
for (const auto & column_ast
: function_node->window_partition_by->children)
{
const auto partition_by_ast = elist->children[1];
fmt::print(stderr, "partition by ast {}\n",
partition_by_ast->dumpTree());
window_description.partition_by = windowArgumentToSortDescription(
partition_by_ast.get(), window_description);
}
if (elist->children.size() == 3)
{
const auto order_by_ast = elist->children[2];
fmt::print(stderr, "order by ast {}\n",
order_by_ast->dumpTree());
window_description.order_by = windowArgumentToSortDescription(
order_by_ast.get(), window_description);
}
if (elist->children.size() > 3)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Too many arguments to window function '{}'",
wrapper_node->formatForErrorMessage());
const auto * with_alias = dynamic_cast<const ASTWithAlias *>(
column_ast.get());
if (!with_alias)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Expected a column in PARTITION BY for window '{}',"
" got '{}'", window_description.window_name,
column_ast->formatForErrorMessage());
}
window_description.partition_by.push_back(
SortColumnDescription(
with_alias->getColumnName(), 1 /* direction */,
1 /* nulls_direction */));
}
}
if (function_node->window_order_by)
{
for (const auto & column_ast
: function_node->window_order_by->children)
{
// Parser should have checked that we have a proper element here.
const auto & order_by_element
= column_ast->as<ASTOrderByElement &>();
// Ignore collation for now.
window_description.order_by.push_back(
SortColumnDescription(
order_by_element.children.front()->getColumnName(),
order_by_element.direction,
order_by_element.nulls_direction));
}
}
window_description.full_sort_description = window_description.partition_by;
window_description.full_sort_description.insert(
window_description.full_sort_description.end(),
@ -573,9 +530,7 @@ bool ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr & actions)
WindowFunctionDescription window_function;
window_function.window_name = window_description.window_name;
window_function.wrapper_node = wrapper_node;
window_function.function_node
= &elist->children.at(0)->as<ASTFunction &>();
window_function.function_node = function_node;
window_function.column_name
= window_function.function_node->getColumnName();
window_function.function_parameters
@ -584,17 +539,10 @@ bool ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr & actions)
window_function.function_node->parameters)
: Array();
// We have to fill actions for window function arguments, so that we are
// then able to find their argumen types. The `actions` passed to this
// functions are temporary and are discarded.
// The same calculation is done in appendWindowFunctionsArguments.
getRootActionsNoMakeSet(wrapper_node->arguments, true /* no subqueries */,
actions);
// We have to separately get actions for the arguments of the aggregate
// function we calculate over window, because the ActionsVisitor does
// not descend into aggregate functions.
getRootActionsNoMakeSet(window_function.function_node->arguments,
true /* no subqueries */, 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.
getRootActionsNoMakeSet(window_function.function_node->clone(),
true, actions);
const ASTs & arguments
= window_function.function_node->arguments->children;
@ -1013,16 +961,9 @@ void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments(
for (const auto & f : window_functions)
{
// Not sure why NoMakeSet, copied from aggregate functions.
getRootActionsNoMakeSet(f.wrapper_node->arguments,
true /* no subqueries */, step.actions());
// We have to separately get actions for the arguments of the aggregate
// function we calculate over window, because the ActionsVisitor does
// not descend into aggregate functions.
// Not sure why NoMakeSet, copied from aggregate functions.
getRootActionsNoMakeSet(f.function_node->arguments,
true /* no subqueries */, step.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.
getRootActionsNoMakeSet(f.function_node->clone(), true, step.actions());
// Add empty INPUT with window function name.
// It is an aggregate function, so it won't be added by getRootActions.
@ -1081,13 +1022,6 @@ void SelectQueryExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain,
for (const auto & child : select_query->select()->children)
{
/// FIXME add proper grammar for window functions
if (const auto * as_function = child->as<ASTFunction>();
as_function && as_function->name == "window")
{
continue;
}
step.required_output.push_back(child->getColumnName());
}
}
@ -1184,13 +1118,6 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActio
ASTs asts = select_query->select()->children;
for (const auto & ast : asts)
{
/// FIXME add proper grammar for window functions
if (const auto * as_function = ast->as<ASTFunction>();
as_function && as_function->name == "window")
{
continue;
}
String result_name = ast->getAliasOrColumnName();
if (required_result_columns.empty() || required_result_columns.count(result_name))
{
@ -1226,16 +1153,6 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActio
}
}
for (const auto & f : window_functions)
{
if (required_result_columns.empty()
|| required_result_columns.count(f.column_name))
{
result_columns.emplace_back(f.column_name, f.column_name);
step.required_output.push_back(f.column_name);
}
}
auto actions = chain.getLastActions();
actions->project(result_columns);
return actions;
@ -1651,7 +1568,6 @@ std::string WindowFunctionDescription::dump() const
{
std::stringstream ss;
ss << "window function '" << column_name << "' over '" << window_name <<"\n";
ss << "wrapper node " << wrapper_node->dumpTree() << "\n";
ss << "function node " << function_node->dumpTree() << "\n";
ss << "aggregate function '" << aggregate_function->getName() << "'\n";
if (function_parameters.size())

View File

@ -19,9 +19,15 @@ void ExpressionInfoMatcher::visit(const ASTPtr & ast, Data & data)
void ExpressionInfoMatcher::visit(const ASTFunction & ast_function, const ASTPtr &, Data & data)
{
if (ast_function.name == "arrayJoin")
{
data.is_array_join = true;
else if (AggregateFunctionFactory::instance().isAggregateFunctionName(ast_function.name))
}
else if (!ast_function.is_window_function
&& AggregateFunctionFactory::instance().isAggregateFunctionName(
ast_function.name))
{
data.is_aggregate_function = true;
}
else
{
const auto & function = FunctionFactory::instance().tryGet(ast_function.name, data.context);

View File

@ -31,9 +31,8 @@ public:
return false;
if (auto * func = node->as<ASTFunction>())
{
if (isAggregateFunction(func->name)
// FIXME temporary hack while we don't have grammar
|| func->name == "window")
if (isAggregateFunction(*func)
|| func->is_window_function)
{
return false;
}
@ -50,7 +49,7 @@ public:
private:
static void visit(const ASTFunction & node, const ASTPtr &, Data & data)
{
if (isAggregateFunction(node.name))
if (isAggregateFunction(node))
{
if (data.assert_no_aggregates)
throw Exception("Aggregate function " + node.getColumnName() + " is found " + String(data.assert_no_aggregates) + " in query",
@ -63,7 +62,7 @@ private:
data.uniq_names.insert(column_name);
data.aggregates.push_back(&node);
}
else if (node.name == "window")
else if (node.is_window_function)
{
if (data.assert_no_windows)
throw Exception("Window function " + node.getColumnName() + " is found " + String(data.assert_no_windows) + " in query",
@ -78,9 +77,13 @@ private:
}
}
static bool isAggregateFunction(const String & name)
static bool isAggregateFunction(const ASTFunction & node)
{
return AggregateFunctionFactory::instance().isAggregateFunctionName(name);
// Aggregate functions can also be calculated as window functions, but
// here we are interested in aggregate functions calculated in GROUP BY.
return !node.is_window_function
&& AggregateFunctionFactory::instance().isAggregateFunctionName(
node.name);
}
};

View File

@ -43,9 +43,14 @@ public:
if (group_by_function_hashes.count(key))
return false;
/// if ORDER BY contains aggregate function it shouldn't be optimized
if (AggregateFunctionFactory::instance().isAggregateFunctionName(ast_function.name))
/// if ORDER BY contains aggregate function or window functions, it
/// shouldn't be optimized
if (ast_function.is_window_function
|| AggregateFunctionFactory::instance().isAggregateFunctionName(
ast_function.name))
{
return false;
}
return true;
}

View File

@ -38,8 +38,12 @@ bool extractIdentifiers(const ASTFunction & func, std::unordered_set<ASTPtr *> &
if (arg_func->name == "lambda")
return false;
if (AggregateFunctionFactory::instance().isAggregateFunctionName(arg_func->name))
if ( arg_func->is_window_function
|| AggregateFunctionFactory::instance().isAggregateFunctionName(
arg_func->name))
{
return false;
}
if (!extractIdentifiers(*arg_func, identifiers))
return false;

View File

@ -1,13 +1,14 @@
#include <Common/typeid_cast.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTWithAlias.h>
#include <Parsers/ASTSubquery.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <Common/SipHash.h>
#include <IO/Operators.h>
#include <Common/SipHash.h>
#include <Common/typeid_cast.h>
#include <IO/Operators.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTWithAlias.h>
namespace DB
{
@ -37,6 +38,14 @@ void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const
(*it)->appendColumnName(ostr);
}
writeChar(')', ostr);
/*
if (window_name)
{
writeCString(" over ", ostr);
window_name->appendColumnName(ostr);
}
*/
}
/** Get the text that identifies this element. */
@ -53,6 +62,21 @@ ASTPtr ASTFunction::clone() const
if (arguments) { res->arguments = arguments->clone(); res->children.push_back(res->arguments); }
if (parameters) { res->parameters = parameters->clone(); res->children.push_back(res->parameters); }
if (window_name)
{
res->set(res->window_name, window_name->clone());
}
if (window_partition_by)
{
res->set(res->window_partition_by, window_partition_by->clone());
}
if (window_order_by)
{
res->set(res->window_order_by, window_order_by->clone());
}
return res;
}

View File

@ -8,6 +8,8 @@
namespace DB
{
class ASTIdentifier;
/** AST for function application or operator.
*/
class ASTFunction : public ASTWithAlias
@ -18,6 +20,11 @@ public:
/// parameters - for parametric aggregate function. Example: quantile(0.9)(x) - what in first parens are 'parameters'.
ASTPtr parameters;
bool is_window_function = false;
ASTIdentifier * window_name;
ASTExpressionList * window_partition_by;
ASTExpressionList * window_order_by;
/// do not print empty parentheses if there are no args - compatibility with new AST for data types and engine names.
bool no_empty_args = false;

View File

@ -1,6 +1,8 @@
#include <Parsers/ASTIndexDeclaration.h>
#include <Common/quoteString.h>
#include <IO/Operators.h>
#include <Parsers/ASTFunction.h>
namespace DB

View File

@ -1,12 +1,12 @@
#pragma once
#include <Parsers/ASTFunction.h>
#include <Parsers/IAST.h>
namespace DB
{
class ASTFunction;
/** name BY expr TYPE typename(args) GRANULARITY int in create query
*/
class ASTIndexDeclaration : public IAST

View File

@ -263,6 +263,7 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserKeyword distinct("DISTINCT");
ParserExpressionList contents(false);
ParserSelectWithUnionQuery select;
ParserKeyword over("OVER");
bool has_distinct_modifier = false;
@ -382,10 +383,96 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
function_node->children.push_back(function_node->parameters);
}
if (over.ignore(pos, expected))
{
function_node->is_window_function = true;
// We are slightly breaking the parser interface by parsing the window
// definition into an existing ASTFunction, hence this weirdness.
ASTPtr function_node_as_iast = function_node;
ParserWindowDefinition window_definition;
if (!window_definition.parse(pos, function_node_as_iast, expected))
{
return false;
}
fmt::print(stderr, "window function ast {}\n", function_node->dumpTree());
}
node = function_node;
return true;
}
bool ParserWindowDefinition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ASTFunction * function = dynamic_cast<ASTFunction *>(node.get());
// Variant 1:
// function_name ( * ) OVER window_name
// FIXME doesn't work anyway for now -- never used anywhere, window names
// can't be defined, and TreeRewriter thinks the window name is a column so
// the query fails.
if (pos->type != TokenType::OpeningRoundBracket)
{
ASTPtr window_name_ast;
ParserIdentifier window_name_parser;
if (window_name_parser.parse(pos, window_name_ast, expected))
{
function->set(function->window_name, window_name_ast);
return true;
}
else
{
return false;
}
}
++pos;
// Variant 2:
// function_name ( * ) OVER ( window_definition )
ParserKeyword keyword_partition_by("PARTITION BY");
ParserNotEmptyExpressionList columns_partition_by(
false /* we don't allow declaring aliases here*/);
ParserKeyword keyword_order_by("ORDER BY");
ParserOrderByExpressionList columns_order_by;
if (keyword_partition_by.ignore(pos, expected))
{
ASTPtr partition_by_ast;
if (columns_partition_by.parse(pos, partition_by_ast, expected))
{
function->set(function->window_partition_by, partition_by_ast);
}
else
{
return false;
}
}
if (keyword_order_by.ignore(pos, expected))
{
ASTPtr order_by_ast;
if (columns_order_by.parse(pos, order_by_ast, expected))
{
function->set(function->window_order_by, order_by_ast);
}
else
{
return false;
}
}
if (pos->type != TokenType::ClosingRoundBracket)
{
expected.add(pos, "')'");
return false;
}
++pos;
return true;
}
bool ParserCodecDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
return ParserList(std::make_unique<ParserIdentifierWithOptionalParameters>(),

View File

@ -124,6 +124,15 @@ protected:
bool allow_function_parameters;
};
struct ParserWindowDefinition : public IParserBase
{
protected:
const char * getName() const override { return "window definition"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserCodecDeclarationList : public IParserBase
{
protected:

View File

@ -19,9 +19,6 @@ namespace DB
namespace ErrorCodes
{
extern const int NOT_A_COLUMN;
extern const int UNKNOWN_TYPE_OF_AST_NODE;
extern const int UNKNOWN_ELEMENT_IN_AST;
extern const int LOGICAL_ERROR;
}
@ -46,7 +43,7 @@ public:
String getColumnName() const;
virtual void appendColumnName(WriteBuffer &) const
{
throw Exception("Trying to get name of not a column: " + getID(), ErrorCodes::NOT_A_COLUMN);
throw Exception("Trying to get name of not a column: " + getID(), ErrorCodes::LOGICAL_ERROR);
}
/** Get the alias, if any, or the canonical name of the column, if it is not. */
@ -58,7 +55,7 @@ public:
/** Set the alias. */
virtual void setAlias(const String & /*to*/)
{
throw Exception("Can't set alias of " + getColumnName(), ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE);
throw Exception("Can't set alias of " + getColumnName(), ErrorCodes::LOGICAL_ERROR);
}
/** Get the text that identifies this element. */
@ -209,7 +206,7 @@ public:
virtual void formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const
{
throw Exception("Unknown element in AST: " + getID(), ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
throw Exception("Unknown element in AST: " + getID(), ErrorCodes::LOGICAL_ERROR);
}
// A simple way to add some user-readable context to an error message.

View File

@ -2,6 +2,7 @@
#include <Parsers/ASTColumnDeclaration.h>
#include <Parsers/ASTConstraintDeclaration.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIndexDeclaration.h>
#include <Parsers/New/AST/ColumnExpr.h>
#include <Parsers/New/AST/ColumnTypeExpr.h>

View File

@ -2,6 +2,7 @@
#include <Interpreters/TreeRewriter.h>
#include <Storages/IndicesDescription.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIndexDeclaration.h>
#include <Parsers/formatAST.h>
#include <Parsers/ParserCreateQuery.h>

View File

@ -10,7 +10,6 @@
#include <Storages/SelectQueryInfo.h>
#include <Storages/MergeTree/MarkRange.h>
#include <Interpreters/ExpressionActions.h>
#include <Parsers/ASTIndexDeclaration.h>
#include <DataTypes/DataTypeLowCardinality.h>
constexpr auto INDEX_FILE_PREFIX = "skp_idx_";