Merge pull request #64452 from Algunenano/more_slow_tests

Reduce the size of some slow tests (2)
This commit is contained in:
Max K 2024-05-29 15:02:10 +00:00 committed by GitHub
commit ce891c70b5
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
11 changed files with 93 additions and 149 deletions

View File

@ -11,7 +11,11 @@ ${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS parallel_ddl"
function query()
{
for _ in {1..50}; do
local it=0
TIMELIMIT=30
while [ $SECONDS -lt "$TIMELIMIT" ] && [ $it -lt 50 ];
do
it=$((it+1))
${CLICKHOUSE_CLIENT} --query "CREATE DATABASE IF NOT EXISTS parallel_ddl"
${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS parallel_ddl"
done

View File

@ -10,7 +10,11 @@ ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS parallel_ddl"
function query()
{
for _ in {1..50}; do
local it=0
TIMELIMIT=30
while [ $SECONDS -lt "$TIMELIMIT" ] && [ $it -lt 50 ];
do
it=$((it+1))
${CLICKHOUSE_CLIENT} --query "CREATE TABLE IF NOT EXISTS parallel_ddl(a Int) ENGINE = Memory"
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS parallel_ddl"
done

View File

@ -19,15 +19,39 @@ trap cleanup EXIT
$CLICKHOUSE_CLIENT -q "create view view_00840 as select count(*),database,table from system.columns group by database,table"
for _ in {1..100}; do
$CLICKHOUSE_CLIENT -nm -q "
drop table if exists view_00840;
create view view_00840 as select count(*),database,table from system.columns group by database,table;
"
done &
for _ in {1..250}; do
$CLICKHOUSE_CLIENT -q "select * from view_00840 order by table" >/dev/null 2>&1 || true
done &
function thread_drop_create()
{
local TIMELIMIT=$((SECONDS+$1))
local it=0
while [ $SECONDS -lt "$TIMELIMIT" ] && [ $it -lt 100 ];
do
it=$((it+1))
$CLICKHOUSE_CLIENT -nm -q "
drop table if exists view_00840;
create view view_00840 as select count(*),database,table from system.columns group by database,table;
"
done
}
function thread_select()
{
local TIMELIMIT=$((SECONDS+$1))
local it=0
while [ $SECONDS -lt "$TIMELIMIT" ] && [ $it -lt 250 ];
do
it=$((it+1))
$CLICKHOUSE_CLIENT -q "select * from view_00840 order by table" >/dev/null 2>&1 || true
done
}
export -f thread_drop_create
export -f thread_select
TIMEOUT=60
thread_drop_create $TIMEOUT &
thread_select $TIMEOUT &
wait
trap '' EXIT

View File

@ -1,100 +0,0 @@
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0
55 0

View File

@ -6,23 +6,33 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
for _ in {1..100}; do $CLICKHOUSE_CLIENT --multiquery --query "
DROP TABLE IF EXISTS mt;
CREATE TABLE mt (x UInt8, k UInt8 DEFAULT 0) ENGINE = SummingMergeTree ORDER BY k;
it=0
TIMELIMIT=31
while [ $SECONDS -lt "$TIMELIMIT" ] && [ $it -lt 100 ];
do
it=$((it+1))
$CLICKHOUSE_CLIENT --multiquery --query "
DROP TABLE IF EXISTS mt;
CREATE TABLE mt (x UInt8, k UInt8 DEFAULT 0) ENGINE = SummingMergeTree ORDER BY k;
INSERT INTO mt (x) VALUES (1);
INSERT INTO mt (x) VALUES (2);
INSERT INTO mt (x) VALUES (3);
INSERT INTO mt (x) VALUES (4);
INSERT INTO mt (x) VALUES (5);
INSERT INTO mt (x) VALUES (6);
INSERT INTO mt (x) VALUES (7);
INSERT INTO mt (x) VALUES (8);
INSERT INTO mt (x) VALUES (9);
INSERT INTO mt (x) VALUES (10);
INSERT INTO mt (x) VALUES (1);
INSERT INTO mt (x) VALUES (2);
INSERT INTO mt (x) VALUES (3);
INSERT INTO mt (x) VALUES (4);
INSERT INTO mt (x) VALUES (5);
INSERT INTO mt (x) VALUES (6);
INSERT INTO mt (x) VALUES (7);
INSERT INTO mt (x) VALUES (8);
INSERT INTO mt (x) VALUES (9);
INSERT INTO mt (x) VALUES (10);
OPTIMIZE TABLE mt FINAL;
SELECT * FROM mt;
OPTIMIZE TABLE mt FINAL;
";
DROP TABLE mt;
"; done
RES=$($CLICKHOUSE_CLIENT --query "SELECT * FROM mt;")
if [ "$RES" != "55 0" ]; then
echo "FAIL. Got: $RES"
fi
$CLICKHOUSE_CLIENT --query "DROP TABLE mt;"
done

View File

@ -1,4 +1,5 @@
#!/usr/bin/env bash
# Tags: no-debug, no-asan, no-tsan, no-msan
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -1,7 +1,8 @@
#!/usr/bin/env bash
# Tags: long, no-parallel
# Tag: no-parallel - to heavy
# Tag: long - to heavy
# Tags: long, no-parallel, disabled
# Tag: no-parallel - too heavy
# Tag: long - too heavy
# Tag: disabled - Always takes 4+ minutes, in serial mode, which is too much to be always run in CI
# This is the regression test when remote peer send some logs for INSERT,
# it is easy to archive using materialized views, with small block size.
@ -49,10 +50,10 @@ insert_client_opts=(
timeout 250s $CLICKHOUSE_CLIENT "${client_opts[@]}" "${insert_client_opts[@]}" -q "insert into function remote('127.2', currentDatabase(), in_02232) select * from numbers(1e6)"
# Kill underlying query of remote() to make KILL faster
# This test is reproducing very interesting bahaviour.
# This test is reproducing very interesting behaviour.
# The block size is 1, so the secondary query creates InterpreterSelectQuery for each row due to pushing to the MV.
# It works extremely slow, and the initial query produces new blocks and writes them to the socket much faster
# then the secondary query can read and process them. Therefore, it fills network buffers in the kernel.
# than the secondary query can read and process them. Therefore, it fills network buffers in the kernel.
# Once a buffer in the kernel is full, send(...) blocks until the secondary query will finish processing data
# that it already has in ReadBufferFromPocoSocket and call recv.
# Or until the kernel will decide to resize the buffer (seems like it has non-trivial rules for that).

View File

@ -1,6 +1,6 @@
990000
990000
900
900
10
990000
900
1
1000000
1000

View File

@ -3,25 +3,25 @@ DROP TABLE IF EXISTS pr_2;
DROP TABLE IF EXISTS numbers_1e6;
CREATE TABLE pr_1 (`a` UInt32) ENGINE = MergeTree ORDER BY a PARTITION BY a % 10 AS
SELECT 10 * intDiv(number, 10) + 1 FROM numbers(1_000_000);
SELECT 10 * intDiv(number, 10) + 1 FROM numbers(1_000);
CREATE TABLE pr_2 (`a` UInt32) ENGINE = MergeTree ORDER BY a AS
SELECT * FROM numbers(1_000_000);
SELECT * FROM numbers(1_000);
WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 10000)
WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 100)
SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.a;
WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 10000)
WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 100)
SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.a
SETTINGS allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3;
-- Testing that it is disabled for allow_experimental_analyzer=0. With analyzer it will be supported (with correct result)
WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 10000)
WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 100)
SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.a
SETTINGS allow_experimental_analyzer = 0, allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; -- { serverError SUPPORT_IS_DISABLED }
-- Disabled for any value of allow_experimental_parallel_reading_from_replicas != 1, not just 2
WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 10000)
WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 100)
SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.a
SETTINGS allow_experimental_analyzer = 0, allow_experimental_parallel_reading_from_replicas = 512, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; -- { serverError SUPPORT_IS_DISABLED }
@ -33,7 +33,7 @@ SETTINGS allow_experimental_parallel_reading_from_replicas = 1, parallel_replica
SELECT *
FROM
(
WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 10000)
WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 100)
SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.a
)
SETTINGS allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3;
@ -45,31 +45,31 @@ FROM
SELECT c + 1
FROM
(
WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 10000)
WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 100)
SELECT count() as c FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.a
)
)
SETTINGS allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3;
CREATE TABLE numbers_1e6
CREATE TABLE numbers_1e3
(
`n` UInt64
)
ENGINE = MergeTree
ORDER BY n
AS SELECT * FROM numbers(1_000_000);
AS SELECT * FROM numbers(1_000);
-- Same but nested CTE's
WITH
cte1 AS
(
SELECT n
FROM numbers_1e6
FROM numbers_1e3
),
cte2 AS
(
SELECT n
FROM numbers_1e6
FROM numbers_1e3
WHERE n IN (cte1)
)
SELECT count()

Binary file not shown.