This commit is contained in:
kssenii 2021-08-14 15:31:55 +03:00
parent 47fb923975
commit 18ab53488f
15 changed files with 117 additions and 274 deletions

View File

@ -56,7 +56,7 @@ InterpreterSelectIntersectExceptQuery::InterpreterSelectIntersectExceptQuery(
ASTSelectIntersectExceptQuery * ast = query_ptr->as<ASTSelectIntersectExceptQuery>();
final_operator = ast->final_operator;
const auto & children = ast->children[0]->children;
const auto & children = ast->children;
size_t num_children = children.size();
/// AST must have been changed by the visitor.

View File

@ -18,87 +18,12 @@ namespace DB
void SelectIntersectExceptQueryMatcher::visit(ASTPtr & ast, Data & data)
{
if (auto * select_intersect_except = ast->as<ASTSelectIntersectExceptQuery>())
{
if (select_intersect_except->final_operator != ASTSelectIntersectExceptQuery::Operator::UNKNOWN)
return;
data.initialize(select_intersect_except);
visit(*select_intersect_except, data);
}
else if (auto * select_union = ast->as<ASTSelectWithUnionQuery>())
{
if (auto * select_union = ast->as<ASTSelectWithUnionQuery>())
visit(*select_union, data);
}
}
void SelectIntersectExceptQueryMatcher::visit(ASTSelectIntersectExceptQuery & ast, Data & data)
{
/* Example: select 1 intersect select 1 intsect select 1 intersect select 1 intersect select 1;
*
* --SelectIntersectExceptQuery --SelectIntersectExceptQuery
* ---ExpressionList ---ExpressionList
* ----SelectQuery ----SelectIntersectExceptQuery
* ----SelectQuery ------ExpressionList
* ----SelectQuery ---> -------SelectIntersectExceptQuery
* ----SelectQuery --------ExpressionList
* ---------SelectQuery
* ---------SelectQuery
* -------SelectQuery
* ----SelectQuery
**/
auto & selects = data.reversed_list_of_selects;
if (selects.empty())
return;
const auto left = selects.back();
selects.pop_back();
const auto right = selects.back();
selects.pop_back();
auto & operators = data.reversed_list_of_operators;
const auto current_operator = operators.back();
operators.pop_back();
auto list_node = std::make_shared<ASTExpressionList>();
list_node->children = {left, right};
if (selects.empty())
{
ast.final_operator = current_operator;
ast.children = {std::move(list_node)};
}
else
{
auto select_intersect_except = std::make_shared<ASTSelectIntersectExceptQuery>();
select_intersect_except->final_operator = {current_operator};
select_intersect_except->list_of_selects = std::move(list_node);
select_intersect_except->children.push_back(select_intersect_except->list_of_selects);
selects.emplace_back(std::move(select_intersect_except));
}
visit(ast, data);
}
void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Data &)
{
/* Example: select 1 union all select 2 except select 1 except select 2 union distinct select 5;
*
* --SelectWithUnionQuery --SelectIntersectExceptQuery
* ---ExpressionList ---ExpressionList
* ----SelectQuery ----SelectIntersectExceptQuery
* ----SelectQuery -----ExpressionList
* ----SelectQuery (except) ---> ------SelectIntersectExceptQuery
* ----SelectQuery (except) -------ExpressionList
* ----SelectQuery --------SelectWithUnionQuery (select 1 union all select 2)
* --------SelectQuery (select 1)
* ------SelectQuery (select 2)
* -----SelectQuery (select 5)
**/
auto & union_modes = ast.list_of_modes;
if (union_modes.empty())
@ -107,8 +32,7 @@ void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Dat
auto selects = std::move(ast.list_of_selects->children);
if (union_modes.size() + 1 != selects.size())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Incorrect ASTSelectWithUnionQuery (modes: {}, selects: {})",
throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect ASTSelectWithUnionQuery (modes: {}, selects: {})",
union_modes.size(), selects.size());
std::reverse(selects.begin(), selects.end());
@ -119,39 +43,70 @@ void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Dat
for (const auto & mode : union_modes)
{
/// Flatten all previous selects into ASTSelectIntersectQuery
if (mode == ASTSelectWithUnionQuery::Mode::EXCEPT)
switch (mode)
{
auto left = std::make_shared<ASTSelectWithUnionQuery>();
left->union_mode = ASTSelectWithUnionQuery::Mode::ALL;
case ASTSelectWithUnionQuery::Mode::EXCEPT:
{
auto left = std::make_shared<ASTSelectWithUnionQuery>();
left->union_mode = ASTSelectWithUnionQuery::Mode::ALL;
left->list_of_selects = std::make_shared<ASTExpressionList>();
left->children.push_back(left->list_of_selects);
left->list_of_selects->children = std::move(children);
left->list_of_selects = std::make_shared<ASTExpressionList>();
left->children.push_back(left->list_of_selects);
left->list_of_selects->children = std::move(children);
left->list_of_modes = std::move(modes);
modes = {};
left->list_of_modes = std::move(modes);
modes = {};
auto right = selects.back();
selects.pop_back();
auto right = selects.back();
selects.pop_back();
auto list_node = std::make_shared<ASTExpressionList>();
list_node->children = {left, right};
auto except_node = std::make_shared<ASTSelectIntersectExceptQuery>();
except_node->final_operator = ASTSelectIntersectExceptQuery::Operator::EXCEPT;
except_node->children = {left, right};
auto select_intersect_except = std::make_shared<ASTSelectIntersectExceptQuery>();
select_intersect_except->final_operator = {ASTSelectIntersectExceptQuery::Operator::EXCEPT};
select_intersect_except->children.emplace_back(std::move(list_node));
select_intersect_except->list_of_selects = std::make_shared<ASTExpressionList>();
select_intersect_except->list_of_selects->children.push_back(select_intersect_except->children[0]);
children = {except_node};
break;
}
case ASTSelectWithUnionQuery::Mode::INTERSECT:
{
bool from_except = false;
const auto * except_ast = typeid_cast<const ASTSelectIntersectExceptQuery *>(children.back().get());
if (except_ast && (except_ast->final_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT))
from_except = true;
children = {select_intersect_except};
}
else if (!selects.empty())
{
auto right = selects.back();
selects.pop_back();
children.emplace_back(std::move(right));
modes.push_back(mode);
ASTPtr left;
if (from_except)
{
left = std::move(children.back()->children[1]);
}
else
{
left = children.back();
children.pop_back();
}
auto right = selects.back();
selects.pop_back();
auto intersect_node = std::make_shared<ASTSelectIntersectExceptQuery>();
intersect_node->final_operator = ASTSelectIntersectExceptQuery::Operator::INTERSECT;
intersect_node->children = {left, right};
if (from_except)
children.back()->children[1] = std::move(intersect_node);
else
children.push_back(std::move(intersect_node));
break;
}
default:
{
auto right = selects.back();
selects.pop_back();
children.emplace_back(std::move(right));
modes.push_back(mode);
break;
}
}
}

