未验证 提交 00836a7c 编写于 作者: A alesapin 提交者: GitHub

Merge pull request #18518 from ClickHouse/fix_bug_with_alter_hung

Control alter sequential execution more correctly
......@@ -23,11 +23,13 @@ void ReplicatedMergeTreeAltersSequence::addMutationForAlter(int alter_version, s
}
void ReplicatedMergeTreeAltersSequence::addMetadataAlter(
int alter_version, bool have_mutation, std::lock_guard<std::mutex> & /*state_lock*/)
int alter_version, std::lock_guard<std::mutex> & /*state_lock*/)
{
/// Data alter (mutation) always added before. See ReplicatedMergeTreeQueue::pullLogsToQueue.
/// So mutation alredy added to this sequence or doesn't exist.
if (!queue_state.count(alter_version))
queue_state.emplace(alter_version, AlterState{.metadata_finished=false, .data_finished=!have_mutation});
else /// Data alter can be added before.
queue_state.emplace(alter_version, AlterState{.metadata_finished=false, .data_finished=true});
else
queue_state[alter_version].metadata_finished = false;
}
......
......@@ -38,9 +38,8 @@ public:
/// Add mutation for alter (alter data stage).
void addMutationForAlter(int alter_version, std::lock_guard<std::mutex> & /*state_lock*/);
/// Add metadata for alter (alter metadata stage). If have_mutation=true, than we expect, that
/// corresponding mutation will be added.
void addMetadataAlter(int alter_version, bool have_mutation, std::lock_guard<std::mutex> & /*state_lock*/);
/// Add metadata for alter (alter metadata stage).
void addMetadataAlter(int alter_version, std::lock_guard<std::mutex> & /*state_lock*/);
/// Finish metadata alter. If corresponding data alter finished, than we can remove
/// alter from sequence.
......
......@@ -158,7 +158,7 @@ void ReplicatedMergeTreeQueue::insertUnlocked(
if (entry->type == LogEntry::ALTER_METADATA)
{
LOG_TRACE(log, "Adding alter metadata version {} to the queue", entry->alter_version);
alter_sequence.addMetadataAlter(entry->alter_version, entry->have_mutation, state_lock);
alter_sequence.addMetadataAlter(entry->alter_version, state_lock);
}
}
......
......@@ -24,6 +24,7 @@ function alter_thread
function kill_mutation_thread
{
while true; do
# find any mutation and kill it
mutation_id=$($CLICKHOUSE_CLIENT --query "SELECT mutation_id FROM system.mutations WHERE is_done=0 and database='${CLICKHOUSE_DATABASE}' and table='concurrent_mutate_kill' LIMIT 1")
if [ ! -z "$mutation_id" ]; then
$CLICKHOUSE_CLIENT --query "KILL MUTATION WHERE mutation_id='$mutation_id'" 1> /dev/null
......@@ -44,7 +45,23 @@ timeout $TIMEOUT bash -c kill_mutation_thread 2> /dev/null &
wait
$CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_mutate_kill"
$CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_mutate_kill MODIFY COLUMN value Int64 SETTINGS replication_alter_partitions_sync=2"
# with timeout alter query can be not finished yet, so to execute new alter
# we use retries
counter=0
while true; do
if $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_mutate_kill MODIFY COLUMN value Int64 SETTINGS replication_alter_partitions_sync=2" 2> /dev/null ; then
break
fi
if [ "$counter" -gt 120 ]
then
break
fi
sleep 0.5
counter=$(($counter + 1))
done
$CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE concurrent_mutate_kill"
$CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE concurrent_mutate_kill FINAL"
$CLICKHOUSE_CLIENT --query "SELECT sum(value) FROM concurrent_mutate_kill"
......
......@@ -36,6 +36,7 @@ function alter_thread
function kill_mutation_thread
{
while true; do
# find any mutation and kill it
mutation_id=$($CLICKHOUSE_CLIENT --query "SELECT mutation_id FROM system.mutations WHERE is_done = 0 and table like 'concurrent_kill_%' and database='${CLICKHOUSE_DATABASE}' LIMIT 1")
if [ ! -z "$mutation_id" ]; then
$CLICKHOUSE_CLIENT --query "KILL MUTATION WHERE mutation_id='$mutation_id'" 1> /dev/null
......@@ -58,7 +59,22 @@ for i in $(seq $REPLICAS); do
$CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_kill_$i"
done
$CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_kill_$i MODIFY COLUMN value Int64 SETTINGS replication_alter_partitions_sync=2"
# with timeout alter query can be not finished yet, so to execute new alter
# we use retries
counter=0
while true; do
if $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_kill_1 MODIFY COLUMN value Int64 SETTINGS replication_alter_partitions_sync=2" 2> /dev/null ; then
break
fi
if [ "$counter" -gt 120 ]
then
break
fi
sleep 0.5
counter=$(($counter + 1))
done
metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/test_01593_concurrent_kill/replicas/$i/' and name = 'metadata_version'")
for i in $(seq $REPLICAS); do
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册