Merge pull request #62457 from ClickHouse/analyzer-fixes-bugs-2

Add more tests from issues
This commit is contained in:
Nikita Mikhaylov 2024-04-16 10:50:16 +00:00 committed by GitHub
commit 29c554ac09
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
76 changed files with 1444 additions and 0 deletions

View File

@ -0,0 +1,2 @@
x1 [('k1',3),('k11',1)]
x1 [('k1',3),('k11',1)]

View File

@ -0,0 +1,16 @@
-- https://github.com/ClickHouse/ClickHouse/issues/58985
DROP TABLE IF EXISTS test_03093;
CREATE TABLE test_03093 (app String, c UInt64, k Map(String, String)) ENGINE=MergeTree ORDER BY app;
INSERT INTO test_03093 VALUES ('x1', 123, {'k1': ''});
INSERT INTO test_03093 VALUES ('x1', 123, {'k1': '', 'k11': ''});
INSERT INTO test_03093 VALUES ('x1', 12, {'k1': ''});
SET allow_experimental_analyzer=1;
select app, arrayZip(untuple(sumMap(k.keys, replicate(1, k.keys)))) from test_03093 PREWHERE c > 1 group by app;
select app, arrayZip(untuple(sumMap(k.keys, replicate(1, k.keys)))) from test_03093 WHERE c > 1 group by app;
DROP TABLE IF EXISTS test_03093;

View File

@ -0,0 +1 @@
1 Ksenia

View File

@ -0,0 +1,16 @@
DROP TABLE IF EXISTS users_03094;
CREATE TABLE users_03094 (name String, age Int16) ENGINE=Memory;
INSERT INTO users_03094 VALUES ('John', 33);
INSERT INTO users_03094 VALUES ('Ksenia', 48);
INSERT INTO users_03094 VALUES ('Alice', 50);
SET allow_experimental_analyzer=1;
SELECT
multiIf((age > 30) or (true), '1', '2') AS a,
max(name)
FROM users_03094
GROUP BY a;
DROP TABLE IF EXISTS users_03094;

View File

@ -0,0 +1 @@
1 1

View File

@ -0,0 +1,29 @@
-- https://github.com/ClickHouse/ClickHouse/issues/36963
DROP TABLE IF EXISTS mt1;
DROP TABLE IF EXISTS mt2;
DROP TABLE IF EXISTS b;
create table mt1 (f1 Int32, f2 Int32) engine = MergeTree() order by f1;
create table mt2 as mt1 engine = MergeTree() order by f1;
create table b as mt1 engine = Buffer(currentDatabase(), mt2, 16, 1, 1, 10000, 1000000, 10000000, 100000000);
create table m as mt1 engine = Merge(currentDatabase(), '^(mt1|b)$');
-- insert some data
insert into mt1 values(1, 1), (2, 2);
insert into b values(3, 3), (4, 4);
OPTIMIZE TABLE b;
OPTIMIZE TABLE mt1;
OPTIMIZE TABLE mt2;
-- do select
select f1, f2
from m
where f1 = 1 and f2 = 1;
DROP TABLE IF EXISTS mt1;
DROP TABLE IF EXISTS mt2;
DROP TABLE IF EXISTS b;

View File

@ -0,0 +1,5 @@
10000
239
1 1
1 1
1 1

View File

@ -0,0 +1,31 @@
-- https://github.com/ClickHouse/ClickHouse/issues/39453
DROP TABLE IF EXISTS test_03096;
CREATE TABLE test_03096
(
`a` UInt32,
`b` UInt32,
`c` UInt32,
`d` UInt32 MATERIALIZED 0,
`sum` UInt32 MATERIALIZED (a + b) + c,
INDEX idx (c, d) TYPE minmax GRANULARITY 1
)
ENGINE = MergeTree
ORDER BY a
SETTINGS index_granularity = 8192;
INSERT INTO test_03096 SELECT number, number % 42, number % 123 FROM numbers(10000);
select count() from test_03096;
select count() from test_03096 where b = 0;
alter table test_03096 update b = 100 where b = 0 SETTINGS mutations_sync=2;
select latest_fail_reason == '', is_done == 1 from system.mutations where table='test_03096' and database = currentDatabase();
alter table test_03096 update b = 123 where c = 0 SETTINGS mutations_sync=2;
select latest_fail_reason == '', is_done == 1 from system.mutations where table='test_03096' and database = currentDatabase();
DROP TABLE IF EXISTS test_03096;

View File

@ -0,0 +1,7 @@
-- https://github.com/ClickHouse/ClickHouse/issues/56521
SYSTEM FLUSH LOGS;
SET allow_experimental_analyzer=1;
SELECT count(1) as num, hostName() as hostName FROM system.query_log as a INNER JOIN system.processes as b on a.query_id = b.query_id and type = 'QueryStart' and dateDiff('second', event_time, now()) > 5 and current_database = currentDatabase() FORMAT Null;

View File

@ -0,0 +1,18 @@
300 \N 3
200 \N 2
100 \N 1
\N 30 3
\N 20 2
\N 10 1
-------------------------
300 30 3
200 20 2
100 10 1
-------------------------
10 3
10 2
10 1
-------------------------
10 3
10 2
10 1

View File

