Merge pull request #27529 from ClickHouse/break_some_tests

Break some tests
This commit is contained in:
tavplubix 2021-08-26 12:40:42 +03:00 committed by GitHub
commit 774af49393
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
20 changed files with 171 additions and 25 deletions

View File

@ -634,6 +634,7 @@ def run_tests_array(all_tests_with_params):
open(stdout_file).read().split('\n')[:100])
status += '\n'
status += "\nstdout:\n{}\n".format(stdout)
status += 'Database: ' + testcase_args.testcase_database
elif stderr:
@ -643,6 +644,7 @@ def run_tests_array(all_tests_with_params):
status += print_test_time(total_time)
status += " - having stderror:\n{}\n".format(
'\n'.join(stderr.split('\n')[:100]))
status += "\nstdout:\n{}\n".format(stdout)
status += 'Database: ' + testcase_args.testcase_database
elif 'Exception' in stdout:
failures += 1

View File

@ -0,0 +1,5 @@
<yandex>
<merge_tree>
<number_of_free_entries_in_pool_to_execute_mutation>8</number_of_free_entries_in_pool_to_execute_mutation>
</merge_tree>
</yandex>

View File

@ -31,6 +31,7 @@ ln -sf $SRC_PATH/config.d/max_concurrent_queries.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/test_cluster_with_incorrect_pw.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/keeper_port.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/logging_no_rotate.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/merge_tree.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/tcp_with_proxy.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/top_level_domains_lists.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/top_level_domains_path.xml $DEST_SERVER_PATH/config.d/

View File

@ -0,0 +1,2 @@
Replication did not hang: synced all replicas of alter_table
Consistency: 1

View File

