Merge pull request #24757 from zdikov/subquery-operators

Subquery operators
This commit is contained in:
Kseniia Sumarokova 2021-08-15 21:33:59 +03:00 committed by GitHub
commit b024c215af
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
28 changed files with 1333 additions and 58 deletions

View File

@ -6,11 +6,12 @@
#include <Columns/ColumnVector.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnNullable.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypesNumber.h>
#include <common/StringRef.h>
#include <Common/assert_cast.h>
#include <DataTypes/DataTypeNullable.h>
#include <AggregateFunctions/IAggregateFunction.h>
#if !defined(ARCADIA_BUILD)
@ -49,6 +50,8 @@ private:
T value;
public:
static constexpr bool is_nullable = false;
bool has() const
{
return has_value;
@ -469,6 +472,8 @@ private:
char small_data[MAX_SMALL_STRING_SIZE]; /// Including the terminating zero.
public:
static constexpr bool is_nullable = false;
bool has() const
{
return size >= 0;
@ -692,6 +697,8 @@ private:
Field value;
public:
static constexpr bool is_nullable = false;
bool has() const
{
return !value.isNull();
@ -975,6 +982,68 @@ struct AggregateFunctionAnyLastData : Data
#endif
};
template <typename Data>
struct AggregateFunctionSingleValueOrNullData : Data
{
static constexpr bool is_nullable = true;
using Self = AggregateFunctionSingleValueOrNullData;
bool first_value = true;
bool is_null = false;
bool changeIfBetter(const IColumn & column, size_t row_num, Arena * arena)
{
if (first_value)
{
first_value = false;
this->change(column, row_num, arena);
return true;
}
else if (!this->isEqualTo(column, row_num))
{
is_null = true;
}
return false;
}
bool changeIfBetter(const Self & to, Arena * arena)
{
if (first_value)
{
first_value = false;
this->change(to, arena);
return true;
}
else if (!this->isEqualTo(to))
{
is_null = true;
}
return false;
}
void insertResultInto(IColumn & to) const
{
if (is_null || first_value)
{
to.insertDefault();
}
else
{
ColumnNullable & col = typeid_cast<ColumnNullable &>(to);
col.getNullMapColumn().insertDefault();
this->Data::insertResultInto(col.getNestedColumn());
}
}
static const char * name() { return "singleValueOrNull"; }
#if USE_EMBEDDED_COMPILER
static constexpr bool is_compilable = false;
#endif
};
/** Implement 'heavy hitters' algorithm.
* Selects most frequent value if its frequency is more than 50% in each thread of execution.
@ -1074,7 +1143,10 @@ public:
DataTypePtr getReturnType() const override
{
return this->argument_types.at(0);
auto result_type = this->argument_types.at(0);
if constexpr (Data::is_nullable)
return makeNullable(result_type);
return result_type;
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override

View File

@ -0,0 +1,27 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/HelpersMinMaxAny.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include "registerAggregateFunctions.h"
namespace DB
{
struct Settings;
namespace
{
AggregateFunctionPtr createAggregateFunctionSingleValueOrNull(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionSingleValueOrNullData>(name, argument_types, parameters, settings));
}
}
void registerAggregateFunctionSingleValueOrNull(AggregateFunctionFactory & factory)
{
factory.registerFunction("singleValueOrNull", createAggregateFunctionSingleValueOrNull);
}
}

View File

@ -48,6 +48,7 @@ void registerAggregateFunctionRankCorrelation(AggregateFunctionFactory &);
void registerAggregateFunctionMannWhitney(AggregateFunctionFactory &);
void registerAggregateFunctionWelchTTest(AggregateFunctionFactory &);
void registerAggregateFunctionStudentTTest(AggregateFunctionFactory &);
void registerAggregateFunctionSingleValueOrNull(AggregateFunctionFactory &);
void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory &);
class AggregateFunctionCombinatorFactory;
@ -113,6 +114,7 @@ void registerAggregateFunctions()
registerAggregateFunctionSequenceNextNode(factory);
registerAggregateFunctionWelchTTest(factory);
registerAggregateFunctionStudentTTest(factory);
registerAggregateFunctionSingleValueOrNull(factory);
registerWindowFunctions(factory);

View File

@ -563,6 +563,7 @@
M(593, ZERO_COPY_REPLICATION_ERROR) \
M(594, BZIP2_STREAM_DECODER_FAILED) \
M(595, BZIP2_STREAM_ENCODER_FAILED) \
M(596, INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH) \
\
M(998, POSTGRESQL_CONNECTION_FAILURE) \
M(999, KEEPER_EXCEPTION) \

View File

@ -1,14 +1,17 @@
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTCheckQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTCreateUserQuery.h>
#include <Parsers/ASTCreateRoleQuery.h>
#include <Parsers/ASTCreateQuotaQuery.h>
#include <Parsers/ASTCreateRoleQuery.h>
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Parsers/ASTCreateSettingsProfileQuery.h>
#include <Parsers/ASTCreateUserQuery.h>
#include <Parsers/ASTDropAccessEntityQuery.h>
#include <Parsers/ASTDropQuery.h>
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/ASTGrantQuery.h>
#include <Parsers/ASTInsertQuery.h>
#include <Parsers/ASTSelectIntersectExceptQuery.h>
#include <Parsers/ASTKillQueryQuery.h>
#include <Parsers/ASTOptimizeQuery.h>
#include <Parsers/ASTRenameQuery.h>
@ -24,11 +27,9 @@
#include <Parsers/ASTShowProcesslistQuery.h>
#include <Parsers/ASTShowTablesQuery.h>
#include <Parsers/ASTUseQuery.h>
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/TablePropertiesQueriesASTs.h>
#include <Parsers/ASTWatchQuery.h>
#include <Parsers/ASTGrantQuery.h>
#include <Parsers/MySQL/ASTCreateQuery.h>
#include <Parsers/TablePropertiesQueriesASTs.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterAlterQuery.h>
@ -44,9 +45,11 @@
#include <Interpreters/InterpreterDropQuery.h>
#include <Interpreters/InterpreterExistsQuery.h>
#include <Interpreters/InterpreterExplainQuery.h>
#include <Interpreters/InterpreterExternalDDLQuery.h>
#include <Interpreters/InterpreterFactory.h>
#include <Interpreters/InterpreterGrantQuery.h>
#include <Interpreters/InterpreterInsertQuery.h>
#include <Interpreters/InterpreterSelectIntersectExceptQuery.h>
#include <Interpreters/InterpreterKillQueryQuery.h>
#include <Interpreters/InterpreterOptimizeQuery.h>
#include <Interpreters/InterpreterRenameQuery.h>
@ -65,7 +68,6 @@
#include <Interpreters/InterpreterSystemQuery.h>
#include <Interpreters/InterpreterUseQuery.h>
#include <Interpreters/InterpreterWatchQuery.h>
#include <Interpreters/InterpreterExternalDDLQuery.h>
#include <Interpreters/OpenTelemetrySpanLog.h>
#include <Parsers/ASTSystemQuery.h>
@ -109,6 +111,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, ContextMut
ProfileEvents::increment(ProfileEvents::SelectQuery);
return std::make_unique<InterpreterSelectWithUnionQuery>(query, context, options);
}
else if (query->as<ASTSelectIntersectExceptQuery>())
{
return std::make_unique<InterpreterSelectIntersectExceptQuery>(query, context, options);
}
else if (query->as<ASTInsertQuery>())
{
ProfileEvents::increment(ProfileEvents::InsertQuery);

View File

@ -0,0 +1,148 @@
#include <Columns/getLeastSuperColumn.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterSelectIntersectExceptQuery.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Parsers/ASTSelectIntersectExceptQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Processors/QueryPlan/IQueryPlanStep.h>
#include <Processors/QueryPlan/IntersectOrExceptStep.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/ExpressionStep.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH;
extern const int LOGICAL_ERROR;
}
static Block getCommonHeader(const Blocks & headers)
{
size_t num_selects = headers.size();
Block common_header = headers.front();
size_t num_columns = common_header.columns();
for (size_t query_num = 1; query_num < num_selects; ++query_num)
{
if (headers[query_num].columns() != num_columns)
throw Exception(ErrorCodes::INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH,
"Different number of columns in IntersectExceptQuery elements:\n {} \nand\n {}",
common_header.dumpNames(), headers[query_num].dumpNames());
}
std::vector<const ColumnWithTypeAndName *> columns(num_selects);
for (size_t column_num = 0; column_num < num_columns; ++column_num)
{
for (size_t i = 0; i < num_selects; ++i)
columns[i] = &headers[i].getByPosition(column_num);
ColumnWithTypeAndName & result_elem = common_header.getByPosition(column_num);
result_elem = getLeastSuperColumn(columns);
}
return common_header;
}
InterpreterSelectIntersectExceptQuery::InterpreterSelectIntersectExceptQuery(
const ASTPtr & query_ptr_,
ContextPtr context_,
const SelectQueryOptions & options_)
: IInterpreterUnionOrSelectQuery(query_ptr_->clone(), context_, options_)
{
ASTSelectIntersectExceptQuery * ast = query_ptr->as<ASTSelectIntersectExceptQuery>();
final_operator = ast->final_operator;
const auto & children = ast->children;
size_t num_children = children.size();
/// AST must have been changed by the visitor.
if (final_operator == Operator::UNKNOWN || num_children != 2)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"SelectIntersectExceptyQuery has not been normalized (number of children: {})",
num_children);
nested_interpreters.resize(num_children);
for (size_t i = 0; i < num_children; ++i)
nested_interpreters[i] = buildCurrentChildInterpreter(children.at(i));
Blocks headers(num_children);
for (size_t query_num = 0; query_num < num_children; ++query_num)
headers[query_num] = nested_interpreters[query_num]->getSampleBlock();
result_header = getCommonHeader(headers);
}
std::unique_ptr<IInterpreterUnionOrSelectQuery>
InterpreterSelectIntersectExceptQuery::buildCurrentChildInterpreter(const ASTPtr & ast_ptr_)
{
if (ast_ptr_->as<ASTSelectWithUnionQuery>())
return std::make_unique<InterpreterSelectWithUnionQuery>(ast_ptr_, context, SelectQueryOptions());
if (ast_ptr_->as<ASTSelectQuery>())
return std::make_unique<InterpreterSelectQuery>(ast_ptr_, context, SelectQueryOptions());
if (ast_ptr_->as<ASTSelectIntersectExceptQuery>())
return std::make_unique<InterpreterSelectIntersectExceptQuery>(ast_ptr_, context, SelectQueryOptions());
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected query: {}", ast_ptr_->getID());
}
void InterpreterSelectIntersectExceptQuery::buildQueryPlan(QueryPlan & query_plan)
{
size_t num_plans = nested_interpreters.size();
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]);
if (!blocksHaveEqualStructure(plans[i]->getCurrentDataStream().header, result_header))
{
auto actions_dag = ActionsDAG::makeConvertingActions(
plans[i]->getCurrentDataStream().header.getColumnsWithTypeAndName(),
result_header.getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Position);
auto converting_step = std::make_unique<ExpressionStep>(plans[i]->getCurrentDataStream(), std::move(actions_dag));
converting_step->setStepDescription("Conversion before UNION");
plans[i]->addStep(std::move(converting_step));
}
data_streams[i] = plans[i]->getCurrentDataStream();
}
auto max_threads = context->getSettingsRef().max_threads;
auto step = std::make_unique<IntersectOrExceptStep>(std::move(data_streams), final_operator, max_threads);
query_plan.unitePlans(std::move(step), std::move(plans));
}
BlockIO InterpreterSelectIntersectExceptQuery::execute()
{
BlockIO res;
QueryPlan query_plan;
buildQueryPlan(query_plan);
auto pipeline = query_plan.buildQueryPipeline(
QueryPlanOptimizationSettings::fromContext(context),
BuildQueryPipelineSettings::fromContext(context));
res.pipeline = std::move(*pipeline);
res.pipeline.addInterpreterContext(context);
return res;
}
void InterpreterSelectIntersectExceptQuery::ignoreWithTotals()
{
for (auto & interpreter : nested_interpreters)
interpreter->ignoreWithTotals();
}
}

View File

@ -0,0 +1,46 @@
#pragma once
#include <Core/QueryProcessingStage.h>
#include <Interpreters/IInterpreter.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/IInterpreterUnionOrSelectQuery.h>
#include <Parsers/ASTSelectIntersectExceptQuery.h>
namespace DB
{
class Context;
class InterpreterSelectQuery;
class QueryPlan;
class InterpreterSelectIntersectExceptQuery : public IInterpreterUnionOrSelectQuery
{
using Operator = ASTSelectIntersectExceptQuery::Operator;
public:
InterpreterSelectIntersectExceptQuery(
const ASTPtr & query_ptr_,
ContextPtr context_,
const SelectQueryOptions & options_);
BlockIO execute() override;
Block getSampleBlock() { return result_header; }
void ignoreWithTotals() override;
private:
static String getName() { return "SelectIntersectExceptQuery"; }
std::unique_ptr<IInterpreterUnionOrSelectQuery>
buildCurrentChildInterpreter(const ASTPtr & ast_ptr_);
void buildQueryPlan(QueryPlan & query_plan) override;
std::vector<std::unique_ptr<IInterpreterUnionOrSelectQuery>> nested_interpreters;
Operator final_operator;
};
}

View File

@ -2,8 +2,10 @@
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/InterpreterSelectIntersectExceptQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSelectIntersectExceptQuery.h>
#include <Parsers/queryToString.h>
#include <Processors/QueryPlan/DistinctStep.h>
#include <Processors/QueryPlan/ExpressionStep.h>
@ -208,8 +210,10 @@ InterpreterSelectWithUnionQuery::buildCurrentChildInterpreter(const ASTPtr & ast
{
if (ast_ptr_->as<ASTSelectWithUnionQuery>())
return std::make_unique<InterpreterSelectWithUnionQuery>(ast_ptr_, context, options, current_required_result_column_names);
else
else if (ast_ptr_->as<ASTSelectQuery>())
return std::make_unique<InterpreterSelectQuery>(ast_ptr_, context, options, current_required_result_column_names);
else
return std::make_unique<InterpreterSelectIntersectExceptQuery>(ast_ptr_, context, options);
}
InterpreterSelectWithUnionQuery::~InterpreterSelectWithUnionQuery() = default;
@ -225,10 +229,14 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_,
}
if (is_subquery)
{
return cache[key]
= InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().subquery().analyze()).getSampleBlock();
}
else
{
return cache[key] = InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().analyze()).getSampleBlock();
}
}

View File

@ -1,5 +1,6 @@
#include <Interpreters/NormalizeSelectWithUnionQueryVisitor.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTSelectIntersectExceptQuery.h>
#include <Common/typeid_cast.h>
namespace DB

View File

@ -0,0 +1,129 @@
#include <Interpreters/SelectIntersectExceptQueryVisitor.h>
#include <Parsers/ASTExpressionList.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/*
* Note: there is a difference between intersect and except behaviour.
* `intersect` is supposed to be a part of the last SelectQuery, i.e. the sequence with no parenthesis:
* select 1 union all select 2 except select 1 intersect 2 except select 2 union distinct select 5;
* is interpreted as:
* select 1 union all select 2 except (select 1 intersect 2) except select 2 union distinct select 5;
* Whereas `except` is applied to all left union part like:
* (((select 1 union all select 2) except (select 1 intersect 2)) except select 2) union distinct select 5;
**/
void SelectIntersectExceptQueryMatcher::visit(ASTPtr & ast, Data & data)
{
if (auto * select_union = ast->as<ASTSelectWithUnionQuery>())
visit(*select_union, data);
}
void SelectIntersectExceptQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Data &)
{
const auto & union_modes = ast.list_of_modes;
if (union_modes.empty())
return;
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: {})",
union_modes.size(), selects.size());
std::reverse(selects.begin(), selects.end());
ASTs children = {selects.back()};
selects.pop_back();
ASTSelectWithUnionQuery::UnionModes modes;
for (const auto & mode : union_modes)
{
switch (mode)
{
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_modes = std::move(modes);
modes = {};
auto right = selects.back();
selects.pop_back();
auto except_node = std::make_shared<ASTSelectIntersectExceptQuery>();
except_node->final_operator = ASTSelectIntersectExceptQuery::Operator::EXCEPT;
except_node->children = {left, right};
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;
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;
}
}
}
if (!selects.empty())
{
auto right = selects.back();
selects.pop_back();
children.emplace_back(std::move(right));
}
ast.union_mode = ASTSelectWithUnionQuery::Mode::Unspecified;
ast.list_of_selects->children = std::move(children);
ast.list_of_modes = std::move(modes);
}
}

