fix test
This commit is contained in:
feng lv 2020-11-09 15:44:11 +00:00
parent 1ca7dcb0a8
commit b4c933e585
10 changed files with 396 additions and 253 deletions

View File

@ -11,6 +11,8 @@
#include <Processors/QueryPlan/UnionStep.h>
#include <Common/typeid_cast.h>
#include <Interpreters/InDepthNodeVisitor.h>
namespace DB
{
@ -21,11 +23,120 @@ namespace ErrorCodes
extern const int EXPECTED_ALL_OR_DISTINCT;
}
struct CustomizeASTSelectWithUnionQueryNormalize
{
using TypeToVisit = ASTSelectWithUnionQuery;
const UnionMode & union_default_mode;
static void getSelectsFromUnionListNode(ASTPtr & ast_select, ASTs & selects)
{
if (auto * inner_union = ast_select->as<ASTSelectWithUnionQuery>())
{
/// We need flatten from last to first
for (auto child = inner_union->list_of_selects->children.rbegin(); child != inner_union->list_of_selects->children.rend();
++child)
getSelectsFromUnionListNode(*child, selects);
return;
}
selects.push_back(std::move(ast_select));
}
void visit(ASTSelectWithUnionQuery & ast, ASTPtr &)
{
auto & union_modes = ast.list_of_modes;
ASTs selects;
auto & select_list = ast.list_of_selects->children;
int i;
for (i = union_modes.size() - 1; i >= 0; --i)
{
/// Rewrite UNION Mode
if (union_modes[i] == ASTSelectWithUnionQuery::Mode::Unspecified)
{
if (union_default_mode == UnionMode::ALL)
union_modes[i] = ASTSelectWithUnionQuery::Mode::ALL;
else if (union_default_mode == UnionMode::DISTINCT)
union_modes[i] = ASTSelectWithUnionQuery::Mode::DISTINCT;
else
throw Exception(
"Expected ALL or DISTINCT in SelectWithUnion query, because setting (union_default_mode) is empty",
DB::ErrorCodes::EXPECTED_ALL_OR_DISTINCT);
}
if (union_modes[i] == ASTSelectWithUnionQuery::Mode::ALL)
{
if (auto * inner_union = select_list[i + 1]->as<ASTSelectWithUnionQuery>())
{
/// Inner_union is an UNION ALL list, just lift up
for (auto child = inner_union->list_of_selects->children.rbegin();
child != inner_union->list_of_selects->children.rend();
++child)
selects.push_back(std::move(*child));
}
else
selects.push_back(std::move(select_list[i + 1]));
}
/// flatten all left nodes and current node to a UNION DISTINCT list
else if (union_modes[i] == ASTSelectWithUnionQuery::Mode::DISTINCT)
{
auto distinct_list = std::make_shared<ASTSelectWithUnionQuery>();
distinct_list->list_of_selects = std::make_shared<ASTExpressionList>();
distinct_list->children.push_back(distinct_list->list_of_selects);
for (int j = i + 1; j >= 0; j--)
{
getSelectsFromUnionListNode(select_list[j], distinct_list->list_of_selects->children);
}
distinct_list->union_mode = ASTSelectWithUnionQuery::Mode::DISTINCT;
// Reverse children list
std::reverse(distinct_list->list_of_selects->children.begin(), distinct_list->list_of_selects->children.end());
distinct_list->is_normalized = true;
selects.push_back(std::move(distinct_list));
break;
}
}
/// No UNION DISTINCT or only one child in select_list
if (i == -1)
{
if (auto * inner_union = select_list[0]->as<ASTSelectWithUnionQuery>())
{
/// Inner_union is an UNION ALL list, just lift it up
for (auto child = inner_union->list_of_selects->children.rbegin(); child != inner_union->list_of_selects->children.rend();
++child)
selects.push_back(std::move(*child));
}
else
selects.push_back(std::move(select_list[0]));
}
// reverse children list
std::reverse(selects.begin(), selects.end());
ast.is_normalized = true;
ast.union_mode = ASTSelectWithUnionQuery::Mode::ALL;
ast.list_of_selects->children = std::move(selects);
}
};
using CustomizeASTSelectWithUnionQueryNormalizeVisitor
= InDepthNodeVisitor<OneTypeMatcher<CustomizeASTSelectWithUnionQueryNormalize>, false>;
InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery(
const ASTPtr & query_ptr_, const Context & context_, const SelectQueryOptions & options_, const Names & required_result_column_names)
: IInterpreterUnionOrSelectQuery(query_ptr_, context_, options_)
{
const auto & ast = query_ptr->as<ASTSelectWithUnionQuery &>();
auto & ast = query_ptr->as<ASTSelectWithUnionQuery &>();
/// Normalize AST Tree
if (!ast.is_normalized)
{
CustomizeASTSelectWithUnionQueryNormalizeVisitor::Data union_default_mode{context->getSettingsRef().union_default_mode};
CustomizeASTSelectWithUnionQueryNormalizeVisitor(union_default_mode).visit(query_ptr);
}
size_t num_children = ast.list_of_selects->children.size();
if (!num_children)
@ -170,51 +281,6 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_,
return cache[key] = InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().analyze()).getSampleBlock();
}
#if 0
size_t InterpreterSelectWithUnionQuery::optimizeUnionList()
{
auto union_distinct_num = 0;
auto union_default_mode = context->getSettingsRef().union_default_mode;
auto & ast = query_ptr->as<ASTSelectWithUnionQuery &>();
size_t num_selects = ast.list_of_selects->children.size();
if (!num_selects)
throw Exception("Logical error: no children in ASTSelectWithUnionQuery", ErrorCodes::LOGICAL_ERROR);
if (num_selects > 1)
{
for (auto & mode : ast.union_modes)
{
if (mode == ASTSelectWithUnionQuery::Mode::Unspecified)
{
if (union_default_mode == UnionMode::ALL)
mode = ASTSelectWithUnionQuery::Mode::ALL;
else if (union_default_mode == UnionMode::DISTINCT)
mode = ASTSelectWithUnionQuery::Mode::DISTINCT;
else
throw Exception(
"Expected ALL or DISTINCT in SelectWithUnion query, because setting (union_default_mode) is empty",
DB::ErrorCodes::EXPECTED_ALL_OR_DISTINCT);
}
}
/// Optimize general cases: if there is UNION DISTINCT, all previous UNION DISTINCT can be rewritten to UNION ALL.
/// Therefore we have at most one UNION DISTINCT in a sequence.
for (auto rit = ast.union_modes.rbegin(); rit != ast.union_modes.rend(); ++rit)
{
if (*rit == ASTSelectWithUnionQuery::Mode::DISTINCT)
{
/// Number of streams need to do a DISTINCT transform after unite
union_distinct_num = ast.union_modes.rend() - rit + 1;
for (auto mode_to_modify = ++rit; mode_to_modify != ast.union_modes.rend(); ++mode_to_modify)
*mode_to_modify = ASTSelectWithUnionQuery::Mode::ALL;
break;
}
}
}
return union_distinct_num;
}
#endif
void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan)
{
@ -228,93 +294,33 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan)
return;
}
/// All UNION streams in the chain does not need to do DISTINCT transform
// if (num_distinct_union == 0)
// {
std::vector<std::unique_ptr<QueryPlan>> plans(num_plans);
DataStreams data_streams(num_plans);
std::vector<std::unique_ptr<QueryPlan>> plans(num_plans);
DataStreams data_streams(num_plans);
for (size_t i = 0; i < num_plans; ++i)
{
plans[i] = std::make_unique<QueryPlan>();
nested_interpreters[i]->buildQueryPlan(*plans[i]);
data_streams[i] = plans[i]->getCurrentDataStream();
}
auto max_threads = context->getSettingsRef().max_threads;
auto union_step = std::make_unique<UnionStep>(std::move(data_streams), result_header, max_threads);
query_plan.unitePlans(std::move(union_step), std::move(plans));
const auto & query = query_ptr->as<ASTSelectWithUnionQuery &>();
if (query.union_mode == ASTSelectWithUnionQuery::Mode::DISTINCT)
{
/// Add distinct transform
const Settings & settings = context->getSettingsRef();
SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode);
auto distinct_step
= std::make_unique<DistinctStep>(query_plan.getCurrentDataStream(), limits, 0, result_header.getNames(), false);
query_plan.addStep(std::move(distinct_step));
}
// }
/// The first union_distinct_num UNION streams need to do a DISTINCT transform after unite
#if 0
else
for (size_t i = 0; i < num_plans; ++i)
{
QueryPlan distinct_query_plan;
plans[i] = std::make_unique<QueryPlan>();
nested_interpreters[i]->buildQueryPlan(*plans[i]);
data_streams[i] = plans[i]->getCurrentDataStream();
}
std::vector<std::unique_ptr<QueryPlan>> plans(num_distinct_union);
DataStreams data_streams(num_distinct_union);
auto max_threads = context->getSettingsRef().max_threads;
auto union_step = std::make_unique<UnionStep>(std::move(data_streams), result_header, max_threads);
for (size_t i = 0; i < num_distinct_union; ++i)
{
plans[i] = std::make_unique<QueryPlan>();
nested_interpreters[i]->buildQueryPlan(*plans[i]);
data_streams[i] = plans[i]->getCurrentDataStream();
}
auto max_threads = context->getSettingsRef().max_threads;
auto union_step = std::make_unique<UnionStep>(std::move(data_streams), result_header, max_threads);
distinct_query_plan.unitePlans(std::move(union_step), std::move(plans));
query_plan.unitePlans(std::move(union_step), std::move(plans));
const auto & query = query_ptr->as<ASTSelectWithUnionQuery &>();
if (query.union_mode == ASTSelectWithUnionQuery::Mode::DISTINCT)
{
/// Add distinct transform
const Settings & settings = context->getSettingsRef();
SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode);
auto distinct_step
= std::make_unique<DistinctStep>(distinct_query_plan.getCurrentDataStream(), limits, 0, result_header.getNames(), false);
auto distinct_step = std::make_unique<DistinctStep>(query_plan.getCurrentDataStream(), limits, 0, result_header.getNames(), false);
distinct_query_plan.addStep(std::move(distinct_step));
/// No other UNION streams after DISTINCT stream
if (num_plans == num_distinct_union)
{
query_plan = std::move(distinct_query_plan);
return;
}
/// Build final UNION step
std::vector<std::unique_ptr<QueryPlan>> final_plans(num_plans - num_distinct_union + 1);
DataStreams final_data_streams(num_plans - num_distinct_union + 1);
final_plans[0] = std::make_unique<QueryPlan>(std::move(distinct_query_plan));
final_data_streams[0] = final_plans[0]->getCurrentDataStream();
for (size_t i = 1; i < num_plans - num_distinct_union + 1; ++i)
{
final_plans[i] = std::make_unique<QueryPlan>();
nested_interpreters[num_distinct_union + i - 1]->buildQueryPlan(*final_plans[i]);
final_data_streams[i] = final_plans[i]->getCurrentDataStream();
}
auto final_union_step = std::make_unique<UnionStep>(std::move(final_data_streams), result_header, max_threads);
query_plan.unitePlans(std::move(final_union_step), std::move(final_plans));
query_plan.addStep(std::move(distinct_step));
}
#endif
}
BlockIO InterpreterSelectWithUnionQuery::execute()

