Fixed tests

This commit is contained in:
Maksim Kita 2023-02-20 19:26:39 +01:00
parent 2672f24ced
commit 40fc36bb6f
34 changed files with 102 additions and 75 deletions

View File

@ -3647,7 +3647,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveQualifiedMatcher(Qu
if (!table_expression_node)
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER,
"Qualified matcher {} does not find table. In scope {}",
matcher_node->formatASTForErrorMessage(),
scope.scope_node->formatASTForErrorMessage());

View File

@ -533,7 +533,8 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
for (auto & output_node : cast_actions_dag->getOutputs())
{
if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output_node->result_name))
if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output_node->result_name) &&
output_node->result_type->canBeInsideNullable())
output_node = &cast_actions_dag->addFunction(to_nullable_function, {output_node}, output_node->result_name);
}

View File

@ -1,3 +1,4 @@
set allow_experimental_analyzer = 1;
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;
@ -10,7 +11,7 @@ select * from (select toLowCardinality(toNullable(dummy)) as val from system.one
select * from (select toLowCardinality(dummy) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as val from system.one) using val;
select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as val from system.one) using val;
select '-';
select * from (select dummy as val from system.one) any left join (select dummy as val from system.one) on val + 0 = val * 1; -- { serverError 352 }
select * from (select dummy as val from system.one) any left join (select dummy as val from system.one) on val + 0 = val * 1; -- { serverError 403 }
select * from (select dummy as val from system.one) any left join (select dummy as rval from system.one) on val + 0 = rval * 1;
select * from (select toLowCardinality(dummy) as val from system.one) any left join (select dummy as rval from system.one) on val + 0 = rval * 1;
select * from (select dummy as val from system.one) any left join (select toLowCardinality(dummy) as rval from system.one) on val + 0 = rval * 1;

View File