View File

@ -0,0 +1,31 @@
#pragma once
#include <unordered_set>
#include <Parsers/IAST.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Parsers/ASTSelectIntersectExceptQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
namespace DB
{
class ASTFunction;
class SelectIntersectExceptQueryMatcher
{
public:
struct Data {};
static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; }
static void visit(ASTPtr & ast, Data &);
static void visit(ASTSelectWithUnionQuery &, Data &);
};
/// Visit children first.
using SelectIntersectExceptQueryVisitor
= InDepthNodeVisitor<SelectIntersectExceptQueryMatcher, false>;
}

View File

@ -46,6 +46,7 @@
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Interpreters/SelectQueryOptions.h>
#include <Interpreters/executeQuery.h>
#include <Interpreters/SelectIntersectExceptQueryVisitor.h>
#include <Common/ProfileEvents.h>
#include <Common/SensitiveDataMasker.h>
@ -490,9 +491,16 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
ApplyWithGlobalVisitor().visit(ast);
}
/// Normalize SelectWithUnionQuery
NormalizeSelectWithUnionQueryVisitor::Data data{context->getSettingsRef().union_default_mode};
NormalizeSelectWithUnionQueryVisitor{data}.visit(ast);
{
SelectIntersectExceptQueryVisitor::Data data;
SelectIntersectExceptQueryVisitor{data}.visit(ast);
}
{
/// Normalize SelectWithUnionQuery
NormalizeSelectWithUnionQueryVisitor::Data data{context->getSettingsRef().union_default_mode};
NormalizeSelectWithUnionQueryVisitor{data}.visit(ast);
}
/// Check the limits.
checkASTSizeLimits(*ast, settings);

