mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Revert "Move conditions from JOIN ON to WHERE"
This commit is contained in:
parent
190846c652
commit
9fe20c1628
@ -16,26 +16,6 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
void addAndTerm(ASTPtr & ast, const ASTPtr & term)
|
||||
{
|
||||
if (!ast)
|
||||
ast = term;
|
||||
else
|
||||
ast = makeASTFunction("and", ast, term);
|
||||
}
|
||||
|
||||
/// If this is an inner join and the expression related to less than 2 tables, then move it to WHERE
|
||||
bool canMoveToWhere(std::pair<size_t, size_t> table_numbers, ASTTableJoin::Kind kind)
|
||||
{
|
||||
return kind == ASTTableJoin::Kind::Inner &&
|
||||
(table_numbers.first == table_numbers.second || table_numbers.first == 0 || table_numbers.second == 0);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void CollectJoinOnKeysMatcher::Data::addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast,
|
||||
const std::pair<size_t, size_t> & table_no)
|
||||
{
|
||||
@ -49,8 +29,7 @@ void CollectJoinOnKeysMatcher::Data::addJoinKeys(const ASTPtr & left_ast, const
|
||||
else
|
||||
throw Exception("Cannot detect left and right JOIN keys. JOIN ON section is ambiguous.",
|
||||
ErrorCodes::AMBIGUOUS_COLUMN_NAME);
|
||||
if (table_no.first != table_no.second && table_no.first > 0 && table_no.second > 0)
|
||||
has_some = true;
|
||||
has_some = true;
|
||||
}
|
||||
|
||||
void CollectJoinOnKeysMatcher::Data::addAsofJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast,
|
||||
@ -99,45 +78,22 @@ void CollectJoinOnKeysMatcher::visit(const ASTFunction & func, const ASTPtr & as
|
||||
{
|
||||
ASTPtr left = func.arguments->children.at(0);
|
||||
ASTPtr right = func.arguments->children.at(1);
|
||||
auto table_numbers = getTableNumbers(left, right, data);
|
||||
|
||||
if (canMoveToWhere(table_numbers, data.kind))
|
||||
{
|
||||
addAndTerm(data.new_where_conditions, ast);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (data.kind == ASTTableJoin::Kind::Inner)
|
||||
{
|
||||
addAndTerm(data.new_on_expression, ast);
|
||||
}
|
||||
data.addJoinKeys(left, right, table_numbers);
|
||||
}
|
||||
auto table_numbers = getTableNumbers(ast, left, right, data);
|
||||
data.addJoinKeys(left, right, table_numbers);
|
||||
}
|
||||
else if (inequality != ASOF::Inequality::None && !data.is_asof)
|
||||
else if (inequality != ASOF::Inequality::None)
|
||||
{
|
||||
ASTPtr left = func.arguments->children.at(0);
|
||||
ASTPtr right = func.arguments->children.at(1);
|
||||
auto table_numbers = getTableNumbers(left, right, data);
|
||||
if (canMoveToWhere(table_numbers, data.kind))
|
||||
{
|
||||
addAndTerm(data.new_where_conditions, ast);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!data.is_asof)
|
||||
throw Exception("JOIN ON inequalities are not supported. Unexpected '" + queryToString(ast) + "'",
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
}
|
||||
else if (inequality != ASOF::Inequality::None && data.is_asof)
|
||||
{
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
if (data.asof_left_key || data.asof_right_key)
|
||||
throw Exception("ASOF JOIN expects exactly one inequality in ON section. Unexpected '" + queryToString(ast) + "'",
|
||||
ErrorCodes::INVALID_JOIN_ON_EXPRESSION);
|
||||
ErrorCodes::INVALID_JOIN_ON_EXPRESSION);
|
||||
|
||||
ASTPtr left = func.arguments->children.at(0);
|
||||
ASTPtr right = func.arguments->children.at(1);
|
||||
auto table_numbers = getTableNumbers(left, right, data);
|
||||
auto table_numbers = getTableNumbers(ast, left, right, data);
|
||||
|
||||
data.addAsofJoinKeys(left, right, table_numbers, inequality);
|
||||
}
|
||||
@ -162,8 +118,7 @@ void CollectJoinOnKeysMatcher::getIdentifiers(const ASTPtr & ast, std::vector<co
|
||||
getIdentifiers(child, out);
|
||||
}
|
||||
|
||||
|
||||
std::pair<size_t, size_t> CollectJoinOnKeysMatcher::getTableNumbers(const ASTPtr & left_ast, const ASTPtr & right_ast,
|
||||
std::pair<size_t, size_t> CollectJoinOnKeysMatcher::getTableNumbers(const ASTPtr & expr, const ASTPtr & left_ast, const ASTPtr & right_ast,
|
||||
Data & data)
|
||||
{
|
||||
std::vector<const ASTIdentifier *> left_identifiers;
|
||||
@ -172,13 +127,23 @@ std::pair<size_t, size_t> CollectJoinOnKeysMatcher::getTableNumbers(const ASTPtr
|
||||
getIdentifiers(left_ast, left_identifiers);
|
||||
getIdentifiers(right_ast, right_identifiers);
|
||||
|
||||
size_t left_idents_table = 0;
|
||||
size_t right_idents_table = 0;
|
||||
if (left_identifiers.empty() || right_identifiers.empty())
|
||||
{
|
||||
throw Exception("Not equi-join ON expression: " + queryToString(expr) + ". No columns in one of equality side.",
|
||||
ErrorCodes::INVALID_JOIN_ON_EXPRESSION);
|
||||
}
|
||||
|
||||
if (!left_identifiers.empty())
|
||||
left_idents_table = getTableForIdentifiers(left_identifiers, data);
|
||||
if (!right_identifiers.empty())
|
||||
right_idents_table = getTableForIdentifiers(right_identifiers, data);
|
||||
size_t left_idents_table = getTableForIdentifiers(left_identifiers, data);
|
||||
size_t right_idents_table = getTableForIdentifiers(right_identifiers, data);
|
||||
|
||||
if (left_idents_table && left_idents_table == right_idents_table)
|
||||
{
|
||||
auto left_name = queryToString(*left_identifiers[0]);
|
||||
auto right_name = queryToString(*right_identifiers[0]);
|
||||
|
||||
throw Exception("In expression " + queryToString(expr) + " columns " + left_name + " and " + right_name
|
||||
+ " are from the same table but from different arguments of equal function", ErrorCodes::INVALID_JOIN_ON_EXPRESSION);
|
||||
}
|
||||
|
||||
return std::make_pair(left_idents_table, right_idents_table);
|
||||
}
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/Aliases.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -31,11 +30,8 @@ public:
|
||||
const TableWithColumnNamesAndTypes & right_table;
|
||||
const Aliases & aliases;
|
||||
const bool is_asof{false};
|
||||
ASTTableJoin::Kind kind;
|
||||
ASTPtr asof_left_key{};
|
||||
ASTPtr asof_right_key{};
|
||||
ASTPtr new_on_expression{};
|
||||
ASTPtr new_where_conditions{};
|
||||
bool has_some{false};
|
||||
|
||||
void addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, const std::pair<size_t, size_t> & table_no);
|
||||
@ -61,7 +57,7 @@ private:
|
||||
static void visit(const ASTFunction & func, const ASTPtr & ast, Data & data);
|
||||
|
||||
static void getIdentifiers(const ASTPtr & ast, std::vector<const ASTIdentifier *> & out);
|
||||
static std::pair<size_t, size_t> getTableNumbers(const ASTPtr & left_ast, const ASTPtr & right_ast, Data & data);
|
||||
static std::pair<size_t, size_t> getTableNumbers(const ASTPtr & expr, const ASTPtr & left_ast, const ASTPtr & right_ast, Data & data);
|
||||
static const ASTIdentifier * unrollAliases(const ASTIdentifier * identifier, const Aliases & aliases);
|
||||
static size_t getTableForIdentifiers(std::vector<const ASTIdentifier *> & identifiers, const Data & data);
|
||||
};
|
||||
|
@ -405,13 +405,13 @@ void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_defaul
|
||||
|
||||
/// Find the columns that are obtained by JOIN.
|
||||
void collectJoinedColumns(TableJoin & analyzed_join, const ASTSelectQuery & select_query,
|
||||
const TablesWithColumns & tables, const Aliases & aliases, ASTPtr & new_where_conditions)
|
||||
const TablesWithColumns & tables, const Aliases & aliases)
|
||||
{
|
||||
const ASTTablesInSelectQueryElement * node = select_query.join();
|
||||
if (!node || tables.size() < 2)
|
||||
return;
|
||||
|
||||
auto & table_join = node->table_join->as<ASTTableJoin &>();
|
||||
const auto & table_join = node->table_join->as<ASTTableJoin &>();
|
||||
|
||||
if (table_join.using_expression_list)
|
||||
{
|
||||
@ -430,33 +430,16 @@ void collectJoinedColumns(TableJoin & analyzed_join, const ASTSelectQuery & sele
|
||||
{
|
||||
bool is_asof = (table_join.strictness == ASTTableJoin::Strictness::Asof);
|
||||
|
||||
CollectJoinOnKeysVisitor::Data data{analyzed_join, tables[0], tables[1], aliases, is_asof, table_join.kind};
|
||||
CollectJoinOnKeysVisitor::Data data{analyzed_join, tables[0], tables[1], aliases, is_asof};
|
||||
CollectJoinOnKeysVisitor(data).visit(table_join.on_expression);
|
||||
if (!data.has_some)
|
||||
throw Exception("Cannot get JOIN keys from JOIN ON section: " + queryToString(table_join.on_expression),
|
||||
ErrorCodes::INVALID_JOIN_ON_EXPRESSION);
|
||||
if (is_asof)
|
||||
{
|
||||
data.asofToJoinKeys();
|
||||
}
|
||||
else if (data.new_on_expression)
|
||||
{
|
||||
table_join.on_expression = data.new_on_expression;
|
||||
new_where_conditions = data.new_where_conditions;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Move joined key related to only one table to WHERE clause
|
||||
void moveJoinedKeyToWhere(ASTSelectQuery * select_query, ASTPtr & new_where_conditions)
|
||||
{
|
||||
if (select_query->where())
|
||||
select_query->setExpression(ASTSelectQuery::Expression::WHERE,
|
||||
makeASTFunction("and", new_where_conditions, select_query->where()));
|
||||
else
|
||||
select_query->setExpression(ASTSelectQuery::Expression::WHERE, new_where_conditions->clone());
|
||||
}
|
||||
|
||||
|
||||
std::vector<const ASTFunction *> getAggregates(ASTPtr & query, const ASTSelectQuery & select_query)
|
||||
{
|
||||
@ -841,11 +824,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
|
||||
|
||||
setJoinStrictness(*select_query, settings.join_default_strictness, settings.any_join_distinct_right_table_keys,
|
||||
result.analyzed_join->table_join);
|
||||
|
||||
ASTPtr new_where_condition = nullptr;
|
||||
collectJoinedColumns(*result.analyzed_join, *select_query, tables_with_columns, result.aliases, new_where_condition);
|
||||
if (new_where_condition)
|
||||
moveJoinedKeyToWhere(select_query, new_where_condition);
|
||||
collectJoinedColumns(*result.analyzed_join, *select_query, tables_with_columns, result.aliases);
|
||||
|
||||
/// rewrite filters for select query, must go after getArrayJoinedColumns
|
||||
if (settings.optimize_respect_aliases && result.metadata_snapshot)
|
||||
|
@ -109,7 +109,7 @@ SELECT
|
||||
t2_00826.a,
|
||||
t2_00826.b
|
||||
FROM t1_00826
|
||||
ALL INNER JOIN t2_00826 ON (((a = t2_00826.a) AND (a = t2_00826.a)) AND (a = t2_00826.a)) AND (b = t2_00826.b)
|
||||
ALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (a = t2_00826.a) AND (a = t2_00826.a) AND (b = t2_00826.b)
|
||||
WHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b)))
|
||||
--- cross split conjunction ---
|
||||
SELECT
|
||||
|
@ -127,7 +127,7 @@ FROM
|
||||
) AS `--.s`
|
||||
CROSS JOIN t3
|
||||
) AS `--.s`
|
||||
ALL INNER JOIN t4 ON ((a = `--t1.a`) AND (a = `--t2.a`)) AND (a = `--t3.a`)
|
||||
ALL INNER JOIN t4 ON (a = `--t1.a`) AND (a = `--t2.a`) AND (a = `--t3.a`)
|
||||
WHERE (a = `--t1.a`) AND (a = `--t2.a`) AND (a = `--t3.a`)
|
||||
SELECT `--t1.a` AS `t1.a`
|
||||
FROM
|
||||
|
@ -23,8 +23,6 @@ join_use_nulls = 1
|
||||
-
|
||||
\N \N
|
||||
-
|
||||
1 1 \N \N
|
||||
2 2 \N \N
|
||||
-
|
||||
1 1 1 1
|
||||
2 2 \N \N
|
||||
@ -51,8 +49,6 @@ join_use_nulls = 0
|
||||
-
|
||||
-
|
||||
-
|
||||
1 1 0 0
|
||||
2 2 0 0
|
||||
-
|
||||
1 1 1 1
|
||||
2 2 0 0
|
||||
|
@ -30,11 +30,11 @@ select * from t left outer join s on (t.a=s.a and t.b=s.b) where s.a is null;
|
||||
select '-';
|
||||
select s.* from t left outer join s on (t.a=s.a and t.b=s.b) where s.a is null;
|
||||
select '-';
|
||||
select t.*, s.* from t left join s on (s.a=t.a and t.b=s.b and t.a=toInt64(2)) order by t.a;
|
||||
select t.*, s.* from t left join s on (s.a=t.a and t.b=s.b and t.a=toInt64(2)) order by t.a; -- {serverError 403 }
|
||||
select '-';
|
||||
select t.*, s.* from t left join s on (s.a=t.a) order by t.a;
|
||||
select '-';
|
||||
select t.*, s.* from t left join s on (t.b=toInt64(2) and s.a=t.a) where s.b=2;
|
||||
select t.*, s.* from t left join s on (t.b=toInt64(2) and s.a=t.a) where s.b=2; -- {serverError 403 }
|
||||
|
||||
select 'join_use_nulls = 0';
|
||||
set join_use_nulls = 0;
|
||||
@ -58,11 +58,11 @@ select '-';
|
||||
select '-';
|
||||
-- select s.* from t left outer join s on (t.a=s.a and t.b=s.b) where s.a is null; -- TODO
|
||||
select '-';
|
||||
select t.*, s.* from t left join s on (s.a=t.a and t.b=s.b and t.a=toInt64(2)) order by t.a;
|
||||
select t.*, s.* from t left join s on (s.a=t.a and t.b=s.b and t.a=toInt64(2)) order by t.a; -- {serverError 403 }
|
||||
select '-';
|
||||
select t.*, s.* from t left join s on (s.a=t.a) order by t.a;
|
||||
select '-';
|
||||
select t.*, s.* from t left join s on (t.b=toInt64(2) and s.a=t.a) where s.b=2;
|
||||
select t.*, s.* from t left join s on (t.b=toInt64(2) and s.a=t.a) where s.b=2; -- {serverError 403 }
|
||||
|
||||
drop table t;
|
||||
drop table s;
|
||||
|
@ -1,140 +0,0 @@
|
||||
---------Q1----------
|
||||
2 2 2 20
|
||||
SELECT
|
||||
a,
|
||||
b,
|
||||
table2.a,
|
||||
table2.b
|
||||
FROM table1
|
||||
ALL INNER JOIN
|
||||
(
|
||||
SELECT
|
||||
a,
|
||||
b
|
||||
FROM table2
|
||||
) AS table2 ON a = table2.a
|
||||
WHERE table2.b = toUInt32(20)
|
||||
---------Q2----------
|
||||
2 2 2 20
|
||||
SELECT
|
||||
a,
|
||||
b,
|
||||
table2.a,
|
||||
table2.b
|
||||
FROM table1
|
||||
ALL INNER JOIN
|
||||
(
|
||||
SELECT
|
||||
a,
|
||||
b
|
||||
FROM table2
|
||||
) AS table2 ON a = table2.a
|
||||
WHERE (table2.a < table2.b) AND (table2.b = toUInt32(20))
|
||||
---------Q3----------
|
||||
---------Q4----------
|
||||
6 40
|
||||
SELECT
|
||||
a,
|
||||
table2.b
|
||||
FROM table1
|
||||
ALL INNER JOIN
|
||||
(
|
||||
SELECT
|
||||
a,
|
||||
b
|
||||
FROM table2
|
||||
) AS table2 ON a = toUInt32(10 - table2.a)
|
||||
WHERE (b = 6) AND (table2.b > 20)
|
||||
---------Q5----------
|
||||
SELECT
|
||||
a,
|
||||
table2.b
|
||||
FROM table1
|
||||
ALL INNER JOIN
|
||||
(
|
||||
SELECT
|
||||
a,
|
||||
b
|
||||
FROM table2
|
||||
WHERE 0
|
||||
) AS table2 ON a = table2.a
|
||||
WHERE 0
|
||||
---------Q6----------
|
||||
---------Q7----------
|
||||
0 0 0 0
|
||||
SELECT
|
||||
a,
|
||||
b,
|
||||
table2.a,
|
||||
table2.b
|
||||
FROM table1
|
||||
ALL INNER JOIN
|
||||
(
|
||||
SELECT
|
||||
a,
|
||||
b
|
||||
FROM table2
|
||||
) AS table2 ON a = table2.a
|
||||
WHERE (table2.b < toUInt32(40)) AND (b < 1)
|
||||
---------Q8----------
|
||||
---------Q9---will not be optimized----------
|
||||
SELECT
|
||||
a,
|
||||
b,
|
||||
table2.a,
|
||||
table2.b
|
||||
FROM table1
|
||||
ALL LEFT JOIN
|
||||
(
|
||||
SELECT
|
||||
a,
|
||||
b
|
||||
FROM table2
|
||||
) AS table2 ON (a = table2.a) AND (b = toUInt32(10))
|
||||
SELECT
|
||||
a,
|
||||
b,
|
||||
table2.a,
|
||||
table2.b
|
||||
FROM table1
|
||||
ALL RIGHT JOIN
|
||||
(
|
||||
SELECT
|
||||
a,
|
||||
b
|
||||
FROM table2
|
||||
) AS table2 ON (a = table2.a) AND (b = toUInt32(10))
|
||||
SELECT
|
||||
a,
|
||||
b,
|
||||
table2.a,
|
||||
table2.b
|
||||
FROM table1
|
||||
ALL FULL OUTER JOIN
|
||||
(
|
||||
SELECT
|
||||
a,
|
||||
b
|
||||
FROM table2
|
||||
) AS table2 ON (a = table2.a) AND (b = toUInt32(10))
|
||||
SELECT
|
||||
a,
|
||||
b,
|
||||
table2.a,
|
||||
table2.b
|
||||
FROM table1
|
||||
ALL FULL OUTER JOIN
|
||||
(
|
||||
SELECT
|
||||
a,
|
||||
b
|
||||
FROM table2
|
||||
) AS table2 ON (a = table2.a) AND (table2.b = toUInt32(10))
|
||||
WHERE a < toUInt32(20)
|
||||
SELECT
|
||||
a,
|
||||
b,
|
||||
table2.a,
|
||||
table2.b
|
||||
FROM table1
|
||||
CROSS JOIN table2
|
@ -1,48 +0,0 @@
|
||||
DROP TABLE IF EXISTS table1;
|
||||
DROP TABLE IF EXISTS table2;
|
||||
|
||||
CREATE TABLE table1 (a UInt32, b UInt32) ENGINE = Memory;
|
||||
CREATE TABLE table2 (a UInt32, b UInt32) ENGINE = Memory;
|
||||
|
||||
INSERT INTO table1 SELECT number, number FROM numbers(10);
|
||||
INSERT INTO table2 SELECT number * 2, number * 20 FROM numbers(6);
|
||||
|
||||
SELECT '---------Q1----------';
|
||||
SELECT * FROM table1 JOIN table2 ON (table1.a = table2.a) AND (table2.b = toUInt32(20));
|
||||
EXPLAIN SYNTAX SELECT * FROM table1 JOIN table2 ON (table1.a = table2.a) AND (table2.b = toUInt32(20));
|
||||
|
||||
SELECT '---------Q2----------';
|
||||
SELECT * FROM table1 JOIN table2 ON (table1.a = table2.a) AND (table2.a < table2.b) AND (table2.b = toUInt32(20));
|
||||
EXPLAIN SYNTAX SELECT * FROM table1 JOIN table2 ON (table1.a = table2.a) AND (table2.a < table2.b) AND (table2.b = toUInt32(20));
|
||||
|
||||
SELECT '---------Q3----------';
|
||||
SELECT * FROM table1 JOIN table2 ON (table1.a = toUInt32(table2.a + 5)) AND (table2.a < table1.b) AND (table2.b > toUInt32(20)); -- { serverError 48 }
|
||||
|
||||
SELECT '---------Q4----------';
|
||||
SELECT table1.a, table2.b FROM table1 INNER JOIN table2 ON (table1.a = toUInt32(10 - table2.a)) AND (table1.b = 6) AND (table2.b > 20);
|
||||
EXPLAIN SYNTAX SELECT table1.a, table2.b FROM table1 INNER JOIN table2 ON (table1.a = toUInt32(10 - table2.a)) AND (table1.b = 6) AND (table2.b > 20);
|
||||
|
||||
SELECT '---------Q5----------';
|
||||
SELECT table1.a, table2.b FROM table1 JOIN table2 ON (table1.a = table2.a) AND (table1.b = 6) AND (table2.b > 20) AND (10 < 6);
|
||||
EXPLAIN SYNTAX SELECT table1.a, table2.b FROM table1 JOIN table2 ON (table1.a = table2.a) AND (table1.b = 6) AND (table2.b > 20) AND (10 < 6);
|
||||
|
||||
SELECT '---------Q6----------';
|
||||
SELECT table1.a, table2.b FROM table1 JOIN table2 ON (table1.b = 6) AND (table2.b > 20); -- { serverError 403 }
|
||||
|
||||
SELECT '---------Q7----------';
|
||||
SELECT * FROM table1 JOIN table2 ON (table1.a = table2.a) AND (table2.b < toUInt32(40)) where table1.b < 1;
|
||||
EXPLAIN SYNTAX SELECT * FROM table1 JOIN table2 ON (table1.a = table2.a) AND (table2.b < toUInt32(40)) where table1.b < 1;
|
||||
SELECT * FROM table1 JOIN table2 ON (table1.a = table2.a) AND (table2.b < toUInt32(40)) where table1.b > 10;
|
||||
|
||||
SELECT '---------Q8----------';
|
||||
SELECT * FROM table1 INNER JOIN table2 ON (table1.a = table2.a) AND (table2.b < toUInt32(table1, 10)); -- { serverError 47 }
|
||||
|
||||
SELECT '---------Q9---will not be optimized----------';
|
||||
EXPLAIN SYNTAX SELECT * FROM table1 LEFT JOIN table2 ON (table1.a = table2.a) AND (table1.b = toUInt32(10));
|
||||
EXPLAIN SYNTAX SELECT * FROM table1 RIGHT JOIN table2 ON (table1.a = table2.a) AND (table1.b = toUInt32(10));
|
||||
EXPLAIN SYNTAX SELECT * FROM table1 FULL JOIN table2 ON (table1.a = table2.a) AND (table1.b = toUInt32(10));
|
||||
EXPLAIN SYNTAX SELECT * FROM table1 FULL JOIN table2 ON (table1.a = table2.a) AND (table2.b = toUInt32(10)) WHERE table1.a < toUInt32(20);
|
||||
EXPLAIN SYNTAX SELECT * FROM table1 , table2;
|
||||
|
||||
DROP TABLE table1;
|
||||
DROP TABLE table2;
|
Loading…
Reference in New Issue
Block a user