rename setting + disable ANY INNER JOIN too

This commit is contained in:
chertus 2019-08-08 17:20:25 +03:00
parent 2fee5ccd34
commit 391574f361
31 changed files with 53 additions and 28 deletions

View File

@ -200,7 +200,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, join_use_nulls, 0, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.") \
\
M(SettingJoinStrictness, join_default_strictness, JoinStrictness::ALL, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.") \
M(SettingBool, any_join_get_any_from_right_table, false, "Enable ANY RIGHT|FULL JOIN that are not consistent with LEFT one. Right table in ANY JOIN has no key copies so it leads to confusing results for RIGHT and FULL JOINs. User expects 't1 ANY LEFT JOIN t2' is equal to 't2 ANY RIGHT JOIN t1'.") \
M(SettingBool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys maping to be consistent with LEFT one.") \
\
M(SettingUInt64, preferred_block_size_bytes, 1000000, "") \
\

View File

@ -546,14 +546,13 @@ void checkJoin(const ASTTablesInSelectQueryElement * join)
const auto & table_join = join->table_join->as<ASTTableJoin &>();
if (table_join.strictness == ASTTableJoin::Strictness::Any ||
table_join.strictness == ASTTableJoin::Strictness::Asof)
if (table_join.kind == ASTTableJoin::Kind::Right ||
table_join.kind == ASTTableJoin::Kind::Full)
throw Exception("ANY RIGHT|FULL JOINs are disabled by default cause of confusing results: "
"'t1 ANY LEFT JOIN t2' is inconsistent with 't2 ANY RIGHT JOIN t1'."
"Default bahaviour is reserved for many-to-one ANY LEFT JOIN and one-to-many ANY RIGHT JOIN."
"Set any_join_get_any_from_right_table=1 to enable many-to-one ANY RIGHT|FULL JOINs.",
if (table_join.strictness == ASTTableJoin::Strictness::Any)
if (table_join.kind != ASTTableJoin::Kind::Left)
throw Exception("Old ANY INNER|RIGHT|FULL JOINs are disabled by default. Their logic would be changed."
"Old logic is many-to-one for all kinds of ANY JOINs. It's equil to apply distinct for right table keys."
"Default bahaviour is reserved for many-to-one LEFT JOIN, one-to-many RIGHT JOIN and one-to-one INNER JOIN."
"It would be equal to apply distinct for keys to right, left and both tables respectively."
"Set any_join_distinct_right_table_keys=1 to enable old bahaviour.",
ErrorCodes::NOT_IMPLEMENTED);
}
@ -597,7 +596,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
if (const ASTTablesInSelectQueryElement * node = select_query->join())
{
if (!settings.any_join_get_any_from_right_table)
if (!settings.any_join_distinct_right_table_keys)
checkJoin(node);
if (settings.enable_optimize_predicate_expression)

View File

@ -1,3 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SELECT a.*, b.* FROM
(
SELECT number AS k FROM system.numbers LIMIT 10

View File

@ -1,3 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SELECT a,b,c,d FROM (SELECT 1 AS a,2 AS b, 3 AS c UNION ALL SELECT 2,3,4 ) ANY INNER JOIN (SELECT 1 AS a,2 AS b,4 AS d UNION ALL SELECT 2,3,5) USING (a) ORDER BY a,b,c,d ASC;
SELECT a,b,c,d FROM (SELECT 1 AS a,2 AS b, 3 AS c UNION ALL SELECT 2,3,4 ) ALL LEFT JOIN (SELECT 1 AS a,2 AS b,4 AS d UNION ALL SELECT 2,3,5) USING (a) ORDER BY a,b,c,d ASC;
SELECT a,b,c,d FROM (SELECT 1 AS a,2 AS b, 3 AS c UNION ALL SELECT 2,3,4 ) ALL LEFT JOIN (SELECT 1 AS a,2 AS b,4 AS d UNION ALL SELECT 2,3,5) USING a,b ORDER BY a,b,c,d ASC;

View File

@ -1,3 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SELECT number, number / 2 AS n, j1, j2 FROM remote('127.0.0.{2,3}', system.numbers) ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) USING n LIMIT 10;
SELECT dummy + 2 AS number, number / 2 AS n, j1, j2 FROM remote('127.0.0.{2,3}', system.one) ANY INNER JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) USING n LIMIT 10;
SELECT number, number / 2 AS n, j1, j2 FROM remote('127.0.0.{2,3}', system.numbers) GLOBAL ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) USING n LIMIT 10;

View File

@ -1,3 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
DROP TABLE IF EXISTS series;
CREATE TABLE series(i UInt32, x_value Float64, y_value Float64) ENGINE = Memory;

View File