@ -0,0 +1,155 @@
-- https://github.com/ClickHouse/ClickHouse/issues/47552
DROP TABLE IF EXISTS clickhouse_alias_issue_1;
DROP TABLE IF EXISTS clickhouse_alias_issue_2;
CREATE TABLE clickhouse_alias_issue_1 (
id bigint,
column_1 Nullable(Float32)
) Engine=Memory;
CREATE TABLE clickhouse_alias_issue_2 (
id bigint,
column_2 Nullable(Float32)
) Engine=Memory;
SET allow_experimental_analyzer = 1;
INSERT INTO `clickhouse_alias_issue_1`
VALUES (1, 100), (2, 200), (3, 300);
INSERT INTO `clickhouse_alias_issue_2`
VALUES (1, 10), (2, 20), (3, 30);
-- This query returns the expected result
-- 300 \N 3
-- 200 \N 2
-- 100 \N 1
-- \N 30 3
-- \N 20 2
-- \N 10 1
SELECT *
FROM
(
SELECT
max(`column_1`) AS `column_1`,
NULL AS `column_2`,
`id`
FROM `clickhouse_alias_issue_1`
GROUP BY
`id`
UNION ALL
SELECT
NULL AS `column_1`,
max(`column_2`) AS `column_2`,
`id`
FROM `clickhouse_alias_issue_2`
GROUP BY
`id`
SETTINGS prefer_column_name_to_alias=1
)
ORDER BY ALL DESC NULLS LAST;
SELECT '-------------------------';
-- This query also returns the expected result
-- 300 30 3
-- 200 20 2
-- 100 10 1
SELECT
max(`column_1`) AS `column_1`,
max(`column_2`) AS `column_2`,
`id`
FROM (
SELECT
max(`column_1`) AS `column_1`,
NULL AS `column_2`,
`id`
FROM `clickhouse_alias_issue_1`
GROUP BY
`id`
UNION ALL
SELECT
NULL AS `column_1`,
max(`column_2`) AS `column_2`,
`id`
FROM `clickhouse_alias_issue_2`
GROUP BY
`id`
SETTINGS prefer_column_name_to_alias=1
) as T1
GROUP BY `id`
ORDER BY `id` DESC
SETTINGS prefer_column_name_to_alias=1;
SELECT '-------------------------';
-- Expected result :
-- 10 3
-- 10 2
-- 10 1
SELECT `column_1` / `column_2`, `id`
FROM (
SELECT
max(`column_1`) AS `column_1`,
max(`column_2`) AS `column_2`,
`id`
FROM (
SELECT
max(`column_1`) AS `column_1`,
NULL AS `column_2`,
`id`
FROM `clickhouse_alias_issue_1`
GROUP BY
`id`
UNION ALL
SELECT
NULL AS `column_1`,
max(`column_2`) AS `column_2`,
`id`
FROM `clickhouse_alias_issue_2`
GROUP BY
`id`
SETTINGS prefer_column_name_to_alias=1
) as T1
GROUP BY `id`
ORDER BY `id` DESC
SETTINGS prefer_column_name_to_alias=1
) as T2
WHERE `column_1` IS NOT NULL AND `column_2` IS NOT NULL
SETTINGS prefer_column_name_to_alias=1;
SELECT '-------------------------';
-- Without the setting, the expected result is the same
-- but the actual result isn't wrong
SELECT `column_1` / `column_2`, `id`
FROM (
SELECT
max(`column_1`) AS `column_1`,
max(`column_2`) AS `column_2`,
`id`
FROM (
SELECT
max(`column_1`) AS `column_1`,
NULL AS `column_2`,
`id`
FROM `clickhouse_alias_issue_1`
GROUP BY
`id`
UNION ALL
SELECT
NULL AS `column_1`,
max(`column_2`) AS `column_2`,
`id`
FROM `clickhouse_alias_issue_2`
GROUP BY
`id`
) as T1
GROUP BY `id`
ORDER BY `id` DESC
) as T2
WHERE `column_1` IS NOT NULL AND `column_2` IS NOT NULL;
DROP TABLE IF EXISTS clickhouse_alias_issue_1;
DROP TABLE IF EXISTS clickhouse_alias_issue_2;

View File

@ -0,0 +1,27 @@
-- https://github.com/ClickHouse/ClickHouse/issues/56503
SET allow_experimental_analyzer = 1;
SELECT
tb1.owner_id AS owner_id,
type
FROM
(
SELECT number AS owner_id
FROM numbers(100)
) AS tb1
CROSS JOIN values('type varchar', 'type1', 'type2', 'type3') AS pt
LEFT JOIN
(
SELECT tb2.owner_id AS owner_id
FROM
(
SELECT number AS owner_id
FROM numbers(100)
GROUP BY owner_id
) AS tb2
) AS merged USING (owner_id)
WHERE tb1.owner_id = merged.owner_id
GROUP BY
tb1.owner_id,
type
FORMAT `Null`;

View File

@ -0,0 +1 @@
1 Ksenia

View File

@ -0,0 +1,13 @@
-- https://github.com/ClickHouse/ClickHouse/issues/59101
SET allow_experimental_analyzer = 1;
CREATE TABLE users (name String, age Int16) ENGINE=Memory;
INSERT INTO users VALUES ('John', 33);
INSERT INTO users VALUES ('Ksenia', 48);
INSERT INTO users VALUES ('Alice', 50);
SELECT
multiIf((age > 30) or (true), '1', '2') AS a,
max(name)
FROM users
GROUP BY a;

View File

@ -0,0 +1,11 @@
0 2 1
---
1
0
0
0
0
---
1
2
3

View File

