mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge pull request #27036 from nickitat/nickitat_select_filter
This commit is contained in:
commit
0858fdf113
@ -1,5 +1,5 @@
|
||||
#include <AggregateFunctions/AggregateFunctionIf.h>
|
||||
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
|
||||
#include <AggregateFunctions/AggregateFunctionIf.h>
|
||||
#include "AggregateFunctionNull.h"
|
||||
|
||||
|
||||
@ -11,6 +11,7 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_AGGREGATION;
|
||||
}
|
||||
|
||||
class AggregateFunctionCombinatorIf final : public IAggregateFunctionCombinator
|
||||
@ -37,6 +38,10 @@ public:
|
||||
const DataTypes & arguments,
|
||||
const Array & params) const override
|
||||
{
|
||||
if (nested_function->getName().find(getName()) != String::npos)
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_AGGREGATION, "nested function for {0}-combinator must not have {0}-combinator", getName());
|
||||
}
|
||||
return std::make_shared<AggregateFunctionIf>(nested_function, arguments, params);
|
||||
}
|
||||
};
|
||||
|
@ -3,10 +3,10 @@
|
||||
|
||||
#include <Poco/String.h>
|
||||
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Parsers/DumpASTNode.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <Parsers/ASTAsterisk.h>
|
||||
#include <Parsers/ASTColumnsTransformers.h>
|
||||
@ -268,7 +268,6 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserIdentifier id_parser;
|
||||
@ -276,6 +275,7 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
ParserKeyword all("ALL");
|
||||
ParserExpressionList contents(false, is_table_function);
|
||||
ParserSelectWithUnionQuery select;
|
||||
ParserKeyword filter("FILTER");
|
||||
ParserKeyword over("OVER");
|
||||
|
||||
bool has_all = false;
|
||||
@ -440,16 +440,27 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
function_node->children.push_back(function_node->parameters);
|
||||
}
|
||||
|
||||
if (over.ignore(pos, expected))
|
||||
if (filter.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. Normally it would take a
|
||||
// reference to ASTPtr and assign it the new node. We only have a pointer
|
||||
// of a different type, hence this workaround with a temporary pointer.
|
||||
ASTPtr function_node_as_iast = function_node;
|
||||
|
||||
ParserFilterClause filter_parser;
|
||||
if (!filter_parser.parse(pos, function_node_as_iast, expected))
|
||||
{
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
if (over.ignore(pos, expected))
|
||||
{
|
||||
function_node->is_window_function = true;
|
||||
|
||||
ASTPtr function_node_as_iast = function_node;
|
||||
|
||||
ParserWindowReference window_reference;
|
||||
if (!window_reference.parse(pos, function_node_as_iast, expected))
|
||||
{
|
||||
@ -504,6 +515,40 @@ bool ParserTableFunctionView::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserFilterClause::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
assert(node);
|
||||
ASTFunction & function = dynamic_cast<ASTFunction &>(*node);
|
||||
|
||||
ParserToken parser_opening_bracket(TokenType::OpeningRoundBracket);
|
||||
if (!parser_opening_bracket.ignore(pos, expected))
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
ParserKeyword parser_where("WHERE");
|
||||
if (!parser_where.ignore(pos, expected))
|
||||
{
|
||||
return false;
|
||||
}
|
||||
ParserExpressionList parser_condition(false);
|
||||
ASTPtr condition;
|
||||
if (!parser_condition.parse(pos, condition, expected) || condition->children.size() != 1)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
ParserToken parser_closing_bracket(TokenType::ClosingRoundBracket);
|
||||
if (!parser_closing_bracket.ignore(pos, expected))
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
function.name += "If";
|
||||
function.arguments->children.push_back(condition->children[0]);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserWindowReference::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
assert(node);
|
||||
|
@ -171,6 +171,13 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
// Allows to make queries like SELECT SUM(<expr>) FILTER(WHERE <cond>) FROM ...
|
||||
class ParserFilterClause : public IParserBase
|
||||
{
|
||||
const char * getName() const override { return "filter"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
// Window reference (the thing that goes after OVER) for window function.
|
||||
// Can be either window name or window definition.
|
||||
class ParserWindowReference : public IParserBase
|
||||
|
@ -1 +1 @@
|
||||
SELECT sumForEachMergeArray(y) FROM (SELECT sumForEachStateForEachIfArrayIfMerge(x) AS y FROM (SELECT sumForEachStateForEachIfArrayIfState([[[1, 2, 3], [4, 5, 6], [7, 8, 9]]], [1], 1) AS x));
|
||||
SELECT sumForEachMergeArray(y) FROM (SELECT sumForEachStateForEachIfArrayMerge(x) AS y FROM (SELECT sumForEachStateForEachIfArrayState([[[1, 2, 3], [4, 5, 6], [7, 8, 9]]], [1]) AS x));
|
||||
|
@ -0,0 +1,2 @@
|
||||
98
|
||||
2450
|
3
tests/queries/0_stateless/02001_select_with_filter.sql
Normal file
3
tests/queries/0_stateless/02001_select_with_filter.sql
Normal file
@ -0,0 +1,3 @@
|
||||
SELECT argMax(number, number + 1) FILTER(WHERE number != 99) FROM numbers(100) ;
|
||||
SELECT sum(number) FILTER(WHERE number % 2 == 0) FROM numbers(100);
|
||||
SELECT sumIfOrNull(number, number % 2 == 1) FILTER(WHERE number % 2 == 0) FROM numbers(100); -- { serverError 184 }
|
Loading…
Reference in New Issue
Block a user