Set replica name to zero copy exclusive lock

This commit is contained in:
Frank Chen 2023-01-20 11:04:36 +08:00
parent c7d12564cb
commit 1fe9bde594
5 changed files with 11 additions and 10 deletions

View File

@ -1363,10 +1363,11 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry(
if (!disk->supportZeroCopyReplication())
only_s3_storage = false;
if (!disks.empty() && only_s3_storage && storage.checkZeroCopyLockExists(entry.new_part_name, disks[0]))
String replica_to_execute_merge;
if (!disks.empty() && only_s3_storage && storage.checkZeroCopyLockExists(entry.new_part_name, disks[0], replica_to_execute_merge))
{
constexpr auto fmt_string = "Not executing merge/mutation for the part {}, waiting other replica to execute it and will fetch after.";
out_postpone_reason = fmt::format(fmt_string, entry.new_part_name);
constexpr auto fmt_string = "Not executing merge/mutation for the part {}, waiting for {} to execute it and will fetch after.";
out_postpone_reason = fmt::format(fmt_string, entry.new_part_name, replica_to_execute_merge;);
return false;
}
}

View File

@ -2,8 +2,8 @@
namespace DB
{
ZeroCopyLock::ZeroCopyLock(const zkutil::ZooKeeperPtr & zookeeper, const std::string & lock_path)
: lock(zkutil::createSimpleZooKeeperLock(zookeeper, lock_path, "part_exclusive_lock", ""))
ZeroCopyLock::ZeroCopyLock(const zkutil::ZooKeeperPtr & zookeeper, const std::string & lock_path, const std::string & lock_message)
: lock(zkutil::createSimpleZooKeeperLock(zookeeper, lock_path, "part_exclusive_lock", lock_message))
{
}
}

View File

@ -12,7 +12,7 @@ namespace DB
/// because due to bad abstraction we use it in MergeTreeData.
struct ZeroCopyLock
{
ZeroCopyLock(const zkutil::ZooKeeperPtr & zookeeper, const std::string & lock_path);
ZeroCopyLock(const zkutil::ZooKeeperPtr & zookeeper, const std::string & lock_path, const std::string & lock_message);
/// Actual lock
std::unique_ptr<zkutil::ZooKeeperLock> lock;

View File

@ -8453,14 +8453,14 @@ Strings StorageReplicatedMergeTree::getZeroCopyPartPath(
return res;
}
bool StorageReplicatedMergeTree::checkZeroCopyLockExists(const String & part_name, const DiskPtr & disk)
bool StorageReplicatedMergeTree::checkZeroCopyLockExists(const String & part_name, const DiskPtr & disk, String & lock_replica)
{
auto path = getZeroCopyPartPath(part_name, disk);
if (path)
{
/// FIXME
auto lock_path = fs::path(*path) / "part_exclusive_lock";
if (getZooKeeper()->exists(lock_path))
if (getZooKeeper()->tryGet(lock_path, lock_replica))
{
return true;
}
@ -8493,7 +8493,7 @@ std::optional<ZeroCopyLock> StorageReplicatedMergeTree::tryCreateZeroCopyExclusi
zookeeper->createIfNotExists(zc_zookeeper_path, "");
/// Create actual lock
ZeroCopyLock lock(zookeeper, zc_zookeeper_path);
ZeroCopyLock lock(zookeeper, zc_zookeeper_path, replica_name);
if (lock.lock->tryLock())
return lock;
else

View File

@ -858,7 +858,7 @@ private:
// Create table id if needed
void createTableSharedID() const;
bool checkZeroCopyLockExists(const String & part_name, const DiskPtr & disk);
bool checkZeroCopyLockExists(const String & part_name, const DiskPtr & disk, String & lock_replica);
std::optional<String> getZeroCopyPartPath(const String & part_name, const DiskPtr & disk);