@ -0,0 +1,53 @@
-- https://github.com/ClickHouse/ClickHouse/issues/23194
-- This test add query-templates for fuzzer
SET allow_experimental_analyzer = 1;
DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier};
CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier};
USE {CLICKHOUSE_DATABASE:Identifier};
CREATE TABLE table (
column UInt64,
nest Nested
(
key Nested (
subkey UInt16
)
)
) ENGINE = Memory();
SELECT t.column FROM table AS t;
USE default;
SELECT column FROM {CLICKHOUSE_DATABASE:Identifier}.table;
USE {CLICKHOUSE_DATABASE:Identifier};
SELECT {CLICKHOUSE_DATABASE:Identifier}.table.column FROM table;
--
SELECT t1.x, t2.x, y FROM
(SELECT x, y FROM VALUES ('x UInt16, y UInt16', (0,1))) AS t1,
(SELECT x, z FROM VALUES ('x UInt16, z UInt16', (2,3))) AS t2;
SELECT '---';
SELECT 1;
SELECT dummy;
SELECT one.dummy;
SELECT system.one.dummy;
SELECT *;
--
SELECT nest.key.subkey FROM table;
SELECT table.nest FROM table ARRAY JOIN nest;
SELECT '---';
SELECT * FROM (SELECT [1, 2, 3] AS arr) ARRAY JOIN arr;
SELECT * FROM table ARRAY JOIN [1, 2, 3] AS arr;

View File

@ -0,0 +1,23 @@
1 1 2
1 2 1
3 6
---
123
---
123 123
123 1
---
555
---
99
---
1
---
1
---
2 2
2 2
---
[2,3]
1 [5,14]
1 [5,14]

View File

@ -0,0 +1,85 @@
-- https://github.com/ClickHouse/ClickHouse/issues/23194
SET allow_experimental_analyzer = 1;
CREATE TEMPORARY TABLE test1 (a String, nest Nested(x String, y String));
SELECT a, nest.* FROM test1 ARRAY JOIN nest;
SELECT a, n.* FROM test1 ARRAY JOIN nest AS n;
CREATE TEMPORARY TABLE test2 (a String, nest Array(Tuple(x String, y String)));
SELECT a, nest.* FROM test2 ARRAY JOIN nest;
SELECT a, n.* FROM test2 ARRAY JOIN nest AS n;
SELECT 1 AS x, x, x + 1;
SELECT x, x + 1, 1 AS x;
SELECT x, 1 + (2 + (3 AS x));
SELECT '---';
SELECT 123 AS x FROM (SELECT a, x FROM (SELECT 1 AS a, 2 AS b));
SELECT '---';
SELECT 123 AS x, (SELECT x) AS y;
SELECT 123 AS x, 123 IN (SELECT x);
SELECT '---';
WITH 123 AS x SELECT 555 FROM (SELECT a, x FROM (SELECT 1 AS a, 2 AS b));
SELECT '---';
-- here we refer to table `test1` (defined as subquery) three times, one of them inside another scalar subquery.
WITH t AS (SELECT 1) SELECT t, (SELECT * FROM t) FROM t; -- { serverError UNKNOWN_IDENTIFIER }
-- throws, because x is not visible outside.
SELECT x FROM (SELECT y FROM VALUES ('y UInt16', (2)) WHERE (1 AS x) = y) AS t; -- { serverError UNKNOWN_IDENTIFIER }
-- throws, because the table name `t` is not visible outside
SELECT t.x FROM (SELECT * FROM (SELECT 1 AS x) AS t); -- { serverError UNKNOWN_IDENTIFIER }
SELECT x FROM (SELECT * FROM (SELECT 99 AS x) AS t);
SELECT '---';
SELECT t.x FROM (SELECT 1 AS x) AS t;
SELECT t.a FROM (SELECT a FROM test1) AS t;
SELECT a FROM (SELECT a FROM test1) AS t;
SELECT '---';
-- this is wrong, the `tbl` name is not exported
SELECT test1.a FROM (SELECT a FROM test1) AS t; -- { serverError UNKNOWN_IDENTIFIER }
-- this is also wrong, the `t2` alias is not exported
SELECT test1.a FROM (SELECT a FROM test1 AS t2) AS t; -- { serverError UNKNOWN_IDENTIFIER }
-- does not work, `x` is not visible;
SELECT x, (SELECT 1 AS x); -- { serverError UNKNOWN_IDENTIFIER }
-- does not work either;
SELECT x IN (SELECT 1 AS x); -- { serverError UNKNOWN_IDENTIFIER }
-- this will work, but keep in mind that there are two different `x`.
SELECT x IN (SELECT 1 AS x) FROM (SELECT 1 AS x);
SELECT '---';
SELECT x + 1 AS x, x FROM (SELECT 1 AS x);
SELECT x, x + 1 AS x FROM (SELECT 1 AS x);
SELECT 1 AS x, 2 AS x; -- { serverError MULTIPLE_EXPRESSIONS_FOR_ALIAS }
SELECT '---';
SELECT arrayMap(x -> x + 1, [1, 2]);
SELECT x, arrayMap((x, y) -> x[1] + y + arrayFirst(x -> x != y, x), arr) FROM (SELECT 1 AS x, [([1, 2], 3), ([4, 5], 6)] AS arr);
SELECT x1, arrayMap((x2, y2) -> x2[1] + y2 + arrayFirst(x3 -> x3 != y2, x2), arr) FROM (SELECT 1 AS x1, [([1, 2], 3), ([4, 5], 6)] AS arr);
SELECT arrayMap(x -> [y * 2, (x + 1) AS y, 1 AS z], [1, 2]), y; -- { serverError UNKNOWN_IDENTIFIER }
-- TODO: this must work
--SELECT arrayMap(x -> [y * 2, (x + 1) AS y, 1 AS z], [1, 2]), z;
SELECT arrayMap(x -> (x + 1) AS y, [3, 5]), arrayMap(x -> (x || 'hello') AS y, ['qq', 'ww']); -- { serverError MULTIPLE_EXPRESSIONS_FOR_ALIAS }

View File

@ -0,0 +1,13 @@
0
2
20
---
0
0
---
1 2
1 2
---
1 1
1 1
---

View File