View File

@ -0,0 +1,41 @@
#include <Parsers/ASTSelectIntersectExceptQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
namespace DB
{
ASTPtr ASTSelectIntersectExceptQuery::clone() const
{
auto res = std::make_shared<ASTSelectIntersectExceptQuery>(*this);
res->children.clear();
for (const auto & child : children)
res->children.push_back(child->clone());
res->final_operator = final_operator;
cloneOutputOptions(*res);
return res;
}
void ASTSelectIntersectExceptQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it)
{
if (it != children.begin())
{
settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "")
<< (final_operator == Operator::INTERSECT ? "INTERSECT" : "EXCEPT")
<< (settings.hilite ? hilite_none : "")
<< settings.nl_or_ws;
}
(*it)->formatImpl(settings, state, frame);
}
}
}

View File

@ -0,0 +1,31 @@
#pragma once
#include <Parsers/ASTQueryWithOutput.h>
namespace DB
{
class ASTSelectIntersectExceptQuery : public ASTQueryWithOutput
{
public:
String getID(char) const override { return "SelectIntersectExceptQuery"; }
ASTPtr clone() const override;
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
const char * getQueryKindString() const override { return "SelectIntersectExcept"; }
enum class Operator
{
UNKNOWN,
INTERSECT,
EXCEPT
};
/// Final operator after applying visitor.
Operator final_operator = Operator::UNKNOWN;
};
}

