mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
some grammar
This commit is contained in:
parent
e335074c39
commit
fef24c81e3
@ -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
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -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())
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -1,6 +1,8 @@
|
||||
#include <Parsers/ASTIndexDeclaration.h>
|
||||
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -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
|
||||
|
@ -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>(),
|
||||
|
@ -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:
|
||||
|
@ -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.
|
||||
|
@ -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>
|
||||
|
@ -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>
|
||||
|
@ -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_";
|
||||
|
Loading…
Reference in New Issue
Block a user