mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Some final fixes
This commit is contained in:
parent
f125fb3fef
commit
f66e8464f9
@ -89,7 +89,6 @@ InterpreterSelectIntersectExceptQuery::buildCurrentChildInterpreter(const ASTPtr
|
||||
if (ast_ptr_->as<ASTSelectIntersectExceptQuery>())
|
||||
return std::make_unique<InterpreterSelectIntersectExceptQuery>(ast_ptr_, context, SelectQueryOptions());
|
||||
|
||||
// if (ast_ptr_->as<ASTSubquery>())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected query: {}", ast_ptr_->getID());
|
||||
}
|
||||
|
||||
@ -140,4 +139,10 @@ BlockIO InterpreterSelectIntersectExceptQuery::execute()
|
||||
return res;
|
||||
}
|
||||
|
||||
void InterpreterSelectIntersectExceptQuery::ignoreWithTotals()
|
||||
{
|
||||
for (auto & interpreter : nested_interpreters)
|
||||
interpreter->ignoreWithTotals();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -28,6 +28,8 @@ public:
|
||||
|
||||
Block getSampleBlock() { return result_header; }
|
||||
|
||||
void ignoreWithTotals() override;
|
||||
|
||||
private:
|
||||
static String getName() { return "SelectIntersectExceptQuery"; }
|
||||
|
||||
@ -36,9 +38,8 @@ private:
|
||||
|
||||
void buildQueryPlan(QueryPlan & query_plan) override;
|
||||
|
||||
void ignoreWithTotals() override {}
|
||||
|
||||
std::vector<std::unique_ptr<IInterpreterUnionOrSelectQuery>> nested_interpreters;
|
||||
|
||||
Operator final_operator;
|
||||
};
|
||||
|
||||
|
@ -12,11 +12,11 @@ namespace ErrorCodes
|
||||
|
||||
/*
|
||||
* Note: there is a difference between intersect and except behaviour.
|
||||
* `intersect` is supposed to be a part of last SelectQuery, i.e. the sequence with no parenthesis:
|
||||
* `intersect` is supposed to be a part of the last SelectQuery, i.e. the sequence with no parenthesis:
|
||||
* select 1 union all select 2 except select 1 intersect 2 except select 2 union distinct select 5;
|
||||
* is interpreted as:
|
||||
* select 1 union all select 2 except (select 1 intersect 2) except select 2 union distinct select 5;
|
||||
* Whereas `except` is applied to all union part like:
|
||||
* Whereas `except` is applied to all left union part like:
|
||||
* (((select 1 union all select 2) except (select 1 intersect 2)) except select 2) union distinct select 5;
|
||||
**/
|
||||
|
||||
@ -28,7 +28,7 @@ void SelectIntersectExceptQueryMatcher::visit(ASTPtr & ast, Data & data)
|
||||
|
||||
void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Data &)
|
||||
{
|
||||
auto & union_modes = ast.list_of_modes;
|
||||
const auto & union_modes = ast.list_of_modes;
|
||||
|
||||
if (union_modes.empty())
|
||||
return;
|
||||
|
@ -30,11 +30,10 @@ void ASTSelectIntersectExceptQuery::formatQueryImpl(const FormatSettings & setti
|
||||
{
|
||||
settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "")
|
||||
<< (final_operator == Operator::INTERSECT ? "INTERSECT" : "EXCEPT")
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
<< (settings.hilite ? hilite_none : "")
|
||||
<< settings.nl_or_ws;
|
||||
}
|
||||
|
||||
if (it != children.begin())
|
||||
settings.ostr << settings.nl_or_ws;
|
||||
(*it)->formatImpl(settings, state, frame);
|
||||
}
|
||||
}
|
||||
|
@ -277,79 +277,6 @@ static bool modifyAST(ASTPtr ast, SubqueryFunctionType type)
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserComparisonExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
bool first = true;
|
||||
|
||||
auto current_depth = pos.depth;
|
||||
while (true)
|
||||
{
|
||||
if (first)
|
||||
{
|
||||
ASTPtr elem;
|
||||
if (!elem_parser.parse(pos, elem, expected))
|
||||
return false;
|
||||
|
||||
node = elem;
|
||||
first = false;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// try to find any of the valid operators
|
||||
const char ** it;
|
||||
Expected stub;
|
||||
for (it = overlapping_operators_to_skip; *it; ++it)
|
||||
if (ParserKeyword{*it}.checkWithoutMoving(pos, stub))
|
||||
break;
|
||||
|
||||
if (*it)
|
||||
break;
|
||||
|
||||
for (it = operators; *it; it += 2)
|
||||
if (parseOperator(pos, *it, expected))
|
||||
break;
|
||||
|
||||
if (!*it)
|
||||
break;
|
||||
|
||||
/// the function corresponding to the operator
|
||||
auto function = std::make_shared<ASTFunction>();
|
||||
|
||||
/// function arguments
|
||||
auto exp_list = std::make_shared<ASTExpressionList>();
|
||||
|
||||
ASTPtr elem;
|
||||
SubqueryFunctionType subquery_function_type = SubqueryFunctionType::NONE;
|
||||
if (ParserKeyword("ANY").ignore(pos, expected))
|
||||
subquery_function_type = SubqueryFunctionType::ANY;
|
||||
else if (ParserKeyword("ALL").ignore(pos, expected))
|
||||
subquery_function_type = SubqueryFunctionType::ALL;
|
||||
else if (!elem_parser.parse(pos, elem, expected))
|
||||
return false;
|
||||
|
||||
if (subquery_function_type != SubqueryFunctionType::NONE && !ParserSubquery().parse(pos, elem, expected))
|
||||
return false;
|
||||
|
||||
/// the first argument of the function is the previous element, the second is the next one
|
||||
function->name = it[1];
|
||||
function->arguments = exp_list;
|
||||
function->children.push_back(exp_list);
|
||||
|
||||
exp_list->children.push_back(node);
|
||||
exp_list->children.push_back(elem);
|
||||
|
||||
if (subquery_function_type != SubqueryFunctionType::NONE && !modifyAST(function, subquery_function_type))
|
||||
return false;
|
||||
|
||||
pos.increaseDepth();
|
||||
node = function;
|
||||
}
|
||||
}
|
||||
|
||||
pos.depth = current_depth;
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
bool first = true;
|
||||
@ -393,7 +320,15 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node
|
||||
auto exp_list = std::make_shared<ASTExpressionList>();
|
||||
|
||||
ASTPtr elem;
|
||||
if (!(remaining_elem_parser ? remaining_elem_parser : first_elem_parser)->parse(pos, elem, expected))
|
||||
SubqueryFunctionType subquery_function_type = SubqueryFunctionType::NONE;
|
||||
if (allow_any_all_operators && ParserKeyword("ANY").ignore(pos, expected))
|
||||
subquery_function_type = SubqueryFunctionType::ANY;
|
||||
else if (allow_any_all_operators && ParserKeyword("ALL").ignore(pos, expected))
|
||||
subquery_function_type = SubqueryFunctionType::ALL;
|
||||
else if (!(remaining_elem_parser ? remaining_elem_parser : first_elem_parser)->parse(pos, elem, expected))
|
||||
return false;
|
||||
|
||||
if (subquery_function_type != SubqueryFunctionType::NONE && !ParserSubquery().parse(pos, elem, expected))
|
||||
return false;
|
||||
|
||||
/// the first argument of the function is the previous element, the second is the next one
|
||||
@ -404,6 +339,9 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node
|
||||
exp_list->children.push_back(node);
|
||||
exp_list->children.push_back(elem);
|
||||
|
||||
if (allow_any_all_operators && subquery_function_type != SubqueryFunctionType::NONE && !modifyAST(function, subquery_function_type))
|
||||
return false;
|
||||
|
||||
/** special exception for the access operator to the element of the array `x[y]`, which
|
||||
* contains the infix part '[' and the suffix ''] '(specified as' [')
|
||||
*/
|
||||
|
@ -121,6 +121,8 @@ private:
|
||||
Operators_t overlapping_operators_to_skip = { (const char *[]){ nullptr } };
|
||||
ParserPtr first_elem_parser;
|
||||
ParserPtr remaining_elem_parser;
|
||||
/// =, !=, <, > ALL (subquery) / ANY (subquery)
|
||||
bool allow_any_all_operators = false;
|
||||
|
||||
public:
|
||||
/** `operators_` - allowed operators and their corresponding functions
|
||||
@ -130,8 +132,10 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
ParserLeftAssociativeBinaryOperatorList(Operators_t operators_, Operators_t overlapping_operators_to_skip_, ParserPtr && first_elem_parser_)
|
||||
: operators(operators_), overlapping_operators_to_skip(overlapping_operators_to_skip_), first_elem_parser(std::move(first_elem_parser_))
|
||||
ParserLeftAssociativeBinaryOperatorList(Operators_t operators_,
|
||||
Operators_t overlapping_operators_to_skip_, ParserPtr && first_elem_parser_, bool allow_any_all_operators_ = false)
|
||||
: operators(operators_), overlapping_operators_to_skip(overlapping_operators_to_skip_),
|
||||
first_elem_parser(std::move(first_elem_parser_)), allow_any_all_operators(allow_any_all_operators_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -341,12 +345,16 @@ class ParserComparisonExpression : public IParserBase
|
||||
private:
|
||||
static const char * operators[];
|
||||
static const char * overlapping_operators_to_skip[];
|
||||
ParserBetweenExpression elem_parser;
|
||||
ParserLeftAssociativeBinaryOperatorList operator_parser {operators,
|
||||
overlapping_operators_to_skip, std::make_unique<ParserBetweenExpression>(), true};
|
||||
|
||||
protected:
|
||||
const char * getName() const override{ return "comparison expression"; }
|
||||
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
|
||||
{
|
||||
return operator_parser.parse(pos, node, expected);
|
||||
}
|
||||
};
|
||||
|
||||
/** Parser for nullity checking with IS (NOT) NULL.
|
||||
@ -355,6 +363,7 @@ class ParserNullityChecking : public IParserBase
|
||||
{
|
||||
private:
|
||||
ParserComparisonExpression elem_parser;
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "nullity checking"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
@ -24,6 +24,7 @@
|
||||
#include <Parsers/ParserWatchQuery.h>
|
||||
#include <Parsers/QueryWithOutputSettingsPushDownVisitor.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -10,8 +10,7 @@ namespace DB
|
||||
|
||||
bool ParserUnionQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
if (!ParserSubquery().parse(pos, node, expected)
|
||||
&& !ParserSelectQuery().parse(pos, node, expected))
|
||||
if (!ParserSubquery().parse(pos, node, expected) && !ParserSelectQuery().parse(pos, node, expected))
|
||||
return false;
|
||||
|
||||
if (const auto * ast_subquery = node->as<ASTSubquery>())
|
||||
|
@ -36,9 +36,6 @@ IntersectOrExceptStep::IntersectOrExceptStep(
|
||||
, max_threads(max_threads_)
|
||||
{
|
||||
input_streams = std::move(input_streams_);
|
||||
if (input_streams.size() == 1)
|
||||
output_stream = input_streams.front();
|
||||
else
|
||||
output_stream = DataStream{.header = header};
|
||||
}
|
||||
|
||||
@ -71,6 +68,7 @@ QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines,
|
||||
});
|
||||
}
|
||||
|
||||
/// For the case of union.
|
||||
cur_pipeline->addTransform(std::make_shared<ResizeProcessor>(header, cur_pipeline->getNumStreams(), 1));
|
||||
}
|
||||
|
||||
|
@ -4,6 +4,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// After visitor is applied, ASTSelectIntersectExcept always has two child nodes.
|
||||
IntersectOrExceptTransform::IntersectOrExceptTransform(const Block & header_, Operator operator_)
|
||||
: IProcessor(InputPorts(2, header_), {header_})
|
||||
, current_operator(operator_)
|
||||
|
@ -70,6 +70,10 @@ select * from (select 1 intersect select 1);
|
||||
1
|
||||
with (select number from numbers(10) intersect select 5) as a select a * 10;
|
||||
50
|
||||
with (select 5 except select 1) as a select a except select 5;
|
||||
with (select number from numbers(10) intersect select 5) as a select a intersect select 1;
|
||||
with (select number from numbers(10) intersect select 5) as a select a except select 1;
|
||||
5
|
||||
select count() from (select number from numbers(10) except select 5);
|
||||
9
|
||||
select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000));
|
||||
@ -102,6 +106,7 @@ select * from (select 1 union all select 2 union all select 3 union all select 4
|
||||
select 1 intersect (select 1 except select 2);
|
||||
1
|
||||
select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4;
|
||||
select 1 intersect select count() from (select 1 except select 2 intersect select 2 union all select 1);
|
||||
explain syntax select 1 intersect select 1;
|
||||
SELECT 1
|
||||
INTERSECT
|
||||
|
@ -21,6 +21,9 @@ select number from numbers(100) intersect select number from numbers(20, 60) exc
|
||||
|
||||
select * from (select 1 intersect select 1);
|
||||
with (select number from numbers(10) intersect select 5) as a select a * 10;
|
||||
with (select 5 except select 1) as a select a except select 5;
|
||||
with (select number from numbers(10) intersect select 5) as a select a intersect select 1;
|
||||
with (select number from numbers(10) intersect select 5) as a select a except select 1;
|
||||
select count() from (select number from numbers(10) except select 5);
|
||||
select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000));
|
||||
select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20));
|
||||
@ -35,6 +38,7 @@ select * from (select 1 union all select 2 union all select 3 union all select 4
|
||||
|
||||
select 1 intersect (select 1 except select 2);
|
||||
select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4;
|
||||
select 1 intersect select count() from (select 1 except select 2 intersect select 2 union all select 1);
|
||||
|
||||
explain syntax select 1 intersect select 1;
|
||||
explain syntax select 1 except select 1;
|
||||
|
@ -29,3 +29,23 @@ select number as a from numbers(10) where a != any (select 5 from numbers(3, 3))
|
||||
7
|
||||
8
|
||||
9
|
||||
select 1 < any (select 1 from numbers(10));
|
||||
0
|
||||
select 1 <= any (select 1 from numbers(10));
|
||||
1
|
||||
select 1 < any (select number from numbers(10));
|
||||
1
|
||||
select 1 > any (select number from numbers(10));
|
||||
1
|
||||
select 1 >= any (select number from numbers(10));
|
||||
1
|
||||
select 11 > all (select number from numbers(10));
|
||||
1
|
||||
select 11 <= all (select number from numbers(11));
|
||||
0
|
||||
select 11 < all (select 11 from numbers(10));
|
||||
0
|
||||
select 11 > all (select 11 from numbers(10));
|
||||
0
|
||||
select 11 >= all (select 11 from numbers(10));
|
||||
1
|
||||
|
@ -8,10 +8,19 @@ select 1 != all (select number from numbers(10));
|
||||
select 1 == all (select 1 from numbers(10));
|
||||
select 1 == all (select number from numbers(10));
|
||||
|
||||
|
||||
select 1 != any (select 1 from numbers(10));
|
||||
select 1 != any (select number from numbers(10));
|
||||
|
||||
select number as a from numbers(10) where a == any (select number from numbers(3, 3));
|
||||
select number as a from numbers(10) where a != any (select 5 from numbers(3, 3));
|
||||
|
||||
select 1 < any (select 1 from numbers(10));
|
||||
select 1 <= any (select 1 from numbers(10));
|
||||
select 1 < any (select number from numbers(10));
|
||||
select 1 > any (select number from numbers(10));
|
||||
select 1 >= any (select number from numbers(10));
|
||||
select 11 > all (select number from numbers(10));
|
||||
select 11 <= all (select number from numbers(11));
|
||||
select 11 < all (select 11 from numbers(10));
|
||||
select 11 > all (select 11 from numbers(10));
|
||||
select 11 >= all (select 11 from numbers(10));
|
||||
|
Loading…
Reference in New Issue
Block a user