View File

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

View File

@ -1713,6 +1713,8 @@ const char * ParserAlias::restricted_keywords[] =
"WHERE",
"WINDOW",
"WITH",
"INTERSECT",
"EXCEPT",
nullptr
};

View File

@ -1,17 +1,21 @@
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/parseIntervalKind.h>
#include <Parsers/ParserUnionQueryElement.h>
#include <Common/StringUtils/StringUtils.h>
namespace DB
{
const char * ParserMultiplicativeExpression::operators[] =
{
"*", "multiply",
@ -108,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);
@ -123,21 +133,33 @@ 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);
}
// SELECT ... UNION SELECT ...
else
{
union_modes.push_back(ASTSelectWithUnionQuery::Mode::Unspecified);
}
return true;
}
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;
@ -169,6 +191,91 @@ static bool parseOperator(IParser::Pos & pos, const char * op, Expected & expect
}
}
enum class SubqueryFunctionType
{
NONE,
ANY,
ALL
};
static bool modifyAST(ASTPtr ast, SubqueryFunctionType type)
{
/* Rewrite in AST:
* = ANY --> IN
* != ALL --> NOT IN
* = ALL --> IN (SELECT singleValueOrNull(*) FROM subquery)
* != ANY --> NOT IN (SELECT singleValueOrNull(*) FROM subquery)
**/
auto * function = assert_cast<ASTFunction *>(ast.get());
String operator_name = function->name;
auto function_equals = operator_name == "equals";
auto function_not_equals = operator_name == "notEquals";
String aggregate_function_name;
if (function_equals || function_not_equals)
{
if (operator_name == "notEquals")
function->name = "notIn";
else
function->name = "in";
if ((type == SubqueryFunctionType::ANY && function_equals)
|| (type == SubqueryFunctionType::ALL && function_not_equals))
{
return true;
}
aggregate_function_name = "singleValueOrNull";
}
else if (operator_name == "greaterOrEquals" || operator_name == "greater")
{
aggregate_function_name = (type == SubqueryFunctionType::ANY ? "min" : "max");
}
else if (operator_name == "lessOrEquals" || operator_name == "less")
{
aggregate_function_name = (type == SubqueryFunctionType::ANY ? "max" : "min");
}
else
return false;
/// subquery --> (SELECT aggregate_function(*) FROM subquery)
auto aggregate_function = makeASTFunction(aggregate_function_name, std::make_shared<ASTAsterisk>());
auto subquery_node = function->children[0]->children[1];
auto table_expression = std::make_shared<ASTTableExpression>();
table_expression->subquery = std::move(subquery_node);
table_expression->children.push_back(table_expression->subquery);
auto tables_in_select_element = std::make_shared<ASTTablesInSelectQueryElement>();
tables_in_select_element->table_expression = std::move(table_expression);
tables_in_select_element->children.push_back(tables_in_select_element->table_expression);
auto tables_in_select = std::make_shared<ASTTablesInSelectQuery>();
tables_in_select->children.push_back(std::move(tables_in_select_element));
auto select_exp_list = std::make_shared<ASTExpressionList>();
select_exp_list->children.push_back(aggregate_function);
auto select_query = std::make_shared<ASTSelectQuery>();
select_query->children.push_back(select_exp_list);
select_query->children.push_back(tables_in_select);
select_query->setExpression(ASTSelectQuery::Expression::SELECT, select_exp_list);
select_query->setExpression(ASTSelectQuery::Expression::TABLES, tables_in_select);
auto select_with_union_query = std::make_shared<ASTSelectWithUnionQuery>();
select_with_union_query->list_of_selects = std::make_shared<ASTExpressionList>();
select_with_union_query->list_of_selects->children.push_back(std::move(select_query));
select_with_union_query->children.push_back(select_with_union_query->list_of_selects);
auto new_subquery = std::make_shared<ASTSubquery>();
new_subquery->children.push_back(select_with_union_query);
ast->children[0]->children.back() = std::move(new_subquery);
return true;
}
bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
@ -213,7 +320,15 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node
auto exp_list = std::make_shared<ASTExpressionList>();
ASTPtr elem;
if (!(remaining_elem_parser ? remaining_elem_parser : first_elem_parser)->parse(pos, elem, expected))
SubqueryFunctionType subquery_function_type = SubqueryFunctionType::NONE;
if (allow_any_all_operators && ParserKeyword("ANY").ignore(pos, expected))
subquery_function_type = SubqueryFunctionType::ANY;
else if (allow_any_all_operators && ParserKeyword("ALL").ignore(pos, expected))
subquery_function_type = SubqueryFunctionType::ALL;
else if (!(remaining_elem_parser ? remaining_elem_parser : first_elem_parser)->parse(pos, elem, expected))
return false;
if (subquery_function_type != SubqueryFunctionType::NONE && !ParserSubquery().parse(pos, elem, expected))
return false;
/// the first argument of the function is the previous element, the second is the next one
@ -224,6 +339,9 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node
exp_list->children.push_back(node);
exp_list->children.push_back(elem);
if (allow_any_all_operators && subquery_function_type != SubqueryFunctionType::NONE && !modifyAST(function, subquery_function_type))
return false;
/** special exception for the access operator to the element of the array `x[y]`, which
* contains the infix part '[' and the suffix ''] '(specified as' [')
*/
@ -855,4 +973,3 @@ bool ParserKeyValuePairsList::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
}
}

