Add retry to check alter_partition_version

This commit is contained in:
Amos Bird 2021-11-03 15:46:33 +08:00
parent b25b778349
commit 710f7d64e4
No known key found for this signature in database
GPG Key ID: 80D430DCBECFEDB4

View File

@ -6257,17 +6257,20 @@ void StorageReplicatedMergeTree::replacePartitionFrom(
/// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet. /// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet.
DataPartsVector src_all_parts = src_data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); DataPartsVector src_all_parts = src_data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id);
DataPartsVector src_parts;
MutableDataPartsVector dst_parts;
Strings block_id_paths;
Strings part_checksums;
auto zookeeper = getZooKeeper();
std::vector<EphemeralLockInZooKeeper> ephemeral_locks;
LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size()); LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size());
static const String TMP_PREFIX = "tmp_replace_from_"; static const String TMP_PREFIX = "tmp_replace_from_";
auto zookeeper = getZooKeeper();
/// Retry if alter_partition_version changes
for (size_t retry = 0; retry < 1000; ++retry)
{
DataPartsVector src_parts;
MutableDataPartsVector dst_parts;
Strings block_id_paths;
Strings part_checksums;
std::vector<EphemeralLockInZooKeeper> ephemeral_locks;
String alter_partition_version_path = zookeeper_path + "/alter_partition_version"; String alter_partition_version_path = zookeeper_path + "/alter_partition_version";
Coordination::Stat alter_partition_version_stat; Coordination::Stat alter_partition_version_stat;
zookeeper->get(alter_partition_version_path, &alter_partition_version_stat); zookeeper->get(alter_partition_version_path, &alter_partition_version_stat);
@ -6400,7 +6403,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(
if (code == Coordination::Error::ZOK) if (code == Coordination::Error::ZOK)
delimiting_block_lock->assumeUnlocked(); delimiting_block_lock->assumeUnlocked();
else if (code == Coordination::Error::ZBADVERSION) else if (code == Coordination::Error::ZBADVERSION)
throw Exception(ErrorCodes::CANNOT_ASSIGN_ALTER, "Cannot assign ALTER PARTITION, because another ALTER PARTITION query was concurrently executed"); continue;
else else
zkutil::KeeperMultiException::check(code, ops, op_results); zkutil::KeeperMultiException::check(code, ops, op_results);
@ -6437,6 +6440,12 @@ void StorageReplicatedMergeTree::replacePartitionFrom(
lock1.reset(); lock1.reset();
waitForLogEntryToBeProcessedIfNecessary(entry, query_context); waitForLogEntryToBeProcessedIfNecessary(entry, query_context);
return;
}
throw Exception(
ErrorCodes::CANNOT_ASSIGN_ALTER, "Cannot assign ALTER PARTITION, because another ALTER PARTITION query was concurrently executed");
} }
void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr query_context) void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr query_context)
@ -6464,6 +6473,9 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
/// A range for log entry to remove parts from the source table (myself). /// A range for log entry to remove parts from the source table (myself).
auto zookeeper = getZooKeeper(); auto zookeeper = getZooKeeper();
/// Retry if alter_partition_version changes
for (size_t retry = 0; retry < 1000; ++retry)
{
String alter_partition_version_path = zookeeper_path + "/alter_partition_version"; String alter_partition_version_path = zookeeper_path + "/alter_partition_version";
Coordination::Stat alter_partition_version_stat; Coordination::Stat alter_partition_version_stat;
zookeeper->get(alter_partition_version_path, &alter_partition_version_stat); zookeeper->get(alter_partition_version_path, &alter_partition_version_stat);
@ -6609,7 +6621,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
Coordination::Error code = zookeeper->tryMulti(ops, op_results); Coordination::Error code = zookeeper->tryMulti(ops, op_results);
if (code == Coordination::Error::ZBADVERSION) if (code == Coordination::Error::ZBADVERSION)
throw Exception(ErrorCodes::CANNOT_ASSIGN_ALTER, "Cannot assign ALTER PARTITION, because another ALTER PARTITION query was concurrently executed"); continue;
else else
zkutil::KeeperMultiException::check(code, ops, op_results); zkutil::KeeperMultiException::check(code, ops, op_results);
@ -6657,6 +6669,11 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
/// Cleaning possibly stored information about parts from /quorum/last_part node in ZooKeeper. /// Cleaning possibly stored information about parts from /quorum/last_part node in ZooKeeper.
cleanLastPartNode(partition_id); cleanLastPartNode(partition_id);
return;
}
throw Exception(ErrorCodes::CANNOT_ASSIGN_ALTER, "Cannot assign ALTER PARTITION, because another ALTER PARTITION query was concurrently executed");
} }
void StorageReplicatedMergeTree::movePartitionToShard( void StorageReplicatedMergeTree::movePartitionToShard(
@ -6984,6 +7001,9 @@ bool StorageReplicatedMergeTree::dropPartImpl(
bool StorageReplicatedMergeTree::dropAllPartsInPartition( bool StorageReplicatedMergeTree::dropAllPartsInPartition(
zkutil::ZooKeeper & zookeeper, String & partition_id, LogEntry & entry, ContextPtr query_context, bool detach) zkutil::ZooKeeper & zookeeper, String & partition_id, LogEntry & entry, ContextPtr query_context, bool detach)
{ {
/// Retry if alter_partition_version changes
for (size_t retry = 0; retry < 1000; ++retry)
{
String alter_partition_version_path = zookeeper_path + "/alter_partition_version"; String alter_partition_version_path = zookeeper_path + "/alter_partition_version";
Coordination::Stat alter_partition_version_stat; Coordination::Stat alter_partition_version_stat;
zookeeper.get(alter_partition_version_path, &alter_partition_version_stat); zookeeper.get(alter_partition_version_path, &alter_partition_version_stat);
@ -7035,8 +7055,7 @@ bool StorageReplicatedMergeTree::dropAllPartsInPartition(
if (code == Coordination::Error::ZOK) if (code == Coordination::Error::ZOK)
delimiting_block_lock->assumeUnlocked(); delimiting_block_lock->assumeUnlocked();
else if (code == Coordination::Error::ZBADVERSION) else if (code == Coordination::Error::ZBADVERSION)
throw Exception(ErrorCodes::CANNOT_ASSIGN_ALTER, continue;
"Cannot assign ALTER PARTITION because another ALTER PARTITION query was concurrently executed");
else else
zkutil::KeeperMultiException::check(code, ops, responses); zkutil::KeeperMultiException::check(code, ops, responses);
@ -7046,6 +7065,9 @@ bool StorageReplicatedMergeTree::dropAllPartsInPartition(
getContext()->getMergeList().cancelInPartition(getStorageID(), partition_id, drop_range_info.max_block); getContext()->getMergeList().cancelInPartition(getStorageID(), partition_id, drop_range_info.max_block);
return true; return true;
}
throw Exception(ErrorCodes::CANNOT_ASSIGN_ALTER,
"Cannot assign ALTER PARTITION because another ALTER PARTITION query was concurrently executed");
} }