better setting name: joined_subquery_requires_alias

This commit is contained in:
chertus 2019-09-02 15:53:31 +03:00
parent 2c0d569fa8
commit bc95027cee
37 changed files with 39 additions and 39 deletions

View File

@ -222,7 +222,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingSeconds, http_receive_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP receive timeout") \
M(SettingBool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown") \
M(SettingBool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.") \
M(SettingBool, subquery_requires_alias, false, "Force subquery in FROM to have an alias for correct name qualification.") \
M(SettingBool, joined_subquery_requires_alias, false, "Force joined subqueries to have aliases for correct name qualification.") \
M(SettingBool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.") \
M(SettingBool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.") \
M(SettingUInt64, odbc_max_field_size, 1024, "Max size of filed can be read from ODBC dictionary. Long strings are truncated.") \

View File

@ -441,7 +441,7 @@ void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast
/// attach an alias to subquery.
/// TODO: remove setting check after testing period
if (data.context.getSettingsRef().subquery_requires_alias)
if (data.context.getSettingsRef().joined_subquery_requires_alias)
{
SetSubqueryAliasVisitor::Data alias_data{String("--.join") + std::to_string(i)};
SetSubqueryAliasVisitor(alias_data).visit(left_table);

View File

@ -103,11 +103,11 @@ void translateQualifiedNames(ASTPtr & query, const ASTSelectQuery & select_query
auto & settings = context.getSettingsRef();
std::vector<TableWithColumnNames> tables_with_columns = getDatabaseAndTablesWithColumnNames(select_query, context);
if (settings.subquery_requires_alias && tables_with_columns.size() > 1)
if (settings.joined_subquery_requires_alias && tables_with_columns.size() > 1)
{
for (auto & pr : tables_with_columns)
if (pr.first.table.empty() && pr.first.alias.empty())
throw Exception("Not unique subquery in FROM requires an alias (or subquery_requires_alias=0 to disable restriction).",
throw Exception("Not unique subquery in FROM requires an alias (or joined_subquery_requires_alias=0 to disable restriction).",
ErrorCodes::ALIAS_REQUIRED);
}

View File

@ -1,4 +1,4 @@
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
SELECT k, s1, s2 FROM (SELECT intDiv(number, 3) AS k, sum(number) AS s1 FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY k) ANY LEFT JOIN (SELECT intDiv(number, 4) AS k, sum(number) AS s2 FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY k) USING k ORDER BY k;
SELECT k, s1, s2 FROM (SELECT intDiv(number, 3) AS k, sum(number) AS s1 FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY k WITH TOTALS) ANY LEFT JOIN (SELECT intDiv(number, 4) AS k, sum(number) AS s2 FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY k) USING k ORDER BY k;

View File

@ -1,5 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
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;

View File

@ -1,5 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
DROP TABLE IF EXISTS series;

View File

@ -1,5 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
DROP TABLE IF EXISTS series;

View File

@ -1,5 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
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;

View File

@ -1,4 +1,4 @@
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
SELECT k, a FROM (SELECT 42 AS k FROM remote('127.0.0.2', system.one)) GLOBAL ALL FULL OUTER JOIN (SELECT 42 AS k, 1 AS a, a) USING k;
SELECT 1 FROM remote('127.0.0.2', system.one) WHERE (1, 1) GLOBAL IN (SELECT 1 AS a, a);

View File

@ -1,5 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
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,6 +1,6 @@
SET join_use_nulls = 0;
SET any_join_distinct_right_table_keys = 1;
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
SELECT k, a, b
FROM

View File

@ -1,5 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
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,5 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
DROP TABLE IF EXISTS local_statements;
DROP TABLE IF EXISTS statements;

View File

@ -1,6 +1,6 @@
SET send_logs_level = 'none';
SET any_join_distinct_right_table_keys = 1;
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
DROP TABLE IF EXISTS test_00597;
DROP TABLE IF EXISTS test_view_00597;

View File

@ -1,4 +1,4 @@
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
-- This test (SELECT) without cache can take tens minutes
DROP TABLE IF EXISTS dict_string;

View File

@ -1,4 +1,4 @@
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
drop table if exists tab1;
drop table if exists tab2;

View File

@ -1,5 +1,5 @@
set any_join_distinct_right_table_keys = 1;
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
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;

View File

@ -30,7 +30,7 @@ select * from persons all inner join children using id;
select * from persons all inner join (select * from children) as j using id;
select * from (select * from persons) as s all inner join (select * from children ) as j using id;
--
set subquery_requires_alias = 0;
set joined_subquery_requires_alias = 0;
select * from persons all inner join (select * from children) using id;
select * from (select * from persons) all inner join (select * from children) using id;
select * from (select * from persons) as s all inner join (select * from children) using id;

View File

@ -1,4 +1,4 @@
set subquery_requires_alias = 0;
set joined_subquery_requires_alias = 0;
drop table if exists t_00725_2;
drop table if exists s_00725_2;

View File

@ -1,4 +1,4 @@
set subquery_requires_alias = 0;
set joined_subquery_requires_alias = 0;
DROP TABLE IF EXISTS left_table;
DROP TABLE IF EXISTS right_table;

View File

@ -1,4 +1,4 @@
set subquery_requires_alias = 0;
set joined_subquery_requires_alias = 0;
select * from (select dummy as val from system.one) any left join (select dummy as val from system.one) using val;
select * from (select toLowCardinality(dummy) as val from system.one) any left join (select dummy as val from system.one) using val;

View File

@ -1,5 +1,5 @@
SET send_logs_level = 'none';
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
DROP TABLE IF EXISTS delta_codec_synthetic;
DROP TABLE IF EXISTS default_codec_synthetic;

View File

@ -1,5 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
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;

View File

@ -14,7 +14,7 @@ INSERT INTO table3 SELECT number * 30, number * 300 FROM numbers(10);
INSERT INTO table5 SELECT number * 5, number * 50, number * 500 FROM numbers(10);
SET allow_experimental_multiple_joins_emulation = 1;
SET subquery_requires_alias = 1;
SET joined_subquery_requires_alias = 1;
select t1.a, t2.b, t3.c from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on t2.b = t3.b;
select t1.a, t2.b, t5.c from table1 as t1 join table2 as t2 on t1.a = t2.a join table5 as t5 on t1.a = t5.a AND t2.b = t5.b;

View File

@ -1,5 +1,5 @@
SET enable_debug_queries = 1;
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
DROP TABLE IF EXISTS t1_00849;
DROP TABLE IF EXISTS t2_00849;

View File

@ -9,7 +9,7 @@ CREATE TABLE t2_00850 (dummy UInt8) ENGINE = Distributed(test_shard_localhost, c
INSERT INTO t_local VALUES (1);
SET asterisk_left_columns_only = 1;
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
SELECT * FROM t1_00850
GLOBAL INNER JOIN

View File

@ -44,7 +44,7 @@ ALL RIGHT JOIN
USING (something)
ORDER BY count_a DESC;
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
SELECT something, count_a, count_b, toTypeName(something) FROM
( SELECT something, count() AS count_a FROM table_a GROUP BY something )

View File

@ -1,4 +1,4 @@
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
set allow_experimental_multiple_joins_emulation = 0;
set allow_experimental_cross_to_join_conversion = 0;

View File

@ -1,4 +1,4 @@
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
select toTypeName(key), toTypeName(value) from (
select 1 as key, '' as value

View File

@ -1,5 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
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 subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
DROP TABLE IF EXISTS ANIMAL;

View File

@ -1,4 +1,4 @@
SET subquery_requires_alias = 1;
SET joined_subquery_requires_alias = 1;
SELECT * FROM (SELECT 1 as A, 2 as B) X
ALL LEFT JOIN (SELECT 3 as A, 2 as B) Y
@ -12,7 +12,7 @@ SELECT * FROM (SELECT 1 as A, 2 as B)
ALL LEFT JOIN (SELECT 3 as A, 2 as B) Y
USING (B); -- { serverError 206 }
set subquery_requires_alias=0;
set joined_subquery_requires_alias = 0;
SELECT * FROM (SELECT 1 as A, 2 as B)
ALL LEFT JOIN (SELECT 3 as A, 2 as B) Y

View File

@ -19,4 +19,4 @@ FROM
) USING EventDate
ORDER BY hits DESC
LIMIT 10
SETTINGS subquery_requires_alias = 0;
SETTINGS joined_subquery_requires_alias = 0;

View File

@ -13,4 +13,4 @@ FROM test.hits ANY LEFT JOIN
GROUP BY EventDate
ORDER BY hits DESC
LIMIT 10
SETTINGS subquery_requires_alias = 0;
SETTINGS joined_subquery_requires_alias = 0;

View File

@ -19,4 +19,4 @@ FROM
) USING domain
ORDER BY hits DESC
LIMIT 10
SETTINGS subquery_requires_alias = 0;
SETTINGS joined_subquery_requires_alias = 0;

View File

@ -1,5 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SET subquery_requires_alias = 0;
SET joined_subquery_requires_alias = 0;
SELECT
loyalty,

View File

@ -1,5 +1,5 @@
set any_join_distinct_right_table_keys = 1;
set subquery_requires_alias = 0;
set joined_subquery_requires_alias = 0;
SELECT
CounterID,