View File

@ -79,14 +79,6 @@ private:
class ParserUnionList : public IParserBase
{
public:
ParserUnionList(ParserPtr && elem_parser_, ParserPtr && s_union_parser_, ParserPtr && s_all_parser_, ParserPtr && s_distinct_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_))
{
}
template <typename ElemFunc, typename SepFunc>
static bool parseUtil(Pos & pos, const ElemFunc & parse_element, const SepFunc & parse_separator)
{
@ -116,10 +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;
ASTSelectWithUnionQuery::UnionModes union_modes;
};
@ -133,6 +121,8 @@ private:
Operators_t overlapping_operators_to_skip = { (const char *[]){ nullptr } };
ParserPtr first_elem_parser;
ParserPtr remaining_elem_parser;
/// =, !=, <, > ALL (subquery) / ANY (subquery)
bool allow_any_all_operators = false;
public:
/** `operators_` - allowed operators and their corresponding functions
@ -142,8 +132,10 @@ public:
{
}
ParserLeftAssociativeBinaryOperatorList(Operators_t operators_, Operators_t overlapping_operators_to_skip_, ParserPtr && first_elem_parser_)
: operators(operators_), overlapping_operators_to_skip(overlapping_operators_to_skip_), first_elem_parser(std::move(first_elem_parser_))
ParserLeftAssociativeBinaryOperatorList(Operators_t operators_,
Operators_t overlapping_operators_to_skip_, ParserPtr && first_elem_parser_, bool allow_any_all_operators_ = false)
: operators(operators_), overlapping_operators_to_skip(overlapping_operators_to_skip_),
first_elem_parser(std::move(first_elem_parser_)), allow_any_all_operators(allow_any_all_operators_)
{
}
@ -353,7 +345,8 @@ class ParserComparisonExpression : public IParserBase
private:
static const char * operators[];
static const char * overlapping_operators_to_skip[];
ParserLeftAssociativeBinaryOperatorList operator_parser {operators, overlapping_operators_to_skip, std::make_unique<ParserBetweenExpression>()};
ParserLeftAssociativeBinaryOperatorList operator_parser {operators,
overlapping_operators_to_skip, std::make_unique<ParserBetweenExpression>(), true};
protected:
const char * getName() const override{ return "comparison expression"; }
@ -364,7 +357,6 @@ protected:
}
};
/** Parser for nullity checking with IS (NOT) NULL.
*/
class ParserNullityChecking : public IParserBase

View File

