From 8058166d5596f6da70fd0bc1a5288f3dcd4c7ea3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 22 May 2021 08:43:24 +0300 Subject: [PATCH] tests: fix flakiness with system.{parts,detached_parts,columns,parts_clumns,tables} --- .../00080_show_tables_and_system_tables.sql | 2 +- ...5_storage_file_and_clickhouse-local_app.sh | 3 +- ...3_max_block_size_system_tables_columns.sql | 2 + .../00696_system_columns_limit.sql | 2 + ...00753_system_columns_and_system_tables.sql | 50 ++++++++----------- ...00821_distributed_storage_with_join_on.sql | 2 + ...ong_concurrent_select_and_drop_deadlock.sh | 2 + ...ookeeper_test_alter_compression_codecs.sql | 16 +++--- .../0_stateless/00938_fix_rwlock_segfault.sh | 1 + .../00941_system_columns_race_condition.sh | 1 + .../00991_system_parts_race_condition.sh | 1 + ...tem_parts_race_condition_zookeeper_long.sh | 1 + ...tem_parts_race_condition_drop_zookeeper.sh | 3 +- .../0_stateless/01004_rename_deadlock.sh | 1 + .../0_stateless/01007_r1r2_w_r2r1_deadlock.sh | 6 ++- ...arallel_alter_add_drop_column_zookeeper.sh | 12 ++--- .../01098_temporary_and_external_tables.sh | 1 + .../0_stateless/01109_exchange_tables.sql | 1 + .../01213_alter_rename_column_zookeeper.sh | 4 +- .../01445_create_table_as_table_function.sh | 1 + .../0_stateless/01451_detach_drop_part.sql | 6 +-- .../01451_replicated_detach_drop_part.sql | 6 +-- ...18_cast_nullable_virtual_system_column.sql | 3 +- .../01533_optimize_skip_merged_partitions.sql | 2 +- .../0_stateless/01560_optimize_on_insert.sql | 2 +- .../0_stateless/01600_parts_states_metrics.sh | 1 + .../0_stateless/01600_parts_types_metrics.sh | 1 + ...676_long_clickhouse_client_autocomplete.sh | 2 + .../0_stateless/01821_table_comment.sql | 2 +- .../1_stateful/00076_system_columns_bytes.sql | 1 + 30 files changed, 79 insertions(+), 59 deletions(-) diff --git a/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql b/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql index b3a3ddd63c8..efb7ccf5f71 100644 --- a/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql +++ b/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql @@ -25,6 +25,6 @@ DROP DATABASE IF EXISTS test_DatabaseMemory; CREATE DATABASE test_DatabaseMemory ENGINE = Memory; CREATE TABLE test_DatabaseMemory.A (A UInt8) ENGINE = Null; -SELECT sum(ignore(*, metadata_modification_time, engine_full, create_table_query)) FROM system.tables; +SELECT sum(ignore(*, metadata_modification_time, engine_full, create_table_query)) FROM system.tables WHERE database = 'test_DatabaseMemory'; DROP DATABASE test_DatabaseMemory; diff --git a/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app.sh b/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app.sh index ef0ec1ae842..a09e9a4adb2 100755 --- a/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app.sh +++ b/tests/queries/0_stateless/00385_storage_file_and_clickhouse-local_app.sh @@ -57,7 +57,8 @@ ${CLICKHOUSE_LOCAL} -q "CREATE TABLE sophisticated_default a UInt8 DEFAULT 3, b UInt8 ALIAS a + 5, c UInt8 -) ENGINE = Memory; SELECT count() FROM system.tables WHERE name='sophisticated_default';" +) ENGINE = Memory; +SELECT count() FROM system.tables WHERE name='sophisticated_default' AND database = currentDatabase();" # Help is not skipped [[ $(${CLICKHOUSE_LOCAL} --help | wc -l) -gt 100 ]] diff --git a/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql b/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql index 9415952e0b3..1d6abf7b252 100644 --- a/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql +++ b/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql @@ -1,3 +1,5 @@ +-- NOTE: database = currentDatabase() is not mandatory + SELECT avg(blockSize()) <= 10 FROM system.tables SETTINGS max_block_size = 10; SELECT avg(blockSize()) <= 10 FROM system.tables LIMIT 10 SETTINGS max_block_size = 10; SELECT (SELECT count() FROM system.tables SETTINGS max_block_size = 10) = (SELECT count() FROM system.tables SETTINGS max_block_size = 9); diff --git a/tests/queries/0_stateless/00696_system_columns_limit.sql b/tests/queries/0_stateless/00696_system_columns_limit.sql index 007fac06053..c7bb53f4cb7 100644 --- a/tests/queries/0_stateless/00696_system_columns_limit.sql +++ b/tests/queries/0_stateless/00696_system_columns_limit.sql @@ -1,3 +1,5 @@ +-- NOTE: database = currentDatabase() is not mandatory + SELECT count() > 0 FROM (SELECT * FROM system.columns LIMIT 0); SELECT count() > 0 FROM (SELECT * FROM system.columns LIMIT 1); SELECT count() > 0 FROM (SELECT * FROM system.columns LIMIT 2); diff --git a/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql b/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql index 862e3693f0e..1037b5a6381 100644 --- a/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql +++ b/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql @@ -13,17 +13,15 @@ CREATE TABLE check_system_tables SETTINGS min_bytes_for_wide_part = 0; SELECT name, partition_key, sorting_key, primary_key, sampling_key, storage_policy, total_rows -FROM system.tables -WHERE name = 'check_system_tables' +FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase() FORMAT PrettyCompactNoEscapes; SELECT name, is_in_partition_key, is_in_sorting_key, is_in_primary_key, is_in_sampling_key -FROM system.columns -WHERE table = 'check_system_tables' +FROM system.columns WHERE table = 'check_system_tables' AND database = currentDatabase() FORMAT PrettyCompactNoEscapes; INSERT INTO check_system_tables VALUES (1, 1, 1); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); DROP TABLE IF EXISTS check_system_tables; @@ -39,13 +37,11 @@ CREATE TABLE check_system_tables ORDER BY date; SELECT name, partition_key, sorting_key, primary_key, sampling_key -FROM system.tables -WHERE name = 'check_system_tables' +FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase() FORMAT PrettyCompactNoEscapes; SELECT name, is_in_partition_key, is_in_sorting_key, is_in_primary_key, is_in_sampling_key -FROM system.columns -WHERE table = 'check_system_tables' +FROM system.columns WHERE table = 'check_system_tables' AND database = currentDatabase() FORMAT PrettyCompactNoEscapes; DROP TABLE IF EXISTS check_system_tables; @@ -59,29 +55,27 @@ CREATE TABLE check_system_tables ) ENGINE = MergeTree(Event, intHash32(UserId), (Counter, Event, intHash32(UserId)), 8192); SELECT name, partition_key, sorting_key, primary_key, sampling_key -FROM system.tables -WHERE name = 'check_system_tables' +FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase() FORMAT PrettyCompactNoEscapes; SELECT name, is_in_partition_key, is_in_sorting_key, is_in_primary_key, is_in_sampling_key -FROM system.columns -WHERE table = 'check_system_tables' +FROM system.columns WHERE table = 'check_system_tables' AND database = currentDatabase() FORMAT PrettyCompactNoEscapes; DROP TABLE IF EXISTS check_system_tables; SELECT 'Check total_bytes/total_rows for TinyLog'; CREATE TABLE check_system_tables (key UInt8) ENGINE = TinyLog(); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); INSERT INTO check_system_tables VALUES (1); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); DROP TABLE check_system_tables; SELECT 'Check total_bytes/total_rows for Memory'; CREATE TABLE check_system_tables (key UInt16) ENGINE = Memory(); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); INSERT INTO check_system_tables VALUES (1); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); DROP TABLE check_system_tables; SELECT 'Check total_bytes/total_rows for Buffer'; @@ -96,33 +90,33 @@ CREATE TABLE check_system_tables (key UInt16) ENGINE = Buffer( 100, 100, /* min_rows /max_rows */ 0, 1e6 /* min_bytes/max_bytes */ ); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); INSERT INTO check_system_tables SELECT * FROM numbers_mt(50); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); SELECT 'Check lifetime_bytes/lifetime_rows for Buffer'; -SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); OPTIMIZE TABLE check_system_tables; -- flush -SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); INSERT INTO check_system_tables SELECT * FROM numbers_mt(50); -SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); OPTIMIZE TABLE check_system_tables; -- flush -SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); INSERT INTO check_system_tables SELECT * FROM numbers_mt(101); -- direct block write (due to min_rows exceeded) -SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); DROP TABLE check_system_tables; DROP TABLE check_system_tables_null; SELECT 'Check total_bytes/total_rows for Set'; CREATE TABLE check_system_tables Engine=Set() AS SELECT * FROM numbers(50); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); INSERT INTO check_system_tables SELECT number+50 FROM numbers(50); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); DROP TABLE check_system_tables; SELECT 'Check total_bytes/total_rows for Join'; CREATE TABLE check_system_tables Engine=Join(ANY, LEFT, number) AS SELECT * FROM numbers(50); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); INSERT INTO check_system_tables SELECT number+50 FROM numbers(50); -SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase(); DROP TABLE check_system_tables; diff --git a/tests/queries/0_stateless/00821_distributed_storage_with_join_on.sql b/tests/queries/0_stateless/00821_distributed_storage_with_join_on.sql index 9e87c739928..f84c20feaea 100644 --- a/tests/queries/0_stateless/00821_distributed_storage_with_join_on.sql +++ b/tests/queries/0_stateless/00821_distributed_storage_with_join_on.sql @@ -1,3 +1,5 @@ +-- NOTE: database = currentDatabase() is not mandatory + DROP TABLE IF EXISTS table1; DROP TABLE IF EXISTS table2; diff --git a/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh b/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh index 0a68225a31a..448e7ad076c 100755 --- a/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh +++ b/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh @@ -1,5 +1,7 @@ #!/usr/bin/env bash +# NOTE: database = $CLICKHOUSE_DATABASE is unwanted + set -e CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) diff --git a/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.sql b/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.sql index 0108bb68743..e8fb073ba7a 100644 --- a/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.sql +++ b/tests/queries/0_stateless/00910_zookeeper_test_alter_compression_codecs.sql @@ -25,8 +25,8 @@ ALTER TABLE alter_compression_codec1 ADD COLUMN alter_column String DEFAULT 'def SYSTEM SYNC REPLICA alter_compression_codec1; SYSTEM SYNC REPLICA alter_compression_codec2; -SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column'; -SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column'; +SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column' AND database = currentDatabase(); +SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column' AND database = currentDatabase(); INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 3, '3'); INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 4, '4'); @@ -37,8 +37,8 @@ SELECT * FROM alter_compression_codec1 ORDER BY id; SELECT * FROM alter_compression_codec2 ORDER BY id; ALTER TABLE alter_compression_codec1 MODIFY COLUMN alter_column CODEC(NONE); -SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column'; -SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column'; +SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column' AND database = currentDatabase(); +SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column' AND database = currentDatabase(); INSERT INTO alter_compression_codec2 VALUES('2018-01-01', 5, '5'); INSERT INTO alter_compression_codec2 VALUES('2018-01-01', 6, '6'); @@ -50,8 +50,8 @@ SET allow_suspicious_codecs = 1; ALTER TABLE alter_compression_codec1 MODIFY COLUMN alter_column CODEC(ZSTD, LZ4HC, LZ4, LZ4, NONE); SYSTEM SYNC REPLICA alter_compression_codec1; SYSTEM SYNC REPLICA alter_compression_codec2; -SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column'; -SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column'; +SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column' AND database = currentDatabase(); +SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column' AND database = currentDatabase(); INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 7, '7'); INSERT INTO alter_compression_codec2 VALUES('2018-01-01', 8, '8'); @@ -62,8 +62,8 @@ SELECT * FROM alter_compression_codec2 ORDER BY id; ALTER TABLE alter_compression_codec1 MODIFY COLUMN alter_column FixedString(100); SYSTEM SYNC REPLICA alter_compression_codec2; -SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column'; -SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column'; +SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column' AND database = currentDatabase(); +SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column' AND database = currentDatabase(); DROP TABLE IF EXISTS alter_compression_codec1; DROP TABLE IF EXISTS alter_compression_codec2; diff --git a/tests/queries/0_stateless/00938_fix_rwlock_segfault.sh b/tests/queries/0_stateless/00938_fix_rwlock_segfault.sh index 5c4253e682b..01519f7a598 100755 --- a/tests/queries/0_stateless/00938_fix_rwlock_segfault.sh +++ b/tests/queries/0_stateless/00938_fix_rwlock_segfault.sh @@ -8,6 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e +# NOTE: database = $CLICKHOUSE_DATABASE is unwanted for _ in {1..100}; do \ $CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables UNION ALL SELECT name FROM system.columns format Null"; done diff --git a/tests/queries/0_stateless/00941_system_columns_race_condition.sh b/tests/queries/0_stateless/00941_system_columns_race_condition.sh index 0a3fc7f3b3f..1b34abc50d7 100755 --- a/tests/queries/0_stateless/00941_system_columns_race_condition.sh +++ b/tests/queries/0_stateless/00941_system_columns_race_condition.sh @@ -13,6 +13,7 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE alter_table (a UInt8, b Int16, c Float32, d function thread1() { + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted while true; do $CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns UNION ALL SELECT name FROM system.columns FORMAT Null"; done } diff --git a/tests/queries/0_stateless/00991_system_parts_race_condition.sh b/tests/queries/0_stateless/00991_system_parts_race_condition.sh index 55ff4d97149..f1b12a319c0 100755 --- a/tests/queries/0_stateless/00991_system_parts_race_condition.sh +++ b/tests/queries/0_stateless/00991_system_parts_race_condition.sh @@ -15,6 +15,7 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE alter_table (a UInt8, b Int16, c Float32, d function thread1() { + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted while true; do $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null"; done } diff --git a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh index fe6246e02f6..793fc8e9575 100755 --- a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh +++ b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh @@ -16,6 +16,7 @@ $CLICKHOUSE_CLIENT -n -q " function thread1() { + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted while true; do $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null"; done } diff --git a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh index d960d8ff91d..32fe31f68c6 100755 --- a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh +++ b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh @@ -8,7 +8,8 @@ set -e function thread1() { - while true; do + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted + while true; do $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null"; done } diff --git a/tests/queries/0_stateless/01004_rename_deadlock.sh b/tests/queries/0_stateless/01004_rename_deadlock.sh index aa9e6f8a5bc..2bde9256bba 100755 --- a/tests/queries/0_stateless/01004_rename_deadlock.sh +++ b/tests/queries/0_stateless/01004_rename_deadlock.sh @@ -28,6 +28,7 @@ function thread2() function thread3() { while true; do + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted $CLICKHOUSE_CLIENT --query "SELECT * FROM system.tables" --format Null done } diff --git a/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh b/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh index 8773a180822..8936d8da775 100755 --- a/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh +++ b/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh @@ -15,14 +15,16 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE b (x UInt8) ENGINE = MergeTree ORDER BY function thread1() { - while true; do + while true; do + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted seq 1 100 | awk '{ print "SELECT x FROM a WHERE x IN (SELECT toUInt8(count()) FROM system.tables);" }' | $CLICKHOUSE_CLIENT -n done } function thread2() { - while true; do + while true; do + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted seq 1 100 | awk '{ print "SELECT x FROM b WHERE x IN (SELECT toUInt8(count()) FROM system.tables);" }' | $CLICKHOUSE_CLIENT -n done } diff --git a/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh b/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh index 7a3e3cf155f..fd0b53cf122 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh @@ -82,14 +82,14 @@ wait echo "Finishing alters" -columns1=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_1'" 2> /dev/null) -columns2=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_2'" 2> /dev/null) -columns3=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_3'" 2> /dev/null) +columns1=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_1' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null) +columns2=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_2' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null) +columns3=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_3' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null) while [ "$columns1" != "$columns2" ] || [ "$columns2" != "$columns3" ]; do - columns1=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_1'" 2> /dev/null) - columns2=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_2'" 2> /dev/null) - columns3=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_3'" 2> /dev/null) + columns1=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_1' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null) + columns2=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_2' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null) + columns3=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_3' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null) sleep 1 done diff --git a/tests/queries/0_stateless/01098_temporary_and_external_tables.sh b/tests/queries/0_stateless/01098_temporary_and_external_tables.sh index bdac3c6fae3..17683852f2b 100755 --- a/tests/queries/0_stateless/01098_temporary_and_external_tables.sh +++ b/tests/queries/0_stateless/01098_temporary_and_external_tables.sh @@ -10,6 +10,7 @@ url="${url_without_session}session_id=test_01098" ${CLICKHOUSE_CURL} -m 30 -sSk "$url" --data "DROP TEMPORARY TABLE IF EXISTS tmp_table" ${CLICKHOUSE_CURL} -m 30 -sSk "$url" --data "CREATE TEMPORARY TABLE tmp_table AS SELECT number AS n FROM numbers(42)" +# NOTE: database = $CLICKHOUSE_DATABASE is unwanted id=$(echo "SELECT uuid FROM system.tables WHERE name='tmp_table' AND is_temporary" | ${CLICKHOUSE_CURL} -m 31 -sSgk "$url" -d @-) internal_table_name="_temporary_and_external_tables.\`_tmp_$id\`" diff --git a/tests/queries/0_stateless/01109_exchange_tables.sql b/tests/queries/0_stateless/01109_exchange_tables.sql index 7fbb36e8ce9..f2b8e4adb1d 100644 --- a/tests/queries/0_stateless/01109_exchange_tables.sql +++ b/tests/queries/0_stateless/01109_exchange_tables.sql @@ -4,6 +4,7 @@ CREATE DATABASE test_01109 ENGINE=Atomic; USE test_01109; CREATE TABLE t0 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBlocks(), * FROM (SELECT toLowCardinality(arrayJoin(['exchange', 'tables']))); +-- NOTE: database = currentDatabase() is not mandatory CREATE TABLE t1 ENGINE=Log() AS SELECT * FROM system.tables AS t JOIN system.databases AS d ON t.database=d.name; CREATE TABLE t2 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBlocks() + (SELECT count() FROM t0), * FROM (SELECT arrayJoin(['hello', 'world'])); diff --git a/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.sh b/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.sh index 5da8de70c46..b82b132be05 100755 --- a/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.sh +++ b/tests/queries/0_stateless/01213_alter_rename_column_zookeeper.sh @@ -32,11 +32,11 @@ $CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE table_for_rename_replicated;" $CLICKHOUSE_CLIENT --query "ALTER TABLE table_for_rename_replicated RENAME COLUMN value1 to renamed_value1" --replication_alter_partitions_sync=0 -while [[ -z $($CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns WHERE name = 'renamed_value1' and table = 'table_for_rename_replicated'" 2>/dev/null) ]]; do +while [[ -z $($CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns WHERE name = 'renamed_value1' and table = 'table_for_rename_replicated' AND database = '$CLICKHOUSE_DATABASE'" 2>/dev/null) ]]; do sleep 0.5 done -$CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns WHERE name = 'renamed_value1' and table = 'table_for_rename_replicated'" +$CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns WHERE name = 'renamed_value1' and table = 'table_for_rename_replicated' AND database = '$CLICKHOUSE_DATABASE'" # SHOW CREATE TABLE takes query from .sql file on disk. # previous select take metadata from memory. So, when previous select says, that return renamed_value1 already exists in table, it's still can have old version on disk. diff --git a/tests/queries/0_stateless/01445_create_table_as_table_function.sh b/tests/queries/0_stateless/01445_create_table_as_table_function.sh index f963c700779..6df5b7d70d1 100755 --- a/tests/queries/0_stateless/01445_create_table_as_table_function.sh +++ b/tests/queries/0_stateless/01445_create_table_as_table_function.sh @@ -6,6 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh +# NOTE: database = $CLICKHOUSE_DATABASE is unwanted ${CLICKHOUSE_CLIENT} --query "CREATE TABLE system.columns AS numbers(10);" 2>&1 | grep -F "Code: 57" > /dev/null && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} --query "CREATE TABLE system.columns engine=Memory AS numbers(10);" 2>&1 | grep -F "Code: 62" > /dev/null && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} --query "CREATE TABLE system.columns AS numbers(10) engine=Memory;" 2>&1 | grep -F "Code: 62" > /dev/null && echo 'OK' || echo 'FAIL' diff --git a/tests/queries/0_stateless/01451_detach_drop_part.sql b/tests/queries/0_stateless/01451_detach_drop_part.sql index d70f4e37de4..a285730e45f 100644 --- a/tests/queries/0_stateless/01451_detach_drop_part.sql +++ b/tests/queries/0_stateless/01451_detach_drop_part.sql @@ -15,13 +15,13 @@ ALTER TABLE mt_01451 DETACH PART 'all_2_2_0'; SELECT v FROM mt_01451 ORDER BY v; -SELECT name FROM system.detached_parts WHERE table = 'mt_01451'; +SELECT name FROM system.detached_parts WHERE table = 'mt_01451' AND database = currentDatabase(); ALTER TABLE mt_01451 ATTACH PART 'all_2_2_0'; SELECT v FROM mt_01451 ORDER BY v; -SELECT name FROM system.detached_parts WHERE table = 'mt_01451'; +SELECT name FROM system.detached_parts WHERE table = 'mt_01451' AND database = currentDatabase(); SELECT '-- drop part --'; @@ -37,6 +37,6 @@ OPTIMIZE TABLE mt_01451 FINAL; SELECT v FROM mt_01451 ORDER BY v; -SELECT name FROM system.parts WHERE table = 'mt_01451' AND active; +SELECT name FROM system.parts WHERE table = 'mt_01451' AND active AND database = currentDatabase(); DROP TABLE mt_01451; diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_part.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_part.sql index 3cd9fc7bc7e..7cefa9bd7ca 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_drop_part.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_part.sql @@ -19,14 +19,14 @@ ALTER TABLE replica2 DETACH PART 'all_1_1_0'; SELECT v FROM replica1 ORDER BY v; -SELECT name FROM system.detached_parts WHERE table = 'replica2'; +SELECT name FROM system.detached_parts WHERE table = 'replica2' AND database = currentDatabase(); ALTER TABLE replica2 ATTACH PART 'all_1_1_0'; SYSTEM SYNC REPLICA replica1; SELECT v FROM replica1 ORDER BY v; -SELECT name FROM system.detached_parts WHERE table = 'replica2'; +SELECT name FROM system.detached_parts WHERE table = 'replica2' AND database = currentDatabase(); SELECT '-- drop part --'; @@ -43,7 +43,7 @@ OPTIMIZE TABLE replica1 FINAL; SELECT v FROM replica1 ORDER BY v; -SELECT name FROM system.parts WHERE table = 'replica2' AND active; +SELECT name FROM system.parts WHERE table = 'replica2' AND active AND database = currentDatabase(); DROP TABLE replica1; DROP TABLE replica2; diff --git a/tests/queries/0_stateless/01518_cast_nullable_virtual_system_column.sql b/tests/queries/0_stateless/01518_cast_nullable_virtual_system_column.sql index 6366bae7b37..9f4ab03e941 100644 --- a/tests/queries/0_stateless/01518_cast_nullable_virtual_system_column.sql +++ b/tests/queries/0_stateless/01518_cast_nullable_virtual_system_column.sql @@ -1,7 +1,8 @@ +-- NOTE: database = currentDatabase() is not mandatory + SELECT database FROM system.tables WHERE database LIKE '%' format Null; SELECT database AS db FROM system.tables WHERE db LIKE '%' format Null; SELECT CAST(database, 'String') AS db FROM system.tables WHERE db LIKE '%' format Null; SELECT CAST('a string', 'Nullable(String)') AS str WHERE str LIKE '%' format Null; SELECT CAST(database, 'Nullable(String)') AS ndb FROM system.tables WHERE ndb LIKE '%' format Null; SELECT 'all tests passed'; - diff --git a/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql index 63bbbd2d721..3e9e5cda23e 100644 --- a/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql +++ b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql @@ -14,7 +14,7 @@ INSERT INTO optimize_final SELECT toDate('2000-01-01'), number + 5 FROM numbers( OPTIMIZE TABLE optimize_final FINAL; -SELECT table, partition, active, level from system.parts where table = 'optimize_final' and active = 1; +SELECT table, partition, active, level from system.parts where table = 'optimize_final' and database = currentDatabase() and active = 1; DROP TABLE optimize_final; diff --git a/tests/queries/0_stateless/01560_optimize_on_insert.sql b/tests/queries/0_stateless/01560_optimize_on_insert.sql index f64f4c75cfe..1c6418a50df 100644 --- a/tests/queries/0_stateless/01560_optimize_on_insert.sql +++ b/tests/queries/0_stateless/01560_optimize_on_insert.sql @@ -38,5 +38,5 @@ DROP TABLE IF EXISTS empty; CREATE TABLE empty (key UInt32, val UInt32, date Datetime) ENGINE=SummingMergeTree(val) PARTITION BY date ORDER BY key; INSERT INTO empty VALUES (1, 1, '2020-01-01'), (1, 1, '2020-01-01'), (1, -2, '2020-01-01'); SELECT * FROM empty ORDER BY key; -SELECT table, partition, active FROM system.parts where table = 'empty' and active = 1; +SELECT table, partition, active FROM system.parts where table = 'empty' and active = 1 and database = currentDatabase(); DROP TABLE empty; diff --git a/tests/queries/0_stateless/01600_parts_states_metrics.sh b/tests/queries/0_stateless/01600_parts_states_metrics.sh index e32210633bb..634cb403f85 100755 --- a/tests/queries/0_stateless/01600_parts_states_metrics.sh +++ b/tests/queries/0_stateless/01600_parts_states_metrics.sh @@ -4,6 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# NOTE: database = $CLICKHOUSE_DATABASE is unwanted verify_sql="SELECT (SELECT sumIf(value, metric = 'PartsCommitted'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics) = (SELECT sum(active), sum(NOT active) FROM system.parts)" diff --git a/tests/queries/0_stateless/01600_parts_types_metrics.sh b/tests/queries/0_stateless/01600_parts_types_metrics.sh index 1a1942be32a..67d2c0c306d 100755 --- a/tests/queries/0_stateless/01600_parts_types_metrics.sh +++ b/tests/queries/0_stateless/01600_parts_types_metrics.sh @@ -7,6 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e set -o pipefail +# NOTE: database = $CLICKHOUSE_DATABASE is unwanted verify_sql="SELECT (SELECT sumIf(value, metric = 'PartsInMemory'), sumIf(value, metric = 'PartsCompact'), sumIf(value, metric = 'PartsWide') FROM system.metrics) = (SELECT countIf(part_type == 'InMemory'), countIf(part_type == 'Compact'), countIf(part_type == 'Wide') FROM system.parts)" diff --git a/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh b/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh index 1ed5c6be272..3bb9da92362 100755 --- a/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh +++ b/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh @@ -4,6 +4,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# NOTE: database = $CLICKHOUSE_DATABASE is superfluous + function test_completion_word() { local w=$1 && shift diff --git a/tests/queries/0_stateless/01821_table_comment.sql b/tests/queries/0_stateless/01821_table_comment.sql index c09f121459c..6d9218d5a42 100644 --- a/tests/queries/0_stateless/01821_table_comment.sql +++ b/tests/queries/0_stateless/01821_table_comment.sql @@ -28,7 +28,7 @@ SELECT name, comment FROM system.tables -WHERE name IN ('t1', 't2', 't3') order by name; +WHERE name IN ('t1', 't2', 't3') AND database = currentDatabase() order by name; SHOW CREATE TABLE t1; diff --git a/tests/queries/1_stateful/00076_system_columns_bytes.sql b/tests/queries/1_stateful/00076_system_columns_bytes.sql index 434054b6980..dc0858e3de6 100644 --- a/tests/queries/1_stateful/00076_system_columns_bytes.sql +++ b/tests/queries/1_stateful/00076_system_columns_bytes.sql @@ -1 +1,2 @@ +-- NOTE: database = currentDatabase() is not mandatory SELECT sum(data_compressed_bytes) > 0, sum(data_uncompressed_bytes) > 0, sum(marks_bytes) > 0 FROM system.columns;