@ -0,0 +1,92 @@
-- Tags: no-parallel
-- Looks like you cannot use the query parameter as a column name.
-- https://github.com/ClickHouse/ClickHouse/issues/23194
SET allow_experimental_analyzer = 1;
DROP DATABASE IF EXISTS db1_03101;
DROP DATABASE IF EXISTS db2_03101;
CREATE DATABASE db1_03101;
CREATE DATABASE db2_03101;
USE db1_03101;
CREATE TABLE db1_03101.tbl
(
col String,
db1_03101 Nested
(
tbl Nested
(
col String
)
)
)
ENGINE = Memory;
SELECT db1_03101.tbl.col FROM db1_03101.tbl;
SELECT db1_03101.* FROM tbl;
SELECT db1_03101 FROM tbl;
SELECT * FROM tbl;
SELECT count(*) FROM tbl;
SELECT * + * FROM VALUES('a UInt16', 1, 10);
SELECT '---';
SELECT * GROUP BY *;
-- not ok as every component of ORDER BY may contain ASC/DESC and COLLATE; though can be supported in some sense
-- but it works
SELECT * ORDER BY *;
SELECT * WHERE *; -- { serverError UNSUPPORTED_METHOD }
SELECT '---';
SELECT * FROM (SELECT 1 AS a) AS t, (SELECT 2 AS b) AS u;
-- equivalent to:
SELECT a, b FROM (SELECT 1 AS a) AS t, (SELECT 2 AS b) AS u;
SELECT '---';
SELECT * FROM (SELECT 1 AS a) AS t, (SELECT 1 AS a) AS u;
-- equivalent to:
SELECT t.a, u.a FROM (SELECT 1 AS a) AS t, (SELECT 1 AS a) AS u;
SELECT '---';
---- TODO: think about it
--CREATE TABLE db1_03101.t
--(
-- a UInt16
--)
--ENGINE = Memory;
--
--CREATE TABLE db2_03101.t
--(
-- a UInt16
--)
--ENGINE = Memory;
--
--SELECT * FROM (SELECT 1 AS a) AS db2_03101.t, (SELECT 1 AS a) AS db1_03101.t;
---- equivalent to:
--SELECT db2_03101.t.a, db1_03101.t.a FROM (SELECT 1 AS a) AS db2_03101.t, (SELECT 1 AS a) AS db1_03101.t;
CREATE TABLE t
(
x String,
nest Nested
(
a String,
b String
)
) ENGINE = Memory;
SELECT * FROM t;
-- equivalent to:
SELECT x, nest.* FROM t;
-- equivalent to:
SELECT x, nest.a, nest.b FROM t;

View File

@ -0,0 +1,25 @@
1
('hello',1) hello
0
---
0
hello 1
---
0
0
---
1 1
1
1
---
3
---
5
5
5
5
5
---
11
---
10 12 11

View File

@ -0,0 +1,103 @@
-- https://github.com/ClickHouse/ClickHouse/issues/23194
SET allow_experimental_analyzer = 1;
DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier};
CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier};
USE {CLICKHOUSE_DATABASE:Identifier};
-- simple tuple access operator
SELECT tuple(1, 'a').1;
-- named tuple or complex column access operator - can be applied to Nested type as well as Array of named Tuple
SELECT CAST(('hello', 1) AS Tuple(hello String, count UInt32)) AS t, t.hello;
-- TODO: this doesn't work
-- https://github.com/ClickHouse/ClickHouse/issues/57361
-- SELECT CAST(('hello', 1) AS Tuple(hello String, count UInt32)).hello;
-- expansion of a tuple or complex column with asterisk
SELECT tuple(1, 'a').*;
SELECT '---';
SELECT CAST(('hello', 1) AS Tuple(name String, count UInt32)).*;
SELECT untuple(CAST(('hello', 1) AS Tuple(name String, count UInt32))); -- will give two columns `name` and `count`.
SELECT '---';
CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.t
(
col String,
hello String,
world String
)
ENGINE = Memory;
CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.u
(
cc String
)
ENGINE = Memory;
SELECT * EXCEPT('hello|world');
-- TODO: Qualified matcher t.* EXCEPT 'hello|world' does not find table.
-- SELECT t.* EXCEPT(hello, world);
-- SELECT {CLICKHOUSE_DATABASE:Identifier}.t.* REPLACE(x + 1 AS x);
SELECT * EXCEPT(hello) REPLACE(x + 1 AS x);
SELECT COLUMNS('^c') FROM t;
SELECT t.COLUMNS('^c') FROM t, u;
SELECT t.COLUMNS('^c') EXCEPT (test_hello, test_world) FROM t, u;
SELECT '---';
SELECT * FROM (SELECT x, x FROM (SELECT 1 AS x));
SELECT x FROM (SELECT x, x FROM (SELECT 1 AS x));
SELECT 1 FROM (SELECT x, x FROM (SELECT 1 AS x));
SELECT '---';
SELECT `plus(1, 2)` FROM (SELECT 1 + 2);
-- Lambda expressions can be aliased. (proposal)
--SELECT arrayMap(plus, [1, 2], [10, 20]);
--SELECT x -> x + 1 AS fun;
SELECT '---';
SELECT x FROM numbers(5 AS x);
SELECT '---';
CREATE TEMPORARY TABLE aliased
(
x UInt8 DEFAULT 0,
y ALIAS x + 1
);
INSERT INTO aliased VALUES (10);
SELECT y FROM aliased;
CREATE TEMPORARY TABLE aliased2
(
x UInt8,
y ALIAS ((x + 1) AS z) + 1
);
SELECT x, y, z FROM aliased2; -- { serverError UNKNOWN_IDENTIFIER }
SELECT '---';
CREATE TEMPORARY TABLE aliased3
(
x UInt8,
y ALIAS z + 1,
z ALIAS x + 1
);
INSERT INTO aliased3 VALUES (10);
SELECT x, y, z FROM aliased3;

View File

