mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #45982 from ClickHouse/Cluster_state_for_disallow_concurrent_backup_restore
Use cluster state data to check concurrent backup/restore
This commit is contained in:
commit
49330b373c
@ -202,4 +202,9 @@ Strings BackupCoordinationLocal::getAllArchiveSuffixes() const
|
||||
return archive_suffixes;
|
||||
}
|
||||
|
||||
bool BackupCoordinationLocal::hasConcurrentBackups(const std::atomic<size_t> & num_active_backups) const
|
||||
{
|
||||
return (num_active_backups > 1);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -52,6 +52,8 @@ public:
|
||||
String getNextArchiveSuffix() override;
|
||||
Strings getAllArchiveSuffixes() const override;
|
||||
|
||||
bool hasConcurrentBackups(const std::atomic<size_t> & num_active_backups) const override;
|
||||
|
||||
private:
|
||||
mutable std::mutex mutex;
|
||||
BackupCoordinationReplicatedTables replicated_tables TSA_GUARDED_BY(mutex);
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Common/ZooKeeper/KeeperException.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/hex.h>
|
||||
#include <Backups/BackupCoordinationStage.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -18,6 +19,8 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
namespace Stage = BackupCoordinationStage;
|
||||
|
||||
/// zookeeper_path/file_names/file_name->checksum_and_size
|
||||
/// zookeeper_path/file_infos/checksum_and_size->info
|
||||
/// zookeeper_path/archive_suffixes
|
||||
@ -160,27 +163,26 @@ namespace
|
||||
{
|
||||
return fmt::format("{:03}", counter); /// Outputs 001, 002, 003, ...
|
||||
}
|
||||
|
||||
/// We try to store data to zookeeper several times due to possible version conflicts.
|
||||
constexpr size_t NUM_ATTEMPTS = 10;
|
||||
}
|
||||
|
||||
BackupCoordinationRemote::BackupCoordinationRemote(
|
||||
const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool remove_zk_nodes_in_destructor_)
|
||||
: zookeeper_path(zookeeper_path_)
|
||||
const String & root_zookeeper_path_, const String & backup_uuid_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_)
|
||||
: root_zookeeper_path(root_zookeeper_path_)
|
||||
, zookeeper_path(root_zookeeper_path_ + "/backup-" + backup_uuid_)
|
||||
, backup_uuid(backup_uuid_)
|
||||
, get_zookeeper(get_zookeeper_)
|
||||
, remove_zk_nodes_in_destructor(remove_zk_nodes_in_destructor_)
|
||||
, is_internal(is_internal_)
|
||||
{
|
||||
createRootNodes();
|
||||
stage_sync.emplace(
|
||||
zookeeper_path_ + "/stage", [this] { return getZooKeeper(); }, &Poco::Logger::get("BackupCoordination"));
|
||||
zookeeper_path + "/stage", [this] { return getZooKeeper(); }, &Poco::Logger::get("BackupCoordination"));
|
||||
}
|
||||
|
||||
BackupCoordinationRemote::~BackupCoordinationRemote()
|
||||
{
|
||||
try
|
||||
{
|
||||
if (remove_zk_nodes_in_destructor)
|
||||
if (!is_internal)
|
||||
removeAllNodes();
|
||||
}
|
||||
catch (...)
|
||||
@ -468,7 +470,7 @@ void BackupCoordinationRemote::updateFileInfo(const FileInfo & file_info)
|
||||
auto zk = getZooKeeper();
|
||||
String size_and_checksum = serializeSizeAndChecksum(std::pair{file_info.size, file_info.checksum});
|
||||
String full_path = zookeeper_path + "/file_infos/" + size_and_checksum;
|
||||
for (size_t attempt = 0; attempt < NUM_ATTEMPTS; ++attempt)
|
||||
for (size_t attempt = 0; attempt < MAX_ZOOKEEPER_ATTEMPTS; ++attempt)
|
||||
{
|
||||
Coordination::Stat stat;
|
||||
auto new_info = deserializeFileInfo(zk->get(full_path, &stat));
|
||||
@ -476,7 +478,7 @@ void BackupCoordinationRemote::updateFileInfo(const FileInfo & file_info)
|
||||
auto code = zk->trySet(full_path, serializeFileInfo(new_info), stat.version);
|
||||
if (code == Coordination::Error::ZOK)
|
||||
return;
|
||||
bool is_last_attempt = (attempt == NUM_ATTEMPTS - 1);
|
||||
bool is_last_attempt = (attempt == MAX_ZOOKEEPER_ATTEMPTS - 1);
|
||||
if ((code != Coordination::Error::ZBADVERSION) || is_last_attempt)
|
||||
throw zkutil::KeeperException(code, full_path);
|
||||
}
|
||||
@ -595,4 +597,51 @@ Strings BackupCoordinationRemote::getAllArchiveSuffixes() const
|
||||
return node_names;
|
||||
}
|
||||
|
||||
bool BackupCoordinationRemote::hasConcurrentBackups(const std::atomic<size_t> &) const
|
||||
{
|
||||
/// If its internal concurrency will be checked for the base backup
|
||||
if (is_internal)
|
||||
return false;
|
||||
|
||||
auto zk = getZooKeeper();
|
||||
std::string backup_stage_path = zookeeper_path +"/stage";
|
||||
|
||||
if (!zk->exists(root_zookeeper_path))
|
||||
zk->createAncestors(root_zookeeper_path);
|
||||
|
||||
for (size_t attempt = 0; attempt < MAX_ZOOKEEPER_ATTEMPTS; ++attempt)
|
||||
{
|
||||
Coordination::Stat stat;
|
||||
zk->get(root_zookeeper_path, &stat);
|
||||
Strings existing_backup_paths = zk->getChildren(root_zookeeper_path);
|
||||
|
||||
for (const auto & existing_backup_path : existing_backup_paths)
|
||||
{
|
||||
if (startsWith(existing_backup_path, "restore-"))
|
||||
continue;
|
||||
|
||||
String existing_backup_uuid = existing_backup_path;
|
||||
existing_backup_uuid.erase(0, String("backup-").size());
|
||||
|
||||
if (existing_backup_uuid == toString(backup_uuid))
|
||||
continue;
|
||||
|
||||
const auto status = zk->get(root_zookeeper_path + "/" + existing_backup_path + "/stage");
|
||||
if (status != Stage::COMPLETED)
|
||||
return true;
|
||||
}
|
||||
|
||||
zk->createIfNotExists(backup_stage_path, "");
|
||||
auto code = zk->trySet(backup_stage_path, Stage::SCHEDULED_TO_START, stat.version);
|
||||
if (code == Coordination::Error::ZOK)
|
||||
break;
|
||||
bool is_last_attempt = (attempt == MAX_ZOOKEEPER_ATTEMPTS - 1);
|
||||
if ((code != Coordination::Error::ZBADVERSION) || is_last_attempt)
|
||||
throw zkutil::KeeperException(code, backup_stage_path);
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -9,11 +9,14 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// We try to store data to zookeeper several times due to possible version conflicts.
|
||||
constexpr size_t MAX_ZOOKEEPER_ATTEMPTS = 10;
|
||||
|
||||
/// Implementation of the IBackupCoordination interface performing coordination via ZooKeeper. It's necessary for "BACKUP ON CLUSTER".
|
||||
class BackupCoordinationRemote : public IBackupCoordination
|
||||
{
|
||||
public:
|
||||
BackupCoordinationRemote(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool remove_zk_nodes_in_destructor_);
|
||||
BackupCoordinationRemote(const String & root_zookeeper_path_, const String & backup_uuid_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_);
|
||||
~BackupCoordinationRemote() override;
|
||||
|
||||
void setStage(const String & current_host, const String & new_stage, const String & message) override;
|
||||
@ -55,6 +58,8 @@ public:
|
||||
String getNextArchiveSuffix() override;
|
||||
Strings getAllArchiveSuffixes() const override;
|
||||
|
||||
bool hasConcurrentBackups(const std::atomic<size_t> & num_active_backups) const override;
|
||||
|
||||
private:
|
||||
zkutil::ZooKeeperPtr getZooKeeper() const;
|
||||
zkutil::ZooKeeperPtr getZooKeeperNoLock() const;
|
||||
@ -63,9 +68,11 @@ private:
|
||||
void prepareReplicatedTables() const;
|
||||
void prepareReplicatedAccess() const;
|
||||
|
||||
const String root_zookeeper_path;
|
||||
const String zookeeper_path;
|
||||
const String backup_uuid;
|
||||
const zkutil::GetZooKeeper get_zookeeper;
|
||||
const bool remove_zk_nodes_in_destructor;
|
||||
const bool is_internal;
|
||||
|
||||
std::optional<BackupCoordinationStageSync> stage_sync;
|
||||
|
||||
|
@ -8,6 +8,10 @@ namespace DB
|
||||
|
||||
namespace BackupCoordinationStage
|
||||
{
|
||||
/// This stage is set after concurrency check so ensure we dont start other backup/restores
|
||||
/// when concurrent backup/restores are not allowed
|
||||
constexpr const char * SCHEDULED_TO_START = "scheduled to start";
|
||||
|
||||
/// Finding all tables and databases which we're going to put to the backup and collecting their metadata.
|
||||
constexpr const char * GATHERING_METADATA = "gathering metadata";
|
||||
|
||||
|
@ -28,7 +28,6 @@ namespace ErrorCodes
|
||||
M(UInt64, replica_num) \
|
||||
M(Bool, internal) \
|
||||
M(String, host_id) \
|
||||
M(String, coordination_zk_path) \
|
||||
M(OptionalUUID, backup_uuid)
|
||||
/// M(Int64, compression_level)
|
||||
|
||||
|
@ -55,10 +55,6 @@ struct BackupSettings
|
||||
/// Cluster's hosts' IDs in the format 'escaped_host_name:port' for all shards and replicas in a cluster specified in BACKUP ON CLUSTER.
|
||||
std::vector<Strings> cluster_host_ids;
|
||||
|
||||
/// Internal, should not be specified by user.
|
||||
/// Path in Zookeeper used to coordinate a distributed backup created by BACKUP ON CLUSTER.
|
||||
String coordination_zk_path;
|
||||
|
||||
/// Internal, should not be specified by user.
|
||||
/// UUID of the backup. If it's not set it will be generated randomly.
|
||||
std::optional<UUID> backup_uuid;
|
||||
|
@ -38,12 +38,12 @@ namespace Stage = BackupCoordinationStage;
|
||||
|
||||
namespace
|
||||
{
|
||||
std::shared_ptr<IBackupCoordination> makeBackupCoordination(const String & coordination_zk_path, const ContextPtr & context, bool is_internal_backup)
|
||||
std::shared_ptr<IBackupCoordination> makeBackupCoordination(const String & root_zk_path, const String & backup_uuid, const ContextPtr & context, bool is_internal_backup)
|
||||
{
|
||||
if (!coordination_zk_path.empty())
|
||||
if (!root_zk_path.empty())
|
||||
{
|
||||
auto get_zookeeper = [global_context = context->getGlobalContext()] { return global_context->getZooKeeper(); };
|
||||
return std::make_shared<BackupCoordinationRemote>(coordination_zk_path, get_zookeeper, !is_internal_backup);
|
||||
return std::make_shared<BackupCoordinationRemote>(root_zk_path, backup_uuid, get_zookeeper, is_internal_backup);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -51,12 +51,12 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
std::shared_ptr<IRestoreCoordination> makeRestoreCoordination(const String & coordination_zk_path, const ContextPtr & context, bool is_internal_backup)
|
||||
std::shared_ptr<IRestoreCoordination> makeRestoreCoordination(const String & root_zk_path, const String & restore_uuid, const ContextPtr & context, bool is_internal_backup)
|
||||
{
|
||||
if (!coordination_zk_path.empty())
|
||||
if (!root_zk_path.empty())
|
||||
{
|
||||
auto get_zookeeper = [global_context = context->getGlobalContext()] { return global_context->getZooKeeper(); };
|
||||
return std::make_shared<RestoreCoordinationRemote>(coordination_zk_path, get_zookeeper, !is_internal_backup);
|
||||
return std::make_shared<RestoreCoordinationRemote>(root_zk_path, restore_uuid, get_zookeeper, is_internal_backup);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -160,26 +160,22 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context
|
||||
else
|
||||
backup_id = toString(*backup_settings.backup_uuid);
|
||||
|
||||
String root_zk_path;
|
||||
|
||||
std::shared_ptr<IBackupCoordination> backup_coordination;
|
||||
if (backup_settings.internal)
|
||||
{
|
||||
/// The following call of makeBackupCoordination() is not essential because doBackup() will later create a backup coordination
|
||||
/// if it's not created here. However to handle errors better it's better to make a coordination here because this way
|
||||
/// if an exception will be thrown in startMakingBackup() other hosts will know about that.
|
||||
backup_coordination = makeBackupCoordination(backup_settings.coordination_zk_path, context, backup_settings.internal);
|
||||
root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups");
|
||||
backup_coordination = makeBackupCoordination(root_zk_path, toString(*backup_settings.backup_uuid), context, backup_settings.internal);
|
||||
}
|
||||
|
||||
auto backup_info = BackupInfo::fromAST(*backup_query->backup_name);
|
||||
String backup_name_for_logging = backup_info.toStringForLogging();
|
||||
try
|
||||
{
|
||||
if (!allow_concurrent_backups && hasConcurrentBackups(backup_settings))
|
||||
{
|
||||
/// addInfo is called here to record the failed backup details
|
||||
addInfo(backup_id, backup_name_for_logging, backup_settings.internal, BackupStatus::BACKUP_FAILED);
|
||||
throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent backups not supported, turn on setting 'allow_concurrent_backups'");
|
||||
}
|
||||
|
||||
addInfo(backup_id, backup_name_for_logging, backup_settings.internal, BackupStatus::CREATING_BACKUP);
|
||||
|
||||
/// Prepare context to use.
|
||||
@ -259,6 +255,7 @@ void BackupsWorker::doBackup(
|
||||
}
|
||||
|
||||
bool on_cluster = !backup_query->cluster.empty();
|
||||
|
||||
assert(mutable_context || (!on_cluster && !called_async));
|
||||
|
||||
/// Checks access rights if this is not ON CLUSTER query.
|
||||
@ -267,22 +264,23 @@ void BackupsWorker::doBackup(
|
||||
if (!on_cluster)
|
||||
context->checkAccess(required_access);
|
||||
|
||||
String root_zk_path;
|
||||
|
||||
ClusterPtr cluster;
|
||||
if (on_cluster)
|
||||
{
|
||||
root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups");
|
||||
backup_query->cluster = context->getMacros()->expand(backup_query->cluster);
|
||||
cluster = context->getCluster(backup_query->cluster);
|
||||
backup_settings.cluster_host_ids = cluster->getHostIDs();
|
||||
if (backup_settings.coordination_zk_path.empty())
|
||||
{
|
||||
String root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups");
|
||||
backup_settings.coordination_zk_path = root_zk_path + "/backup-" + toString(*backup_settings.backup_uuid);
|
||||
}
|
||||
}
|
||||
|
||||
/// Make a backup coordination.
|
||||
if (!backup_coordination)
|
||||
backup_coordination = makeBackupCoordination(backup_settings.coordination_zk_path, context, backup_settings.internal);
|
||||
backup_coordination = makeBackupCoordination(root_zk_path, toString(*backup_settings.backup_uuid), context, backup_settings.internal);
|
||||
|
||||
if (!allow_concurrent_backups && backup_coordination->hasConcurrentBackups(std::ref(num_active_backups)))
|
||||
throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent backups not supported, turn on setting 'allow_concurrent_backups'");
|
||||
|
||||
/// Opens a backup for writing.
|
||||
BackupFactory::CreateParams backup_create_params;
|
||||
@ -402,7 +400,8 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
|
||||
/// The following call of makeRestoreCoordination() is not essential because doRestore() will later create a restore coordination
|
||||
/// if it's not created here. However to handle errors better it's better to make a coordination here because this way
|
||||
/// if an exception will be thrown in startRestoring() other hosts will know about that.
|
||||
restore_coordination = makeRestoreCoordination(restore_settings.coordination_zk_path, context, restore_settings.internal);
|
||||
auto root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups");
|
||||
restore_coordination = makeRestoreCoordination(root_zk_path, toString(*restore_settings.restore_uuid), context, restore_settings.internal);
|
||||
}
|
||||
|
||||
try
|
||||
@ -410,13 +409,6 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
|
||||
auto backup_info = BackupInfo::fromAST(*restore_query->backup_name);
|
||||
String backup_name_for_logging = backup_info.toStringForLogging();
|
||||
|
||||
if (!allow_concurrent_restores && hasConcurrentRestores(restore_settings))
|
||||
{
|
||||
/// addInfo is called here to record the failed restore details
|
||||
addInfo(restore_id, backup_name_for_logging, restore_settings.internal, BackupStatus::RESTORING);
|
||||
throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent restores not supported, turn on setting 'allow_concurrent_restores'");
|
||||
}
|
||||
|
||||
addInfo(restore_id, backup_name_for_logging, restore_settings.internal, BackupStatus::RESTORING);
|
||||
|
||||
/// Prepare context to use.
|
||||
@ -496,18 +488,18 @@ void BackupsWorker::doRestore(
|
||||
backup_open_params.context = context;
|
||||
backup_open_params.backup_info = backup_info;
|
||||
backup_open_params.base_backup_info = restore_settings.base_backup_info;
|
||||
backup_open_params.backup_uuid = restore_settings.restore_uuid;
|
||||
backup_open_params.password = restore_settings.password;
|
||||
BackupPtr backup = BackupFactory::instance().createBackup(backup_open_params);
|
||||
|
||||
String current_database = context->getCurrentDatabase();
|
||||
|
||||
String root_zk_path;
|
||||
/// Checks access rights if this is ON CLUSTER query.
|
||||
/// (If this isn't ON CLUSTER query RestorerFromBackup will check access rights later.)
|
||||
ClusterPtr cluster;
|
||||
bool on_cluster = !restore_query->cluster.empty();
|
||||
if (on_cluster)
|
||||
{
|
||||
root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups");
|
||||
restore_query->cluster = context->getMacros()->expand(restore_query->cluster);
|
||||
cluster = context->getCluster(restore_query->cluster);
|
||||
restore_settings.cluster_host_ids = cluster->getHostIDs();
|
||||
@ -529,14 +521,11 @@ void BackupsWorker::doRestore(
|
||||
}
|
||||
|
||||
/// Make a restore coordination.
|
||||
if (on_cluster && restore_settings.coordination_zk_path.empty())
|
||||
{
|
||||
String root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups");
|
||||
restore_settings.coordination_zk_path = root_zk_path + "/restore-" + toString(UUIDHelpers::generateV4());
|
||||
}
|
||||
|
||||
if (!restore_coordination)
|
||||
restore_coordination = makeRestoreCoordination(restore_settings.coordination_zk_path, context, restore_settings.internal);
|
||||
restore_coordination = makeRestoreCoordination(root_zk_path, toString(*restore_settings.restore_uuid), context, restore_settings.internal);
|
||||
|
||||
if (!allow_concurrent_restores && restore_coordination->hasConcurrentRestores(std::ref(num_active_restores)))
|
||||
throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent restores not supported, turn on setting 'allow_concurrent_restores'");
|
||||
|
||||
/// Do RESTORE.
|
||||
if (on_cluster)
|
||||
@ -726,58 +715,6 @@ std::vector<BackupsWorker::Info> BackupsWorker::getAllInfos() const
|
||||
return res_infos;
|
||||
}
|
||||
|
||||
std::vector<BackupsWorker::Info> BackupsWorker::getAllActiveBackupInfos() const
|
||||
{
|
||||
std::vector<Info> res_infos;
|
||||
std::lock_guard lock{infos_mutex};
|
||||
for (const auto & info : infos | boost::adaptors::map_values)
|
||||
{
|
||||
if (info.status==BackupStatus::CREATING_BACKUP)
|
||||
res_infos.push_back(info);
|
||||
}
|
||||
return res_infos;
|
||||
}
|
||||
|
||||
std::vector<BackupsWorker::Info> BackupsWorker::getAllActiveRestoreInfos() const
|
||||
{
|
||||
std::vector<Info> res_infos;
|
||||
std::lock_guard lock{infos_mutex};
|
||||
for (const auto & info : infos | boost::adaptors::map_values)
|
||||
{
|
||||
if (info.status==BackupStatus::RESTORING)
|
||||
res_infos.push_back(info);
|
||||
}
|
||||
return res_infos;
|
||||
}
|
||||
|
||||
bool BackupsWorker::hasConcurrentBackups(const BackupSettings & backup_settings) const
|
||||
{
|
||||
/// Check if there are no concurrent backups
|
||||
if (num_active_backups)
|
||||
{
|
||||
/// If its an internal backup and we currently have 1 active backup, it could be the original query, validate using backup_uuid
|
||||
if (!(num_active_backups == 1 && backup_settings.internal && getAllActiveBackupInfos().at(0).id == toString(*backup_settings.backup_uuid)))
|
||||
{
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
bool BackupsWorker::hasConcurrentRestores(const RestoreSettings & restore_settings) const
|
||||
{
|
||||
/// Check if there are no concurrent restores
|
||||
if (num_active_restores)
|
||||
{
|
||||
/// If its an internal restore and we currently have 1 active restore, it could be the original query, validate using iz
|
||||
if (!(num_active_restores == 1 && restore_settings.internal && getAllActiveRestoreInfos().at(0).id == toString(*restore_settings.restore_uuid)))
|
||||
{
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
void BackupsWorker::shutdown()
|
||||
{
|
||||
bool has_active_backups_and_restores = (num_active_backups || num_active_restores);
|
||||
|
@ -117,11 +117,6 @@ private:
|
||||
void setNumFilesAndSize(const OperationID & id, size_t num_files, UInt64 total_size, size_t num_entries,
|
||||
UInt64 uncompressed_size, UInt64 compressed_size, size_t num_read_files, UInt64 num_read_bytes);
|
||||
|
||||
std::vector<Info> getAllActiveBackupInfos() const;
|
||||
std::vector<Info> getAllActiveRestoreInfos() const;
|
||||
bool hasConcurrentBackups(const BackupSettings & backup_settings) const;
|
||||
bool hasConcurrentRestores(const RestoreSettings & restore_settings) const;
|
||||
|
||||
ThreadPool backups_thread_pool;
|
||||
ThreadPool restores_thread_pool;
|
||||
|
||||
|
@ -114,6 +114,10 @@ public:
|
||||
|
||||
/// Returns the list of all the archive suffixes which were generated.
|
||||
virtual Strings getAllArchiveSuffixes() const = 0;
|
||||
|
||||
/// This function is used to check if concurrent backups are running
|
||||
/// other than the backup passed to the function
|
||||
virtual bool hasConcurrentBackups(const std::atomic<size_t> & num_active_backups) const = 0;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -34,6 +34,10 @@ public:
|
||||
/// Sets that this replica is going to restore a ReplicatedAccessStorage.
|
||||
/// The function returns false if this access storage is being already restored by another replica.
|
||||
virtual bool acquireReplicatedAccessStorage(const String & access_storage_zk_path) = 0;
|
||||
|
||||
/// This function is used to check if concurrent restores are running
|
||||
/// other than the restore passed to the function
|
||||
virtual bool hasConcurrentRestores(const std::atomic<size_t> & num_active_restores) const = 0;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -42,4 +42,9 @@ bool RestoreCoordinationLocal::acquireReplicatedAccessStorage(const String &)
|
||||
return true;
|
||||
}
|
||||
|
||||
bool RestoreCoordinationLocal::hasConcurrentRestores(const std::atomic<size_t> & num_active_restores) const
|
||||
{
|
||||
return (num_active_restores > 1);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -35,6 +35,8 @@ public:
|
||||
/// The function returns false if this access storage is being already restored by another replica.
|
||||
bool acquireReplicatedAccessStorage(const String & access_storage_zk_path) override;
|
||||
|
||||
bool hasConcurrentRestores(const std::atomic<size_t> & num_active_restores) const override;
|
||||
|
||||
private:
|
||||
std::set<std::pair<String /* database_zk_path */, String /* table_name */>> acquired_tables_in_replicated_databases;
|
||||
std::unordered_set<String /* table_zk_path */> acquired_data_in_replicated_tables;
|
||||
|
@ -1,28 +1,33 @@
|
||||
#include <Backups/RestoreCoordinationRemote.h>
|
||||
#include <Common/ZooKeeper/KeeperException.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
|
||||
#include <Backups/BackupCoordinationStage.h>
|
||||
#include <Backups/BackupCoordinationRemote.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace Stage = BackupCoordinationStage;
|
||||
|
||||
RestoreCoordinationRemote::RestoreCoordinationRemote(
|
||||
const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool remove_zk_nodes_in_destructor_)
|
||||
: zookeeper_path(zookeeper_path_)
|
||||
const String & root_zookeeper_path_, const String & restore_uuid_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_)
|
||||
: root_zookeeper_path(root_zookeeper_path_)
|
||||
, zookeeper_path(root_zookeeper_path_ + "/restore-" + restore_uuid_)
|
||||
, restore_uuid(restore_uuid_)
|
||||
, get_zookeeper(get_zookeeper_)
|
||||
, remove_zk_nodes_in_destructor(remove_zk_nodes_in_destructor_)
|
||||
, is_internal(is_internal_)
|
||||
{
|
||||
createRootNodes();
|
||||
|
||||
stage_sync.emplace(
|
||||
zookeeper_path_ + "/stage", [this] { return getZooKeeper(); }, &Poco::Logger::get("RestoreCoordination"));
|
||||
zookeeper_path + "/stage", [this] { return getZooKeeper(); }, &Poco::Logger::get("RestoreCoordination"));
|
||||
}
|
||||
|
||||
RestoreCoordinationRemote::~RestoreCoordinationRemote()
|
||||
{
|
||||
try
|
||||
{
|
||||
if (remove_zk_nodes_in_destructor)
|
||||
if (!is_internal)
|
||||
removeAllNodes();
|
||||
}
|
||||
catch (...)
|
||||
@ -129,4 +134,49 @@ void RestoreCoordinationRemote::removeAllNodes()
|
||||
zk->removeRecursive(zookeeper_path);
|
||||
}
|
||||
|
||||
bool RestoreCoordinationRemote::hasConcurrentRestores(const std::atomic<size_t> &) const
|
||||
{
|
||||
/// If its internal concurrency will be checked for the base restore
|
||||
if (is_internal)
|
||||
return false;
|
||||
|
||||
auto zk = getZooKeeper();
|
||||
std::string path = zookeeper_path +"/stage";
|
||||
|
||||
if (! zk->exists(root_zookeeper_path))
|
||||
zk->createAncestors(root_zookeeper_path);
|
||||
|
||||
for (size_t attempt = 0; attempt < MAX_ZOOKEEPER_ATTEMPTS; ++attempt)
|
||||
{
|
||||
Coordination::Stat stat;
|
||||
zk->get(root_zookeeper_path, &stat);
|
||||
Strings existing_restore_paths = zk->getChildren(root_zookeeper_path);
|
||||
for (const auto & existing_restore_path : existing_restore_paths)
|
||||
{
|
||||
if (startsWith(existing_restore_path, "backup-"))
|
||||
continue;
|
||||
|
||||
String existing_restore_uuid = existing_restore_path;
|
||||
existing_restore_uuid.erase(0, String("restore-").size());
|
||||
|
||||
if (existing_restore_uuid == toString(restore_uuid))
|
||||
continue;
|
||||
|
||||
|
||||
const auto status = zk->get(root_zookeeper_path + "/" + existing_restore_path + "/stage");
|
||||
if (status != Stage::COMPLETED)
|
||||
return true;
|
||||
}
|
||||
|
||||
zk->createIfNotExists(path, "");
|
||||
auto code = zk->trySet(path, Stage::SCHEDULED_TO_START, stat.version);
|
||||
if (code == Coordination::Error::ZOK)
|
||||
break;
|
||||
bool is_last_attempt = (attempt == MAX_ZOOKEEPER_ATTEMPTS - 1);
|
||||
if ((code != Coordination::Error::ZBADVERSION) || is_last_attempt)
|
||||
throw zkutil::KeeperException(code, path);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -11,7 +11,7 @@ namespace DB
|
||||
class RestoreCoordinationRemote : public IRestoreCoordination
|
||||
{
|
||||
public:
|
||||
RestoreCoordinationRemote(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool remove_zk_nodes_in_destructor_);
|
||||
RestoreCoordinationRemote(const String & root_zookeeper_path_, const String & restore_uuid_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_);
|
||||
~RestoreCoordinationRemote() override;
|
||||
|
||||
/// Sets the current stage and waits for other hosts to come to this stage too.
|
||||
@ -31,6 +31,8 @@ public:
|
||||
/// The function returns false if this access storage is being already restored by another replica.
|
||||
bool acquireReplicatedAccessStorage(const String & access_storage_zk_path) override;
|
||||
|
||||
bool hasConcurrentRestores(const std::atomic<size_t> & num_active_restores) const override;
|
||||
|
||||
private:
|
||||
zkutil::ZooKeeperPtr getZooKeeper() const;
|
||||
void createRootNodes();
|
||||
@ -38,9 +40,11 @@ private:
|
||||
|
||||
class ReplicatedDatabasesMetadataSync;
|
||||
|
||||
const String root_zookeeper_path;
|
||||
const String zookeeper_path;
|
||||
const String restore_uuid;
|
||||
const zkutil::GetZooKeeper get_zookeeper;
|
||||
const bool remove_zk_nodes_in_destructor;
|
||||
const bool is_internal;
|
||||
|
||||
std::optional<BackupCoordinationStageSync> stage_sync;
|
||||
|
||||
|
@ -163,7 +163,6 @@ namespace
|
||||
M(RestoreUDFCreationMode, create_function) \
|
||||
M(Bool, internal) \
|
||||
M(String, host_id) \
|
||||
M(String, coordination_zk_path) \
|
||||
M(OptionalUUID, restore_uuid)
|
||||
|
||||
|
||||
|
@ -118,13 +118,9 @@ struct RestoreSettings
|
||||
/// Cluster's hosts' IDs in the format 'escaped_host_name:port' for all shards and replicas in a cluster specified in BACKUP ON CLUSTER.
|
||||
std::vector<Strings> cluster_host_ids;
|
||||
|
||||
/// Internal, should not be specified by user.
|
||||
/// Path in Zookeeper used to coordinate restoring process while executing by RESTORE ON CLUSTER.
|
||||
String coordination_zk_path;
|
||||
|
||||
/// Internal, should not be specified by user.
|
||||
/// UUID of the restore. If it's not set it will be generated randomly.
|
||||
/// This is used to validate internal restores when allow_concurrent_restores is turned off
|
||||
/// This is used to generate coordination path and for concurrency check
|
||||
std::optional<UUID> restore_uuid;
|
||||
|
||||
static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query);
|
||||
|
Loading…
Reference in New Issue
Block a user