@ -1,27 +1,27 @@
#include <Parsers/ParserQueryWithOutput.h>
#include <Parsers/ParserShowTablesQuery.h>
#include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/ParserTablePropertiesQuery.h>
#include <Parsers/ParserDescribeTableQuery.h>
#include <Parsers/ParserShowProcesslistQuery.h>
#include <Parsers/ParserCheckQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ParserRenameQuery.h>
#include <Parsers/ParserAlterQuery.h>
#include <Parsers/ParserDropQuery.h>
#include <Parsers/ParserKillQueryQuery.h>
#include <Parsers/ParserOptimizeQuery.h>
#include <Parsers/ParserWatchQuery.h>
#include <Parsers/ParserSetQuery.h>
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ParserAlterQuery.h>
#include <Parsers/ParserCheckQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ParserDescribeTableQuery.h>
#include <Parsers/ParserDropQuery.h>
#include <Parsers/ParserExplainQuery.h>
#include <Parsers/ParserKillQueryQuery.h>
#include <Parsers/ParserOptimizeQuery.h>
#include <Parsers/ParserQueryWithOutput.h>
#include <Parsers/ParserRenameQuery.h>
#include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/ParserSetQuery.h>
#include <Parsers/ParserShowAccessEntitiesQuery.h>
#include <Parsers/ParserShowAccessQuery.h>
#include <Parsers/ParserShowCreateAccessEntityQuery.h>
#include <Parsers/ParserShowGrantsQuery.h>
#include <Parsers/ParserShowPrivilegesQuery.h>
#include <Parsers/ParserExplainQuery.h>
#include <Parsers/ParserShowProcesslistQuery.h>
#include <Parsers/ParserShowTablesQuery.h>
#include <Parsers/ParserTablePropertiesQuery.h>
#include <Parsers/ParserWatchQuery.h>
#include <Parsers/QueryWithOutputSettingsPushDownVisitor.h>

View File

@ -10,12 +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"));
ParserUnionList parser;
if (!parser.parse(pos, list_node, expected))
return false;

View File

@ -0,0 +1,87 @@
#include <Processors/QueryPlan/IntersectOrExceptStep.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionActions.h>
#include <Processors/QueryPipeline.h>
#include <Processors/Sources/NullSource.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/IntersectOrExceptTransform.h>
#include <Processors/ResizeProcessor.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
static Block checkHeaders(const DataStreams & input_streams_)
{
if (input_streams_.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot perform intersect/except on empty set of query plan steps");
Block res = input_streams_.front().header;
for (const auto & stream : input_streams_)
assertBlocksHaveEqualStructure(stream.header, res, "IntersectOrExceptStep");
return res;
}
IntersectOrExceptStep::IntersectOrExceptStep(
DataStreams input_streams_ , Operator operator_ , size_t max_threads_)
: header(checkHeaders(input_streams_))
, current_operator(operator_)
, max_threads(max_threads_)
{
input_streams = std::move(input_streams_);
output_stream = DataStream{.header = header};
}
QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings &)
{
auto pipeline = std::make_unique<QueryPipeline>();
QueryPipelineProcessorsCollector collector(*pipeline, this);
if (pipelines.empty())
{
pipeline->init(Pipe(std::make_shared<NullSource>(output_stream->header)));
processors = collector.detachProcessors();
return pipeline;
}
for (auto & cur_pipeline : pipelines)
{
/// Just in case.
if (!isCompatibleHeader(cur_pipeline->getHeader(), getOutputStream().header))
{
auto converting_dag = ActionsDAG::makeConvertingActions(
cur_pipeline->getHeader().getColumnsWithTypeAndName(),
getOutputStream().header.getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Name);
auto converting_actions = std::make_shared<ExpressionActions>(std::move(converting_dag));
cur_pipeline->addSimpleTransform([&](const Block & cur_header)
{
return std::make_shared<ExpressionTransform>(cur_header, converting_actions);
});
}
/// For the case of union.
cur_pipeline->addTransform(std::make_shared<ResizeProcessor>(header, cur_pipeline->getNumStreams(), 1));
}
*pipeline = QueryPipeline::unitePipelines(std::move(pipelines), max_threads);
pipeline->addTransform(std::make_shared<IntersectOrExceptTransform>(header, current_operator));
processors = collector.detachProcessors();
return pipeline;
}
void IntersectOrExceptStep::describePipeline(FormatSettings & settings) const
{
IQueryPlanStep::describePipeline(processors, settings);
}
}

View File

@ -0,0 +1,30 @@
#pragma once
#include <Processors/QueryPlan/IQueryPlanStep.h>
#include <Parsers/ASTSelectIntersectExceptQuery.h>
namespace DB
{
class IntersectOrExceptStep : public IQueryPlanStep
{
using Operator = ASTSelectIntersectExceptQuery::Operator;
public:
/// max_threads is used to limit the number of threads for result pipeline.
IntersectOrExceptStep(DataStreams input_streams_, Operator operator_, size_t max_threads_ = 0);
String getName() const override { return "IntersectOrExcept"; }
QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) override;
void describePipeline(FormatSettings & settings) const override;
private:
Block header;
Operator current_operator;
size_t max_threads;
Processors processors;
};
}

View File

