mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-29 02:52:13 +00:00
Add retry to check alter_partition_version
This commit is contained in:
parent
b25b778349
commit
710f7d64e4
@ -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.
|
||||
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());
|
||||
|
||||
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";
|
||||
Coordination::Stat 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)
|
||||
delimiting_block_lock->assumeUnlocked();
|
||||
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
|
||||
zkutil::KeeperMultiException::check(code, ops, op_results);
|
||||
|
||||
@ -6437,6 +6440,12 @@ void StorageReplicatedMergeTree::replacePartitionFrom(
|
||||
lock1.reset();
|
||||
|
||||
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)
|
||||
@ -6464,6 +6473,9 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
|
||||
|
||||
/// A range for log entry to remove parts from the source table (myself).
|
||||
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";
|
||||
Coordination::Stat 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);
|
||||
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
|
||||
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.
|
||||
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(
|
||||
@ -6984,6 +7001,9 @@ bool StorageReplicatedMergeTree::dropPartImpl(
|
||||
bool StorageReplicatedMergeTree::dropAllPartsInPartition(
|
||||
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";
|
||||
Coordination::Stat 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)
|
||||
delimiting_block_lock->assumeUnlocked();
|
||||
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
|
||||
zkutil::KeeperMultiException::check(code, ops, responses);
|
||||
|
||||
@ -7046,6 +7065,9 @@ bool StorageReplicatedMergeTree::dropAllPartsInPartition(
|
||||
getContext()->getMergeList().cancelInPartition(getStorageID(), partition_id, drop_range_info.max_block);
|
||||
|
||||
return true;
|
||||
}
|
||||
throw Exception(ErrorCodes::CANNOT_ASSIGN_ALTER,
|
||||
"Cannot assign ALTER PARTITION because another ALTER PARTITION query was concurrently executed");
|
||||
}
|
||||
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user