View File

@ -17,27 +17,11 @@ class ASTFunction;
class SelectIntersectExceptQueryMatcher
{
public:
struct Data
{
Data() = default;
void initialize(const ASTSelectIntersectExceptQuery * select_intersect_except)
{
reversed_list_of_selects = select_intersect_except->list_of_selects->clone()->children;
reversed_list_of_operators = select_intersect_except->list_of_operators;
std::reverse(reversed_list_of_selects.begin(), reversed_list_of_selects.end());
std::reverse(reversed_list_of_operators.begin(), reversed_list_of_operators.end());
}
ASTs reversed_list_of_selects;
ASTSelectIntersectExceptQuery::Operators reversed_list_of_operators;
};
struct Data {};
static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; }
static void visit(ASTPtr & ast, Data &);
static void visit(ASTSelectIntersectExceptQuery &, Data &);
static void visit(ASTSelectWithUnionQuery &, Data &);
};

View File

@ -14,10 +14,6 @@ ASTPtr ASTSelectIntersectExceptQuery::clone() const
for (const auto & child : children)
res->children.push_back(child->clone());
if (res->list_of_selects)
res->list_of_selects = list_of_selects->clone();
res->list_of_operators = list_of_operators;
res->final_operator = final_operator;
cloneOutputOptions(*res);
@ -28,44 +24,18 @@ void ASTSelectIntersectExceptQuery::formatQueryImpl(const FormatSettings & setti
{
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
auto operator_to_str = [&](auto current_operator)
for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it)
{
if (current_operator == Operator::INTERSECT)
return "INTERSECT";
else
return "EXCEPT";
};
for (ASTs::const_iterator it = list_of_selects->children.begin(); it != list_of_selects->children.end(); ++it)
{
if (it != list_of_selects->children.begin())
if (it != children.begin())
{
settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "")
<< operator_to_str(list_of_operators[it - list_of_selects->children.begin() - 1])
<< (final_operator == Operator::INTERSECT ? "INTERSECT" : "EXCEPT")
<< (settings.hilite ? hilite_none : "");
}
if (auto * node = (*it)->as<ASTSelectWithUnionQuery>())
{
settings.ostr << settings.nl_or_ws << indent_str;
if (node->list_of_selects->children.size() == 1)
{
(node->list_of_selects->children.at(0))->formatImpl(settings, state, frame);
}
else
{
auto sub_query = std::make_shared<ASTSubquery>();
sub_query->children.push_back(*it);
sub_query->formatImpl(settings, state, frame);
}
}
else
{
if (it != list_of_selects->children.begin())
settings.ostr << settings.nl_or_ws;
(*it)->formatImpl(settings, state, frame);
}
if (it != children.begin())
settings.ostr << settings.nl_or_ws;
(*it)->formatImpl(settings, state, frame);
}
}

