mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
support expressions in window frame -- draft
This commit is contained in:
parent
6bf62d8653
commit
40b1dc1812
@ -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});
|
||||||
|
@ -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");
|
||||||
}
|
}
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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
|
||||||
|
Loading…
Reference in New Issue
Block a user