未验证 提交 eb54e2d7 编写于 作者: T tavplubix 提交者: GitHub

Merge pull request #19984 from ClickHouse/backport/20.12/19684

Backport #19684 to 20.12: Should fail ddl query as soon as possible if table is shutdown
......@@ -903,7 +903,18 @@ bool DDLWorker::tryExecuteQueryOnLeaderReplica(
while (stopwatch.elapsedSeconds() <= MAX_EXECUTION_TIMEOUT_SEC)
{
StorageReplicatedMergeTree::Status status;
replicated_storage->getStatus(status);
// Has to get with zk fields to get active replicas field
replicated_storage->getStatus(status, true);
// Should return as soon as possible if the table is dropped.
bool replica_dropped = replicated_storage->is_dropped;
bool all_replicas_likely_detached = status.active_replicas == 0 && !DatabaseCatalog::instance().isTableExist(replicated_storage->getStorageID(), context);
if (replica_dropped || all_replicas_likely_detached)
{
LOG_WARNING(log, ", task {} will not be executed.", task.entry_name);
task.execution_status = ExecutionStatus(ErrorCodes::UNFINISHED, "Cannot execute replicated DDL query, table is dropped or detached permanently");
return false;
}
/// Any replica which is leader tries to take lock
if (status.is_leader && lock->tryLock())
......
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
function thread_create_drop_table {
while true; do
REPLICA=$(($RANDOM % 10))
$CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS t1 (x UInt64, s Array(Nullable(String))) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01671/test_01671', 'r_$REPLICA') order by x" 2>/dev/null
sleep 0.0$RANDOM
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS t1"
done
}
function thread_alter_table {
while true; do
$CLICKHOUSE_CLIENT --query "ALTER TABLE $CLICKHOUSE_DATABASE.t1 on cluster test_shard_localhost ADD COLUMN newcol UInt32" >/dev/null 2>&1
sleep 0.0$RANDOM
done
}
export -f thread_create_drop_table
export -f thread_alter_table
timeout 20 bash -c "thread_create_drop_table" &
timeout 20 bash -c 'thread_alter_table' &
wait
sleep 1
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS t1";
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册