View File

@ -24,11 +24,6 @@ public:
EXCEPT
};
using Operators = std::vector<Operator>;
ASTPtr list_of_selects;
Operators list_of_operators;
/// Final operator after applying visitor.
Operator final_operator = Operator::UNKNOWN;
};

View File

@ -23,7 +23,8 @@ public:
Unspecified,
ALL,
DISTINCT,
EXCEPT
EXCEPT,
INTERSECT
};
using UnionModes = std::vector<Mode>;

View File

@ -9,6 +9,7 @@
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/parseIntervalKind.h>
#include <Parsers/ParserUnionQueryElement.h>
#include <Common/StringUtils/StringUtils.h>
@ -111,12 +112,18 @@ bool ParserList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserUnionQueryElement elem_parser;
ParserKeyword s_union_parser("UNION");
ParserKeyword s_all_parser("ALL");
ParserKeyword s_distinct_parser("DISTINCT");
ParserKeyword s_except_parser("EXCEPT");
ParserKeyword s_intersect_parser("INTERSECT");
ASTs elements;
auto parse_element = [&]
{
ASTPtr element;
if (!elem_parser->parse(pos, element, expected))
if (!elem_parser.parse(pos, element, expected))
return false;
elements.push_back(element);
@ -126,15 +133,15 @@ bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
/// Parse UNION type
auto parse_separator = [&]
{
if (s_union_parser->ignore(pos, expected))
if (s_union_parser.ignore(pos, expected))
{
// SELECT ... UNION ALL SELECT ...
if (s_all_parser->check(pos, expected))
if (s_all_parser.check(pos, expected))
{
union_modes.push_back(ASTSelectWithUnionQuery::Mode::ALL);
}
// SELECT ... UNION DISTINCT SELECT ...
else if (s_distinct_parser->check(pos, expected))
else if (s_distinct_parser.check(pos, expected))
{
union_modes.push_back(ASTSelectWithUnionQuery::Mode::DISTINCT);
}
@ -145,11 +152,16 @@ bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
return true;
}
else if (s_except_parser->check(pos, expected))
else if (s_except_parser.check(pos, expected))
{
union_modes.push_back(ASTSelectWithUnionQuery::Mode::EXCEPT);
return true;
}
else if (s_intersect_parser.check(pos, expected))
{
union_modes.push_back(ASTSelectWithUnionQuery::Mode::INTERSECT);
return true;
}
return false;
};

View File

@ -79,20 +79,6 @@ private:
class ParserUnionList : public IParserBase
{
public:
ParserUnionList(
ParserPtr && elem_parser_,
ParserPtr && s_union_parser_,
ParserPtr && s_all_parser_,
ParserPtr && s_distinct_parser_,
ParserPtr && s_except_parser_)
: elem_parser(std::move(elem_parser_))
, s_union_parser(std::move(s_union_parser_))
, s_all_parser(std::move(s_all_parser_))
, s_distinct_parser(std::move(s_distinct_parser_))
, s_except_parser(std::move(s_except_parser_))
{
}
template <typename ElemFunc, typename SepFunc>
static bool parseUtil(Pos & pos, const ElemFunc & parse_element, const SepFunc & parse_separator)
{
@ -122,11 +108,6 @@ protected:
const char * getName() const override { return "list of union elements"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
private:
ParserPtr elem_parser;
ParserPtr s_union_parser;
ParserPtr s_all_parser;
ParserPtr s_distinct_parser;
ParserPtr s_except_parser;
ASTSelectWithUnionQuery::UnionModes union_modes;
};

View File

@ -7,7 +7,6 @@
#include <Parsers/ParserDescribeTableQuery.h>
#include <Parsers/ParserDropQuery.h>
#include <Parsers/ParserExplainQuery.h>
#include <Parsers/ParserSelectIntersectExceptQuery.h>
#include <Parsers/ParserKillQueryQuery.h>
#include <Parsers/ParserOptimizeQuery.h>
#include <Parsers/ParserQueryWithOutput.h>

View File

@ -1,62 +0,0 @@
#include <Parsers/ASTSelectIntersectExceptQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ParserSelectIntersectExceptQuery.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ASTExpressionList.h>
namespace DB
{
bool ParserSelectIntersectExceptQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword intersect_keyword("INTERSECT");
ASTs elements;
ASTSelectIntersectExceptQuery::Operators operators;
auto parse_element = [&]() -> bool
{
ASTPtr element;
if (!ParserSelectQuery().parse(pos, element, expected) && !ParserSubquery().parse(pos, element, expected))
return false;
elements.push_back(element);
return true;
};
auto parse_separator = [&]() -> bool
{
if (!intersect_keyword.ignore(pos))
return false;
operators.emplace_back(ASTSelectIntersectExceptQuery::Operator::INTERSECT);
return true;
};
if (!ParserUnionList::parseUtil(pos, parse_element, parse_separator))
return false;
if (operators.empty() || elements.empty())
return false;
if (operators.size() + 1 != elements.size())
return false;
auto list_node = std::make_shared<ASTExpressionList>();
list_node->children = std::move(elements);
auto intersect_or_except_ast = std::make_shared<ASTSelectIntersectExceptQuery>();
node = intersect_or_except_ast;
intersect_or_except_ast->list_of_selects = list_node;
intersect_or_except_ast->children.push_back(intersect_or_except_ast->list_of_selects);
intersect_or_except_ast->list_of_operators = operators;
return true;
}
}

View File

@ -1,14 +0,0 @@
#pragma once
#include <Parsers/IParserBase.h>
namespace DB
{
class ParserSelectIntersectExceptQuery : public IParserBase
{
protected:
const char * getName() const override { return "INTERSECT or EXCEPT"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}

View File

@ -10,13 +10,7 @@ namespace DB
bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ASTPtr list_node;
ParserUnionList parser(
std::make_unique<ParserUnionQueryElement>(),
std::make_unique<ParserKeyword>("UNION"),
std::make_unique<ParserKeyword>("ALL"),
std::make_unique<ParserKeyword>("DISTINCT"),
std::make_unique<ParserKeyword>("EXCEPT"));
ParserUnionList parser;
if (!parser.parse(pos, list_node, expected))
return false;

View File

@ -2,7 +2,6 @@
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/ParserUnionQueryElement.h>
#include <Parsers/ParserSelectIntersectExceptQuery.h>
#include <Common/typeid_cast.h>
@ -12,7 +11,6 @@ namespace DB
bool ParserUnionQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserSubquery().parse(pos, node, expected)
&& !ParserSelectIntersectExceptQuery().parse(pos, node, expected)
&& !ParserSelectQuery().parse(pos, node, expected))
return false;

View File

@ -99,3 +99,26 @@ select * from (select 1 union all select 2 union all select 3 union all select 4
2
3
5
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;
explain syntax select 1 intersect select 1;
SELECT 1
INTERSECT
SELECT 1
explain syntax select 1 except select 1;
SELECT 1
EXCEPT
SELECT 1
explain syntax select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4;
SELECT 1
UNION ALL
SELECT 2
EXCEPT
SELECT 2
EXCEPT
SELECT 1
UNION ALL
SELECT 1
EXCEPT
SELECT 4

View File

@ -32,3 +32,10 @@ select 1 union all select 1 intersect select 2;
select * from (select 1 union all select 2 union all select 3 union all select 4 except select 3 union all select 5) order by 1;
select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5) order by 1;
select * from (select 1 union all select 2 union all select 3 union all select 4 intersect select 3 union all select 5 except select 1) order by 1;
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;
explain syntax select 1 intersect select 1;
explain syntax select 1 except select 1;
explain syntax select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4;