fix some tests

This commit is contained in:
Anton Popov 2023-12-11 12:15:42 +00:00
parent 951f971d73
commit 30b09f4287
17 changed files with 35 additions and 20 deletions

View File

@ -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;

View File

@ -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;

View File

@ -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

View File

@ -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;

View File

@ -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;

View File

@ -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);

View File

@ -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

View File

@ -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;

View File

@ -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

View File

@ -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';

View File

@ -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)

View File

@ -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"

View File

@ -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)

View File

@ -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 *

View File

@ -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"

View File

@ -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 *

View File

@ -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;