Merge pull request #48337 from vitlibar/add-logging-for-concurrency-checks-for-backups

Add logging for concurrency checks for backups
This commit is contained in:
Alexander Tokmakov 2023-04-04 02:56:10 +03:00 committed by GitHub
commit 72ca7a9724
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 36 additions and 7 deletions

View File

@ -8,7 +8,8 @@
namespace DB
{
BackupCoordinationLocal::BackupCoordinationLocal(bool plain_backup_) : file_infos(plain_backup_)
BackupCoordinationLocal::BackupCoordinationLocal(bool plain_backup_)
: log(&Poco::Logger::get("BackupCoordinationLocal")), file_infos(plain_backup_)
{
}
@ -125,7 +126,12 @@ bool BackupCoordinationLocal::startWritingFile(size_t data_file_index)
bool BackupCoordinationLocal::hasConcurrentBackups(const std::atomic<size_t> & num_active_backups) const
{
return (num_active_backups > 1);
if (num_active_backups > 1)
{
LOG_WARNING(log, "Found concurrent backups: num_active_backups={}", num_active_backups);
return true;
}
return false;
}
}

View File

@ -52,6 +52,8 @@ public:
bool hasConcurrentBackups(const std::atomic<size_t> & num_active_backups) const override;
private:
Poco::Logger * const log;
BackupCoordinationReplicatedTables TSA_GUARDED_BY(replicated_tables_mutex) replicated_tables;
BackupCoordinationReplicatedAccess TSA_GUARDED_BY(replicated_access_mutex) replicated_access;
BackupCoordinationReplicatedSQLObjects TSA_GUARDED_BY(replicated_sql_objects_mutex) replicated_sql_objects;

View File

@ -164,17 +164,18 @@ BackupCoordinationRemote::BackupCoordinationRemote(
, current_host_index(findCurrentHostIndex(all_hosts, current_host))
, plain_backup(plain_backup_)
, is_internal(is_internal_)
, log(&Poco::Logger::get("BackupCoordinationRemote"))
{
zookeeper_retries_info = ZooKeeperRetriesInfo(
"BackupCoordinationRemote",
&Poco::Logger::get("BackupCoordinationRemote"),
log,
keeper_settings.keeper_max_retries,
keeper_settings.keeper_retry_initial_backoff_ms,
keeper_settings.keeper_retry_max_backoff_ms);
createRootNodes();
stage_sync.emplace(
zookeeper_path + "/stage", [this] { return getZooKeeper(); }, &Poco::Logger::get("BackupCoordination"));
zookeeper_path + "/stage", [this] { return getZooKeeper(); }, log);
}
BackupCoordinationRemote::~BackupCoordinationRemote()
@ -664,7 +665,10 @@ bool BackupCoordinationRemote::hasConcurrentBackups(const std::atomic<size_t> &)
const auto status = zk->get(root_zookeeper_path + "/" + existing_backup_path + "/stage");
if (status != Stage::COMPLETED)
{
LOG_WARNING(log, "Found a concurrent backup: {}, current backup: {}", existing_backup_uuid, toString(backup_uuid));
return true;
}
}
zk->createIfNotExists(backup_stage_path, "");

View File

@ -104,6 +104,7 @@ private:
const size_t current_host_index;
const bool plain_backup;
const bool is_internal;
Poco::Logger * const log;
mutable ZooKeeperRetriesInfo zookeeper_retries_info;
std::optional<BackupCoordinationStageSync> stage_sync;

View File

@ -1,10 +1,14 @@
#include <Backups/RestoreCoordinationLocal.h>
#include <Common/logger_useful.h>
namespace DB
{
RestoreCoordinationLocal::RestoreCoordinationLocal() = default;
RestoreCoordinationLocal::RestoreCoordinationLocal() : log(&Poco::Logger::get("RestoreCoordinationLocal"))
{
}
RestoreCoordinationLocal::~RestoreCoordinationLocal() = default;
void RestoreCoordinationLocal::setStage(const String &, const String &)
@ -49,7 +53,12 @@ bool RestoreCoordinationLocal::acquireReplicatedSQLObjects(const String &, UserD
bool RestoreCoordinationLocal::hasConcurrentRestores(const std::atomic<size_t> & num_active_restores) const
{
return (num_active_restores > 1);
if (num_active_restores > 1)
{
LOG_WARNING(log, "Found concurrent backups: num_active_restores={}", num_active_restores);
return true;
}
return false;
}
}

View File

@ -42,6 +42,8 @@ public:
bool hasConcurrentRestores(const std::atomic<size_t> & num_active_restores) const override;
private:
Poco::Logger * const log;
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;
mutable std::mutex mutex;

View File

@ -25,11 +25,12 @@ RestoreCoordinationRemote::RestoreCoordinationRemote(
, current_host(current_host_)
, current_host_index(BackupCoordinationRemote::findCurrentHostIndex(all_hosts, current_host))
, is_internal(is_internal_)
, log(&Poco::Logger::get("RestoreCoordinationRemote"))
{
createRootNodes();
stage_sync.emplace(
zookeeper_path + "/stage", [this] { return getZooKeeper(); }, &Poco::Logger::get("RestoreCoordination"));
zookeeper_path + "/stage", [this] { return getZooKeeper(); }, log);
}
RestoreCoordinationRemote::~RestoreCoordinationRemote()
@ -197,7 +198,10 @@ bool RestoreCoordinationRemote::hasConcurrentRestores(const std::atomic<size_t>
const auto status = zk->get(root_zookeeper_path + "/" + existing_restore_path + "/stage");
if (status != Stage::COMPLETED)
{
LOG_WARNING(log, "Found a concurrent restore: {}, current restore: {}", existing_restore_uuid, toString(restore_uuid));
return true;
}
}
zk->createIfNotExists(path, "");

View File

@ -59,6 +59,7 @@ private:
const String current_host;
const size_t current_host_index;
const bool is_internal;
Poco::Logger * const log;
std::optional<BackupCoordinationStageSync> stage_sync;