ClickHouse/tests/queries/0_stateless/replication.lib

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

141 lines
7.5 KiB
Plaintext
Raw Normal View History

#!/usr/bin/env bash
# shellcheck source=./mergetree_mutations.lib
. "$CURDIR"/mergetree_mutations.lib
2021-08-11 15:24:47 +00:00
function try_sync_replicas()
{
2021-08-24 15:56:32 +00:00
table_name_prefix=$1
time_left=$2
2021-08-24 15:56:32 +00:00
2021-08-16 09:18:52 +00:00
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
2021-08-24 15:56:32 +00:00
WHERE (database = currentDatabase()) AND (table LIKE '$table_name_prefix%') AND (last_exception LIKE '%No active replica has part%') AND (partition_id NOT IN (
2021-08-16 09:18:52 +00:00
SELECT partition_id
FROM system.parts
2021-08-24 15:56:32 +00:00
WHERE (database = currentDatabase()) AND (table LIKE '$table_name_prefix%')
2021-08-16 09:18:52 +00:00
))")
2021-08-24 15:56:32 +00:00
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%'")
2021-08-16 09:18:52 +00:00
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
# Do not start new merges (it can make SYNC a bit faster)
$CLICKHOUSE_CLIENT -q "ALTER TABLE $t MODIFY SETTING max_replicated_merges_in_queue=0"
2024-03-14 11:13:33 +00:00
done
2024-03-14 11:13:33 +00:00
i=0
for t in "${tables_arr[@]}"
do
$CLICKHOUSE_CLIENT --receive_timeout $time_left -q "SYSTEM SYNC REPLICA $t STRICT" || ($CLICKHOUSE_CLIENT -q \
"select 'sync failed, queue:', * from system.replication_queue where database=currentDatabase() and table='$t' order by database, table, node_name" && exit 1) &
pids[${i}]=$!
i=$((i + 1))
done
2023-08-16 22:49:00 +00:00
for pid in "${pids[@]}"; do
wait $pid || (echo "Failed to sync some replicas" && exit 1)
done
2021-08-24 15:56:32 +00:00
echo "Replication did not hang: synced all replicas of $table_name_prefix"
}
function check_replication_consistency()
{
2021-08-24 15:56:32 +00:00
table_name_prefix=$1
check_query_part=$2
# Try to kill some mutations because sometimes tests run too much (it's not guarenteed to kill all mutations, see below)
# Try multiple replicas, because queries are not finished yet, and "global" KILL MUTATION may fail due to another query (like DROP TABLE)
readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$table_name_prefix%'")
for t in "${tables_arr[@]}"
do
2024-07-24 11:43:13 +00:00
${CLICKHOUSE_CLIENT} -q "KILL MUTATION WHERE database=currentDatabase() AND table='$t'" > /dev/null 2>/dev/null ||:
done
2021-10-09 15:20:13 +00:00
# Wait for all queries to finish (query may still be running if thread is killed by timeout)
num_tries=0
while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE current_database=currentDatabase() AND query LIKE '%$table_name_prefix%'") -ne 1 ]]; do
sleep 1;
2022-06-14 14:11:16 +00:00
num_tries=$((num_tries+1))
if [ $num_tries -eq 250 ]; then
echo "Queries for $table_name_prefix did not finish automatically after 250+ seconds"
echo "==================== QUERIES ===================="
2022-06-21 16:47:55 +00:00
$CLICKHOUSE_CLIENT -q "SELECT * FROM system.processes WHERE current_database=currentDatabase() AND query LIKE '%$table_name_prefix%' FORMAT Vertical"
echo "==================== STACK TRACES ===================="
$CLICKHOUSE_CLIENT -q "SELECT query_id, thread_name, thread_id, arrayStringConcat(arrayMap(x -> demangle(addressToSymbol(x)), trace), '\n') FROM system.stack_trace where query_id IN (SELECT query_id FROM system.processes WHERE current_database=currentDatabase() AND query LIKE '%$table_name_prefix%') SETTINGS allow_introspection_functions=1 FORMAT Vertical"
echo "==================== MUTATIONS ===================="
$CLICKHOUSE_CLIENT -q "SELECT * FROM system.mutations WHERE database=currentDatabase() FORMAT Vertical"
2021-10-09 15:20:13 +00:00
break
fi
done
2021-08-23 19:28:30 +00:00
# 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)
2021-08-24 15:56:32 +00:00
res=$($CLICKHOUSE_CLIENT -q "SELECT count() - sum(is_readonly) FROM system.replicas WHERE database=currentDatabase() AND table LIKE '$table_name_prefix%'")
2021-08-23 19:28:30 +00:00
if [ $res -eq 0 ]; then
# Print dummy lines
2021-08-24 15:56:32 +00:00
echo "Replication did not hang: synced all replicas of $table_name_prefix"
2021-08-23 19:28:30 +00:00
echo "Consistency: 1"
return 0
fi
# Touch all data to check that it's readable (and trigger PartCheckThread if needed)
# it's important to disable prefer warmed unmerged parts because
# otherwise it can read non-syncrhonized state of replicas
2024-07-29 19:16:19 +00:00
while ! $CLICKHOUSE_CLIENT --prefer_warmed_unmerged_parts_seconds=0 --max_result_rows 0 --max_result_bytes 0 --max_rows_to_read 0 --max_bytes_to_read 0 -q "SELECT * FROM merge(currentDatabase(), '$table_name_prefix') FORMAT Null" 2>/dev/null; do
sleep 1;
num_tries=$((num_tries+1))
if [ $num_tries -eq 250 ]; then
break
fi
done
time_left=$((300 - num_tries))
2021-08-16 12:36:12 +00:00
# Trigger pullLogsToQueue(...) and updateMutations(...) on some replica to make it pull all mutations, so it will be possible to kill them
2021-08-24 15:56:32 +00:00
some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$table_name_prefix%' ORDER BY rand() LIMIT 1")
2023-03-27 23:39:36 +00:00
$CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA $some_table PULL" 1>/dev/null 2>/dev/null ||:
2021-08-24 15:56:32 +00:00
some_table=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database=currentDatabase() AND name like '$table_name_prefix%' ORDER BY rand() LIMIT 1")
2023-03-27 23:39:36 +00:00
$CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA $some_table PULL" 1>/dev/null 2>/dev/null ||:
2021-08-16 12:36:12 +00:00
# Forcefully cancel mutations to avoid waiting for them to finish. Kills the remaining mutations
2021-08-24 15:56:32 +00:00
${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
2021-08-24 15:56:32 +00:00
wait_for_all_mutations "$table_name_prefix%"
try_sync_replicas "$table_name_prefix" "$time_left" || exit 1
2021-08-11 15:24:47 +00:00
# it's important to disable prefer warmed unmerged parts because
# otherwise it can read non-syncrhonized state of replicas
2024-07-26 12:17:17 +00:00
# also, disable the limit that is set for tests globally
res=$($CLICKHOUSE_CLIENT --prefer_warmed_unmerged_parts_seconds=0 --max_rows_to_read=0 -q \
"SELECT
2021-08-12 15:58:49 +00:00
if((countDistinct(data) as c) == 0, 1, c)
FROM
(
2021-08-24 15:56:32 +00:00
SELECT _table, ($check_query_part) AS data
FROM merge(currentDatabase(), '$table_name_prefix') GROUP BY _table
2021-08-12 10:39:06 +00:00
)")
echo "Consistency: $res"
if [ $res -ne 1 ]; then
echo "Replicas have diverged:"
2021-08-24 15:56:32 +00:00
$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"
2021-08-12 10:39:06 +00:00
echo "Good luck with debugging..."
exit 1
2021-08-11 15:24:47 +00:00
fi
}
2023-08-16 22:49:00 +00:00
# vi: ft=bash