@ -0,0 +1 @@
succeed

View File

@ -0,0 +1,25 @@
-- https://github.com/ClickHouse/ClickHouse/issues/50271
drop table if exists t1;
drop table if exists t2;
set allow_experimental_analyzer=1;
create table t1 (c3 String, primary key(c3)) engine = MergeTree;
create table t2 (c11 String, primary key(c11)) engine = MergeTree;
insert into t1 values ('succeed');
insert into t2 values ('succeed');
select
ref_0.c11 as c_2_c30_0
from
t2 as ref_0
cross join (select
ref_1.c3 as c_6_c28_15
from
t1 as ref_1
) as subq_0
where subq_0.c_6_c28_15 = (select c11 from t2 order by c11 limit 1);
drop table if exists t1;
drop table if exists t2;

View File

@ -0,0 +1,2 @@
5
6

View File

@ -0,0 +1,30 @@
-- https://github.com/ClickHouse/ClickHouse/issues/54954
DROP TABLE IF EXISTS loans;
CREATE TABLE loans (loan_number int, security_id text) ENGINE=Memory;
SET allow_experimental_analyzer=1;
INSERT INTO loans VALUES (1, 'AAA');
INSERT INTO loans VALUES (1, 'AAA');
INSERT INTO loans VALUES (1, 'AAA');
INSERT INTO loans VALUES (1, 'AAA');
INSERT INTO loans VALUES (1, 'AAA');
INSERT INTO loans VALUES (1, 'BBB');
INSERT INTO loans VALUES (1, 'BBB');
INSERT INTO loans VALUES (1, 'BBB');
INSERT INTO loans VALUES (1, 'BBB');
INSERT INTO loans VALUES (1, 'BBB');
INSERT INTO loans VALUES (1, 'BBB');
with block_0 as (
select * from loans
),
block_1 as (
select sum(loan_number) as loan_number from block_0 group by security_id
)
select loan_number from block_1 where loan_number > 3 order by loan_number settings prefer_column_name_to_alias = 1;
DROP TABLE IF EXISTS loans;

View File

@ -0,0 +1,6 @@
Alice 50
John 33
Ksenia 48
Alice 50
John 33
Ksenia 48

View File

@ -0,0 +1,45 @@
-- https://github.com/ClickHouse/ClickHouse/issues/56466
SET allow_experimental_analyzer=1;
DROP TABLE IF EXISTS users;
CREATE TABLE users (uid Int16, name String, age Int16) ENGINE=Memory;
INSERT INTO users VALUES (1231, 'John', 33);
INSERT INTO users VALUES (6666, 'Ksenia', 48);
INSERT INTO users VALUES (8888, 'Alice', 50);
-- The query works when using a single SELECT *
SELECT *
FROM
(
SELECT
name,
age
FROM users
)
GROUP BY
1,
2
ORDER BY ALL;
-- It doesn't when the GROUP BY is nested deeper
SELECT *
FROM
(
SELECT *
FROM
(
SELECT
name,
age
FROM users
)
GROUP BY
1,
2
)
ORDER BY ALL;
DROP TABLE IF EXISTS users;

View File

@ -0,0 +1,26 @@
-- https://github.com/ClickHouse/ClickHouse/issues/11000
DROP TABLE IF EXISTS test_table_01;
DROP TABLE IF EXISTS test_table_02;
DROP TABLE IF EXISTS test_view_01;
SET allow_experimental_analyzer = 1;
CREATE TABLE test_table_01 (
column Int32
) ENGINE = Memory();
CREATE TABLE test_table_02 (
column Int32
) ENGINE = Memory();
CREATE VIEW test_view_01 AS
SELECT
t1.column,
t2.column
FROM test_table_01 AS t1
INNER JOIN test_table_02 AS t2 ON t1.column = t2.column;
DROP TABLE IF EXISTS test_table_01;
DROP TABLE IF EXISTS test_table_02;
DROP TABLE IF EXISTS test_view_01;

View File

@ -0,0 +1,45 @@
-- https://github.com/ClickHouse/ClickHouse/issues/10894
DROP TABLE IF EXISTS event;
DROP TABLE IF EXISTS user;
DROP TABLE IF EXISTS mv;
CREATE TABLE event (
`event_time` DateTime,
`event_name` String,
`user_id` String
)
ENGINE = MergeTree()
ORDER BY (event_time, event_name);
CREATE TABLE user (
`user_id` String,
`user_type` String
)
ENGINE = MergeTree()
ORDER BY (user_id);
INSERT INTO event VALUES ('2020-05-01 00:00:01', 'install', '1'), ('2020-05-01 00:00:02', 'install', '2'), ('2020-05-01 00:00:03', 'install', '3');
INSERT INTO user VALUES ('1', 'type_1'), ('2', 'type_2'), ('3', 'type_3');
CREATE MATERIALIZED VIEW mv
(
`event_time` DateTime,
`event_name` String,
`user_id` String,
`user_type` String
)
ENGINE = MergeTree()
ORDER BY (event_time, event_name) POPULATE AS
SELECT
e.event_time,
e.event_name,
e.user_id,
u.user_type
FROM event e
INNER JOIN user u ON u.user_id = e.user_id;
DROP TABLE IF EXISTS event;
DROP TABLE IF EXISTS user;
DROP TABLE IF EXISTS mv;

View File

