Merge pull request #35047 from Avogar/random-settings

Add next batch of random settings in functional tests
This commit is contained in:
Kruglov Pavel 2022-03-24 13:38:43 +01:00 committed by GitHub
commit 87ff97f910
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
20 changed files with 35 additions and 21 deletions

View File

@ -373,6 +373,11 @@ class SettingsRandomizer:
"priority": lambda: int(abs(random.gauss(0, 2))),
"output_format_parallel_formatting": lambda: random.randint(0, 1),
"input_format_parallel_parsing": lambda: random.randint(0, 1),
"min_chunk_bytes_for_parallel_parsing": lambda: max(1024, int(random.gauss(10 * 1024 * 1024, 5 * 1000 * 1000))),
"max_read_buffer_size": lambda: random.randint(1, 20) if random.random() < 0.1 else random.randint(500000, 1048576),
"prefer_localhost_replica": lambda: random.randint(0, 1),
"max_block_size": lambda: random.randint(8000, 100000),
"max_threads": lambda: random.randint(1, 64),
}
@staticmethod

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: no-fasttest
# Tags: no-fasttest, no-random-settings
# set -x

View File

@ -22,3 +22,4 @@ for m in gz br xz zst lz4 bz2
do
${CLICKHOUSE_CLIENT} --query "SELECT count() < 4000000, max(x) FROM file('${CLICKHOUSE_DATABASE}/${m}.tsv.${m}', RowBinary, 'x UInt8', 'none')"
done

View File

@ -1,5 +1,7 @@
-- Tags: no-parallel, no-fasttest
SET prefer_localhost_replica=1;
DROP TABLE IF EXISTS file;
DROP TABLE IF EXISTS url;
DROP TABLE IF EXISTS view;

View File

@ -3,6 +3,7 @@
-- set insert_distributed_sync = 1; -- see https://github.com/ClickHouse/ClickHouse/issues/18971
SET allow_experimental_parallel_reading_from_replicas = 0; -- see https://github.com/ClickHouse/ClickHouse/issues/34525
SET prefer_localhost_replica = 1;
DROP TABLE IF EXISTS local_01099_a;
DROP TABLE IF EXISTS local_01099_b;

View File

@ -1,4 +1,4 @@
-- Tags: distributed, no-replicated-database, no-parallel, no-fasttest
-- Tags: distributed, no-replicated-database, no-parallel, no-fasttest, no-random-settings
SET allow_experimental_live_view = 1;

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: no-replicated-database, no-parallel, no-fasttest, no-tsan, no-asan
# Tags: no-replicated-database, no-parallel, no-fasttest, no-tsan, no-asan, no-random-settings
# Tag no-fasttest: max_memory_usage_for_user can interfere another queries running concurrently
# Regression for MemoryTracker that had been incorrectly accounted

View File

@ -1,3 +1,5 @@
-- Tags: no-random-settings
DROP TABLE IF EXISTS buf_dest;
DROP TABLE IF EXISTS buf;

View File

@ -1,6 +1,7 @@
DROP TABLE IF EXISTS select_final;
SET do_not_merge_across_partitions_select_final = 1;
SET max_threads = 0;
CREATE TABLE select_final (t DateTime, x Int32, string String) ENGINE = ReplacingMergeTree() PARTITION BY toYYYYMM(t) ORDER BY (x, t);

View File

@ -1,4 +1,4 @@
-- Tags: no-parallel, no-fasttest
-- Tags: no-parallel, no-fasttest, no-random-settings
SET max_memory_usage = 1, max_untracked_memory = 1000000, max_threads=40;
select 'test', count(*) from zeros_mt(1000000) where not ignore(zero); -- { serverError 241 }

View File