View File

@ -49,7 +49,6 @@ private:
std::unique_ptr<IInterpreterUnionOrSelectQuery>
buildCurrentChildInterpreter(const ASTPtr & ast_ptr_, const Names & current_required_result_column_names);
// size_t optimizeUnionList();
};
}

View File

@ -3,6 +3,8 @@
#include <Parsers/ASTSubquery.h>
#include <Common/typeid_cast.h>
#include <iostream>
namespace DB
{
@ -16,6 +18,8 @@ ASTPtr ASTSelectWithUnionQuery::clone() const
res->union_mode = union_mode;
res->list_of_modes = list_of_modes;
cloneOutputOptions(*res);
return res;
}
@ -38,15 +42,24 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F
for (ASTs::const_iterator it = list_of_selects->children.begin(); it != list_of_selects->children.end(); ++it)
{
if (it != list_of_selects->children.begin())
settings.ostr
<< settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "")
<< "UNION "
<< mode_to_str(union_mode) << (settings.hilite ? hilite_none : "");
settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") << "UNION "
<< mode_to_str((is_normalized) ? union_mode : list_of_modes[it - list_of_selects->children.begin() - 1])
<< (settings.hilite ? hilite_none : "");
if (auto * node = (*it)->as<ASTSelectWithUnionQuery>())
{
auto sub_query = std::make_shared<ASTSubquery>();
sub_query->children.push_back(*it);
sub_query->formatImpl(settings, state, frame);
if (node->list_of_selects->children.size() == 1)
{
if (it != list_of_selects->children.begin())
settings.ostr << settings.nl_or_ws;
(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
{

View File

@ -27,6 +27,10 @@ public:
Mode union_mode;
UnionModes list_of_modes;
bool is_normalized = false;
ASTPtr list_of_selects;
};

View File

@ -128,7 +128,7 @@ bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
// SELECT ... UNION SELECT ...
else
union_modes.push_back(ASTSelectWithUnionQuery::Mode::DISTINCT);
union_modes.push_back(ASTSelectWithUnionQuery::Mode::Unspecified);
return true;
}
return false;

View File

@ -6,84 +6,6 @@
namespace DB
{
static void getSelectsFromUnionListNode(ASTPtr & ast_select, ASTs & selects)
{
if (auto * inner_union = ast_select->as<ASTSelectWithUnionQuery>())
{
/// We need flatten from last to first
for (auto child = inner_union->list_of_selects->children.rbegin(); child != inner_union->list_of_selects->children.rend(); ++child)
getSelectsFromUnionListNode(*child, selects);
return;
}
selects.push_back(std::move(ast_select));
}
void normalizeSelectList(ASTs & select_list, const ASTSelectWithUnionQuery::UnionModes & union_modes, ASTs & selects)
{
int i;
for (i = union_modes.size() - 1; i >= 0; --i)
{
if (union_modes[i] == ASTSelectWithUnionQuery::Mode::ALL)
{
if (auto * inner_union = select_list[i + 1]->as<ASTSelectWithUnionQuery>())
{
/// If inner_union is an UNION ALL list, just lift up
if (inner_union->union_mode == ASTSelectWithUnionQuery::Mode::ALL)
{
for (auto child = inner_union->list_of_selects->children.rbegin();
child != inner_union->list_of_selects->children.rend();
++child)
selects.push_back(std::move(*child));
}
/// inner_union is an UNION DISTINCT list,
// we cann't lift up
else
selects.push_back(std::move(select_list[i + 1]));
}
else
selects.push_back(std::move(select_list[i + 1]));
}
/// flatten all left nodes and current node to a UNION DISTINCT list
else if (union_modes[i] == ASTSelectWithUnionQuery::Mode::DISTINCT)
{
auto distinct_list = std::make_shared<ASTSelectWithUnionQuery>();
distinct_list->list_of_selects = std::make_shared<ASTExpressionList>();
distinct_list->children.push_back(distinct_list->list_of_selects);
for (int j = i + 1; j >= 0; j--)
{
getSelectsFromUnionListNode(select_list[j], distinct_list->list_of_selects->children);
}
distinct_list->union_mode = ASTSelectWithUnionQuery::Mode::DISTINCT;
// Reverse children list
std::reverse(distinct_list->list_of_selects->children.begin(), distinct_list->list_of_selects->children.end());
selects.push_back(std::move(distinct_list));
return;
}
}
/// No UNION DISTINCT or only one SELECT in select_list
if (i == -1)
{
if (auto * inner_union = select_list[0]->as<ASTSelectWithUnionQuery>())
{
/// If inner_union is an UNION ALL list, just lift it up
if (inner_union->union_mode == ASTSelectWithUnionQuery::Mode::ALL)
{
for (auto child = inner_union->list_of_selects->children.rbegin();
child != inner_union->list_of_selects->children.rend();
++child)
selects.push_back(std::move(*child));
}
/// inner_union is an UNION DISTINCT list,
// we cann't lift it up
else
selects.push_back(std::move(select_list[i + 1]));
}
else
selects.push_back(std::move(select_list[0]));
}
}
bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
@ -102,7 +24,7 @@ bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
/// so flatten may change it's semantics. For example:
/// flatten `SELECT 1 UNION (SELECT 1 UNION ALL SELETC 1)` -> `SELECT 1 UNION SELECT 1 UNION ALL SELECT 1`
/// Before normalize, if we got only one child which is ASTSelectWithUnionQuery, just lift it up
/// If we got only one child which is ASTSelectWithUnionQuery, just lift it up
auto & expr_list = list_node->as<ASTExpressionList &>();
if (expr_list.children.size() == 1)
{
@ -116,25 +38,9 @@ bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
auto select_with_union_query = std::make_shared<ASTSelectWithUnionQuery>();
node = select_with_union_query;
select_with_union_query->list_of_selects = std::make_shared<ASTExpressionList>();
select_with_union_query->list_of_selects = list_node; //std::make_shared<ASTExpressionList>();
select_with_union_query->children.push_back(select_with_union_query->list_of_selects);
auto union_modes = parser.getUnionModes();
normalizeSelectList(expr_list.children, union_modes, select_with_union_query->list_of_selects->children);
/// We need reverse children list
std::reverse(select_with_union_query->list_of_selects->children.begin(), select_with_union_query->list_of_selects->children.end());
select_with_union_query->union_mode = ASTSelectWithUnionQuery::Mode::ALL;
/// After normalize, if we only have one ASTSelectWithUnionQuery child, lift if up
if (select_with_union_query->list_of_selects->children.size() == 1)
{
if (select_with_union_query->list_of_selects->children.at(0)->as<ASTSelectWithUnionQuery>())
{
node = std::move(select_with_union_query->list_of_selects->children.at(0));
}
}
select_with_union_query->list_of_modes = parser.getUnionModes();
return true;
}

View File

@ -8,6 +8,11 @@
1
1
1
a
a
a
a
a
1
1
all
@ -44,6 +49,28 @@ all
1
1
1
1
1
1
1
1
a
a
a
a
a
a
a
a
a
a
a
a
1
1
1
1
all
all
all
1
@ -67,3 +94,16 @@ all
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1

View File

@ -1,26 +1,59 @@
SELECT 1;
(((((((SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1;
(((((((SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1;
SELECT 'a' UNION ALL SELECT 'a' UNION ALL SELECT 'a' UNION SELECT 'a';
SELECT 'a' UNION ALL (SELECT 'a' UNION ALL SELECT 'a' UNION SELECT 'a');
SELECT 'a' UNION SELECT 'a' UNION SELECT 'a' UNION ALL SELECT'a';
SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION ALL SELECT 1;
SELECT 'all' UNION SELECT 'all' UNION ALL SELECT 'all';
SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1;
SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION ALL SELECT 1;
SELECT 1 UNION (SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1 UNION ALL SELECT 1);
SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1 UNION ALL SELECT 1);
SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1)))))));
SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1)))))));
SELECT * FROM (SELECT 1 UNION ALL (SELECT 1 UNION SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1)));
SET union_default_mode='ALL';
(((((((SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1) UNION SELECT 1;
(((((((SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1) UNION ALL SELECT 1;
SELECT 'a' UNION ALL SELECT 'a' UNION ALL SELECT 'a' UNION SELECT 'a';
SELECT 'a' UNION ALL (SELECT 'a' UNION ALL SELECT 'a' UNION SELECT 'a');
SELECT 'a' UNION SELECT 'a' UNION SELECT 'a' UNION ALL SELECT'a';
SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION ALL SELECT 1;
SELECT 'all' UNION SELECT 'all' UNION ALL SELECT 'all';
SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1;
SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION ALL SELECT 1;
SELECT 1 UNION (SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1 UNION ALL SELECT 1);
SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1 UNION ALL SELECT 1);
SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1 UNION (SELECT 1)))))));
SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL(SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1)))))));
SELECT * FROM (SELECT 1 UNION ALL (SELECT 1 UNION SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1)));

