Tiny improvements

This commit is contained in:
alesapin 2021-12-30 12:57:38 +03:00
parent cbdba89d65
commit 91e1ac437e
2 changed files with 34 additions and 31 deletions

View File

@ -7042,35 +7042,35 @@ String StorageReplicatedMergeTree::getTableSharedID() const
void StorageReplicatedMergeTree::createTableSharedID()
{
if (table_shared_id == UUIDHelpers::Nil)
if (table_shared_id != UUIDHelpers::Nil)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table shared id already initialized");
zkutil::ZooKeeperPtr zookeeper = getZooKeeper();
String zookeeper_table_id_path = fs::path(zookeeper_path) / "table_shared_id";
String id;
if (!zookeeper->tryGet(zookeeper_table_id_path, id))
{
zkutil::ZooKeeperPtr zookeeper = getZooKeeper();
String zookeeper_table_id_path = fs::path(zookeeper_path) / "table_shared_id";
String id;
if (!zookeeper->tryGet(zookeeper_table_id_path, id))
{
UUID table_id_candidate;
auto storage_id = getStorageID();
if (storage_id.uuid != UUIDHelpers::Nil)
table_id_candidate = storage_id.uuid;
else
table_id_candidate = UUIDHelpers::generateV4();
UUID table_id_candidate;
auto storage_id = getStorageID();
if (storage_id.uuid != UUIDHelpers::Nil)
table_id_candidate = storage_id.uuid;
else
table_id_candidate = UUIDHelpers::generateV4();
id = toString(table_id_candidate);
id = toString(table_id_candidate);
auto code = zookeeper->tryCreate(zookeeper_table_id_path, id, zkutil::CreateMode::Persistent);
if (code == Coordination::Error::ZNODEEXISTS)
{ /// Other replica create node early
id = zookeeper->get(zookeeper_table_id_path);
}
else if (code != Coordination::Error::ZOK)
{
throw zkutil::KeeperException(code, zookeeper_table_id_path);
}
auto code = zookeeper->tryCreate(zookeeper_table_id_path, id, zkutil::CreateMode::Persistent);
if (code == Coordination::Error::ZNODEEXISTS)
{ /// Other replica create node early
id = zookeeper->get(zookeeper_table_id_path);
}
else if (code != Coordination::Error::ZOK)
{
throw zkutil::KeeperException(code, zookeeper_table_id_path);
}
table_shared_id = parseFromString<UUID>(id);
}
table_shared_id = parseFromString<UUID>(id);
}
@ -7123,12 +7123,12 @@ bool StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & par
if (ref_count > 0) /// Keep part shard info for frozen backups
return false;
return unlockSharedDataById(part.getUniqueId(), getTableSharedID(), name, replica_name, disk, zookeeper, *getSettings(), log,
return unlockSharedDataByID(part.getUniqueId(), getTableSharedID(), name, replica_name, disk, zookeeper, *getSettings(), log,
zookeeper_path);
}
bool StorageReplicatedMergeTree::unlockSharedDataById(String id, const String & table_uuid, const String & part_name,
bool StorageReplicatedMergeTree::unlockSharedDataByID(String id, const String & table_uuid, const String & part_name,
const String & replica_name_, DiskPtr disk, zkutil::ZooKeeperPtr zookeeper_ptr, const MergeTreeSettings & settings,
Poco::Logger * logger, const String & zookeeper_path_old)
{
@ -7552,6 +7552,8 @@ void StorageReplicatedMergeTree::createZeroCopyLockNode(const zkutil::ZooKeeperP
namespace
{
/// Special metadata used during freeze table. Required for zero-copy
/// replication.
struct FreezeMetaData
{
public:
@ -7673,7 +7675,7 @@ bool StorageReplicatedMergeTree::removeSharedDetachedPart(DiskPtr disk, const St
if (ref_count == 0)
{
String id = disk->getUniqueId(checksums);
keep_shared = !StorageReplicatedMergeTree::unlockSharedDataById(id, table_uuid, part_name,
keep_shared = !StorageReplicatedMergeTree::unlockSharedDataByID(id, table_uuid, part_name,
detached_replica_name, disk, zookeeper, getContext()->getReplicatedMergeTreeSettings(), log,
detached_zookeeper_path);
}

View File

@ -243,7 +243,7 @@ public:
/// Unlock shared data part in zookeeper by part id
/// Return true if data unlocked
/// Return false if data is still used by another node
static bool unlockSharedDataById(String id, const String & table_uuid, const String & part_name, const String & replica_name_,
static bool unlockSharedDataByID(String id, const String & table_uuid, const String & part_name, const String & replica_name_,
DiskPtr disk, zkutil::ZooKeeperPtr zookeeper_, const MergeTreeSettings & settings, Poco::Logger * logger,
const String & zookeeper_path_old);
@ -410,6 +410,9 @@ private:
ThrottlerPtr replicated_fetches_throttler;
ThrottlerPtr replicated_sends_throttler;
/// Global ID, synced via ZooKeeper between replicas
UUID table_shared_id;
template <class Func>
void foreachCommittedParts(Func && func, bool select_sequential_consistency) const;
@ -749,6 +752,7 @@ private:
bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid,
const String & zookeeper_name, const String & replica_name, const String & zookeeper_path);
/// Create freeze metadata for table and save in zookeeper. Required only if zero-copy replication enabled.
void createAndStoreFreezeMetadata(DiskPtr disk, DataPartPtr part, String backup_part_path) const override;
// Create table id if needed
@ -770,9 +774,6 @@ protected:
std::unique_ptr<MergeTreeSettings> settings_,
bool has_force_restore_data_flag,
bool allow_renaming_);
/// Global ID, synced via ZooKeeper between replicas
UUID table_shared_id;
};
String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info);