@ -0,0 +1,197 @@
#include <Processors/Transforms/IntersectOrExceptTransform.h>
namespace DB
{
/// After visitor is applied, ASTSelectIntersectExcept always has two child nodes.
IntersectOrExceptTransform::IntersectOrExceptTransform(const Block & header_, Operator operator_)
: IProcessor(InputPorts(2, header_), {header_})
, current_operator(operator_)
{
const Names & columns = header_.getNames();
size_t num_columns = columns.empty() ? header_.columns() : columns.size();
key_columns_pos.reserve(columns.size());
for (size_t i = 0; i < num_columns; ++i)
{
auto pos = columns.empty() ? i : header_.getPositionByName(columns[i]);
key_columns_pos.emplace_back(pos);
}
}
IntersectOrExceptTransform::Status IntersectOrExceptTransform::prepare()
{
auto & output = outputs.front();
if (output.isFinished())
{
for (auto & in : inputs)
in.close();
return Status::Finished;
}
if (!output.canPush())
{
for (auto & input : inputs)
input.setNotNeeded();
return Status::PortFull;
}
if (current_output_chunk)
{
output.push(std::move(current_output_chunk));
}
if (finished_second_input)
{
if (inputs.front().isFinished())
{
output.finish();
return Status::Finished;
}
}
else if (inputs.back().isFinished())
{
finished_second_input = true;
}
if (!has_input)
{
InputPort & input = finished_second_input ? inputs.front() : inputs.back();
input.setNeeded();
if (!input.hasData())
return Status::NeedData;
current_input_chunk = input.pull();
has_input = true;
}
return Status::Ready;
}
void IntersectOrExceptTransform::work()
{
if (!finished_second_input)
{
accumulate(std::move(current_input_chunk));
}
else
{
filter(current_input_chunk);
current_output_chunk = std::move(current_input_chunk);
}
has_input = false;
}
template <typename Method>
void IntersectOrExceptTransform::addToSet(Method & method, const ColumnRawPtrs & columns, size_t rows, SetVariants & variants) const
{
typename Method::State state(columns, key_sizes, nullptr);
for (size_t i = 0; i < rows; ++i)
state.emplaceKey(method.data, i, variants.string_pool);
}
template <typename Method>
size_t IntersectOrExceptTransform::buildFilter(
Method & method, const ColumnRawPtrs & columns, IColumn::Filter & filter, size_t rows, SetVariants & variants) const
{
typename Method::State state(columns, key_sizes, nullptr);
size_t new_rows_num = 0;
for (size_t i = 0; i < rows; ++i)
{
auto find_result = state.findKey(method.data, i, variants.string_pool);
filter[i] = current_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT ? !find_result.isFound() : find_result.isFound();
if (filter[i])
++new_rows_num;
}
return new_rows_num;
}
void IntersectOrExceptTransform::accumulate(Chunk chunk)
{
auto num_rows = chunk.getNumRows();
auto columns = chunk.detachColumns();
ColumnRawPtrs column_ptrs;
column_ptrs.reserve(key_columns_pos.size());
for (auto pos : key_columns_pos)
column_ptrs.emplace_back(columns[pos].get());
if (!data)
data.emplace();
if (data->empty())
data->init(SetVariants::chooseMethod(column_ptrs, key_sizes));
auto & data_set = *data;
switch (data->type)
{
case SetVariants::Type::EMPTY:
break;
#define M(NAME) \
case SetVariants::Type::NAME: \
addToSet(*data_set.NAME, column_ptrs, num_rows, data_set); \
break;
APPLY_FOR_SET_VARIANTS(M)
#undef M
}
}
void IntersectOrExceptTransform::filter(Chunk & chunk)
{
auto num_rows = chunk.getNumRows();
auto columns = chunk.detachColumns();
ColumnRawPtrs column_ptrs;
column_ptrs.reserve(key_columns_pos.size());
for (auto pos : key_columns_pos)
column_ptrs.emplace_back(columns[pos].get());
if (!data)
data.emplace();
if (data->empty())
data->init(SetVariants::chooseMethod(column_ptrs, key_sizes));
size_t new_rows_num = 0;
IColumn::Filter filter(num_rows);
auto & data_set = *data;
switch (data->type)
{
case SetVariants::Type::EMPTY:
break;
#define M(NAME) \
case SetVariants::Type::NAME: \
new_rows_num = buildFilter(*data_set.NAME, column_ptrs, filter, num_rows, data_set); \
break;
APPLY_FOR_SET_VARIANTS(M)
#undef M
}
if (!new_rows_num)
return;
for (auto & column : columns)
column = column->filter(filter, -1);
chunk.setColumns(std::move(columns), new_rows_num);
}
}

View File

@ -0,0 +1,51 @@
#pragma once
#include <Processors/IProcessor.h>
#include <Interpreters/SetVariants.h>
#include <Core/ColumnNumbers.h>
#include <Parsers/ASTSelectIntersectExceptQuery.h>
namespace DB
{
class IntersectOrExceptTransform : public IProcessor
{
using Operator = ASTSelectIntersectExceptQuery::Operator;
public:
IntersectOrExceptTransform(const Block & header_, Operator operator_);
String getName() const override { return "IntersectOrExcept"; }
protected:
Status prepare() override;
void work() override;
private:
Operator current_operator;
ColumnNumbers key_columns_pos;
std::optional<SetVariants> data;
Sizes key_sizes;
Chunk current_input_chunk;
Chunk current_output_chunk;
bool finished_second_input = false;
bool has_input = false;
void accumulate(Chunk chunk);
void filter(Chunk & chunk);
template <typename Method>
void addToSet(Method & method, const ColumnRawPtrs & key_columns, size_t rows, SetVariants & variants) const;
template <typename Method>
size_t buildFilter(Method & method, const ColumnRawPtrs & columns,
IColumn::Filter & filter, size_t rows, SetVariants & variants) const;
};
}

View File