@ -3,15 +3,17 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# shellcheck source=./replication.lib
. "$CURDIR"/replication.lib
set -e
$CLICKHOUSE_CLIENT -n -q "
DROP TABLE IF EXISTS alter_table;
DROP TABLE IF EXISTS alter_table2;
DROP TABLE IF EXISTS alter_table0;
DROP TABLE IF EXISTS alter_table1;
CREATE TABLE alter_table (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r1') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0;
CREATE TABLE alter_table2 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r2') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0
CREATE TABLE alter_table0 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r1') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0;
CREATE TABLE alter_table1 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r2') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0
"
function thread1()
@ -22,22 +24,22 @@ function thread1()
function thread2()
{
while true; do $CLICKHOUSE_CLIENT -n --query "ALTER TABLE alter_table ADD COLUMN h String DEFAULT '0'; ALTER TABLE alter_table MODIFY COLUMN h UInt64; ALTER TABLE alter_table DROP COLUMN h;"; done
while true; do $CLICKHOUSE_CLIENT -n --query "ALTER TABLE alter_table0 ADD COLUMN h String DEFAULT '0'; ALTER TABLE alter_table0 MODIFY COLUMN h UInt64; ALTER TABLE alter_table0 DROP COLUMN h;"; done
}
function thread3()
{
while true; do $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table SELECT rand(1), rand(2), 1 / rand(3), toString(rand(4)), [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), (rand(8), rand(9)) FROM numbers(100000)"; done
while true; do $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table0 SELECT rand(1), rand(2), 1 / rand(3), toString(rand(4)), [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), (rand(8), rand(9)) FROM numbers(100000)"; done
}
function thread4()
{
while true; do $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE alter_table FINAL"; done
while true; do $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE alter_table0 FINAL"; done
}
function thread5()
{
while true; do $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table DELETE WHERE cityHash64(a,b,c,d,e,g) % 1048576 < 524288"; done
while true; do $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table0 DELETE WHERE cityHash64(a,b,c,d,e,g) % 1048576 < 524288"; done
}
# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout
@ -74,8 +76,9 @@ timeout $TIMEOUT bash -c thread4 2> /dev/null &
timeout $TIMEOUT bash -c thread5 2> /dev/null &
wait
check_replication_consistency "alter_table" "count(), sum(a), sum(b), round(sum(c))"
$CLICKHOUSE_CLIENT -n -q "DROP TABLE alter_table;" &
$CLICKHOUSE_CLIENT -n -q "DROP TABLE alter_table2;" &
$CLICKHOUSE_CLIENT -n -q "DROP TABLE alter_table0;" &
$CLICKHOUSE_CLIENT -n -q "DROP TABLE alter_table1;" &
wait

View File

@ -0,0 +1,2 @@
Replication did not hang: synced all replicas of alter_table_
Consistency: 1

View File

@ -3,6 +3,8 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# shellcheck source=./replication.lib
. "$CURDIR"/replication.lib
set -e
@ -99,6 +101,8 @@ timeout $TIMEOUT bash -c thread6 2>&1 | grep "was not completely removed from Zo
wait
check_replication_consistency "alter_table_" "count(), sum(a), sum(b), round(sum(c))"
for i in {0..9}; do
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS alter_table_$i" 2>&1 | grep "was not completely removed from ZooKeeper" &
done

View File

@ -1,6 +1,8 @@
Starting alters
Finishing alters
Equal number of columns
Replication did not hang: synced all replicas of concurrent_alter_add_drop_
Consistency: 1
0
0
0

View File

@ -3,6 +3,8 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# shellcheck source=./replication.lib
. "$CURDIR"/replication.lib
REPLICAS=3
@ -101,6 +103,8 @@ while [[ $(timeout 120 ${CLICKHOUSE_CLIENT} --query "ALTER TABLE concurrent_alte
sleep 1
done
check_replication_consistency "concurrent_alter_add_drop_" "count(), sum(key), sum(cityHash64(value0))"
for i in $(seq $REPLICAS); do
$CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_alter_add_drop_$i"
$CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.mutations WHERE is_done = 0 and table = 'concurrent_alter_add_drop_$i'"

View File

@ -5,6 +5,8 @@
1725
Starting alters
Finishing alters
Replication did not hang: synced all replicas of concurrent_alter_mt_
Consistency: 1
1
0
1

View File

@ -3,6 +3,8 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# shellcheck source=./replication.lib
. "$CURDIR"/replication.lib
REPLICAS=5
@ -112,6 +114,8 @@ while [[ $(timeout 120 ${CLICKHOUSE_CLIENT} --query "ALTER TABLE concurrent_alte
sleep 1
done
check_replication_consistency "concurrent_alter_mt_" "count(), sum(key), sum(cityHash64(value1)), sum(cityHash64(value2))"
for i in $(seq $REPLICAS); do
$CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_alter_mt_$i"
$CLICKHOUSE_CLIENT --query "SELECT SUM(toUInt64(value1)) > $INITIAL_SUM FROM concurrent_alter_mt_$i"

View File

@ -1 +1,3 @@
Replication did not hang
Replication did not hang: synced all replicas of dst_
Consistency: 1
Replication did not hang: synced all replicas of src_

View File

@ -3,6 +3,8 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# shellcheck source=./replication.lib
. "$CURDIR"/replication.lib
declare -A engines
engines[0]="MergeTree"
@ -116,13 +118,8 @@ timeout $TIMEOUT bash -c optimize_thread &
timeout $TIMEOUT bash -c drop_part_thread &
wait
for ((i=0; i<16; i++)) do
# The size of log is big, so increase timeout.
$CLICKHOUSE_CLIENT --receive_timeout 600 -q "SYSTEM SYNC REPLICA dst_$i" &
$CLICKHOUSE_CLIENT --receive_timeout 600 -q "SYSTEM SYNC REPLICA src_$i" 2>/dev/null &
done
wait
echo "Replication did not hang"
check_replication_consistency "dst_" "count(), sum(p), sum(k), sum(v)"
try_sync_replicas "src_"
for ((i=0; i<16; i++)) do
$CLICKHOUSE_CLIENT -q "DROP TABLE dst_$i" 2>&1| grep -Fv "is already started to be removing" &

View File

@ -14,3 +14,5 @@ CREATE TABLE default.concurrent_kill_4\n(\n `key` UInt64,\n `value` Int64\
Metadata version on replica 5 equal with first replica, OK
CREATE TABLE default.concurrent_kill_5\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_long_default/{shard}\', \'{replica}5\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192
499999500000
Replication did not hang: synced all replicas of concurrent_kill_
Consistency: 1

View File

@ -3,6 +3,8 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# shellcheck source=./replication.lib
. "$CURDIR"/replication.lib
REPLICAS=5
@ -59,10 +61,6 @@ timeout $TIMEOUT bash -c kill_mutation_thread 2> /dev/null &
wait
for i in $(seq $REPLICAS); do
$CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_kill_$i"
done
# with timeout alter query can be not finished yet, so to execute new alter
# we use retries
counter=0
@ -80,7 +78,7 @@ while true; do
done
metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1/replicas/r1$i/' and name = 'metadata_version'")
metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1/replicas/r11/' and name = 'metadata_version'")
for i in $(seq $REPLICAS); do
replica_metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1/replicas/r1$i/' and name = 'metadata_version'")
@ -95,6 +93,8 @@ done
$CLICKHOUSE_CLIENT --query "SELECT sum(value) FROM concurrent_kill_1"
check_replication_consistency "concurrent_kill_" "count(), sum(key), sum(cityHash64(value))"
for i in $(seq $REPLICAS); do
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_kill_$i"
done

View File

@ -1 +1,3 @@
Replication did not hang: synced all replicas of ttl_table
Consistency: 1
1

View File

@ -3,6 +3,8 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# shellcheck source=./replication.lib
. "$CURDIR"/replication.lib
NUM_REPLICAS=5
@ -58,14 +60,16 @@ timeout $TIMEOUT bash -c optimize_thread 2> /dev/null &
timeout $TIMEOUT bash -c optimize_thread 2> /dev/null &
wait
for i in $(seq 1 $NUM_REPLICAS); do
$CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA ttl_table$i"
# disable ttl merges before checking consistency
$CLICKHOUSE_CLIENT --query "ALTER TABLE ttl_table$i MODIFY SETTING max_replicated_merges_with_ttl_in_queue=0"
done
check_replication_consistency "ttl_table" "count(), sum(toUInt64(key))"
$CLICKHOUSE_CLIENT --query "SELECT * FROM system.replication_queue where table like 'ttl_table%' and database = '${CLICKHOUSE_DATABASE}' and type='MERGE_PARTS' and last_exception != '' FORMAT Vertical"
$CLICKHOUSE_CLIENT --query "SELECT COUNT() > 0 FROM system.part_log where table like 'ttl_table%' and database = '${CLICKHOUSE_DATABASE}'"
for i in $(seq 1 $NUM_REPLICAS); do
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS ttl_table$i" &
done

View File

@ -20,3 +20,23 @@ function wait_for_mutation()
done
}
function wait_for_all_mutations()
{
local table=$1
local database=$2
database=${database:="${CLICKHOUSE_DATABASE}"}
for i in {1..200}
do
sleep 1
if [[ $(${CLICKHOUSE_CLIENT} --query="SELECT coalesce(minOrNull(is_done), 1) FROM system.mutations WHERE database='$database' AND table like '$table'") -eq 1 ]]; then
break
fi
if [[ $i -eq 200 ]]; then
echo "Timed out while waiting for mutation to execute!"
fi
done
}

View File

@ -0,0 +1,87 @@
#!/usr/bin/env bash
# shellcheck source=./mergetree_mutations.lib
. "$CURDIR"/mergetree_mutations.lib
function try_sync_replicas()
{
table_name_prefix=$1
readarray -t empty_partitions_arr < <(${CLICKHOUSE_CLIENT} -q \
"SELECT DISTINCT substr(new_part_name, 1, position(new_part_name, '_') - 1) AS partition_id
FROM system.replication_queue
WHERE (database = currentDatabase()) AND (table LIKE '$table_name_prefix%') AND (last_exception LIKE '%No active replica has part%') AND (partition_id NOT IN (
SELECT partition_id
FROM system.parts
WHERE (database = currentDatabase()) AND (table LIKE '$table_name_prefix%')
))")
readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$table_name_prefix%' AND engine like '%Replicated%'")
for t in "${tables_arr[@]}"
do
for p in "${empty_partitions_arr[@]}"
do
# Avoid "Empty part ... is not created instead of lost part because there are no parts in partition"
$CLICKHOUSE_CLIENT -q "ALTER TABLE $t DROP PARTITION ID '$p'" 2>/dev/null
done
done
for t in "${tables_arr[@]}"
do
# The size of log may be big, so increase timeout.
$CLICKHOUSE_CLIENT --receive_timeout 400 -q "SYSTEM SYNC REPLICA $t" || $CLICKHOUSE_CLIENT -q \
"select 'sync failed, queue:', * from system.replication_queue where database=currentDatabase() and table='$t' order by database, table, node_name" &
done
wait
echo "Replication did not hang: synced all replicas of $table_name_prefix"
}
function check_replication_consistency()
{
table_name_prefix=$1
check_query_part=$2
# Do not check anything if all replicas are readonly,
# because is this case all replicas are probably lost (it may happen and it's not a bug)
res=$($CLICKHOUSE_CLIENT -q "SELECT count() - sum(is_readonly) FROM system.replicas WHERE database=currentDatabase() AND table LIKE '$table_name_prefix%'")
if [ $res -eq 0 ]; then
# Print dummy lines
echo "Replication did not hang: synced all replicas of $table_name_prefix"
echo "Consistency: 1"
return 0
fi
# Trigger pullLogsToQueue(...) and updateMutations(...) on some replica to make it pull all mutations, so it will be possible to kill them
some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$table_name_prefix%' ORDER BY rand() LIMIT 1")
$CLICKHOUSE_CLIENT --receive_timeout 3 -q "SYSTEM SYNC REPLICA $some_table" 1>/dev/null 2>/dev/null ||:
some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$table_name_prefix%' ORDER BY rand() LIMIT 1")
$CLICKHOUSE_CLIENT --receive_timeout 3 -q "SYSTEM SYNC REPLICA $some_table" 1>/dev/null 2>/dev/null ||:
# Forcefully cancel mutations to avoid waiting for them to finish
${CLICKHOUSE_CLIENT} -q "KILL MUTATION WHERE database=currentDatabase() AND table like '$table_name_prefix%'" > /dev/null
# SYNC REPLICA is not enough if some MUTATE_PARTs are not assigned yet
wait_for_all_mutations "$table_name_prefix%"
try_sync_replicas "$table_name_prefix"
res=$($CLICKHOUSE_CLIENT -q \
"SELECT
if((countDistinct(data) as c) == 0, 1, c)
FROM
(
SELECT _table, ($check_query_part) AS data
FROM merge(currentDatabase(), '$table_name_prefix') GROUP BY _table
)")
echo "Consistency: $res"
if [ $res -ne 1 ]; then
echo "Replicas have diverged:"
$CLICKHOUSE_CLIENT -q "select 'data', _table, $check_query_part, arraySort(groupArrayDistinct(_part)) from merge(currentDatabase(), '$table_name_prefix') group by _table order by _table"
$CLICKHOUSE_CLIENT -q "select 'queue', * from system.replication_queue where database=currentDatabase() and table like '$table_name_prefix%' order by database, table, node_name"
$CLICKHOUSE_CLIENT -q "select 'mutations', * from system.mutations where database=currentDatabase() and table like '$table_name_prefix%' order by database, table, mutation_id"
$CLICKHOUSE_CLIENT -q "select 'parts', * from system.parts where database=currentDatabase() and table like '$table_name_prefix%' order by database, table, name"
echo "Good luck with debugging..."
fi
}

View File

@ -322,6 +322,7 @@
"01076_parallel_alter_replicated_zookeeper",
"01079_parallel_alter_add_drop_column_zookeeper",
"01079_parallel_alter_detach_table_zookeeper",
"01079_parallel_alter_modify_zookeeper_long",
"01080_check_for_error_incorrect_size_of_nested_column",
"01083_expressions_in_engine_arguments",
"01084_regexp_empty",