@ -0,0 +1,15 @@
-- https://github.com/ClickHouse/ClickHouse/issues/448
DROP TABLE IF EXISTS a;
DROP TABLE iF EXISTS b;
CREATE TABLE a ( a UInt64, b UInt64) ENGINE = Memory;
CREATE TABLE b ( b UInt64) ENGINE = Memory;
SET allow_experimental_analyzer = 1;
SET joined_subquery_requires_alias = 0;
CREATE MATERIALIZED VIEW view_4 ( bb UInt64, cnt UInt64) Engine=MergeTree ORDER BY bb POPULATE AS SELECT bb, count() AS cnt FROM (SELECT a, b AS j, b AS bb FROM a INNER JOIN (SELECT b AS j, b AS bb FROM b ) USING (j)) GROUP BY bb; -- { serverError UNKNOWN_IDENTIFIER }
DROP TABLE IF EXISTS a;
DROP TABLE iF EXISTS b;

View File

@ -0,0 +1,2 @@
1 UInt8
1 UInt8

View File

@ -0,0 +1,5 @@
-- https://github.com/ClickHouse/ClickHouse/issues/8030
SET allow_experimental_analyzer=1;
DESCRIBE (SELECT 1, 1 UNION ALL SELECT 1, 2);

View File

@ -0,0 +1,53 @@
-- https://github.com/ClickHouse/ClickHouse/issues/32139
SET allow_experimental_analyzer=1;
WITH
data AS (
SELECT
rand64() AS val1,
rand64() AS val2,
rand64() AS val3,
rand64() AS val4,
rand64() AS val5,
rand64() AS val6,
rand64() AS val7,
rand64() AS val8,
rand64() AS val9,
rand64() AS val10,
rand64() AS val11,
rand64() AS val12,
rand64() AS val13,
rand64() AS val14
FROM numbers(10)
),
(SELECT avg(val1) FROM data) AS value1,
(SELECT avg(val2) FROM data) AS value2,
(SELECT avg(val3) FROM data) AS value3,
(SELECT avg(val4) FROM data) AS value4,
(SELECT avg(val5) FROM data) AS value5,
(SELECT avg(val6) FROM data) AS value6,
(SELECT avg(val7) FROM data) AS value7,
(SELECT avg(val8) FROM data) AS value8,
(SELECT avg(val9) FROM data) AS value9,
(SELECT avg(val10) FROM data) AS value10,
(SELECT avg(val11) FROM data) AS value11,
(SELECT avg(val12) FROM data) AS value12,
(SELECT avg(val13) FROM data) AS value13,
(SELECT avg(val14) FROM data) AS value14
SELECT
value1 AS v1,
value2 AS v2,
value3 AS v3,
value4 AS v4,
value5 AS v5,
value6 AS v6,
value7 AS v7,
value8 AS v8,
value9 AS v9,
value10 AS v10,
value11 AS v11,
value12 AS v12,
value13 AS v13,
value14 AS v14
FORMAT Null;

View File

@ -0,0 +1,2 @@
1
1

View File

@ -0,0 +1,33 @@
-- https://github.com/ClickHouse/ClickHouse/issues/47288
SET allow_experimental_analyzer=1;
select 1 as `c0`
from (
select C.`` AS ``
from (
select 2 as bb
) A
LEFT JOIN (
select '1' as ``
) C ON 1 = 1
LEFT JOIN (
select 1 as a
) D ON 1 = 1
) as `T0`
where `T0`.`` = '1';
select 1 as `c0`
from (
select C.`` AS ``
from (
select 2 as bb
) A
LEFT JOIN (
select '1' as ``
) C ON 1 = 1
LEFT JOIN (
select 1 as a
) D ON 1 = 1
) as `T0`
where `T0`.`` = '1';

View File

@ -0,0 +1,18 @@
-- https://github.com/ClickHouse/ClickHouse/issues/50705
set allow_experimental_analyzer=1;
SELECT
count(s0.number),
s1.half
FROM system.numbers AS s0
INNER JOIN
(
SELECT
number,
number / 2 AS half
FROM system.numbers
LIMIT 10
) AS s1 ON s0.number = s1.number
GROUP BY s0.number > 5
LIMIT 10 -- {serverError NOT_AN_AGGREGATE}

View File

@ -0,0 +1,2 @@
101 2
101 2

View File

@ -0,0 +1,27 @@
-- https://github.com/ClickHouse/ClickHouse/issues/54511
DROP TABLE IF EXISTS my_first_table;
CREATE TABLE my_first_table
(
user_id UInt32,
message String,
timestamp DateTime,
metric Float32
)
ENGINE = MergeTree
PRIMARY KEY (user_id, timestamp);
INSERT INTO my_first_table (user_id, message, timestamp, metric) VALUES
(101, 'Hello, ClickHouse!', now(), -1.0 ), (102, 'Insert a lot of rows per batch', yesterday(), 1.41421 ), (102, 'Sort your data based on your commonly-used queries', today(), 2.718 ), (101, 'Granules are the smallest chunks of data read', now() + 5, 3.14159 );
SET allow_experimental_analyzer=1;
SELECT
user_id
, (count(user_id) OVER (PARTITION BY user_id)) AS count
FROM my_first_table
WHERE timestamp > 0 and user_id IN (101)
LIMIT 2 BY user_id;
DROP TABLE IF EXISTS my_first_table;

View File

@ -0,0 +1,21 @@
-- https://github.com/ClickHouse/ClickHouse/pull/62457
drop table if exists t;
create table t (ID String) Engine= Memory() ;
insert into t values('a'),('b'),('c');
-- This optimization is disabled by default and even its description says that it could lead to
-- inconsistencies for distributed queries.
set optimize_if_transform_strings_to_enum=0;
set allow_experimental_analyzer=1;
SELECT multiIf( ((multiIf(ID='' AND (ID = 'a' OR ID = 'c' OR ID = 'b'),'a','x') as y) = 'c') OR
(multiIf(ID='' AND (ID = 'a' OR ID = 'c' OR ID = 'b'),'a','x') = 'b') OR
(multiIf(ID='' AND (ID = 'a' OR ID = 'c' OR ID = 'b'),'a','x') = 'd') OR
(multiIf(ID='' AND (ID = 'a' OR ID = 'c' OR ID = 'b'),'a','x') = 'e'),'test', 'x'
) AS alias
FROM remote('127.0.0.{1,2}', currentDatabase(), t)
GROUP BY alias;
drop table if exists t;

