CROSS to INNER JOIN (analyze expressions)

This commit is contained in:
chertus 2019-02-04 21:45:31 +03:00
parent 7bd76c89de
commit 27ba299c95
5 changed files with 254 additions and 51 deletions

View File

@ -5,6 +5,7 @@
#include <Parsers/ASTSelectQuery.h> #include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h> #include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTExpressionList.h> #include <Parsers/ASTExpressionList.h>
#include <Parsers/ParserTablesInSelectQuery.h> #include <Parsers/ParserTablesInSelectQuery.h>
#include <Parsers/ExpressionListParsers.h> #include <Parsers/ExpressionListParsers.h>
@ -19,23 +20,112 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
/// TODO: array join aliases? /// It checks if where expression could be moved to JOIN ON expression partially or entirely.
struct CheckColumnsVisitorData class CheckExpressionVisitorData
{ {
using TypeToVisit = ASTIdentifier; public:
using TypeToVisit = const ASTFunction;
const std::vector<DatabaseAndTableWithAlias> & tables; CheckExpressionVisitorData(const std::vector<DatabaseAndTableWithAlias> & tables_)
size_t visited; : tables(tables_)
size_t found; , save_where(false)
, flat_ands(true)
{}
size_t allMatch() const { return visited == found; } void visit(const ASTFunction & node, ASTPtr & ast)
void visit(ASTIdentifier & node, ASTPtr &)
{ {
++visited; if (node.name == "and")
for (const auto & t : tables) {
if (IdentifierSemantic::canReferColumnToTable(node, t)) if (!node.arguments || node.arguments->children.empty())
++found; throw Exception("Logical error: function requires argiment", ErrorCodes::LOGICAL_ERROR);
for (auto & child : node.arguments->children)
{
if (auto func = typeid_cast<const ASTFunction *>(child.get()))
{
if (func->name == "and")
flat_ands = false;
visit(*func, child);
}
else
save_where = true;
}
}
else if (node.name == "equals")
{
if (checkEquals(node))
asts_to_join_on.push_back(ast);
else
save_where = true;
}
else
save_where = true;
}
bool matchAny() const { return !asts_to_join_on.empty(); }
bool matchAll() const { return matchAny() && !save_where; }
bool canReuseWhere() const { return matchAll() && flat_ands; }
ASTPtr makeOnExpression()
{
if (asts_to_join_on.size() == 1)
return asts_to_join_on[0]->clone();
std::vector<ASTPtr> arguments;
arguments.reserve(asts_to_join_on.size());
for (auto & ast : asts_to_join_on)
arguments.emplace_back(ast->clone());
return makeASTFunction("and", std::move(arguments));
}
private:
const std::vector<DatabaseAndTableWithAlias> & tables;
std::vector<ASTPtr> asts_to_join_on;
bool save_where;
bool flat_ands;
bool checkEquals(const ASTFunction & node)
{
if (!node.arguments)
throw Exception("Logical error: function requires argiment", ErrorCodes::LOGICAL_ERROR);
if (node.arguments->children.size() != 2)
return false;
auto left = typeid_cast<const ASTIdentifier *>(node.arguments->children[0].get());
auto right = typeid_cast<const ASTIdentifier *>(node.arguments->children[1].get());
if (!left || !right)
return false;
return checkIdentifiers(*left, *right);
}
/// Check if the identifiers are from different joined tables. If it's a self joint, tables should have aliases.
/// select * from t1 a cross join t2 b where a.x = b.x
bool checkIdentifiers(const ASTIdentifier & left, const ASTIdentifier & right)
{
/// {best_match, berst_table_pos}
std::pair<size_t, size_t> left_best{0, 0};
std::pair<size_t, size_t> right_best{0, 0};
for (size_t i = 0; i < tables.size(); ++i)
{
size_t match = IdentifierSemantic::canReferColumnToTable(left, tables[i]);
if (match > left_best.first)
{
left_best.first = match;
left_best.second = i;
}
match = IdentifierSemantic::canReferColumnToTable(right, tables[i]);
if (match > right_best.first)
{
right_best.first = match;
right_best.second = i;
}
}
return left_best.first && right_best.first && (left_best.second != right_best.second);
} }
}; };
@ -100,27 +190,33 @@ std::vector<ASTPtr *> CrossToInnerJoinMatcher::visit(ASTPtr & ast, Data & data)
void CrossToInnerJoinMatcher::visit(ASTSelectQuery & select, ASTPtr & ast, Data & data) void CrossToInnerJoinMatcher::visit(ASTSelectQuery & select, ASTPtr & ast, Data & data)
{ {
using CheckColumnsMatcher = OneTypeMatcher<CheckColumnsVisitorData>; using CheckExpressionMatcher = OneTypeMatcher<CheckExpressionVisitorData, false>;
using CheckColumnsVisitor = InDepthNodeVisitor<CheckColumnsMatcher, true>; using CheckExpressionVisitor = InDepthNodeVisitor<CheckExpressionMatcher, true>;
std::vector<DatabaseAndTableWithAlias> table_names; std::vector<DatabaseAndTableWithAlias> table_names;
ASTPtr ast_join = getCrossJoin(select, table_names); ASTPtr ast_join = getCrossJoin(select, table_names);
if (!ast_join) if (!ast_join)
return; return;
/// check Identifier names from where expression CheckExpressionVisitor::Data visitor_data{table_names};
CheckColumnsVisitor::Data columns_data{table_names, 0, 0}; CheckExpressionVisitor(visitor_data).visit(select.where_expression);
CheckColumnsVisitor(columns_data).visit(select.where_expression);
if (!columns_data.allMatch()) if (visitor_data.matchAny())
return; {
auto & join = typeid_cast<ASTTableJoin &>(*ast_join);
join.kind = ASTTableJoin::Kind::Inner;
join.strictness = ASTTableJoin::Strictness::All;
auto & join = typeid_cast<ASTTableJoin &>(*ast_join); if (visitor_data.canReuseWhere())
join.kind = ASTTableJoin::Kind::Inner; join.on_expression.swap(select.where_expression);
join.strictness = ASTTableJoin::Strictness::All; /// TODO: do we need it? else
join.on_expression = visitor_data.makeOnExpression();
join.on_expression.swap(select.where_expression); if (visitor_data.matchAll())
join.children.push_back(join.on_expression); select.where_expression.reset();
join.children.push_back(join.on_expression);
}
ast = ast->clone(); /// rewrite AST in right manner ast = ast->clone(); /// rewrite AST in right manner
data.done = true; data.done = true;

View File

@ -53,7 +53,7 @@ private:
}; };
/// Simple matcher for one node type without complex traversal logic. /// Simple matcher for one node type without complex traversal logic.
template <typename _Data> template <typename _Data, bool _visit_children = true>
class OneTypeMatcher class OneTypeMatcher
{ {
public: public:
@ -62,7 +62,7 @@ public:
static constexpr const char * label = ""; static constexpr const char * label = "";
static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; } static bool needChildVisit(ASTPtr &, const ASTPtr &) { return _visit_children; }
static std::vector<ASTPtr *> visit(ASTPtr & ast, Data & data) static std::vector<ASTPtr *> visit(ASTPtr & ast, Data & data)
{ {

View File

@ -192,7 +192,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
if (!internal) if (!internal)
logQuery(query.substr(0, settings.log_queries_cut_to_length), context); logQuery(query.substr(0, settings.log_queries_cut_to_length), context);
if (settings.allow_experimental_multiple_joins_emulation) if (!internal && settings.allow_experimental_multiple_joins_emulation)
{ {
JoinToSubqueryTransformVisitor::Data join_to_subs_data; JoinToSubqueryTransformVisitor::Data join_to_subs_data;
JoinToSubqueryTransformVisitor(join_to_subs_data).visit(ast); JoinToSubqueryTransformVisitor(join_to_subs_data).visit(ast);
@ -200,7 +200,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
logQuery(queryToString(*ast), context); logQuery(queryToString(*ast), context);
} }
if (settings.allow_experimental_cross_to_join_conversion) if (!internal && settings.allow_experimental_cross_to_join_conversion)
{ {
CrossToInnerJoinVisitor::Data cross_to_inner; CrossToInnerJoinVisitor::Data cross_to_inner;
CrossToInnerJoinVisitor(cross_to_inner).visit(ast); CrossToInnerJoinVisitor(cross_to_inner).visit(ast);

View File

@ -1,23 +1,79 @@
cross cross
1 1 1 1 1 1 1 1
1 1 1 2
2 2 2 \N
1 1 1 1
1 1 1 2
2 2 2 \N 2 2 2 \N
cross nullable cross nullable
1 1 1 1 1 1 1 1
2 2 1 2
1 1 1 1
2 2 1 2
cross nullable vs not nullable cross nullable vs not nullable
1 1 1 1 1 1 1 1
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 3)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableExpression (children 1)\n Identifier t2\n TableJoin\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n 2 2 1 2
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 3)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableExpression (children 1)\n Identifier t2\n TableJoin\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 2)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableJoin (children 1)\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n TableExpression (children 1)\n Identifier t2\n
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 2)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableJoin (children 1)\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n TableExpression (children 1)\n Identifier t2\n
cross
1 1 1 1 1 1 1 1
2 2 1 2
cross self
1 1 1 1
2 2 2 2
1 1 1 1
2 2 2 2
cross one table expr
1 1 1 1
1 1 1 2
1 1 2 \N
1 1 3 \N
2 2 1 1
2 2 1 2
2 2 2 \N 2 2 2 \N
cross nullable 2 2 3 \N
1 1 1 1
1 1 1 2
1 1 2 \N
1 1 3 \N
2 2 1 1
2 2 1 2
2 2 2 \N
2 2 3 \N
cross multiple ands
1 1 1 1
1 1 1 1
cross and inside and
1 1 1 1
1 1 1 1
cross split conjunction
1 1 1 1 1 1 1 1
cross nullable vs not nullable
1 1 1 1 1 1 1 1
comma comma
1 1 1 1 1 1 1 1
1 1 1 2
2 2 2 \N 2 2 2 \N
comma nullable comma nullable
1 1 1 1 1 1 1 1
2 2 1 2
cross
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 3)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableExpression (children 1)\n Identifier t2\n TableJoin\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 2)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableJoin (children 1)\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n TableExpression (children 1)\n Identifier t2\n
cross nullable
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 3)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableExpression (children 1)\n Identifier t2\n TableJoin\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 2)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableJoin (children 1)\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n TableExpression (children 1)\n Identifier t2\n
cross nullable vs not nullable
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 3)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableExpression (children 1)\n Identifier t2\n TableJoin\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.b\n
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 2)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableJoin (children 1)\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.b\n TableExpression (children 1)\n Identifier t2\n
cross self
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 3)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1 (alias x)\n TablesInSelectQueryElement (children 2)\n TableExpression (children 1)\n Identifier t1 (alias y)\n TableJoin\n Function and (children 1)\n ExpressionList (children 2)\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier x.a\n Identifier y.a\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier x.b\n Identifier y.b\n
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 2)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1 (alias x)\n TablesInSelectQueryElement (children 2)\n TableJoin (children 1)\n Function and (children 1)\n ExpressionList (children 2)\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier x.a\n Identifier y.a\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier x.b\n Identifier y.b\n TableExpression (children 1)\n Identifier t1 (alias y)\n
cross one table expr
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 3)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableExpression (children 1)\n Identifier t2\n TableJoin\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t1.b\n
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 3)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableJoin\n TableExpression (children 1)\n Identifier t2\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t1.b\n
cross multiple ands
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 3)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableExpression (children 1)\n Identifier t2\n TableJoin\n Function and (children 1)\n ExpressionList (children 2)\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.b\n Identifier t2.b\n
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 2)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableJoin (children 1)\n Function and (children 1)\n ExpressionList (children 2)\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.b\n Identifier t2.b\n TableExpression (children 1)\n Identifier t2\n
cross and inside and
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 3)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableExpression (children 1)\n Identifier t2\n TableJoin\n Function and (children 1)\n ExpressionList (children 2)\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n Function and (children 1)\n ExpressionList (children 2)\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n Function and (children 1)\n ExpressionList (children 2)\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.b\n Identifier t2.b\n
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 2)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableJoin (children 1)\n Function and (children 1)\n ExpressionList (children 4)\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.b\n Identifier t2.b\n TableExpression (children 1)\n Identifier t2\n
cross split conjunction
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 3)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableExpression (children 1)\n Identifier t2\n TableJoin\n Function and (children 1)\n ExpressionList (children 4)\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.b\n Identifier t2.b\n Function greaterOrEquals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Literal UInt64_1\n Function greater (children 1)\n ExpressionList (children 2)\n Identifier t2.b\n Literal UInt64_0\n
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 3)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableJoin (children 1)\n Function and (children 1)\n ExpressionList (children 2)\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.b\n Identifier t2.b\n TableExpression (children 1)\n Identifier t2\n Function and (children 1)\n ExpressionList (children 4)\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.b\n Identifier t2.b\n Function greaterOrEquals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Literal UInt64_1\n Function greater (children 1)\n ExpressionList (children 2)\n Identifier t2.b\n Literal UInt64_0\n