@ -6,6 +6,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
${CLICKHOUSE_CLIENT} -q "create table insert_big_json(a String, b String) engine=MergeTree() order by tuple()";
python3 -c "[print('{{\"a\":\"{}\", \"b\":\"{}\"'.format('clickhouse'* 1000000, 'dbms' * 1000000)) for i in range(10)]; [print('{{\"a\":\"{}\", \"b\":\"{}\"}}'.format('clickhouse'* 100000, 'dbms' * 100000)) for i in range(10)]" 2>/dev/null | ${CLICKHOUSE_CLIENT} --input_format_parallel_parsing=1 --max_memory_usage=0 -q "insert into insert_big_json FORMAT JSONEachRow" 2>&1 | grep -q "min_chunk_bytes_for_parallel_parsing" && echo "Ok." || echo "FAIL" ||:
python3 -c "[print('{{\"a\":\"{}\", \"b\":\"{}\"'.format('clickhouse'* 1000000, 'dbms' * 1000000)) for i in range(10)]; [print('{{\"a\":\"{}\", \"b\":\"{}\"}}'.format('clickhouse'* 100000, 'dbms' * 100000)) for i in range(10)]" 2>/dev/null | ${CLICKHOUSE_CLIENT} --min_chunk_bytes_for_parallel_parsing=10485760 --max_threads=0 --input_format_parallel_parsing=1 --max_memory_usage=0 -q "insert into insert_big_json FORMAT JSONEachRow" 2>&1 | grep -q "min_chunk_bytes_for_parallel_parsing" && echo "Ok." || echo "FAIL" ||:
${CLICKHOUSE_CLIENT} -q "drop table insert_big_json"

View File

@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: br' "${CLICKHOUSE_URL}&enable_http_compression=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) FORMAT JSON" | brotli -d | tail -n30 | head -n23
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: br' "${CLICKHOUSE_URL}&enable_http_compression=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) FORMAT JSON SETTINGS max_block_size=65505" | brotli -d | tail -n30 | head -n23

View File

@ -1,4 +1,4 @@
-- Tags: long, distributed
-- Tags: long, distributed, no-random-settings
drop table if exists data_01730;

View File

@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip' "${CLICKHOUSE_URL}&enable_http_compression=1&http_zlib_compression_level=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(100000) FORMAT JSON" | gzip -d | tail -n30 | head -n23
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip' "${CLICKHOUSE_URL}&enable_http_compression=1&http_zlib_compression_level=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(100000) FORMAT JSON SETTINGS max_block_size=65505" | gzip -d | tail -n30 | head -n23

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: no-fasttest
# Tags: no-fasttest, no-random-settings
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -12,6 +12,7 @@ mkdir -p ${user_files_path}/
cp $CUR_DIR/data_zstd/test_01946.zstd ${user_files_path}/
${CLICKHOUSE_CLIENT} --multiline --multiquery --query "
set min_chunk_bytes_for_parallel_parsing=10485760;
set max_read_buffer_size = 65536;
set input_format_parallel_parsing = 0;
select * from file('test_01946.zstd', 'JSONEachRow', 'foo String') limit 30 format Null;

View File