View File

@ -0,0 +1,15 @@
-- Tags: no-replicated-database
-- https://github.com/ClickHouse/ClickHouse/issues/58500
SET allow_experimental_analyzer=1;
drop table if exists t;
create table t (ID UInt8) Engine= Memory() ;
insert into t values(1),(2),(3);
with a as (select 1 as column_a) , b as (select 2 as column_b)
select * FROM remote('127.0.0.{1,2}', currentDatabase(), t) as c
inner join a on ID=column_a inner join b on ID=column_b;
drop table if exists t;

View File

@ -0,0 +1,2 @@
mycheck
1

View File

@ -0,0 +1,5 @@
-- https://github.com/ClickHouse/ClickHouse/issues/44412
SET allow_experimental_analyzer=1;
SELECT EXISTS(SELECT 1) AS mycheck FORMAT TSVWithNames;

View File

@ -0,0 +1,19 @@
-- https://github.com/ClickHouse/ClickHouse/issues/39923
SET allow_experimental_analyzer=1;
SELECT
errors.name AS labels,
value,
'ch_errors_total' AS name
FROM system.errors
LIMIT 1
FORMAT Null;
SELECT
map('name', errors.name) AS labels,
value,
'ch_errors_total' AS name
FROM system.errors
LIMIT 1
FORMAT Null;

View File

@ -0,0 +1,7 @@
100000 1419251193922930845
1000 15159681593610358603
10000 17240395714154645476
1000 15159681593610358603
10000 17240395714154645476
100000 1419251193922930845

View File

@ -0,0 +1,14 @@
-- https://github.com/ClickHouse/ClickHouse/issues/39855
SET allow_experimental_analyzer=1;
create table x(
a UInt64,
`sipHash64(a)` UInt64
) engine = MergeTree order by a;
insert into x select number, number from VALUES('number UInt64', 1000, 10000, 100000);
select a, sipHash64(a) from x order by sipHash64(a);
select '';
select a, sipHash64(a) from x order by `sipHash64(a)`;

View File

@ -0,0 +1,17 @@
-- https://github.com/ClickHouse/ClickHouse/issues/4596
SET allow_experimental_analyzer=1;
CREATE TABLE a1 ( ANIMAL Nullable(String) ) engine = MergeTree order by tuple();
insert into a1 values('CROCO');
select count()
from a1 a
join a1 b on (a.ANIMAL = b.ANIMAL)
join a1 c on (c.ANIMAL = b.ANIMAL)
where a.ANIMAL = 'CROCO';
select count()
from a1 a
join a1 b on (a.ANIMAL = b.ANIMAL)
join a1 c on (c.ANIMAL = b.ANIMAL)
prewhere a.ANIMAL = 'CROCO';

View File

@ -0,0 +1,9 @@
query1
15 10
15 20
query2
15 10
15 20
query3
15 10
15 20

View File

@ -0,0 +1,21 @@
-- https://github.com/ClickHouse/ClickHouse/issues/47422
SET allow_experimental_analyzer=1;
DROP TEMPORARY TABLE IF EXISTS test;
CREATE TEMPORARY TABLE test (a Float32, id UInt64);
INSERT INTO test VALUES (10,10),(20,20);
SELECT 'query1';
-- alias clash (a is redefined in CTE)
-- 21.8: no error, bad result
-- 21.9 and newer: error "Block structure mismatch in (columns with identical name must have identical structure) stream"
WITH avg(a) OVER () AS a SELECT a, id FROM test SETTINGS allow_experimental_window_functions = 1;
SELECT 'query2';
-- no aliases clash, good result
WITH avg(a) OVER () AS a2 SELECT a2, id FROM test SETTINGS allow_experimental_window_functions = 1;
SELECT 'query3';
-- aliases clash without CTE
SELECT avg(a) OVER () AS a, id FROM test SETTINGS allow_experimental_window_functions = 1;

View File

@ -0,0 +1,20 @@
localhost 9000 0 0 0
localhost 9000 0 0 0
9 1
10 2
11 2
9 1
10 2
11 2
9 1
10 2
11 2
9 1
10 2
11 2
9 1
10 2
11 2
9 1
10 2
11 2

View File

@ -0,0 +1,76 @@
-- Tags: no-replicated-database
-- https://github.com/ClickHouse/ClickHouse/issues/8547
SET allow_experimental_analyzer=1;
SET distributed_foreground_insert=1;
CREATE TABLE a1_replicated ON CLUSTER test_shard_localhost (
day Date,
id UInt32
)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/a1_replicated', '1_replica')
ORDER BY tuple();
CREATE TABLE a1 (
day Date,
id UInt32
)
ENGINE = Distributed('test_shard_localhost', currentDatabase(), a1_replicated, id);
CREATE TABLE b1_replicated ON CLUSTER test_shard_localhost (
day Date,
id UInt32
)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/b1_replicated', '1_replica')
ORDER BY tuple();
CREATE TABLE b1 (
day Date,
id UInt32
)
ENGINE = Distributed('test_shard_localhost', currentDatabase(), b1_replicated, id);
INSERT INTO a1 (day, id) VALUES ('2019-01-01', 9), ('2019-01-01', 10), ('2019-01-02', 10), ('2019-01-01', 11);
INSERT INTO b1 (day, id) VALUES ('2019-01-01', 9), ('2019-01-01', 10), ('2019-01-02', 11), ('2019-01-01', 11);
SET distributed_product_mode='local';
SELECT id, count()
FROM a1 AS a1
LEFT JOIN b1 AS b1 ON a1.id = b1.id
GROUP BY id
ORDER BY id;
SELECT id, count()
FROM a1 a1
LEFT JOIN (SELECT id FROM b1 b1) b1 ON a1.id = b1.id
GROUP BY id
ORDER BY id;
SELECT id, count()
FROM (SELECT id FROM a1) a1
LEFT JOIN (SELECT id FROM b1) b1 ON a1.id = b1.id
GROUP BY id
ORDER BY id;
SET distributed_product_mode='global';
SELECT id, count()
FROM a1 AS a1
LEFT JOIN b1 AS b1 ON a1.id = b1.id
GROUP BY id
ORDER BY id;
SELECT id, count()
FROM a1 a1
LEFT JOIN (SELECT id FROM b1 b1) b1 ON a1.id = b1.id
GROUP BY id
ORDER BY id;
SELECT id, count()
FROM (SELECT id FROM a1) a1
LEFT JOIN (SELECT id FROM b1) b1 ON a1.id = b1.id
GROUP BY id
ORDER BY id;

