mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Better
This commit is contained in:
parent
47fb923975
commit
18ab53488f
@ -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.
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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 &);
|
||||
};
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
|
@ -23,7 +23,8 @@ public:
|
||||
Unspecified,
|
||||
ALL,
|
||||
DISTINCT,
|
||||
EXCEPT
|
||||
EXCEPT,
|
||||
INTERSECT
|
||||
};
|
||||
|
||||
using UnionModes = std::vector<Mode>;
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
@ -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>
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
@ -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;
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user