@ -1,3 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
DROP TABLE IF EXISTS series;
CREATE TABLE series(i UInt32, x_value Float64, y_value Float64) ENGINE = Memory;

View File

@ -1,4 +1,4 @@
SET any_join_get_any_from_right_table = 1;
SET any_join_distinct_right_table_keys = 1;
SELECT k, x, y FROM (SELECT arrayJoin([1, 2, 3]) AS k, 'Hello' AS x) ANY FULL JOIN (SELECT range(k) AS y, arrayJoin([3, 4, 5]) AS k) USING k WHERE k < 10 ORDER BY k;
SELECT k, x FROM (SELECT arrayJoin([1, 2, 3]) AS k, 'Hello' AS x) ANY FULL JOIN (SELECT range(k) AS y, arrayJoin([3, 4, 5]) AS k) USING k WHERE k < 10 ORDER BY k;
@ -24,7 +24,7 @@ INSERT INTO t2_00203 VALUES ('ccc', 4, 3, 2), ('ddd', 7, 6, 5);
SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY FULL JOIN t2_00203 USING (k3, k1, k2 AS k2_alias) ORDER BY k1, k2, k3;
SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY RIGHT JOIN t2_00203 USING (k3, k1, k2 AS k2_alias) ORDER BY k1, k2, k3;
SET any_join_get_any_from_right_table = 0;
SET any_join_distinct_right_table_keys = 0;
SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY FULL JOIN t2_00203 USING (k3, k1, k2 AS k2_alias) ORDER BY k1, k2, k3; -- { serverError 48 }
SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY RIGHT JOIN t2_00203 USING (k3, k1, k2 AS k2_alias) ORDER BY k1, k2, k3; -- { serverError 48 }

View File

@ -1,4 +1,4 @@
SET any_join_get_any_from_right_table = 1;
SET any_join_distinct_right_table_keys = 1;
SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b) USING a, b;
SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b) USING b, a;

View File

@ -1,3 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SELECT
floor((ReferrerTimestamp - InstallTimestamp) / 86400) AS DaysSinceInstallations
FROM

View File

@ -1 +1,2 @@
set any_join_distinct_right_table_keys = 1;
select a from (select (1, 2) as a) any inner join (select (1, 2) as a) using a;

View File

@ -1,4 +1,4 @@
SET any_join_get_any_from_right_table = 1;
SET any_join_distinct_right_table_keys = 1;
select x, y from (select 1 as x, 2 as y, x, y);
select x, y from (select 1 as x, 1 as y, x, y);

View File

@ -1,5 +1,5 @@
SET join_use_nulls = 0;
SET any_join_get_any_from_right_table = 1;
SET any_join_distinct_right_table_keys = 1;
SELECT k, a, b
FROM

View File

@ -1,5 +1,5 @@
SET join_use_nulls = 0;
SET any_join_get_any_from_right_table = 1;
SET any_join_distinct_right_table_keys = 1;
SELECT k, a, b
FROM

View File

@ -14,7 +14,7 @@ INSERT INTO test2_00550 VALUES ('a');
CREATE TABLE test3_00550 ( id String, name String ) ENGINE = StripeLog;
INSERT INTO test3_00550 VALUES ('a', 'aaa');
INSERT INTO test1_00550 SELECT id, name FROM test2_00550 ANY INNER JOIN test3_00550 USING (id);
INSERT INTO test1_00550 SELECT id, name FROM test2_00550 ANY INNER JOIN test3_00550 USING (id) SETTINGS any_join_distinct_right_table_keys=1;
INSERT INTO test1_00550 SELECT id, name FROM test2_00550 ANY LEFT OUTER JOIN test3_00550 USING (id);
DROP TABLE test1_00550;

View File

@ -1,3 +1,3 @@
SET any_join_get_any_from_right_table = 1;
SET any_join_distinct_right_table_keys = 1;
SET max_block_size = 10;
SELECT * FROM (select toUInt64(1) s limit 1) any right join (select number s, s as x from numbers(11)) using (s) ORDER BY s;

View File