@ -1,3 +1,5 @@
SET allow_experimental_analyzer = 1;
{% for join_algorithm in ['partial_merge', 'full_sorting_merge', 'grace_hash'] -%}
SET join_algorithm = '{{ join_algorithm }}';
@ -12,7 +14,7 @@ SELECT * FROM (SELECT toLowCardinality(toNullable(dummy)) AS val FROM system.one
SELECT * FROM (SELECT toLowCardinality(dummy) AS val FROM system.one) s1 ANY LEFT JOIN (SELECT toLowCardinality(toNullable(dummy)) AS val FROM system.one) s2 USING val ORDER BY val;
SELECT * FROM (SELECT toLowCardinality(toNullable(dummy)) AS val FROM system.one) s1 ANY LEFT JOIN (SELECT toLowCardinality(toNullable(dummy)) AS val FROM system.one) s2 USING val ORDER BY val;
SELECT '-';
SELECT * FROM (SELECT dummy AS val FROM system.one) s1 ANY LEFT JOIN (SELECT dummy AS val FROM system.one) s2 ON val + 0 = val * 1 ORDER BY val; -- { serverError 352 }
SELECT * FROM (SELECT dummy AS val FROM system.one) s1 ANY LEFT JOIN (SELECT dummy AS val FROM system.one) s2 ON val + 0 = val * 1 ORDER BY val; -- { serverError 403 }
SELECT * FROM (SELECT dummy AS val FROM system.one) s1 ANY LEFT JOIN (SELECT dummy AS rval FROM system.one) s2 ON val + 0 = rval * 1 ORDER BY val;
SELECT * FROM (SELECT toLowCardinality(dummy) AS val FROM system.one) s1 ANY LEFT JOIN (SELECT dummy AS rval FROM system.one) s2 ON val + 0 = rval * 1 ORDER BY val;
SELECT * FROM (SELECT dummy AS val FROM system.one) s1 ANY LEFT JOIN (SELECT toLowCardinality(dummy) AS rval FROM system.one) s2 ON val + 0 = rval * 1 ORDER BY val;

View File

@ -1,3 +1,5 @@
SET allow_experimental_analyzer = 1;
SELECT number FROM numbers(10) LIMIT 0 + 1;
SELECT number FROM numbers(10) LIMIT 1 - 1;
SELECT number FROM numbers(10) LIMIT 2 - 1;
@ -9,7 +11,7 @@ SELECT number FROM numbers(10) LIMIT now(); -- { serverError 440 }
SELECT number FROM numbers(10) LIMIT today(); -- { serverError 440 }
SELECT number FROM numbers(10) LIMIT toUInt8('1');
SELECT number FROM numbers(10) LIMIT toFloat32('1');
SELECT number FROM numbers(10) LIMIT rand(); -- { serverError 36 }
SELECT number FROM numbers(10) LIMIT rand(); -- { serverError 440 }
SELECT count() <= 1 FROM (SELECT number FROM numbers(10) LIMIT randConstant() % 2);

View File

@ -1,2 +1,2 @@
1 2 3
1 2 3
2 1 3
2 1 3

View File

@ -1,3 +1,4 @@
SET allow_experimental_analyzer = 1;
SET joined_subquery_requires_alias = 1;
SELECT * FROM (SELECT 1 as A, 2 as B) X

View File

@ -1,7 +1,7 @@
drop table if exists tab;
create table tab (x UInt64) engine = MergeTree order by tuple();
insert into tab select number as n from numbers(20) nums
insert into tab select n from (SELECT number AS n FROM numbers(20)) nums
semi left join (select number * 10 as n from numbers(2)) js2 using(n)
settings max_block_size = 5;
select * from tab order by x;

View File

@ -1,3 +1,5 @@
SET allow_experimental_analyzer = 1;
DROP TABLE IF EXISTS ints;
CREATE TABLE ints (i64 Int64, i32 Int32) ENGINE = Memory;
@ -10,6 +12,6 @@ SELECT '-';
SELECT * FROM ints l INNER JOIN ints r USING i64 ORDER BY l.i32, r.i32;
SELECT '-';
SELECT count() FROM ( SELECT [1], count(1) ) AS t1 ALL RIGHT JOIN ( SELECT number AS s FROM numbers(2) ) AS t2 USING (s); -- { serverError NOT_FOUND_COLUMN_IN_BLOCK }
SELECT count() FROM ( SELECT [1], count(1) ) AS t1 ALL RIGHT JOIN ( SELECT number AS s FROM numbers(2) ) AS t2 USING (s); -- { serverError UNKNOWN_IDENTIFIER }
DROP TABLE ints;

View File

@ -1,6 +1,6 @@
SET join_algorithm = 'hash';
SELECT number as n, j FROM numbers(4) nums
SELECT n, j FROM (SELECT number as n FROM numbers(4)) nums
ANY LEFT JOIN (
SELECT number * 2 AS n, number + 10 AS j
FROM numbers(4000)
@ -10,7 +10,7 @@ ORDER BY n;
SET max_rows_in_join = 1000;
SELECT number as n, j FROM numbers(4) nums
SELECT n, j FROM (SELECT number AS n FROM numbers(4)) nums
ANY LEFT JOIN (
SELECT number * 2 AS n, number + 10 AS j
FROM numbers(4000)
@ -20,7 +20,7 @@ ORDER BY n; -- { serverError 191 }
SET join_algorithm = 'partial_merge';
SELECT number as n, j FROM numbers(4) nums
SELECT n, j FROM (SELECT number as n FROM numbers(4)) nums
ANY LEFT JOIN (
SELECT number * 2 AS n, number + 10 AS j
FROM numbers(4000)
@ -30,7 +30,7 @@ ORDER BY n;
SET partial_merge_join_optimizations = 1;
SELECT number as n, j FROM numbers(4) nums
SELECT n, j FROM (SELECT number AS n FROM numbers(4)) nums
ANY LEFT JOIN (
SELECT number * 2 AS n, number + 10 AS j
FROM numbers(4000)
@ -40,7 +40,7 @@ ORDER BY n;
SET join_algorithm = 'auto';
SELECT number as n, j FROM numbers(4) nums
SELECT n, j FROM (SELECT number AS n FROM numbers(4)) nums
ANY LEFT JOIN (
SELECT number * 2 AS n, number + 10 AS j
FROM numbers(4000)
@ -50,7 +50,7 @@ ORDER BY n;
SET max_rows_in_join = '10';
SELECT number as n, j FROM numbers(4) nums
SELECT n, j FROM (SELECT number AS n FROM numbers(4)) nums
ANY LEFT JOIN (
SELECT number * 2 AS n, number + 10 AS j
FROM numbers(4000)

View File

@ -1,87 +1,87 @@
SET join_algorithm = 'partial_merge';
SELECT count(1), uniqExact(1) FROM (
SELECT materialize(1) as k FROM numbers(1) nums
SELECT k FROM (SELECT materialize(1) AS k FROM numbers(1)) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT materialize(1) as k FROM numbers(1) nums
SELECT k FROM (SELECT materialize(1) as k FROM numbers(1)) nums
JOIN (SELECT 1 AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT 1 as k FROM numbers(1) nums
SELECT k FROM (SELECT 1 AS k FROM numbers(1)) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT 1 as k FROM numbers(1) nums
SELECT k FROM (SELECT 1 as k FROM numbers(1)) nums
JOIN (SELECT 1 AS k, number n FROM numbers(100000)) j
USING k);
SELECT 'first nullable';
SELECT count(1), uniqExact(1) FROM (
SELECT materialize(toNullable(1)) as k FROM numbers(1) nums
SELECT k FROM (SELECT materialize(toNullable(1)) AS k FROM numbers(1)) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT materialize(toNullable(1)) as k FROM numbers(1) nums
SELECT k FROM (SELECT materialize(toNullable(1)) as k FROM numbers(1)) nums
JOIN (SELECT 1 AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT toNullable(1) as k FROM numbers(1) nums
SELECT k FROM (SELECT toNullable(1) as k FROM numbers(1)) nums
JOIN (SELECT materialize(1) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT toNullable(1) as k FROM numbers(1) nums
SELECT k FROM (SELECT toNullable(1) as k FROM numbers(1)) nums
JOIN (SELECT 1 AS k, number n FROM numbers(100000)) j
USING k);
SELECT 'second nullable';
SELECT count(1), uniqExact(1) FROM (
SELECT materialize(1) as k FROM numbers(1) nums
SELECT k FROM (SELECT materialize(1) as k FROM numbers(1)) nums
JOIN (SELECT materialize(toNullable(1)) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT materialize(1) as k FROM numbers(1) nums
SELECT k FROM (SELECT materialize(1) as k FROM numbers(1)) nums
JOIN (SELECT toNullable(1) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT 1 as k FROM numbers(1) nums
SELECT k FROM (SELECT 1 as k FROM numbers(1)) nums
JOIN (SELECT materialize(toNullable(1)) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT 1 as k FROM numbers(1) nums
SELECT k FROM (SELECT 1 as k FROM numbers(1)) nums
JOIN (SELECT toNullable(1) AS k, number n FROM numbers(100000)) j
USING k);
SELECT 'both nullable';
SELECT count(1), uniqExact(1) FROM (
SELECT materialize(toNullable(1)) as k FROM numbers(1) nums
SELECT k FROM (SELECT materialize(toNullable(1)) as k FROM numbers(1)) nums
JOIN (SELECT materialize(toNullable(1)) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT materialize(toNullable(1)) as k FROM numbers(1) nums
SELECT k FROM (SELECT materialize(toNullable(1)) as k FROM numbers(1)) nums
JOIN (SELECT toNullable(1) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT toNullable(1) as k FROM numbers(1) nums
SELECT k FROM (SELECT toNullable(1) as k FROM numbers(1)) nums
JOIN (SELECT materialize(toNullable(1)) AS k, number n FROM numbers(100000)) j
USING k);
SELECT count(1), uniqExact(1) FROM (
SELECT toNullable(1) as k FROM numbers(1) nums
SELECT k FROM (SELECT toNullable(1) as k FROM numbers(1)) nums
JOIN (SELECT toNullable(1) AS k, number n FROM numbers(100000)) j
USING k);

View File

@ -1,3 +1,5 @@
SET allow_experimental_analyzer = 1;
DROP TABLE IF EXISTS part;
DROP TABLE IF EXISTS supplier;
DROP TABLE IF EXISTS partsupp;
@ -180,7 +182,7 @@ order by
n_name,
s_name,
p_partkey
limit 100; -- { serverError 47 }
limit 100; -- { serverError 1 }
select 3;
select
@ -598,7 +600,7 @@ where
lineitem
where
l_partkey = p_partkey
); -- { serverError 47 }
); -- { serverError 1 }
select 18;
select
@ -709,7 +711,7 @@ where
and s_nationkey = n_nationkey
and n_name = 'CANADA'
order by
s_name; -- { serverError 47 }
s_name; -- { serverError 1 }
select 21, 'fail: exists, not exists'; -- TODO
-- select

View File

@ -1,7 +1,7 @@
SELECT
t1.*,
t2.*,
'world',
'world' AS constant,
isConstant('world')
FROM
(
@ -19,7 +19,7 @@ LEFT JOIN
SELECT
t1.*,
t2.*,
123,
123 AS constant,
isConstant('world')
FROM
(

View File

@ -13,9 +13,9 @@ SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' PASSWORD '' TAB
LIFETIME(0)
LAYOUT(hashed());
SELECT dictGetInt32('system.dict1', 'element_count', toUInt64(dict_key)) AS join_key,
SELECT join_key,
toTimeZone(dictGetDateTime('system.dict1', 'loading_start_time', toUInt64(dict_key)), 'UTC') AS datetime
FROM (select 1 AS dict_key) js1
FROM (select dictGetInt32('system.dict1', 'element_count', toUInt64(dict_key)) AS join_key, 1 AS dict_key) js1
LEFT JOIN (SELECT toInt32(2) AS join_key) js2
USING (join_key)
WHERE now() >= datetime;

View File

@ -33,9 +33,9 @@ select 'cluster() GROUP BY ORDER BY';
select * from cluster(test_cluster_two_shards, currentDatabase(), dist_01223) group by key order by key;
select 'LEFT JOIN';
select toInt32(number) key, b.key from numbers(2) a left join (select distinct * from dist_01223) b using key order by b.key;
select a.key, b.key from (SELECT toInt32(number) key from numbers(2)) a left join (select distinct * from dist_01223) b using key order by b.key;
select 'RIGHT JOIN';
select toInt32(number) key, b.key from numbers(2) a right join (select distinct * from dist_01223) b using key order by b.key;
select a.key, b.key from (SELECT toInt32(number) key from numbers(2)) a right join (select distinct * from dist_01223) b using key order by b.key;
-- more data for GROUP BY
insert into data_01223 select number%3 from numbers(30);

View File

@ -2,7 +2,7 @@
hello 1 3 world
9
9 (0,1)
key tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), 1) tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), 2) tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), 3) tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), 4) tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), 5)
key tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'1\') tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'2\') tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'3\') tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'4\') tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'5\')
1 20 20 10 20 30
2 11 20 10 20 30
3 70 20 10 20 30

View File

@ -1,3 +1,5 @@
SET allow_experimental_analyzer = 1;
select untuple((* except (b),)) from (select 1 a, 2 b, 3 c);
select 'hello', untuple((* except (b),)), 'world' from (select 1 a, 2 b, 3 c);
select argMax(untuple(x)) from (select (number, number + 1) as x from numbers(10));

View File

@ -1,5 +1,7 @@
-- Tags: global
SET allow_experimental_analyzer = 1;
DROP TABLE IF EXISTS local_table;
DROP TABLE IF EXISTS dist_table;
@ -10,7 +12,7 @@ INSERT INTO local_table SELECT number AS id, toString(number) AS val FROM number
CREATE TABLE dist_table AS local_table
ENGINE = Distributed('test_cluster_two_shards_localhost', currentDatabase(), local_table);
SELECT uniq(d.val) FROM dist_table AS d GLOBAL LEFT JOIN numbers(100) AS t USING id; -- { serverError 284 }
SELECT uniq(d.val) FROM dist_table AS d GLOBAL LEFT JOIN numbers(100) AS t USING id; -- { serverError 47 }
SELECT uniq(d.val) FROM dist_table AS d GLOBAL LEFT JOIN local_table AS t USING id;
DROP TABLE local_table;

View File

@ -1,6 +1,6 @@
┌─a─┬──────────b─┬─c─┬──────────d─┬─t2.'0.10'─┐
│ a │ 2018-01-01 │ │ 1970-01-01 │ │
│ b │ 2018-01-01 │ B │ 2018-01-01 │ 0.10
│ c │ 2018-01-01 │ C │ 2018-01-01 │ 0.10
└───┴────────────┴───┴────────────┴───────────┘
┌─a─┬──────────b─┬─c─┬──────────d─┬─'0.10'─┬─c─┐
│ a │ 2018-01-01 │ │ 1970-01-01 │
│ b │ 2018-01-01 │ B │ 2018-01-01 │ 0.10 │ B
│ c │ 2018-01-01 │ C │ 2018-01-01 │ 0.10 │ C
└───┴────────────┴───┴────────────┴───────────┘
\N \N \N \N 0 0

View File

@ -1,3 +1,5 @@
SET allow_experimental_analyzer = 1;
DROP TABLE IF EXISTS table1;
DROP TABLE IF EXISTS table2;

View File

@ -1,3 +1,4 @@
1 1 1
2 1 1
1 2 1
1 1 2

View File

@ -1,8 +1,7 @@
SET allow_experimental_analyzer = 1;
SET joined_subquery_requires_alias = 0;
select * FROM (SELECT 1), (SELECT 1), (SELECT 1); -- { serverError 352 }
-- This queries work by luck.
-- Feel free to remove then if it is the only failed test.
select * FROM (SELECT 1), (SELECT 1), (SELECT 1);
select * from (select 2), (select 1) as a, (select 1) as b;
select * from (select 1) as a, (select 2), (select 1) as b;
select * from (select 1) as a, (select 1) as b, (select 2);

View File

@ -1,7 +1,9 @@
-- Tags: global
SET allow_experimental_analyzer = 1;
SELECT
cityHash64(number GLOBAL IN (NULL, -2147483648, -9223372036854775808), nan, 1024, NULL, NULL, 1.000100016593933, NULL),
(NULL, cityHash64(inf, -2147483648, NULL, NULL, 10.000100135803223), cityHash64(1.1754943508222875e-38, NULL, NULL, NULL), 2147483647)
FROM cluster(test_cluster_two_shards_localhost, numbers((NULL, cityHash64(0., 65536, NULL, NULL, 10000000000., NULL), 0) GLOBAL IN (some_identifier), 65536))
WHERE number GLOBAL IN [1025] --{serverError 284}
WHERE number GLOBAL IN [1025] --{serverError 36}

View File

@ -1,3 +1,5 @@
SET allow_experimental_analyzer = 1;
DROP TABLE IF EXISTS t_tuple_element;
CREATE TABLE t_tuple_element(t1 Tuple(a UInt32, s String), t2 Tuple(UInt32, String)) ENGINE = Memory;
@ -16,7 +18,7 @@ EXPLAIN SYNTAX SELECT tupleElement(t1, 'a') FROM t_tuple_element;
SELECT tupleElement(number, 1) FROM numbers(1); -- { serverError 43 }
SELECT tupleElement(t1) FROM t_tuple_element; -- { serverError 42 }
SELECT tupleElement(t1, 'b') FROM t_tuple_element; -- { serverError 47 }
SELECT tupleElement(t1, 'b') FROM t_tuple_element; -- { serverError 10 }
SELECT tupleElement(t1, 0) FROM t_tuple_element; -- { serverError 127 }
SELECT tupleElement(t1, 3) FROM t_tuple_element; -- { serverError 127 }
SELECT tupleElement(t1, materialize('a')) FROM t_tuple_element; -- { serverError 43 }
@ -28,7 +30,7 @@ SELECT tupleElement(t2, 1) FROM t_tuple_element;
EXPLAIN SYNTAX SELECT tupleElement(t2, 1) FROM t_tuple_element;
SELECT tupleElement(t2) FROM t_tuple_element; -- { serverError 42 }
SELECT tupleElement(t2, 'a') FROM t_tuple_element; -- { serverError 47 }
SELECT tupleElement(t2, 'a') FROM t_tuple_element; -- { serverError 10 }
SELECT tupleElement(t2, 0) FROM t_tuple_element; -- { serverError 127 }
SELECT tupleElement(t2, 3) FROM t_tuple_element; -- { serverError 127 }
SELECT tupleElement(t2, materialize(1)) FROM t_tuple_element; -- { serverError 43 }

View File

@ -30,7 +30,7 @@
"meta":
[
{
"type": "Tuple(UInt64, UInt64)"
"type": "Tuple(`max(number)` UInt64, `count(number)` UInt64)"
}
],

View File

@ -7,4 +7,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
echo "#1"
${CLICKHOUSE_CLIENT} --query='SELECT count() FROM numbers(100) FORMAT JSON;' | grep -a -v "elapsed"
echo "#2"
${CLICKHOUSE_CLIENT} --query='SELECT (SELECT max(number), count(number) FROM numbers(100000) as n) SETTINGS max_block_size = 65505 FORMAT JSON;' | grep -a -v "elapsed" | grep -v "_subquery"
${CLICKHOUSE_CLIENT} --query='SELECT (SELECT max(number), count(number) FROM numbers(100000) as n) SETTINGS max_block_size = 65505, allow_experimental_analyzer = 1 FORMAT JSON;' | grep -a -v "elapsed" | grep -v "_subquery"

View File

@ -3,11 +3,11 @@
"meta":
[
{
"name": "dictGet(02154_test_dictionary, 'value', toUInt64(0))",
"name": "dictGet('02154_test_dictionary', 'value', toUInt64(0))",
"type": "String"
},
{
"name": "dictGet(02154_test_dictionary, 'value', toUInt64(1))",
"name": "dictGet('02154_test_dictionary', 'value', toUInt64(1))",
"type": "String"
}
],
@ -15,8 +15,8 @@
"data":
[
{
"dictGet(02154_test_dictionary, 'value', toUInt64(0))": "Value",
"dictGet(02154_test_dictionary, 'value', toUInt64(1))": ""
"dictGet('02154_test_dictionary', 'value', toUInt64(0))": "Value",
"dictGet('02154_test_dictionary', 'value', toUInt64(1))": ""
}
],

View File

@ -32,6 +32,7 @@ $CLICKHOUSE_CLIENT -q """
echo """
SELECT dictGet(02154_test_dictionary, 'value', toUInt64(0)), dictGet(02154_test_dictionary, 'value', toUInt64(1))
SETTINGS allow_experimental_analyzer = 1
FORMAT JSON
""" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&wait_end_of_query=1&output_format_write_statistics=0" -d @-

View File

@ -1,2 +1,2 @@
WITH 2 AS `b.c`, [4, 5] AS a, 6 AS u, 3 AS v, 2 AS d, TRUE AS e, 1 AS f, 0 AS g, 2 AS h, 'Hello' AS i, 'World' AS j, TIMESTAMP '2022-02-02 02:02:02' AS w, [] AS k, (1, 2) AS l, 2 AS m, 3 AS n, [] AS o, [1] AS p, 1 AS q, q AS r, 1 AS s, 1 AS t
SELECT INTERVAL CASE CASE WHEN NOT -a[b.c] * u DIV v + d IS NOT NULL AND e OR f BETWEEN g AND h THEN i ELSE j END WHEN w THEN k END || [l, (m, n)] MINUTE IS NULL OR NOT o::Array(INT) = p <> q < r > s != t AS upyachka;
SELECT INTERVAL CASE CASE WHEN NOT -a[`b.c`] * u DIV v + d IS NOT NULL AND e OR f BETWEEN g AND h THEN i ELSE j END WHEN w THEN k END || [l, (m, n)] MINUTE IS NULL OR NOT o::Array(INT) = p <> q < r > s != t AS upyachka;

View File

@ -5,5 +5,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_LOCAL --query="SELECT n" 2>&1 | grep -q "Code: 47. DB::Exception: Missing columns:" && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_LOCAL --query="SELECT n SETTINGS allow_experimental_analyzer = 1" 2>&1 | grep -q "Code: 47. DB::Exception:" && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_LOCAL --query="SELECT n -- { serverError 47 }"

View File

@ -1,3 +1,5 @@
SET allow_experimental_analyzer = 1;
DROP TABLE IF EXISTS github_events;
CREATE TABLE github_events
@ -59,6 +61,6 @@ CREATE TABLE github_events
)
ENGINE = MergeTree ORDER BY (event_type, repo_name, created_at);
with top_repos as ( select repo_name from github_events where event_type = 'WatchEvent' and toDate(created_at) = today() - 1 group by repo_name order by count() desc limit 100 union distinct select repo_name from github_events where event_type = 'WatchEvent' and toMonday(created_at) = toMonday(today() - interval 1 week) group by repo_name order by count() desc limit 100 union distinct select repo_name from github_events where event_type = 'WatchEvent' and toStartOfMonth(created_at) = toStartOfMonth(today()) - interval 1 month group by repo_name order by count() desc limit 100 union distinct select repo_name from github_events where event_type = 'WatchEvent' and toYear(created_at) = toYear(today()) - 1 group by repo_name order by count() desc limit 100 ), last_day as ( select repo_name, count() as count_last_day, rowNumberInAllBlocks() + 1 as position_last_day from github_events where repo_name in (select repo_name from top_repos) and toDate(created_at) = today() - 1 group by repo_name order by count_last_day desc ), last_week as ( select repo_name, count() as count_last_week, rowNumberInAllBlocks() + 1 as position_last_week from github_events where repo_name in (select repo_name from top_repos) and toMonday(created_at) = toMonday(today()) - interval 1 week group by repo_name order by count_last_week desc ), last_month as ( select repo_name, count() as count_last_month, rowNumberInAllBlocks() + 1 as position_last_month from github_events where repo_name in (select repo_name from top_repos) and toStartOfMonth(created_at) = toStartOfMonth(today()) - interval 1 month group by repo_name order by count_last_month desc ) select d.repo_name, columns(count) from last_day d join last_week w on d.repo_name = w.repo_name join last_month m on d.repo_name = m.repo_name FORMAT TabSeparatedWithNamesAndTypes; -- { serverError 47 }
with top_repos as ( select repo_name from github_events where event_type = 'WatchEvent' and toDate(created_at) = today() - 1 group by repo_name order by count() desc limit 100 union distinct select repo_name from github_events where event_type = 'WatchEvent' and toMonday(created_at) = toMonday(today() - interval 1 week) group by repo_name order by count() desc limit 100 union distinct select repo_name from github_events where event_type = 'WatchEvent' and toStartOfMonth(created_at) = toStartOfMonth(today()) - interval 1 month group by repo_name order by count() desc limit 100 union distinct select repo_name from github_events where event_type = 'WatchEvent' and toYear(created_at) = toYear(today()) - 1 group by repo_name order by count() desc limit 100 ), last_day as ( select repo_name, count() as count_last_day, rowNumberInAllBlocks() + 1 as position_last_day from github_events where repo_name in (select repo_name from top_repos) and toDate(created_at) = today() - 1 group by repo_name order by count_last_day desc ), last_week as ( select repo_name, count() as count_last_week, rowNumberInAllBlocks() + 1 as position_last_week from github_events where repo_name in (select repo_name from top_repos) and toMonday(created_at) = toMonday(today()) - interval 1 week group by repo_name order by count_last_week desc ), last_month as ( select repo_name, count() as count_last_month, rowNumberInAllBlocks() + 1 as position_last_month from github_events where repo_name in (select repo_name from top_repos) and toStartOfMonth(created_at) = toStartOfMonth(today()) - interval 1 month group by repo_name order by count_last_month desc ) select d.repo_name, columns(count) from last_day d join last_week w on d.repo_name = w.repo_name join last_month m on d.repo_name = m.repo_name;
DROP TABLE github_events;

View File

@ -1,11 +1,13 @@
-- https://github.com/ClickHouse/ClickHouse/issues/34697
SET allow_experimental_analyzer = 1;
SELECT table1_id FROM (
SELECT first.table1_id
FROM (SELECT number+1 as table1_id FROM numbers(1)) as first
JOIN (SELECT number+1 as table2_id FROM numbers(1)) as second ON first.table1_id = second.table2_id
JOIN (SELECT number+1 as table3_id FROM numbers(1)) as third ON first.table1_id = third.table3_id
); -- { serverError UNKNOWN_IDENTIFIER }
);
SELECT table1_id FROM (
SELECT first.table1_id

View File

@ -37,7 +37,7 @@ QUERY id: 0
JOIN TREE
TABLE id: 7, table_name: system.numbers
LIMIT
CONSTANT id: 17, constant_value: UInt64_10, constant_value_type: UInt8
CONSTANT id: 17, constant_value: UInt64_10, constant_value_type: UInt64
google
google
google
@ -80,7 +80,7 @@ QUERY id: 0
JOIN TREE
TABLE id: 9, table_name: system.numbers
LIMIT
CONSTANT id: 19, constant_value: UInt64_10, constant_value_type: UInt8
CONSTANT id: 19, constant_value: UInt64_10, constant_value_type: UInt64
other1
other1
google1
@ -124,7 +124,7 @@ QUERY id: 0
JOIN TREE
TABLE id: 9, table_name: system.numbers
LIMIT
CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt8
CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt64
google1
google1
google1
@ -171,7 +171,7 @@ QUERY id: 0
JOIN TREE
TABLE id: 11, table_name: system.numbers
LIMIT
CONSTANT id: 22, constant_value: UInt64_10, constant_value_type: UInt8
CONSTANT id: 22, constant_value: UInt64_10, constant_value_type: UInt64
google
google
google
@ -225,7 +225,7 @@ QUERY id: 0
JOIN TREE
TABLE id: 12, table_name: system.numbers
LIMIT
CONSTANT id: 22, constant_value: UInt64_10, constant_value_type: UInt8
CONSTANT id: 22, constant_value: UInt64_10, constant_value_type: UInt64
other
other
google
@ -276,7 +276,7 @@ QUERY id: 0
JOIN TREE
TABLE id: 10, table_name: system.numbers
LIMIT
CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt8
CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt64
google google
google google
google google
@ -343,7 +343,7 @@ QUERY id: 0
JOIN TREE
TABLE id: 9, table_name: system.numbers
LIMIT
CONSTANT id: 19, constant_value: UInt64_10, constant_value_type: UInt8
CONSTANT id: 19, constant_value: UInt64_10, constant_value_type: UInt64
other other
other other
google google
@ -404,7 +404,7 @@ QUERY id: 0
JOIN TREE
TABLE id: 7, table_name: system.numbers
LIMIT
CONSTANT id: 17, constant_value: UInt64_10, constant_value_type: UInt8
CONSTANT id: 17, constant_value: UInt64_10, constant_value_type: UInt64
\N
\N
\N
@ -444,7 +444,7 @@ QUERY id: 0
JOIN TREE
TABLE id: 11, table_name: system.numbers
LIMIT
CONSTANT id: 12, constant_value: UInt64_10, constant_value_type: UInt8
CONSTANT id: 12, constant_value: UInt64_10, constant_value_type: UInt64
\N
\N
\N
@ -473,7 +473,7 @@ QUERY id: 0
JOIN TREE
TABLE id: 5, table_name: system.numbers
LIMIT
CONSTANT id: 9, constant_value: UInt64_10, constant_value_type: UInt8
CONSTANT id: 9, constant_value: UInt64_10, constant_value_type: UInt64
other
other
google
@ -502,7 +502,7 @@ QUERY id: 0
JOIN TREE
TABLE id: 5, table_name: system.numbers
LIMIT
CONSTANT id: 9, constant_value: UInt64_10, constant_value_type: UInt8
CONSTANT id: 9, constant_value: UInt64_10, constant_value_type: UInt64
google
google
google
@ -534,4 +534,4 @@ QUERY id: 0
JOIN TREE
TABLE id: 7, table_name: system.numbers
LIMIT
CONSTANT id: 11, constant_value: UInt64_10, constant_value_type: UInt8
CONSTANT id: 11, constant_value: UInt64_10, constant_value_type: UInt64