mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
fix some tests
This commit is contained in:
parent
951f971d73
commit
30b09f4287
@ -4,6 +4,9 @@ SET max_rows_to_group_by = 100000;
|
||||
SET max_block_size = 100001;
|
||||
SET group_by_overflow_mode = 'any';
|
||||
|
||||
-- Settings 'max_rows_to_group_by' and 'max_bytes_before_external_group_by' are mutually exclusive.
|
||||
SET max_bytes_before_external_group_by = 0;
|
||||
|
||||
DROP TABLE IF EXISTS numbers500k;
|
||||
CREATE TABLE numbers500k (number UInt32) ENGINE = TinyLog;
|
||||
|
||||
|
@ -12,7 +12,7 @@ SELECT x, s, k FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LE
|
||||
SELECT 1, x, 2, s, 3, k, 4 FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 USING k;
|
||||
|
||||
SELECT t1.k, t1.s, t2.x
|
||||
FROM ( SELECT number AS k, 'a' AS s FROM numbers(2) GROUP BY number WITH TOTALS ) AS t1
|
||||
FROM ( SELECT number AS k, 'a' AS s FROM numbers(2) GROUP BY number WITH TOTALS ORDER BY number) AS t1
|
||||
ANY LEFT JOIN t2 AS t2 USING(k);
|
||||
|
||||
DROP TABLE t2;
|
||||
|
@ -36,9 +36,9 @@ GROUP BY ORDER BY
|
||||
1
|
||||
GROUP BY w/ ALIAS
|
||||
0
|
||||
1
|
||||
0
|
||||
1
|
||||
1
|
||||
ORDER BY w/ ALIAS
|
||||
0
|
||||
func(aggregate function) GROUP BY
|
||||
|
@ -34,7 +34,7 @@ SELECT uniq(number) u FROM remote('127.0.0.{2,3}', currentDatabase(), data_00184
|
||||
|
||||
-- cover possible tricky issues
|
||||
SELECT 'GROUP BY w/ ALIAS';
|
||||
SELECT n FROM remote('127.0.0.{2,3}', currentDatabase(), data_00184) GROUP BY number AS n SETTINGS distributed_group_by_no_merge=2;
|
||||
SELECT n FROM remote('127.0.0.{2,3}', currentDatabase(), data_00184) GROUP BY number AS n ORDER BY n SETTINGS distributed_group_by_no_merge=2;
|
||||
|
||||
SELECT 'ORDER BY w/ ALIAS';
|
||||
SELECT n FROM remote('127.0.0.{2,3}', currentDatabase(), data_00184) ORDER BY number AS n LIMIT 1 SETTINGS distributed_group_by_no_merge=2;
|
||||
|
@ -8,4 +8,7 @@ SELECT quantilesExact(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0
|
||||
SELECT quantilesTDigest(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001);
|
||||
SELECT quantilesDeterministic(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(x, x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001);
|
||||
|
||||
-- The result slightly differs but it's ok since `quantilesDeterministic` is an approximate function.
|
||||
SET max_bytes_before_external_group_by = 0;
|
||||
|
||||
SELECT round(1000000 / (number + 1)) AS k, count() AS c, arrayMap(x -> round(x, 6), quantilesDeterministic(0.1, 0.5, 0.9)(number, intHash64(number))) AS q1, quantilesExact(0.1, 0.5, 0.9)(number) AS q2 FROM (SELECT number FROM system.numbers LIMIT 1000000) GROUP BY k ORDER BY k;
|
||||
|
@ -7,4 +7,8 @@ DROP TABLE IF EXISTS arena;
|
||||
|
||||
SELECT length(arrayReduce('groupUniqArray', [[1, 2], [1], emptyArrayUInt8(), [1], [1, 2]]));
|
||||
SELECT min(x), max(x) FROM (SELECT length(arrayReduce('groupUniqArray', [hex(number), hex(number+1), hex(number)])) AS x FROM system.numbers LIMIT 100000);
|
||||
|
||||
-- Disable external aggregation because the state is reset for each new block of data in 'runningAccumulate' function.
|
||||
SET max_bytes_before_external_group_by = 0;
|
||||
|
||||
SELECT sum(length(runningAccumulate(x))) FROM (SELECT groupUniqArrayState(toString(number % 10)) AS x, number FROM (SELECT * FROM system.numbers LIMIT 11) GROUP BY number ORDER BY number);
|
||||
|
@ -48,7 +48,8 @@ SELECT
|
||||
intDiv(number, 25) AS n,
|
||||
avgState(number) AS s
|
||||
FROM numbers(2500)
|
||||
GROUP BY n;
|
||||
GROUP BY n
|
||||
ORDER BY n;
|
||||
|
||||
SET force_primary_key = 1, enable_optimize_predicate_expression = 1;
|
||||
|
||||
@ -60,7 +61,8 @@ FROM
|
||||
finalizeAggregation(s)
|
||||
FROM test_00808_push_down_with_finalizeAggregation
|
||||
)
|
||||
WHERE (n >= 2) AND (n <= 5);
|
||||
WHERE (n >= 2) AND (n <= 5)
|
||||
ORDER BY n;
|
||||
|
||||
EXPLAIN SYNTAX SELECT *
|
||||
FROM
|
||||
|
@ -4,6 +4,9 @@
|
||||
|
||||
set optimize_distributed_group_by_sharding_key=1;
|
||||
|
||||
-- Some queries in this test require sorting after aggregation.
|
||||
set max_bytes_before_external_group_by = 0;
|
||||
|
||||
drop table if exists dist_01247;
|
||||
drop table if exists data_01247;
|
||||
|
||||
|
@ -28,7 +28,7 @@ opts=(
|
||||
"--prefer_localhost_replica=0"
|
||||
)
|
||||
$CLICKHOUSE_CLIENT "${opts[@]}" --format CSV -nm <<EOL
|
||||
select count(), * from dist_01247 group by number limit 1;
|
||||
select count(), * from dist_01247 group by number order by number limit 1;
|
||||
EOL
|
||||
|
||||
# expect zero new network errors
|
||||
|
@ -1,7 +1,7 @@
|
||||
-- Tags: long, no-fasttest
|
||||
|
||||
-- The result slightly differs but it's ok since `uniqueTheta` is an approximate function.
|
||||
set group_by_two_level_threshold_bytes = 0;
|
||||
set max_bytes_before_external_group_by = 0;
|
||||
|
||||
SELECT 'uniqTheta';
|
||||
|
||||
|
@ -113,7 +113,7 @@ FROM
|
||||
)
|
||||
ORDER BY number DESC
|
||||
) AS t2
|
||||
ORDER BY number
|
||||
ORDER BY t1.number, t2.number
|
||||
-- explain
|
||||
Expression (Projection)
|
||||
Sorting (Sorting for ORDER BY)
|
||||
|
@ -97,7 +97,7 @@ FROM
|
||||
)
|
||||
ORDER BY number DESC
|
||||
) AS t2
|
||||
ORDER BY number"
|
||||
ORDER BY t1.number, t2.number"
|
||||
run_query "$query"
|
||||
|
||||
echo "-- CROSS JOIN with subqueries, ORDER BY in main query -> all ORDER BY clauses will be removed in subqueries"
|
||||
|
@ -113,7 +113,7 @@ FROM
|
||||
)
|
||||
ORDER BY number DESC
|
||||
) AS t2
|
||||
ORDER BY number
|
||||
ORDER BY t1.number, t2.number
|
||||
-- explain
|
||||
Expression (Project names)
|
||||
Sorting (Sorting for ORDER BY)
|
||||
|
@ -73,7 +73,7 @@ FROM
|
||||
SELECT DISTINCT number AS n
|
||||
FROM numbers(2)
|
||||
) as y
|
||||
ORDER BY n
|
||||
ORDER BY x.n, y.n
|
||||
-- explain
|
||||
Expression (Projection)
|
||||
Distinct
|
||||
@ -155,7 +155,7 @@ FROM
|
||||
FROM VALUES('Hello', 'World', 'Goodbye')
|
||||
) AS words
|
||||
ARRAY JOIN [0, 1] AS arr
|
||||
ORDER BY arr
|
||||
ORDER BY c1, arr
|
||||
-- explain
|
||||
Expression (Projection)
|
||||
Distinct
|
||||
@ -169,9 +169,9 @@ Expression (Projection)
|
||||
Expression (Before ORDER BY)
|
||||
ReadFromStorage (Values)
|
||||
-- execute
|
||||
Goodbye
|
||||
Hello
|
||||
World
|
||||
Goodbye
|
||||
-- WITH FILL: do _not_ remove outer DISTINCT because new rows are generated between inner and outer DISTINCTs
|
||||
-- query
|
||||
SELECT DISTINCT *
|
||||
|
@ -60,7 +60,7 @@ FROM
|
||||
SELECT DISTINCT number AS n
|
||||
FROM numbers(2)
|
||||
) as y
|
||||
ORDER BY n"
|
||||
ORDER BY x.n, y.n"
|
||||
run_query "$query"
|
||||
|
||||
echo "-- DISTINCT duplicates with several columns"
|
||||
@ -99,7 +99,7 @@ FROM
|
||||
FROM VALUES('Hello', 'World', 'Goodbye')
|
||||
) AS words
|
||||
ARRAY JOIN [0, 1] AS arr
|
||||
ORDER BY arr"
|
||||
ORDER BY c1, arr"
|
||||
run_query "$query"
|
||||
|
||||
echo "-- WITH FILL: do _not_ remove outer DISTINCT because new rows are generated between inner and outer DISTINCTs"
|
||||
|
@ -74,7 +74,7 @@ FROM
|
||||
SELECT DISTINCT number AS n
|
||||
FROM numbers(2)
|
||||
) as y
|
||||
ORDER BY n
|
||||
ORDER BY x.n, y.n
|
||||
-- explain
|
||||
Expression (Project names)
|
||||
Distinct (DISTINCT)
|
||||
@ -157,7 +157,7 @@ FROM
|
||||
FROM VALUES('Hello', 'World', 'Goodbye')
|
||||
) AS words
|
||||
ARRAY JOIN [0, 1] AS arr
|
||||
ORDER BY arr
|
||||
ORDER BY c1, arr
|
||||
-- explain
|
||||
Expression (Project names)
|
||||
Distinct (DISTINCT)
|
||||
@ -172,9 +172,9 @@ Expression (Project names)
|
||||
Expression ((Projection + Change column names to column identifiers))
|
||||
ReadFromStorage (Values)
|
||||
-- execute
|
||||
Goodbye
|
||||
Hello
|
||||
World
|
||||
Goodbye
|
||||
-- WITH FILL: do _not_ remove outer DISTINCT because new rows are generated between inner and outer DISTINCTs
|
||||
-- query
|
||||
SELECT DISTINCT *
|
||||
|
@ -76,10 +76,10 @@ GROUP BY g;
|
||||
select '=';
|
||||
SELECT MAX(left.c0), min2(left.c0, -(-left.c0) * (radians(left.c0) - radians(left.c0))) as g, (((-1925024212 IS NOT NULL) IS NOT NULL) != radians(tan(1216286224))) AND cos(lcm(MAX(left.c0), -1966575216) OR (MAX(left.c0) * 1180517420)) as h, not h, h is null
|
||||
FROM t2 AS left
|
||||
GROUP BY g HAVING h SETTINGS enable_optimize_predicate_expression = 0;
|
||||
GROUP BY g HAVING h ORDER BY g DESC SETTINGS enable_optimize_predicate_expression = 0;
|
||||
select '=';
|
||||
SELECT MAX(left.c0), min2(left.c0, -(-left.c0) * (radians(left.c0) - radians(left.c0))) as g, (((-1925024212 IS NOT NULL) IS NOT NULL) != radians(tan(1216286224))) AND cos(lcm(MAX(left.c0), -1966575216) OR (MAX(left.c0) * 1180517420)) as h, not h, h is null
|
||||
FROM t2 AS left
|
||||
GROUP BY g HAVING h SETTINGS enable_optimize_predicate_expression = 1;
|
||||
GROUP BY g HAVING h ORDER BY g DESC SETTINGS enable_optimize_predicate_expression = 1;
|
||||
|
||||
DROP TABLE IF EXISTS t2;
|
||||
|
Loading…
Reference in New Issue
Block a user