@ -1,3 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
drop table IF EXISTS joinbug;
CREATE TABLE joinbug (

View File

@ -1,4 +1,4 @@
SET any_join_get_any_from_right_table = 1;
SET any_join_distinct_right_table_keys = 1;
SELECT k, a1, b1, a2, b2 FROM (SELECT 0 AS k, 'hello' AS a1, 123 AS b1, a1) ANY FULL OUTER JOIN (SELECT 1 AS k, 'hello' AS a2, 456 AS b2, a2) USING (k) ORDER BY k;
SELECT k, a, b FROM (SELECT 0 AS k, 'hello' AS a, 123 AS b, a) ANY FULL OUTER JOIN (SELECT 1 AS k) USING (k) ORDER BY k;

View File

@ -1,3 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
DROP TABLE IF EXISTS local_statements;
DROP TABLE IF EXISTS statements;

View File

@ -1,5 +1,5 @@
SET send_logs_level = 'none';
SET any_join_get_any_from_right_table = 1;
SET any_join_distinct_right_table_keys = 1;
DROP TABLE IF EXISTS test_00597;
DROP TABLE IF EXISTS test_view_00597;

View File

@ -1,3 +1,5 @@
set any_join_distinct_right_table_keys = 1;
SELECT * FROM (SELECT 1 AS id, 2 AS value);
SELECT * FROM (SELECT 1 AS id, 2 AS value, 3 AS A) ANY INNER JOIN (SELECT 1 AS id, 4 AS values, 5 AS D) USING id;
SELECT *, d.* FROM ( SELECT 1 AS id, 2 AS value ) ANY INNER JOIN ( SELECT 1 AS id, 3 AS values ) AS d USING id;

View File

@ -1,4 +1,4 @@
SET any_join_get_any_from_right_table = 1;
SET any_join_distinct_right_table_keys = 1;
SELECT * FROM (SELECT 1 AS a, 'x' AS b) join (SELECT 1 as a, 'y' as b) using a;
SELECT * FROM (SELECT 1 AS a, 'x' AS b) left join (SELECT 1 as a, 'y' as b) using a;
@ -10,6 +10,6 @@ SELECT * FROM (SELECT 1 AS a, 'x' AS b) any left join (SELECT 1 as a, 'y' as b)
SELECT * FROM (SELECT 1 AS a, 'x' AS b) any full join (SELECT 1 as a, 'y' as b) using a;
SELECT * FROM (SELECT 1 AS a, 'x' AS b) any right join (SELECT 1 as a, 'y' as b) using a;
SET any_join_get_any_from_right_table = 0;
SET any_join_distinct_right_table_keys = 0;
SELECT * FROM (SELECT 1 AS a, 'x' AS b) any full join (SELECT 1 as a, 'y' as b) using a; -- { serverError 48 }
SELECT * FROM (SELECT 1 AS a, 'x' AS b) any right join (SELECT 1 as a, 'y' as b) using a; -- { serverError 48 }

View File

@ -1,4 +1,4 @@
SET any_join_get_any_from_right_table = 1;
SET any_join_distinct_right_table_keys = 1;
SELECT * FROM (SELECT dummy as a, (toUInt8(0), toUInt8(0)) AS tup FROM system.one)
JOIN (SELECT dummy as a, (toUInt8(0), toUInt8(0)) AS tup FROM system.one)

View File

@ -1,4 +1,4 @@
SET any_join_get_any_from_right_table = 1;
SET any_join_distinct_right_table_keys = 1;
DROP TABLE IF EXISTS t1_00848;
DROP TABLE IF EXISTS t2_00848;

View File

@ -1,3 +1,5 @@
set any_join_distinct_right_table_keys = 1;
drop table if exists fooL;
drop table if exists fooR;
create table fooL (a Int32, v String) engine = Memory;

View File

@ -1,4 +1,4 @@
SET any_join_get_any_from_right_table = 1;
SET any_join_distinct_right_table_keys = 1;
SELECT * FROM (SELECT 1 AS x) ALL LEFT JOIN (SELECT 1 AS x) USING x;
SELECT * FROM (SELECT 1 AS x) ALL LEFT JOIN (SELECT 2 AS x) USING x;

View File

@ -1,4 +1,4 @@
SET any_join_get_any_from_right_table = 1;
SET any_join_distinct_right_table_keys = 1;
DROP TABLE IF EXISTS test.filtered_table1;
DROP TABLE IF EXISTS test.filtered_table2;

View File

@ -1,3 +1,4 @@
SET any_join_distinct_right_table_keys = 1;
SET join_use_nulls = 1;
SELECT number FROM system.numbers ANY INNER JOIN (SELECT number, ['test'] FROM system.numbers LIMIT 1) USING (number) LIMIT 1;
SELECT number FROM system.numbers ANY LEFT JOIN (SELECT number, ['test'] FROM system.numbers LIMIT 1) USING (number) LIMIT 1;
SELECT number FROM system.numbers ANY LEFT JOIN (SELECT number, ['test'] FROM system.numbers LIMIT 1) USING (number) LIMIT 1;

View File

@ -1,3 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SELECT
loyalty,
count()

View File

@ -1,3 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
USE test;
DROP TABLE IF EXISTS join;

View File

@ -1,4 +1,4 @@
SET any_join_get_any_from_right_table = 1;
set any_join_distinct_right_table_keys = 1;
SELECT
CounterID,