mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Merge pull request #55132 from ClickHouse/fix_deadlock_backups
Fix deadlock in backups
This commit is contained in:
commit
180562adfb
@ -218,9 +218,36 @@ namespace
|
||||
}
|
||||
|
||||
|
||||
BackupsWorker::BackupsWorker(ContextPtr global_context, size_t num_backup_threads, size_t num_restore_threads, bool allow_concurrent_backups_, bool allow_concurrent_restores_)
|
||||
: backups_thread_pool(std::make_unique<ThreadPool>(CurrentMetrics::BackupsThreads, CurrentMetrics::BackupsThreadsActive, num_backup_threads, /* max_free_threads = */ 0, num_backup_threads))
|
||||
, restores_thread_pool(std::make_unique<ThreadPool>(CurrentMetrics::RestoreThreads, CurrentMetrics::RestoreThreadsActive, num_restore_threads, /* max_free_threads = */ 0, num_restore_threads))
|
||||
BackupsWorker::BackupsWorker(
|
||||
ContextPtr global_context,
|
||||
size_t num_backup_threads,
|
||||
size_t num_restore_threads,
|
||||
bool allow_concurrent_backups_,
|
||||
bool allow_concurrent_restores_)
|
||||
: backups_thread_pool(std::make_unique<ThreadPool>(
|
||||
CurrentMetrics::BackupsThreads,
|
||||
CurrentMetrics::BackupsThreadsActive,
|
||||
num_backup_threads,
|
||||
/* max_free_threads = */ 0,
|
||||
num_backup_threads))
|
||||
, restores_thread_pool(std::make_unique<ThreadPool>(
|
||||
CurrentMetrics::RestoreThreads,
|
||||
CurrentMetrics::RestoreThreadsActive,
|
||||
num_restore_threads,
|
||||
/* max_free_threads = */ 0,
|
||||
num_restore_threads))
|
||||
, backup_async_executor_pool(std::make_unique<ThreadPool>(
|
||||
CurrentMetrics::BackupsThreads,
|
||||
CurrentMetrics::BackupsThreadsActive,
|
||||
num_backup_threads,
|
||||
num_backup_threads,
|
||||
num_backup_threads))
|
||||
, restore_async_executor_pool(std::make_unique<ThreadPool>(
|
||||
CurrentMetrics::RestoreThreads,
|
||||
CurrentMetrics::RestoreThreadsActive,
|
||||
num_restore_threads,
|
||||
num_restore_threads,
|
||||
num_restore_threads))
|
||||
, log(&Poco::Logger::get("BackupsWorker"))
|
||||
, allow_concurrent_backups(allow_concurrent_backups_)
|
||||
, allow_concurrent_restores(allow_concurrent_restores_)
|
||||
@ -286,8 +313,16 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context
|
||||
|
||||
if (backup_settings.async)
|
||||
{
|
||||
backups_thread_pool->scheduleOrThrowOnError(
|
||||
[this, backup_query, backup_id, backup_name_for_logging, backup_info, backup_settings, backup_coordination, context_in_use, mutable_context]
|
||||
backup_async_executor_pool->scheduleOrThrowOnError(
|
||||
[this,
|
||||
backup_query,
|
||||
backup_id,
|
||||
backup_name_for_logging,
|
||||
backup_info,
|
||||
backup_settings,
|
||||
backup_coordination,
|
||||
context_in_use,
|
||||
mutable_context]
|
||||
{
|
||||
doBackup(
|
||||
backup_query,
|
||||
@ -631,18 +666,25 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
|
||||
|
||||
if (restore_settings.async)
|
||||
{
|
||||
restores_thread_pool->scheduleOrThrowOnError(
|
||||
[this, restore_query, restore_id, backup_name_for_logging, backup_info, restore_settings, restore_coordination, context_in_use]
|
||||
{
|
||||
doRestore(
|
||||
restore_query,
|
||||
restore_id,
|
||||
backup_name_for_logging,
|
||||
backup_info,
|
||||
restore_settings,
|
||||
restore_coordination,
|
||||
context_in_use,
|
||||
/* called_async= */ true);
|
||||
restore_async_executor_pool->scheduleOrThrowOnError(
|
||||
[this,
|
||||
restore_query,
|
||||
restore_id,
|
||||
backup_name_for_logging,
|
||||
backup_info,
|
||||
restore_settings,
|
||||
restore_coordination,
|
||||
context_in_use]
|
||||
{
|
||||
doRestore(
|
||||
restore_query,
|
||||
restore_id,
|
||||
backup_name_for_logging,
|
||||
backup_info,
|
||||
restore_settings,
|
||||
restore_coordination,
|
||||
context_in_use,
|
||||
/* called_async= */ true);
|
||||
});
|
||||
}
|
||||
else
|
||||
@ -1009,6 +1051,8 @@ void BackupsWorker::shutdown()
|
||||
|
||||
backups_thread_pool->wait();
|
||||
restores_thread_pool->wait();
|
||||
backup_async_executor_pool->wait();
|
||||
restore_async_executor_pool->wait();
|
||||
|
||||
if (has_active_backups_and_restores)
|
||||
LOG_INFO(log, "All backup and restore tasks have finished");
|
||||
|
@ -91,6 +91,9 @@ private:
|
||||
std::unique_ptr<ThreadPool> backups_thread_pool;
|
||||
std::unique_ptr<ThreadPool> restores_thread_pool;
|
||||
|
||||
std::unique_ptr<ThreadPool> backup_async_executor_pool;
|
||||
std::unique_ptr<ThreadPool> restore_async_executor_pool;
|
||||
|
||||
std::unordered_map<BackupOperationID, BackupOperationInfo> infos;
|
||||
std::shared_ptr<BackupLog> backup_log;
|
||||
std::condition_variable status_changed;
|
||||
|
@ -11,4 +11,6 @@
|
||||
<upload_part_size_multiply_factor>2</upload_part_size_multiply_factor>
|
||||
</multipart>
|
||||
</s3>
|
||||
<backup_threads>1</backup_threads>
|
||||
<restore_threads>1</restore_threads>
|
||||
</clickhouse>
|
||||
|
Loading…
Reference in New Issue
Block a user