@ -0,0 +1,129 @@
-- { echo }
select 1 intersect select 1;
1
select 2 intersect select 1;
select 1 except select 1;
select 2 except select 1;
2
select number from numbers(20) intersect select number from numbers(5, 5);
5
6
7
8
9
select number from numbers(10) except select number from numbers(5);
5
6
7
8
9
select number, number+10 from numbers(12) except select number+5, number+15 from numbers(10);
0 10
1 11
2 12
3 13
4 14
select 1 except select 2 intersect select 1;
1
select 1 except select 2 intersect select 2;
1
select 1 intersect select 1 except select 2;
1
select 1 intersect select 1 except select 1;
select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 1;
1
select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2;
1
select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2 except select 1;
select number from numbers(10) except select 5;
0
1
2
3
4
6
7
8
9
select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20);
20
21
22
23
24
25
26
27
28
29
50
51
52
53
54
55
56
57
58
59
select * from (select 1 intersect select 1);
1
with (select number from numbers(10) intersect select 5) as a select a * 10;
50
with (select 5 except select 1) as a select a except select 5;
with (select number from numbers(10) intersect select 5) as a select a intersect select 1;
with (select number from numbers(10) intersect select 5) as a select a except select 1;
5
select count() from (select number from numbers(10) except select 5);
9
select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000));
600000
select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20));
20
select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20) union all select number from numbers(100, 10));
30
select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000) except select number from numbers(300000, 200000) except select number from numbers(600000, 200000));
200000
select 1 union all select 1 intersect select 1;
1
1
select 1 union all select 1 intersect select 2;
1
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;
1
2
4
5
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;
1
2
3
5
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;
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;
select 1 intersect select count() from (select 1 except select 2 intersect select 2 union all select 1);
explain syntax select 1 intersect select 1;
SELECT 1
INTERSECT
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

@ -0,0 +1,45 @@
-- { echo }
select 1 intersect select 1;
select 2 intersect select 1;
select 1 except select 1;
select 2 except select 1;
select number from numbers(20) intersect select number from numbers(5, 5);
select number from numbers(10) except select number from numbers(5);
select number, number+10 from numbers(12) except select number+5, number+15 from numbers(10);
select 1 except select 2 intersect select 1;
select 1 except select 2 intersect select 2;
select 1 intersect select 1 except select 2;
select 1 intersect select 1 except select 1;
select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 1;
select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2;
select 1 intersect select 1 except select 2 intersect select 1 except select 3 intersect select 2 except select 1;
select number from numbers(10) except select 5;
select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20);
select * from (select 1 intersect select 1);
with (select number from numbers(10) intersect select 5) as a select a * 10;
with (select 5 except select 1) as a select a except select 5;
with (select number from numbers(10) intersect select 5) as a select a intersect select 1;
with (select number from numbers(10) intersect select 5) as a select a except select 1;
select count() from (select number from numbers(10) except select 5);
select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000));
select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20));
select count() from (select number from numbers(100) intersect select number from numbers(20, 60) except select number from numbers(30, 20) except select number from numbers(60, 20) union all select number from numbers(100, 10));
select count() from (select number from numbers(1000000) intersect select number from numbers(200000, 600000) except select number from numbers(300000, 200000) except select number from numbers(600000, 200000));
select 1 union all select 1 intersect select 1;
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;
select 1 intersect select count() from (select 1 except select 2 intersect select 2 union all select 1);
explain syntax select 1 intersect select 1;
explain syntax select 1 except select 1;
explain syntax select 1 union all select 2 except (select 2 except select 1 union all select 1) except select 4;

View File

@ -0,0 +1,51 @@
-- { echo }
select 1 == any (select number from numbers(10));
1
select 1 == any (select number from numbers(2, 10));
0
select 1 != all (select 1 from numbers(10));
0
select 1 != all (select number from numbers(10));
0
select 1 == all (select 1 from numbers(10));
1
select 1 == all (select number from numbers(10));
0
select 1 != any (select 1 from numbers(10));
0
select 1 != any (select number from numbers(10));
1
select number as a from numbers(10) where a == any (select number from numbers(3, 3));
3
4
5
select number as a from numbers(10) where a != any (select 5 from numbers(3, 3));
0
1
2
3
4
6
7
8
9
select 1 < any (select 1 from numbers(10));
0
select 1 <= any (select 1 from numbers(10));
1
select 1 < any (select number from numbers(10));
1
select 1 > any (select number from numbers(10));
1
select 1 >= any (select number from numbers(10));
1
select 11 > all (select number from numbers(10));
1
select 11 <= all (select number from numbers(11));
0
select 11 < all (select 11 from numbers(10));
0
select 11 > all (select 11 from numbers(10));
0
select 11 >= all (select 11 from numbers(10));
1

View File

@ -0,0 +1,26 @@
-- { echo }
select 1 == any (select number from numbers(10));
select 1 == any (select number from numbers(2, 10));
select 1 != all (select 1 from numbers(10));
select 1 != all (select number from numbers(10));
select 1 == all (select 1 from numbers(10));
select 1 == all (select number from numbers(10));
select 1 != any (select 1 from numbers(10));
select 1 != any (select number from numbers(10));
select number as a from numbers(10) where a == any (select number from numbers(3, 3));
select number as a from numbers(10) where a != any (select 5 from numbers(3, 3));
select 1 < any (select 1 from numbers(10));
select 1 <= any (select 1 from numbers(10));
select 1 < any (select number from numbers(10));
select 1 > any (select number from numbers(10));
select 1 >= any (select number from numbers(10));
select 11 > all (select number from numbers(10));
select 11 <= all (select number from numbers(11));
select 11 < all (select 11 from numbers(10));
select 11 > all (select 11 from numbers(10));
select 11 >= all (select 11 from numbers(10));