Merge pull request #27036 from nickitat/nickitat_select_filter

This commit is contained in:
Vladimir C 2021-08-24 10:18:55 +03:00 committed by GitHub
commit 0858fdf113
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 70 additions and 8 deletions

View File

@ -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);
}
};

View File

@ -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);

View File

@ -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

View File

@ -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));

View File

@ -0,0 +1,2 @@
98
2450

View 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 }