View File

@ -1,3 +1,4 @@
SET enable_debug_queries = 1;
USE test; USE test;
DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t1;
@ -7,36 +8,86 @@ CREATE TABLE t1 (a Int8, b Nullable(Int8)) ENGINE = Memory;
CREATE TABLE t2 (a Int8, b Nullable(Int8)) ENGINE = Memory; CREATE TABLE t2 (a Int8, b Nullable(Int8)) ENGINE = Memory;
INSERT INTO t1 values (1,1), (2,2); INSERT INTO t1 values (1,1), (2,2);
INSERT INTO t2 values (1,1); INSERT INTO t2 values (1,1), (1,2);
INSERT INTO t2 (a) values (2), (3); INSERT INTO t2 (a) values (2), (3);
SELECT 'cross'; SELECT 'cross';
SET allow_experimental_cross_to_join_conversion = 0;
SELECT * FROM t1 cross join t2 where t1.a = t2.a;
SET allow_experimental_cross_to_join_conversion = 1;
SELECT * FROM t1 cross join t2 where t1.a = t2.a; SELECT * FROM t1 cross join t2 where t1.a = t2.a;
SELECT 'cross nullable'; SELECT 'cross nullable';
SET allow_experimental_cross_to_join_conversion = 0;
SELECT * FROM t1 cross join t2 where t1.b = t2.b;
SET allow_experimental_cross_to_join_conversion = 1;
SELECT * FROM t1 cross join t2 where t1.b = t2.b; SELECT * FROM t1 cross join t2 where t1.b = t2.b;
SELECT 'cross nullable vs not nullable'; SELECT 'cross nullable vs not nullable';
SET allow_experimental_cross_to_join_conversion = 0;
SELECT * FROM t1 cross join t2 where t1.a = t2.b; SELECT * FROM t1 cross join t2 where t1.a = t2.b;
SET allow_experimental_cross_to_join_conversion = 1;
SET enable_debug_queries = 1; SELECT * FROM t1 cross join t2 where t1.a = t2.b;
AST SELECT * FROM t1 cross join t2 where t1.a = t2.a; SELECT 'cross self';
AST SELECT * FROM t1, t2 where t1.a = t2.a; SET allow_experimental_cross_to_join_conversion = 0;
SELECT * FROM t1 x cross join t1 y where x.a = y.a and x.b = y.b;
SET allow_experimental_cross_to_join_conversion = 1;
SELECT * FROM t1 x cross join t1 y where x.a = y.a and x.b = y.b;
SELECT 'cross one table expr';
SET allow_experimental_cross_to_join_conversion = 0;
SELECT * FROM t1 cross join t2 where t1.a = t1.b order by (t1.a, t2.a, t2.b);
SET allow_experimental_cross_to_join_conversion = 1;
SELECT * FROM t1 cross join t2 where t1.a = t1.b order by (t1.a, t2.a, t2.b);
SELECT 'cross multiple ands';
SET allow_experimental_cross_to_join_conversion = 0;
--SELECT * FROM t1 cross join t2 where t1.a = t2.a and t1.a = t2.a and t1.b = t2.b and t1.a = t2.a;
SELECT * FROM t1 cross join t2 where t1.a = t2.a and t1.b = t2.b;
SET allow_experimental_cross_to_join_conversion = 1;
SELECT * FROM t1 cross join t2 where t1.a = t2.a and t1.b = t2.b;
SELECT 'cross and inside and';
SET allow_experimental_cross_to_join_conversion = 0;
--SELECT * FROM t1 cross join t2 where t1.a = t2.a and (t1.a = t2.a and (t1.a = t2.a and t1.b = t2.b));
--SELECT * FROM t1 x cross join t2 y where t1.a = t2.a and (t1.b = t2.b and (x.a = y.a and x.b = y.b));
SELECT * FROM t1 cross join t2 where t1.a = t2.a and (t1.b = t2.b and 1);
SET allow_experimental_cross_to_join_conversion = 1;
SELECT * FROM t1 cross join t2 where t1.a = t2.a and (t1.b = t2.b and 1);
SELECT 'cross split conjunction';
SET allow_experimental_cross_to_join_conversion = 0;
SELECT * FROM t1 cross join t2 where t1.a = t2.a and t1.b = t2.b and t1.a >= 1 and t2.b = 1;
SET allow_experimental_cross_to_join_conversion = 1;
SELECT * FROM t1 cross join t2 where t1.a = t2.a and t1.b = t2.b and t1.a >= 1 and t2.b = 1;
SET allow_experimental_cross_to_join_conversion = 1; SET allow_experimental_cross_to_join_conversion = 1;
AST SELECT * FROM t1 cross join t2 where t1.a = t2.a;
AST SELECT * FROM t1, t2 where t1.a = t2.a;
SELECT 'cross';
SELECT * FROM t1 cross join t2 where t1.a = t2.a;
SELECT 'cross nullable';
SELECT * FROM t1 cross join t2 where t1.b = t2.b;
SELECT 'cross nullable vs not nullable';
SELECT * FROM t1 cross join t2 where t1.a = t2.b;
SELECT 'comma'; SELECT 'comma';
SELECT * FROM t1, t2 where t1.a = t2.a; SELECT * FROM t1, t2 where t1.a = t2.a;
SELECT 'comma nullable'; SELECT 'comma nullable';
SELECT * FROM t1, t2 where t1.b = t2.b; SELECT * FROM t1, t2 where t1.b = t2.b;
SELECT 'cross';
SET allow_experimental_cross_to_join_conversion = 0; AST SELECT * FROM t1 cross join t2 where t1.a = t2.a;
SET allow_experimental_cross_to_join_conversion = 1; AST SELECT * FROM t1 cross join t2 where t1.a = t2.a;
SELECT 'cross nullable';
SET allow_experimental_cross_to_join_conversion = 0; AST SELECT * FROM t1, t2 where t1.a = t2.a;
SET allow_experimental_cross_to_join_conversion = 1; AST SELECT * FROM t1, t2 where t1.a = t2.a;
SELECT 'cross nullable vs not nullable';
SET allow_experimental_cross_to_join_conversion = 0; AST SELECT * FROM t1 cross join t2 where t1.a = t2.b;
SET allow_experimental_cross_to_join_conversion = 1; AST SELECT * FROM t1 cross join t2 where t1.a = t2.b;
SELECT 'cross self';
SET allow_experimental_cross_to_join_conversion = 0; AST SELECT * FROM t1 x cross join t1 y where x.a = y.a and x.b = y.b;
SET allow_experimental_cross_to_join_conversion = 1; AST SELECT * FROM t1 x cross join t1 y where x.a = y.a and x.b = y.b;
SELECT 'cross one table expr';
SET allow_experimental_cross_to_join_conversion = 0; AST SELECT * FROM t1 cross join t2 where t1.a = t1.b;
SET allow_experimental_cross_to_join_conversion = 1; AST SELECT * FROM t1 cross join t2 where t1.a = t1.b;
SELECT 'cross multiple ands';
SET allow_experimental_cross_to_join_conversion = 0; AST SELECT * FROM t1 cross join t2 where t1.a = t2.a and t1.b = t2.b;
SET allow_experimental_cross_to_join_conversion = 1; AST SELECT * FROM t1 cross join t2 where t1.a = t2.a and t1.b = t2.b;
SELECT 'cross and inside and';
SET allow_experimental_cross_to_join_conversion = 0; AST SELECT * FROM t1 cross join t2 where t1.a = t2.a and (t1.a = t2.a and (t1.a = t2.a and t1.b = t2.b));
SET allow_experimental_cross_to_join_conversion = 1; AST SELECT * FROM t1 cross join t2 where t1.a = t2.a and (t1.a = t2.a and (t1.a = t2.a and t1.b = t2.b));
SELECT 'cross split conjunction';
SET allow_experimental_cross_to_join_conversion = 0; AST SELECT * FROM t1 cross join t2 where t1.a = t2.a and t1.b = t2.b and t1.a >= 1 and t2.b > 0;
SET allow_experimental_cross_to_join_conversion = 1; AST SELECT * FROM t1 cross join t2 where t1.a = t2.a and t1.b = t2.b and t1.a >= 1 and t2.b > 0;
DROP TABLE t1; DROP TABLE t1;
DROP TABLE t2; DROP TABLE t2;