View File

@ -0,0 +1,9 @@
number
30
30
number
30
30
column
30
30

View File

@ -0,0 +1,10 @@
-- https://github.com/ClickHouse/ClickHouse/issues/33000
SET allow_experimental_analyzer=1;
SET param_test_a=30;
WITH {test_a:UInt32} as column SELECT column as number FROM numbers(2) FORMAT TSVWithNames;
WITH {test_a:UInt32} as column SELECT {test_a:UInt32} as number FROM numbers(2) FORMAT TSVWithNames;
WITH {test_a:UInt32} as column SELECT column FROM numbers(2) FORMAT TSVWithNames;

View File

@ -0,0 +1,2 @@
redefined
redefined

View File

@ -0,0 +1,39 @@
-- https://github.com/ClickHouse/ClickHouse/issues/14739
SET allow_experimental_analyzer=1;
drop table if exists test_subquery;
CREATE TABLE test_subquery
ENGINE = Memory AS
SELECT 'base' AS my_field;
-- query 1
SELECT my_field
FROM
(
SELECT
*,
'redefined' AS my_field
from test_subquery
);
-- query 2
SELECT my_field
FROM
(
SELECT
'redefined' AS my_field,
*
from test_subquery
);
-- query 3
-- it works with old analyzer
SELECT my_field
FROM
(
SELECT
*,
'redefined' AS my_field
from (select * from test_subquery)
); -- {serverError AMBIGUOUS_COLUMN_NAME}

View File

@ -0,0 +1,3 @@
北京 ['北京']
上海 ['北京','上海']
西安 ['北京','上海','西安']

View File

@ -0,0 +1,9 @@
-- Tags: no-fasttest
-- https://github.com/ClickHouse/ClickHouse/issues/44039
SET allow_experimental_analyzer=1;
create table test_window_collate(c1 String, c2 String) engine=MergeTree order by c1;
insert into test_window_collate values('1', '上海');
insert into test_window_collate values('1', '北京');
insert into test_window_collate values('1', '西安');
select c2, groupArray(c2) over (partition by c1 order by c2 asc collate 'zh_Hans_CN') as res from test_window_collate order by c2 asc collate 'zh_Hans_CN';

View File

@ -0,0 +1,2 @@
1 10 1 1
2 12 2 2

View File

@ -0,0 +1,28 @@
-- https://github.com/ClickHouse/ClickHouse/issues/22923
SET allow_experimental_analyzer=1;
SET prefer_localhost_replica=0;
create table "t0" (a Int64, b Int64) engine = MergeTree() partition by a order by a;
create table "dist_t0" (a Int64, b Int64) engine = Distributed(test_shard_localhost, currentDatabase(), t0);
insert into t0 values (1, 10), (2, 12);
SELECT * FROM (
WITH
b AS
(
SELECT toInt64(number) AS a
FROM numbers(10)
),
c AS
(
SELECT toInt64(number) AS a
FROM numbers(10)
)
SELECT *
FROM dist_t0 AS a
LEFT JOIN b AS b ON a.a = b.a
LEFT JOIN c AS c ON a.a = c.a
)
ORDER BY ALL;

View File

@ -0,0 +1,4 @@
0 2
3 2
6 2
9 2

View File

@ -0,0 +1,20 @@
-- https://github.com/ClickHouse/ClickHouse/issues/23865
SET allow_experimental_analyzer=1;
create table table_local engine = Memory AS select * from numbers(10);
create table table_dist engine = Distributed('test_cluster_two_shards', currentDatabase(),table_local) AS table_local;
with
x as (
select number
from numbers(10)
where number % 3=0),
y as (
select number, count()
from table_dist
where number in (select * from x)
group by number
)
select * from y
ORDER BY ALL;

View File

@ -0,0 +1 @@
1234 1234 1234

View File

@ -0,0 +1,17 @@
-- https://github.com/ClickHouse/ClickHouse/issues/29748
SET allow_experimental_analyzer=1;
create table events ( distinct_id String ) engine = Memory;
INSERT INTO events VALUES ('1234'), ('1');
WITH cte1 as (
SELECT '1234' as x
), cte2 as (
SELECT '1234' as x
)
SELECT *
FROM events AS events
JOIN cte2 ON cte2.x = events.distinct_id
JOIN cte1 ON cte1.x = cte2.x
limit 1;

View File

@ -0,0 +1,7 @@
SET allow_experimental_analyzer=1;
SELECT v.x, r.a, sum(c)
FROM (select 1 x, 2 c) AS v
ANY LEFT JOIN (SELECT 1 x, 2 a) AS r ON v.x = r.x
GROUP BY v.x; -- { serverError NOT_AN_AGGREGATE}