mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
fix
fix test
This commit is contained in:
parent
1ca7dcb0a8
commit
b4c933e585
@ -11,6 +11,8 @@
|
|||||||
#include <Processors/QueryPlan/UnionStep.h>
|
#include <Processors/QueryPlan/UnionStep.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
|
|
||||||
|
#include <Interpreters/InDepthNodeVisitor.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -21,11 +23,120 @@ namespace ErrorCodes
|
|||||||
extern const int EXPECTED_ALL_OR_DISTINCT;
|
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(
|
InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery(
|
||||||
const ASTPtr & query_ptr_, const Context & context_, const SelectQueryOptions & options_, const Names & required_result_column_names)
|
const ASTPtr & query_ptr_, const Context & context_, const SelectQueryOptions & options_, const Names & required_result_column_names)
|
||||||
: IInterpreterUnionOrSelectQuery(query_ptr_, context_, options_)
|
: 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();
|
size_t num_children = ast.list_of_selects->children.size();
|
||||||
if (!num_children)
|
if (!num_children)
|
||||||
@ -170,51 +281,6 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_,
|
|||||||
return cache[key] = InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().analyze()).getSampleBlock();
|
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)
|
void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan)
|
||||||
{
|
{
|
||||||
@ -228,93 +294,33 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan)
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// All UNION streams in the chain does not need to do DISTINCT transform
|
std::vector<std::unique_ptr<QueryPlan>> plans(num_plans);
|
||||||
// if (num_distinct_union == 0)
|
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)
|
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
|
|
||||||
{
|
{
|
||||||
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);
|
auto max_threads = context->getSettingsRef().max_threads;
|
||||||
DataStreams data_streams(num_distinct_union);
|
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)
|
query_plan.unitePlans(std::move(union_step), std::move(plans));
|
||||||
{
|
|
||||||
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));
|
|
||||||
|
|
||||||
|
const auto & query = query_ptr->as<ASTSelectWithUnionQuery &>();
|
||||||
|
if (query.union_mode == ASTSelectWithUnionQuery::Mode::DISTINCT)
|
||||||
|
{
|
||||||
/// Add distinct transform
|
/// Add distinct transform
|
||||||
const Settings & settings = context->getSettingsRef();
|
const Settings & settings = context->getSettingsRef();
|
||||||
SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode);
|
SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode);
|
||||||
|
|
||||||
auto distinct_step
|
auto distinct_step = std::make_unique<DistinctStep>(query_plan.getCurrentDataStream(), limits, 0, result_header.getNames(), false);
|
||||||
= std::make_unique<DistinctStep>(distinct_query_plan.getCurrentDataStream(), limits, 0, result_header.getNames(), false);
|
|
||||||
|
|
||||||
distinct_query_plan.addStep(std::move(distinct_step));
|
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));
|
|
||||||
}
|
}
|
||||||
#endif
|
|
||||||
}
|
}
|
||||||
|
|
||||||
BlockIO InterpreterSelectWithUnionQuery::execute()
|
BlockIO InterpreterSelectWithUnionQuery::execute()
|
||||||
|
@ -49,7 +49,6 @@ private:
|
|||||||
std::unique_ptr<IInterpreterUnionOrSelectQuery>
|
std::unique_ptr<IInterpreterUnionOrSelectQuery>
|
||||||
buildCurrentChildInterpreter(const ASTPtr & ast_ptr_, const Names & current_required_result_column_names);
|
buildCurrentChildInterpreter(const ASTPtr & ast_ptr_, const Names & current_required_result_column_names);
|
||||||
|
|
||||||
// size_t optimizeUnionList();
|
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -3,6 +3,8 @@
|
|||||||
#include <Parsers/ASTSubquery.h>
|
#include <Parsers/ASTSubquery.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
|
|
||||||
|
#include <iostream>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -16,6 +18,8 @@ ASTPtr ASTSelectWithUnionQuery::clone() const
|
|||||||
|
|
||||||
res->union_mode = union_mode;
|
res->union_mode = union_mode;
|
||||||
|
|
||||||
|
res->list_of_modes = list_of_modes;
|
||||||
|
|
||||||
cloneOutputOptions(*res);
|
cloneOutputOptions(*res);
|
||||||
return 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)
|
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 != list_of_selects->children.begin())
|
||||||
settings.ostr
|
settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") << "UNION "
|
||||||
<< settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "")
|
<< mode_to_str((is_normalized) ? union_mode : list_of_modes[it - list_of_selects->children.begin() - 1])
|
||||||
<< "UNION "
|
<< (settings.hilite ? hilite_none : "");
|
||||||
<< mode_to_str(union_mode) << (settings.hilite ? hilite_none : "");
|
|
||||||
if (auto * node = (*it)->as<ASTSelectWithUnionQuery>())
|
if (auto * node = (*it)->as<ASTSelectWithUnionQuery>())
|
||||||
{
|
{
|
||||||
auto sub_query = std::make_shared<ASTSubquery>();
|
if (node->list_of_selects->children.size() == 1)
|
||||||
sub_query->children.push_back(*it);
|
{
|
||||||
sub_query->formatImpl(settings, state, frame);
|
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
|
else
|
||||||
{
|
{
|
||||||
|
@ -27,6 +27,10 @@ public:
|
|||||||
|
|
||||||
Mode union_mode;
|
Mode union_mode;
|
||||||
|
|
||||||
|
UnionModes list_of_modes;
|
||||||
|
|
||||||
|
bool is_normalized = false;
|
||||||
|
|
||||||
ASTPtr list_of_selects;
|
ASTPtr list_of_selects;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -128,7 +128,7 @@ bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
}
|
}
|
||||||
// SELECT ... UNION SELECT ...
|
// SELECT ... UNION SELECT ...
|
||||||
else
|
else
|
||||||
union_modes.push_back(ASTSelectWithUnionQuery::Mode::DISTINCT);
|
union_modes.push_back(ASTSelectWithUnionQuery::Mode::Unspecified);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
return false;
|
return false;
|
||||||
|
@ -6,84 +6,6 @@
|
|||||||
|
|
||||||
namespace DB
|
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)
|
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:
|
/// 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`
|
/// 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 &>();
|
auto & expr_list = list_node->as<ASTExpressionList &>();
|
||||||
if (expr_list.children.size() == 1)
|
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>();
|
auto select_with_union_query = std::make_shared<ASTSelectWithUnionQuery>();
|
||||||
|
|
||||||
node = select_with_union_query;
|
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);
|
select_with_union_query->children.push_back(select_with_union_query->list_of_selects);
|
||||||
|
select_with_union_query->list_of_modes = parser.getUnionModes();
|
||||||
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));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
@ -8,6 +8,11 @@
|
|||||||
1
|
1
|
||||||
1
|
1
|
||||||
1
|
1
|
||||||
|
a
|
||||||
|
a
|
||||||
|
a
|
||||||
|
a
|
||||||
|
a
|
||||||
1
|
1
|
||||||
1
|
1
|
||||||
all
|
all
|
||||||
@ -44,6 +49,28 @@ all
|
|||||||
1
|
1
|
||||||
1
|
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
|
||||||
all
|
all
|
||||||
1
|
1
|
||||||
@ -67,3 +94,16 @@ all
|
|||||||
1
|
1
|
||||||
1
|
1
|
||||||
1
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
@ -1,26 +1,59 @@
|
|||||||
SELECT 1;
|
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 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 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 1 UNION SELECT 1 UNION SELECT 1 UNION ALL SELECT 1;
|
||||||
|
|
||||||
SELECT 'all' UNION SELECT 'all' UNION ALL SELECT 'all';
|
SELECT 'all' UNION SELECT 'all' UNION ALL SELECT 'all';
|
||||||
|
|
||||||
SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1;
|
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 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 (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 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 (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 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)));
|
SELECT * FROM (SELECT 1 UNION ALL (SELECT 1 UNION SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1)));
|
||||||
|
|
||||||
SET union_default_mode='ALL';
|
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 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 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 1 UNION SELECT 1 UNION SELECT 1 UNION ALL SELECT 1;
|
||||||
|
|
||||||
SELECT 'all' UNION SELECT 'all' UNION ALL SELECT 'all';
|
SELECT 'all' UNION SELECT 'all' UNION ALL SELECT 'all';
|
||||||
|
|
||||||
SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION DISTINCT SELECT 1;
|
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 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 (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 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 (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 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)));
|
SELECT * FROM (SELECT 1 UNION ALL (SELECT 1 UNION SELECT 1 UNION ALL (SELECT 1 UNION ALL SELECT 1)));
|
||||||
|
@ -18,16 +18,17 @@ Union
|
|||||||
Expression (Projection)
|
Expression (Projection)
|
||||||
Expression (Before ORDER BY and SELECT)
|
Expression (Before ORDER BY and SELECT)
|
||||||
ReadFromStorage (Read from SystemOne)
|
ReadFromStorage (Read from SystemOne)
|
||||||
Union
|
Distinct
|
||||||
Expression (Projection)
|
Union
|
||||||
Expression (Before ORDER BY and SELECT)
|
Expression (Projection)
|
||||||
ReadFromStorage (Read from SystemOne)
|
Expression (Before ORDER BY and SELECT)
|
||||||
Expression (Projection)
|
ReadFromStorage (Read from SystemOne)
|
||||||
Expression (Before ORDER BY and SELECT)
|
Expression (Projection)
|
||||||
ReadFromStorage (Read from SystemOne)
|
Expression (Before ORDER BY and SELECT)
|
||||||
Expression (Projection)
|
ReadFromStorage (Read from SystemOne)
|
||||||
Expression (Before ORDER BY and SELECT)
|
Expression (Projection)
|
||||||
ReadFromStorage (Read from SystemOne)
|
Expression (Before ORDER BY and SELECT)
|
||||||
|
ReadFromStorage (Read from SystemOne)
|
||||||
Distinct
|
Distinct
|
||||||
Union
|
Union
|
||||||
Expression (Projection)
|
Expression (Projection)
|
||||||
@ -79,6 +80,132 @@ Union
|
|||||||
Expression (Projection)
|
Expression (Projection)
|
||||||
Expression (Before ORDER BY and SELECT)
|
Expression (Before ORDER BY and SELECT)
|
||||||
ReadFromStorage (Read from SystemOne)
|
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 (Projection)
|
||||||
Expression (Before ORDER BY and SELECT)
|
Expression (Before ORDER BY and SELECT)
|
||||||
ReadFromStorage (Read from SystemOne)
|
ReadFromStorage (Read from SystemOne)
|
||||||
@ -116,11 +243,10 @@ Union
|
|||||||
Expression (Projection)
|
Expression (Projection)
|
||||||
Expression (Before ORDER BY and SELECT)
|
Expression (Before ORDER BY and SELECT)
|
||||||
ReadFromStorage (Read from SystemOne)
|
ReadFromStorage (Read from SystemOne)
|
||||||
Distinct
|
Union
|
||||||
Union
|
Expression (Projection)
|
||||||
Expression (Projection)
|
Expression (Before ORDER BY and SELECT)
|
||||||
Expression (Before ORDER BY and SELECT)
|
ReadFromStorage (Read from SystemOne)
|
||||||
ReadFromStorage (Read from SystemOne)
|
Expression (Projection)
|
||||||
Expression (Projection)
|
Expression (Before ORDER BY and SELECT)
|
||||||
Expression (Before ORDER BY and SELECT)
|
ReadFromStorage (Read from SystemOne)
|
||||||
ReadFromStorage (Read from SystemOne)
|
|
||||||
|
@ -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 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 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) UNION SELECT 1))))))))))))));
|
||||||
EXPLAIN (((((((((((((((((((((((((((((SELECT 1 UNION ALL SELECT 1)))))))))))))))))))))))))))));
|
EXPLAIN (((((((((((((((((((((((((((((SELECT 1 UNION SELECT 1)))))))))))))))))))))))))))));
|
||||||
EXPLAIN (((((((((((((((((((((((((((((SELECT 1 UNION DISTINCT 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)))))))))))))))))))))))))))));
|
||||||
|
Loading…
Reference in New Issue
Block a user