View File

@ -18,16 +18,17 @@ Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Distinct
Union
Expression (Projection)
@ -79,6 +80,132 @@ Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
@ -116,11 +243,10 @@ Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
ReadFromStorage (Read from SystemOne)

View File

@ -1,13 +1,29 @@
EXPLAIN SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1;
EXPLAIN (SELECT 1 UNION ALL SELECT 1) UNION ALL SELECT 1;
EXPLAIN SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1);
EXPLAIN SELECT 1 UNION (SELECT 1 UNION ALL SELECT 1);
EXPLAIN SELECT 1 UNION DISTINCT SELECT 1 UNION DISTINCT SELECT 1;
EXPLAIN SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1;
EXPLAIN (SELECT 1 UNION DISTINCT SELECT 1) UNION DISTINCT SELECT 1;
EXPLAIN SELECT 1 UNION DISTINCT (SELECT 1 UNION DISTINCT SELECT 1);
EXPLAIN SELECT 1 UNION DISTINCT (SELECT 1 UNION SELECT 1);
EXPLAIN (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1 UNION DISTINCT SELECT 1))) UNION ALL (((SELECT 1) UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1 ) UNION DISTINCT SELECT 1))));
EXPLAIN (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1 UNION SELECT 1))) UNION ALL (((SELECT 1) UNION (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION SELECT 1 ) UNION DISTINCT SELECT 1))));
EXPLAIN (((((((((((((((SELECT 1 UNION ALL SELECT 1) UNION ALL SELECT 1))))))))))))));
EXPLAIN (((((((((((((((((((((((((((((SELECT 1 UNION ALL SELECT 1)))))))))))))))))))))))))))));
EXPLAIN (((((((((((((((((((((((((((((SELECT 1 UNION DISTINCT SELECT 1)))))))))))))))))))))))))))));
EXPLAIN (((((((((((((((SELECT 1 UNION ALL SELECT 1) UNION SELECT 1))))))))))))));
EXPLAIN (((((((((((((((((((((((((((((SELECT 1 UNION SELECT 1)))))))))))))))))))))))))))));
EXPLAIN (((((((((((((((((((((((((((((SELECT 1 UNION SELECT 1)))))))))))))))))))))))))))));
SET union_default_mode='ALL';
EXPLAIN SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1;
EXPLAIN (SELECT 1 UNION ALL SELECT 1) UNION ALL SELECT 1;
EXPLAIN SELECT 1 UNION (SELECT 1 UNION ALL SELECT 1);
EXPLAIN SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1;
EXPLAIN (SELECT 1 UNION DISTINCT SELECT 1) UNION DISTINCT SELECT 1;
EXPLAIN SELECT 1 UNION DISTINCT (SELECT 1 UNION SELECT 1);
EXPLAIN (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1 UNION SELECT 1))) UNION ALL (((SELECT 1) UNION (SELECT 1 UNION ALL (SELECT 1 UNION ALL (SELECT 1 UNION SELECT 1 ) UNION DISTINCT SELECT 1))));
EXPLAIN (((((((((((((((SELECT 1 UNION ALL SELECT 1) UNION SELECT 1))))))))))))));
EXPLAIN (((((((((((((((((((((((((((((SELECT 1 UNION SELECT 1)))))))))))))))))))))))))))));
EXPLAIN (((((((((((((((((((((((((((((SELECT 1 UNION SELECT 1)))))))))))))))))))))))))))));