@ -1,13 +1,14 @@
#!/usr/bin/env bash
# Tags: no-random-settings
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=1"
url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=1&max_insert_threads=0&group_by_two_level_threshold=100000&group_by_two_level_threshold_bytes=50000000&distributed_aggregation_memory_efficient=1&fsync_metadata=1&priority=1&output_format_parallel_formatting=0&input_format_parallel_parsing=0&min_chunk_bytes_for_parallel_parsing=4031398&max_read_buffer_size=554729&prefer_localhost_replica=0&max_block_size=51672&max_threads=20"
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS async_inserts"
${CLICKHOUSE_CLIENT} -q "CREATE TABLE async_inserts (id UInt32, s String) ENGINE = MergeTree ORDER BY id"
${CLICKHOUSE_CLIENT} --max_insert_threads=0 --group_by_two_level_threshold=100000 --group_by_two_level_threshold_bytes=50000000 --distributed_aggregation_memory_efficient=1 --fsync_metadata=1 --priority=1 --output_format_parallel_formatting=0 --input_format_parallel_parsing=0 --min_chunk_bytes_for_parallel_parsing=4031398 --max_read_buffer_size=554729 --prefer_localhost_replica=0 --max_block_size=51672 --max_threads=20 -q "DROP TABLE IF EXISTS async_inserts"
${CLICKHOUSE_CLIENT} --max_insert_threads=0 --group_by_two_level_threshold=100000 --group_by_two_level_threshold_bytes=50000000 --distributed_aggregation_memory_efficient=1 --fsync_metadata=1 --priority=1 --output_format_parallel_formatting=0 --input_format_parallel_parsing=0 --min_chunk_bytes_for_parallel_parsing=4031398 --max_read_buffer_size=554729 --prefer_localhost_replica=0 --max_block_size=51672 --max_threads=20 -q "CREATE TABLE async_inserts (id UInt32, s String) ENGINE = MergeTree ORDER BY id"
${CLICKHOUSE_CURL} -sS "$url" -d 'INSERT INTO async_inserts FORMAT CSV
1,"a"
@ -22,7 +23,7 @@ ${CLICKHOUSE_CURL} -sS "$url" -d 'INSERT INTO async_inserts FORMAT CSV
wait
${CLICKHOUSE_CLIENT} -q "SELECT * FROM async_inserts ORDER BY id"
${CLICKHOUSE_CLIENT} -q "SELECT name, rows, level FROM system.parts WHERE table = 'async_inserts' AND database = '$CLICKHOUSE_DATABASE' ORDER BY name"
${CLICKHOUSE_CLIENT} --max_insert_threads=0 --group_by_two_level_threshold=100000 --group_by_two_level_threshold_bytes=50000000 --distributed_aggregation_memory_efficient=1 --fsync_metadata=1 --priority=1 --output_format_parallel_formatting=0 --input_format_parallel_parsing=0 --min_chunk_bytes_for_parallel_parsing=4031398 --max_read_buffer_size=554729 --prefer_localhost_replica=0 --max_block_size=51672 --max_threads=20 -q "SELECT * FROM async_inserts ORDER BY id"
${CLICKHOUSE_CLIENT} --max_insert_threads=0 --group_by_two_level_threshold=100000 --group_by_two_level_threshold_bytes=50000000 --distributed_aggregation_memory_efficient=1 --fsync_metadata=1 --priority=1 --output_format_parallel_formatting=0 --input_format_parallel_parsing=0 --min_chunk_bytes_for_parallel_parsing=4031398 --max_read_buffer_size=554729 --prefer_localhost_replica=0 --max_block_size=51672 --max_threads=20 -q "SELECT name, rows, level FROM system.parts WHERE table = 'async_inserts' AND database = '$CLICKHOUSE_DATABASE' ORDER BY name"
${CLICKHOUSE_CLIENT} -q "DROP TABLE async_inserts"
${CLICKHOUSE_CLIENT} --max_insert_threads=0 --group_by_two_level_threshold=100000 --group_by_two_level_threshold_bytes=50000000 --distributed_aggregation_memory_efficient=1 --fsync_metadata=1 --priority=1 --output_format_parallel_formatting=0 --input_format_parallel_parsing=0 --min_chunk_bytes_for_parallel_parsing=4031398 --max_read_buffer_size=554729 --prefer_localhost_replica=0 --max_block_size=51672 --max_threads=20 -q "DROP TABLE async_inserts"

View File

@ -29,5 +29,5 @@ $CLICKHOUSE_CLIENT -q "create table test_dist engine=Distributed('test_shard_loc
$CLICKHOUSE_CLIENT -q "detach table test_dist"
$CLICKHOUSE_CLIENT -q "drop table test"
$CLICKHOUSE_CLIENT -q "attach table test_dist"
$CLICKHOUSE_CLIENT -q "select * from test_dist" 2>&1 | grep -q "UNKNOWN_TABLE" && echo "OK" || echo "FAIL"
$CLICKHOUSE_CLIENT --prefer_localhost_replica=1 -q "select * from test_dist" 2>&1 | grep -q "UNKNOWN_TABLE" && echo "OK" || echo "FAIL"

View File

@ -1,4 +1,4 @@
-- Tags: replica, distributed
-- Tags: replica, distributed, no-random-settings
SET max_parallel_replicas = 2;
SELECT EventTime::DateTime('Asia/Dubai') FROM remote('127.0.0.{1|2}', test, hits) ORDER BY EventTime DESC LIMIT 10

View File

@ -13,9 +13,9 @@ do
$CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime('Asia/Dubai'), b String) ENGINE=Memory()"
echo "$format, false";
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \
$CLICKHOUSE_CLIENT --max_block_size=65505 --output_format_parallel_formatting=false -q \
"SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Asia/Dubai') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 50000 Format $format" | \
$CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=false -q "INSERT INTO parsing_with_names FORMAT $format"
$CLICKHOUSE_CLIENT --max_block_size=65505 --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=false -q "INSERT INTO parsing_with_names FORMAT $format"
$CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names"
@ -23,9 +23,9 @@ do
$CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime('Asia/Dubai'), b String) ENGINE=Memory()"
echo "$format, true";
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \
$CLICKHOUSE_CLIENT --max_block_size=65505 --output_format_parallel_formatting=false -q \
"SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Asia/Dubai') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 50000 Format $format" | \
$CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=true -q "INSERT INTO parsing_with_names FORMAT $format"
$CLICKHOUSE_CLIENT --max_block_size=65505 --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=true -q "INSERT INTO parsing_with_names FORMAT $format"
$CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names"