support expressions in window frame -- draft

This commit is contained in:
Alexander Kuzmenkov 2021-05-28 20:49:47 +03:00
parent 6bf62d8653
commit 40b1dc1812
4 changed files with 55 additions and 25 deletions

View File

@ -14,17 +14,18 @@
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <Columns/IColumn.h> #include <Columns/IColumn.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ArrayJoinAction.h> #include <Interpreters/ArrayJoinAction.h>
#include <Interpreters/Context.h>
#include <Interpreters/DictionaryReader.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExternalDictionariesLoader.h> #include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/HashJoin.h>
#include <Interpreters/JoinSwitcher.h>
#include <Interpreters/MergeJoin.h>
#include <Interpreters/Set.h> #include <Interpreters/Set.h>
#include <Interpreters/TableJoin.h> #include <Interpreters/TableJoin.h>
#include <Interpreters/JoinSwitcher.h>
#include <Interpreters/HashJoin.h>
#include <Interpreters/MergeJoin.h>
#include <Interpreters/DictionaryReader.h>
#include <Interpreters/Context.h>
#include <Processors/QueryPlan/ExpressionStep.h> #include <Processors/QueryPlan/ExpressionStep.h>
@ -474,7 +475,8 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions)
return !aggregates().empty(); return !aggregates().empty();
} }
void makeWindowDescriptionFromAST(const WindowDescriptions & existing_descriptions, void makeWindowDescriptionFromAST(const Context & context,
const WindowDescriptions & existing_descriptions,
WindowDescription & desc, const IAST * ast) WindowDescription & desc, const IAST * ast)
{ {
const auto & definition = ast->as<const ASTWindowDefinition &>(); const auto & definition = ast->as<const ASTWindowDefinition &>();
@ -580,6 +582,22 @@ void makeWindowDescriptionFromAST(const WindowDescriptions & existing_descriptio
} }
desc.frame = definition.frame; desc.frame = definition.frame;
if (definition.frame.end_type == WindowFrame::BoundaryType::Offset)
{
auto [value, _] = evaluateConstantExpression(
definition.frame_end_offset,
const_pointer_cast<Context>(context.shared_from_this()));
desc.frame.end_offset = value;
}
if (definition.frame.begin_type == WindowFrame::BoundaryType::Offset)
{
auto [value, _] = evaluateConstantExpression(
definition.frame_begin_offset,
const_pointer_cast<Context>(context.shared_from_this()));
desc.frame.begin_offset = value;
}
} }
void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions) void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
@ -605,7 +623,8 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
const auto & elem = ptr->as<const ASTWindowListElement &>(); const auto & elem = ptr->as<const ASTWindowListElement &>();
WindowDescription desc; WindowDescription desc;
desc.window_name = elem.name; desc.window_name = elem.name;
makeWindowDescriptionFromAST(window_descriptions, desc, elem.definition.get()); makeWindowDescriptionFromAST(*getContext(), window_descriptions,
desc, elem.definition.get());
auto [it, inserted] = window_descriptions.insert( auto [it, inserted] = window_descriptions.insert(
{desc.window_name, desc}); {desc.window_name, desc});
@ -690,7 +709,8 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
const ASTWindowDefinition &>(); const ASTWindowDefinition &>();
WindowDescription desc; WindowDescription desc;
desc.window_name = definition.getDefaultWindowName(); desc.window_name = definition.getDefaultWindowName();
makeWindowDescriptionFromAST(window_descriptions, desc, &definition); makeWindowDescriptionFromAST(*getContext(), window_descriptions,
desc, &definition);
auto [it, inserted] = window_descriptions.insert( auto [it, inserted] = window_descriptions.insert(
{desc.window_name, desc}); {desc.window_name, desc});

View File

@ -28,6 +28,18 @@ ASTPtr ASTWindowDefinition::clone() const
result->frame = frame; result->frame = frame;
if (frame_begin_offset)
{
result->frame_begin_offset = frame_begin_offset->clone();
result->children.push_back(result->frame_begin_offset);
}
if (frame_end_offset)
{
result->frame_end_offset = frame_end_offset->clone();
result->children.push_back(result->frame_end_offset);
}
return result; return result;
} }
@ -93,8 +105,7 @@ void ASTWindowDefinition::formatImpl(const FormatSettings & settings,
} }
else else
{ {
settings.ostr << applyVisitor(FieldVisitorToString(), frame_begin_offset->formatImpl(settings, state, format_frame);
frame.begin_offset);
settings.ostr << " " settings.ostr << " "
<< (!frame.begin_preceding ? "FOLLOWING" : "PRECEDING"); << (!frame.begin_preceding ? "FOLLOWING" : "PRECEDING");
} }
@ -109,8 +120,7 @@ void ASTWindowDefinition::formatImpl(const FormatSettings & settings,
} }
else else
{ {
settings.ostr << applyVisitor(FieldVisitorToString(), frame_end_offset->formatImpl(settings, state, format_frame);
frame.end_offset);
settings.ostr << " " settings.ostr << " "
<< (!frame.end_preceding ? "FOLLOWING" : "PRECEDING"); << (!frame.end_preceding ? "FOLLOWING" : "PRECEDING");
} }

View File

@ -16,8 +16,12 @@ struct ASTWindowDefinition : public IAST
ASTPtr order_by; ASTPtr order_by;
// Be careful: offsets can contain constant expressions such as INTERVAL 1 DAY,
// that are evaluated later by ExpressionAnalyzer. The WindowFrame struct
// can be incomplete after parsing.
WindowFrame frame; WindowFrame frame;
ASTPtr frame_begin_offset;
ASTPtr frame_end_offset;
ASTPtr clone() const override; ASTPtr clone() const override;

View File

@ -569,16 +569,14 @@ static bool tryParseFrameDefinition(ASTWindowDefinition * node, IParser::Pos & p
} }
else else
{ {
ParserLiteral parser_literal; ParserExpression parser_expression;
ASTPtr ast_literal;
if (keyword_unbounded.ignore(pos, expected)) if (keyword_unbounded.ignore(pos, expected))
{ {
node->frame.begin_type = WindowFrame::BoundaryType::Unbounded; node->frame.begin_type = WindowFrame::BoundaryType::Unbounded;
} }
else if (parser_literal.parse(pos, ast_literal, expected)) else if (parser_expression.parse(pos, node->frame_begin_offset, expected))
{ {
const Field & value = ast_literal->as<ASTLiteral &>().value; // We will evaluate the expression for offset expression later.
node->frame.begin_offset = value;
node->frame.begin_type = WindowFrame::BoundaryType::Offset; node->frame.begin_type = WindowFrame::BoundaryType::Offset;
} }
else else
@ -618,16 +616,14 @@ static bool tryParseFrameDefinition(ASTWindowDefinition * node, IParser::Pos & p
} }
else else
{ {
ParserLiteral parser_literal; ParserExpression parser_expression;
ASTPtr ast_literal;
if (keyword_unbounded.ignore(pos, expected)) if (keyword_unbounded.ignore(pos, expected))
{ {
node->frame.end_type = WindowFrame::BoundaryType::Unbounded; node->frame.end_type = WindowFrame::BoundaryType::Unbounded;
} }
else if (parser_literal.parse(pos, ast_literal, expected)) else if (parser_expression.parse(pos, node->frame_end_offset, expected))
{ {
const Field & value = ast_literal->as<ASTLiteral &>().value; // We will evaluate the expression for offset expression later.
node->frame.end_offset = value;
node->frame.end_type = WindowFrame::BoundaryType::Offset; node->frame.end_type = WindowFrame::BoundaryType::Offset;
} }
else else