mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
Add logging for concurrency checks for backups.
This commit is contained in:
parent
d70e3d0ddf
commit
3f4aadfe7d
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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,8 +665,11 @@ 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, "");
|
||||
auto code = zk->trySet(backup_stage_path, Stage::SCHEDULED_TO_START, stat.version);
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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,8 +198,11 @@ 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, "");
|
||||
auto code = zk->trySet(path, Stage::SCHEDULED_TO_START, stat.version);
|
||||
|
@ -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;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user