mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #60002 from vitlibar/add-config-setting-remove-backup-files-after-failure
Add new config setting "backups.remove_backup_files_after_failure"
This commit is contained in:
commit
db119eed49
@ -1569,6 +1569,11 @@
|
||||
|
||||
<backups>
|
||||
<allowed_path>backups</allowed_path>
|
||||
|
||||
<!-- If the BACKUP command fails and this setting is true then the files
|
||||
copied before the failure will be removed automatically.
|
||||
-->
|
||||
<remove_backup_files_after_failure>true</remove_backup_files_after_failure>
|
||||
</backups>
|
||||
|
||||
<!-- This allows to disable exposing addresses in stack traces for security reasons.
|
||||
|
@ -144,6 +144,13 @@ BackupImpl::BackupImpl(
|
||||
|
||||
BackupImpl::~BackupImpl()
|
||||
{
|
||||
if ((open_mode == OpenMode::WRITE) && !is_internal_backup && !writing_finalized && !std::uncaught_exceptions() && !std::current_exception())
|
||||
{
|
||||
/// It is suspicious to destroy BackupImpl without finalization while writing a backup when there is no exception.
|
||||
LOG_ERROR(log, "BackupImpl is not finalized when destructor is called. Stack trace: {}", StackTrace().toString());
|
||||
chassert(false && "BackupImpl is not finalized when destructor is called.");
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
close();
|
||||
@ -195,10 +202,6 @@ void BackupImpl::close()
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
closeArchive(/* finalize= */ false);
|
||||
|
||||
if (!is_internal_backup && writer && !writing_finalized)
|
||||
removeAllFilesAfterFailure();
|
||||
|
||||
writer.reset();
|
||||
reader.reset();
|
||||
coordination.reset();
|
||||
@ -1005,14 +1008,18 @@ void BackupImpl::setCompressedSize()
|
||||
}
|
||||
|
||||
|
||||
void BackupImpl::removeAllFilesAfterFailure()
|
||||
void BackupImpl::tryRemoveAllFiles()
|
||||
{
|
||||
if (open_mode != OpenMode::WRITE)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Backup is not opened for writing");
|
||||
|
||||
if (is_internal_backup)
|
||||
return; /// Let the initiator remove unnecessary files.
|
||||
return;
|
||||
|
||||
try
|
||||
{
|
||||
LOG_INFO(log, "Removing all files of backup {} after failure", backup_name_for_logging);
|
||||
LOG_INFO(log, "Removing all files of backup {}", backup_name_for_logging);
|
||||
closeArchive(/* finalize= */ false);
|
||||
|
||||
Strings files_to_remove;
|
||||
if (use_archive)
|
||||
|
@ -81,8 +81,9 @@ public:
|
||||
size_t copyFileToDisk(const String & file_name, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) const override;
|
||||
size_t copyFileToDisk(const SizeAndChecksum & size_and_checksum, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) const override;
|
||||
void writeFile(const BackupFileInfo & info, BackupEntryPtr entry) override;
|
||||
void finalizeWriting() override;
|
||||
bool supportsWritingInMultipleThreads() const override { return !use_archive; }
|
||||
void finalizeWriting() override;
|
||||
void tryRemoveAllFiles() override;
|
||||
|
||||
private:
|
||||
void open();
|
||||
@ -107,8 +108,6 @@ private:
|
||||
bool checkLockFile(bool throw_if_failed) const;
|
||||
void removeLockFile();
|
||||
|
||||
void removeAllFilesAfterFailure();
|
||||
|
||||
/// Calculates and sets `compressed_size`.
|
||||
void setCompressedSize();
|
||||
|
||||
|
@ -375,11 +375,12 @@ private:
|
||||
};
|
||||
|
||||
|
||||
BackupsWorker::BackupsWorker(ContextMutablePtr global_context, size_t num_backup_threads, size_t num_restore_threads, bool allow_concurrent_backups_, bool allow_concurrent_restores_, bool test_inject_sleep_)
|
||||
BackupsWorker::BackupsWorker(ContextMutablePtr global_context, size_t num_backup_threads, size_t num_restore_threads)
|
||||
: thread_pools(std::make_unique<ThreadPools>(num_backup_threads, num_restore_threads))
|
||||
, allow_concurrent_backups(allow_concurrent_backups_)
|
||||
, allow_concurrent_restores(allow_concurrent_restores_)
|
||||
, test_inject_sleep(test_inject_sleep_)
|
||||
, allow_concurrent_backups(global_context->getConfigRef().getBool("backups.allow_concurrent_backups", true))
|
||||
, allow_concurrent_restores(global_context->getConfigRef().getBool("backups.allow_concurrent_restores", true))
|
||||
, remove_backup_files_after_failure(global_context->getConfigRef().getBool("backups.remove_backup_files_after_failure", true))
|
||||
, test_inject_sleep(global_context->getConfigRef().getBool("backups.test_inject_sleep", false))
|
||||
, log(getLogger("BackupsWorker"))
|
||||
, backup_log(global_context->getBackupLog())
|
||||
, process_list(global_context->getProcessList())
|
||||
@ -411,6 +412,9 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context
|
||||
auto backup_query = std::static_pointer_cast<ASTBackupQuery>(query->clone());
|
||||
auto backup_settings = BackupSettings::fromBackupQuery(*backup_query);
|
||||
|
||||
auto backup_info = BackupInfo::fromAST(*backup_query->backup_name);
|
||||
String backup_name_for_logging = backup_info.toStringForLogging();
|
||||
|
||||
if (!backup_settings.backup_uuid)
|
||||
backup_settings.backup_uuid = UUIDHelpers::generateV4();
|
||||
|
||||
@ -424,22 +428,28 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context
|
||||
backup_id = toString(*backup_settings.backup_uuid);
|
||||
|
||||
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(context, backup_settings, /* remote= */ true);
|
||||
}
|
||||
BackupMutablePtr backup;
|
||||
|
||||
auto backup_info = BackupInfo::fromAST(*backup_query->backup_name);
|
||||
String backup_name_for_logging = backup_info.toStringForLogging();
|
||||
/// Called in exception handlers below. This lambda function can be called on a separate thread, so it can't capture local variables by reference.
|
||||
auto on_exception = [this](BackupMutablePtr & backup_, const OperationID & backup_id_, const String & backup_name_for_logging_,
|
||||
const BackupSettings & backup_settings_, const std::shared_ptr<IBackupCoordination> & backup_coordination_)
|
||||
{
|
||||
/// Something bad happened, the backup has not built.
|
||||
tryLogCurrentException(log, fmt::format("Failed to make {} {}", (backup_settings_.internal ? "internal backup" : "backup"), backup_name_for_logging_));
|
||||
setStatusSafe(backup_id_, getBackupStatusFromCurrentException());
|
||||
sendCurrentExceptionToCoordination(backup_coordination_);
|
||||
|
||||
if (backup_ && remove_backup_files_after_failure)
|
||||
backup_->tryRemoveAllFiles();
|
||||
backup_.reset();
|
||||
};
|
||||
|
||||
try
|
||||
{
|
||||
String base_backup_name;
|
||||
if (backup_settings.base_backup_info)
|
||||
base_backup_name = backup_settings.base_backup_info->toStringForLogging();
|
||||
|
||||
try
|
||||
{
|
||||
addInfo(backup_id,
|
||||
backup_name_for_logging,
|
||||
base_backup_name,
|
||||
@ -448,6 +458,14 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context
|
||||
context->getProcessListElement(),
|
||||
BackupStatus::CREATING_BACKUP);
|
||||
|
||||
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(context, backup_settings, /* remote= */ true);
|
||||
}
|
||||
|
||||
/// Prepare context to use.
|
||||
ContextPtr context_in_use = context;
|
||||
ContextMutablePtr mutable_context;
|
||||
@ -468,7 +486,7 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context
|
||||
/// process_list_element_holder is used to make an element in ProcessList live while BACKUP is working asynchronously.
|
||||
auto process_list_element = context_in_use->getProcessListElement();
|
||||
|
||||
thread_pool.scheduleOrThrowOnError(
|
||||
scheduleFromThreadPool<void>(
|
||||
[this,
|
||||
backup_query,
|
||||
backup_id,
|
||||
@ -478,10 +496,14 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context
|
||||
backup_coordination,
|
||||
context_in_use,
|
||||
mutable_context,
|
||||
thread_group = CurrentThread::getGroup(),
|
||||
on_exception,
|
||||
process_list_element_holder = process_list_element ? process_list_element->getProcessListEntry() : nullptr]
|
||||
{
|
||||
BackupMutablePtr backup_async;
|
||||
try
|
||||
{
|
||||
doBackup(
|
||||
backup_async,
|
||||
backup_query,
|
||||
backup_id,
|
||||
backup_name_for_logging,
|
||||
@ -489,14 +511,19 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context
|
||||
backup_settings,
|
||||
backup_coordination,
|
||||
context_in_use,
|
||||
mutable_context,
|
||||
thread_group,
|
||||
/* called_async= */ true);
|
||||
});
|
||||
mutable_context);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
on_exception(backup_async, backup_id, backup_name_for_logging, backup_settings, backup_coordination);
|
||||
}
|
||||
},
|
||||
thread_pool, "BackupWorker");
|
||||
}
|
||||
else
|
||||
{
|
||||
doBackup(
|
||||
backup,
|
||||
backup_query,
|
||||
backup_id,
|
||||
backup_name_for_logging,
|
||||
@ -504,25 +531,21 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context
|
||||
backup_settings,
|
||||
backup_coordination,
|
||||
context_in_use,
|
||||
mutable_context,
|
||||
nullptr,
|
||||
/* called_async= */ false);
|
||||
mutable_context);
|
||||
}
|
||||
|
||||
return backup_id;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, fmt::format("Failed to start {} {}", (backup_settings.internal ? "internal backup" : "backup"), backup_name_for_logging));
|
||||
/// Something bad happened, the backup has not built.
|
||||
setStatusSafe(backup_id, getBackupStatusFromCurrentException());
|
||||
sendCurrentExceptionToCoordination(backup_coordination);
|
||||
on_exception(backup, backup_id, backup_name_for_logging, backup_settings, backup_coordination);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void BackupsWorker::doBackup(
|
||||
BackupMutablePtr & backup,
|
||||
const std::shared_ptr<ASTBackupQuery> & backup_query,
|
||||
const OperationID & backup_id,
|
||||
const String & backup_name_for_logging,
|
||||
@ -530,24 +553,10 @@ void BackupsWorker::doBackup(
|
||||
BackupSettings backup_settings,
|
||||
std::shared_ptr<IBackupCoordination> backup_coordination,
|
||||
const ContextPtr & context,
|
||||
ContextMutablePtr mutable_context,
|
||||
ThreadGroupPtr thread_group,
|
||||
bool called_async)
|
||||
ContextMutablePtr mutable_context)
|
||||
{
|
||||
SCOPE_EXIT_SAFE(
|
||||
if (called_async && thread_group)
|
||||
CurrentThread::detachFromGroupIfNotDetached();
|
||||
);
|
||||
|
||||
try
|
||||
{
|
||||
if (called_async && thread_group)
|
||||
CurrentThread::attachToGroup(thread_group);
|
||||
if (called_async)
|
||||
setThreadName("BackupWorker");
|
||||
|
||||
bool on_cluster = !backup_query->cluster.empty();
|
||||
assert(mutable_context || (!on_cluster && !called_async));
|
||||
assert(!on_cluster || mutable_context);
|
||||
|
||||
/// Checks access rights if this is not ON CLUSTER query.
|
||||
/// (If this is ON CLUSTER query executeDDLQueryOnCluster() will check access rights later.)
|
||||
@ -588,7 +597,7 @@ void BackupsWorker::doBackup(
|
||||
backup_create_params.use_same_s3_credentials_for_base_backup = backup_settings.use_same_s3_credentials_for_base_backup;
|
||||
backup_create_params.read_settings = getReadSettingsForBackup(context, backup_settings);
|
||||
backup_create_params.write_settings = getWriteSettingsForBackup(context);
|
||||
BackupMutablePtr backup = BackupFactory::instance().createBackup(backup_create_params);
|
||||
backup = BackupFactory::instance().createBackup(backup_create_params);
|
||||
|
||||
/// Write the backup.
|
||||
if (on_cluster)
|
||||
@ -623,6 +632,9 @@ void BackupsWorker::doBackup(
|
||||
}
|
||||
|
||||
/// Write the backup entries to the backup.
|
||||
chassert(backup);
|
||||
chassert(backup_coordination);
|
||||
chassert(context);
|
||||
buildFileInfosForBackupEntries(backup, backup_entries, backup_create_params.read_settings, backup_coordination, context->getProcessListElement());
|
||||
writeBackupEntries(backup, std::move(backup_entries), backup_id, backup_coordination, backup_settings.internal, context->getProcessListElement());
|
||||
|
||||
@ -656,22 +668,6 @@ void BackupsWorker::doBackup(
|
||||
/// NOTE: setStatus is called after setNumFilesAndSize in order to have actual information in a backup log record
|
||||
setStatus(backup_id, BackupStatus::BACKUP_CREATED);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Something bad happened, the backup has not built.
|
||||
if (called_async)
|
||||
{
|
||||
tryLogCurrentException(log, fmt::format("Failed to make {} {}", (backup_settings.internal ? "internal backup" : "backup"), backup_name_for_logging));
|
||||
setStatusSafe(backup_id, getBackupStatusFromCurrentException());
|
||||
sendCurrentExceptionToCoordination(backup_coordination);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// setStatus() and sendCurrentExceptionToCoordination() will be called by startMakingBackup().
|
||||
throw;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void BackupsWorker::buildFileInfosForBackupEntries(const BackupPtr & backup, const BackupEntries & backup_entries, const ReadSettings & read_settings, std::shared_ptr<IBackupCoordination> backup_coordination, QueryStatusPtr process_list_element)
|
||||
@ -800,6 +796,9 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
|
||||
auto restore_query = std::static_pointer_cast<ASTBackupQuery>(query->clone());
|
||||
auto restore_settings = RestoreSettings::fromRestoreQuery(*restore_query);
|
||||
|
||||
auto backup_info = BackupInfo::fromAST(*restore_query->backup_name);
|
||||
String backup_name_for_logging = backup_info.toStringForLogging();
|
||||
|
||||
if (!restore_settings.restore_uuid)
|
||||
restore_settings.restore_uuid = UUIDHelpers::generateV4();
|
||||
|
||||
@ -813,18 +812,19 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
|
||||
restore_id = toString(*restore_settings.restore_uuid);
|
||||
|
||||
std::shared_ptr<IRestoreCoordination> restore_coordination;
|
||||
if (restore_settings.internal)
|
||||
|
||||
/// Called in exception handlers below. This lambda function can be called on a separate thread, so it can't capture local variables by reference.
|
||||
auto on_exception = [this](const OperationID & restore_id_, const String & backup_name_for_logging_,
|
||||
const RestoreSettings & restore_settings_, const std::shared_ptr<IRestoreCoordination> & restore_coordination_)
|
||||
{
|
||||
/// 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(context, restore_settings, /* remote= */ true);
|
||||
}
|
||||
/// Something bad happened, some data were not restored.
|
||||
tryLogCurrentException(log, fmt::format("Failed to restore from {} {}", (restore_settings_.internal ? "internal backup" : "backup"), backup_name_for_logging_));
|
||||
setStatusSafe(restore_id_, getRestoreStatusFromCurrentException());
|
||||
sendCurrentExceptionToCoordination(restore_coordination_);
|
||||
};
|
||||
|
||||
try
|
||||
{
|
||||
auto backup_info = BackupInfo::fromAST(*restore_query->backup_name);
|
||||
String backup_name_for_logging = backup_info.toStringForLogging();
|
||||
String base_backup_name;
|
||||
if (restore_settings.base_backup_info)
|
||||
base_backup_name = restore_settings.base_backup_info->toStringForLogging();
|
||||
@ -837,6 +837,14 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
|
||||
context->getProcessListElement(),
|
||||
BackupStatus::RESTORING);
|
||||
|
||||
if (restore_settings.internal)
|
||||
{
|
||||
/// 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(context, restore_settings, /* remote= */ true);
|
||||
}
|
||||
|
||||
/// Prepare context to use.
|
||||
ContextMutablePtr context_in_use = context;
|
||||
bool on_cluster = !restore_query->cluster.empty();
|
||||
@ -856,7 +864,7 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
|
||||
/// process_list_element_holder is used to make an element in ProcessList live while RESTORE is working asynchronously.
|
||||
auto process_list_element = context_in_use->getProcessListElement();
|
||||
|
||||
thread_pool.scheduleOrThrowOnError(
|
||||
scheduleFromThreadPool<void>(
|
||||
[this,
|
||||
restore_query,
|
||||
restore_id,
|
||||
@ -865,8 +873,10 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
|
||||
restore_settings,
|
||||
restore_coordination,
|
||||
context_in_use,
|
||||
thread_group = CurrentThread::getGroup(),
|
||||
on_exception,
|
||||
process_list_element_holder = process_list_element ? process_list_element->getProcessListEntry() : nullptr]
|
||||
{
|
||||
try
|
||||
{
|
||||
doRestore(
|
||||
restore_query,
|
||||
@ -875,10 +885,15 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
|
||||
backup_info,
|
||||
restore_settings,
|
||||
restore_coordination,
|
||||
context_in_use,
|
||||
thread_group,
|
||||
/* called_async= */ true);
|
||||
});
|
||||
context_in_use);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
on_exception(restore_id, backup_name_for_logging, restore_settings, restore_coordination);
|
||||
}
|
||||
},
|
||||
thread_pool,
|
||||
"RestoreWorker");
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -889,18 +904,14 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
|
||||
backup_info,
|
||||
restore_settings,
|
||||
restore_coordination,
|
||||
context_in_use,
|
||||
nullptr,
|
||||
/* called_async= */ false);
|
||||
context_in_use);
|
||||
}
|
||||
|
||||
return restore_id;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Something bad happened, the backup has not built.
|
||||
setStatusSafe(restore_id, getRestoreStatusFromCurrentException());
|
||||
sendCurrentExceptionToCoordination(restore_coordination);
|
||||
on_exception(restore_id, backup_name_for_logging, restore_settings, restore_coordination);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
@ -913,22 +924,8 @@ void BackupsWorker::doRestore(
|
||||
const BackupInfo & backup_info,
|
||||
RestoreSettings restore_settings,
|
||||
std::shared_ptr<IRestoreCoordination> restore_coordination,
|
||||
ContextMutablePtr context,
|
||||
ThreadGroupPtr thread_group,
|
||||
bool called_async)
|
||||
ContextMutablePtr context)
|
||||
{
|
||||
SCOPE_EXIT_SAFE(
|
||||
if (called_async && thread_group)
|
||||
CurrentThread::detachFromGroupIfNotDetached();
|
||||
);
|
||||
|
||||
try
|
||||
{
|
||||
if (called_async && thread_group)
|
||||
CurrentThread::attachToGroup(thread_group);
|
||||
if (called_async)
|
||||
setThreadName("RestoreWorker");
|
||||
|
||||
/// Open the backup for reading.
|
||||
BackupFactory::CreateParams backup_open_params;
|
||||
backup_open_params.open_mode = IBackup::OpenMode::READ;
|
||||
@ -1025,22 +1022,6 @@ void BackupsWorker::doRestore(
|
||||
LOG_INFO(log, "Restored from {} {} successfully", (restore_settings.internal ? "internal backup" : "backup"), backup_name_for_logging);
|
||||
setStatus(restore_id, BackupStatus::RESTORED);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Something bad happened, the backup has not built.
|
||||
if (called_async)
|
||||
{
|
||||
tryLogCurrentException(log, fmt::format("Failed to restore from {} {}", (restore_settings.internal ? "internal backup" : "backup"), backup_name_for_logging));
|
||||
setStatusSafe(restore_id, getRestoreStatusFromCurrentException());
|
||||
sendCurrentExceptionToCoordination(restore_coordination);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// setStatus() and sendCurrentExceptionToCoordination() will be called by startRestoring().
|
||||
throw;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void BackupsWorker::restoreTablesData(const OperationID & restore_id, BackupPtr backup, DataRestoreTasks && tasks, ThreadPool & thread_pool, QueryStatusPtr process_list_element)
|
||||
|
@ -38,20 +38,15 @@ class ProcessList;
|
||||
class BackupsWorker
|
||||
{
|
||||
public:
|
||||
BackupsWorker(
|
||||
ContextMutablePtr global_context,
|
||||
size_t num_backup_threads,
|
||||
size_t num_restore_threads,
|
||||
bool allow_concurrent_backups_,
|
||||
bool allow_concurrent_restores_,
|
||||
bool test_inject_sleep_);
|
||||
|
||||
BackupsWorker(ContextMutablePtr global_context, size_t num_backup_threads, size_t num_restore_threads);
|
||||
~BackupsWorker();
|
||||
|
||||
/// Waits until all tasks have been completed.
|
||||
void shutdown();
|
||||
|
||||
/// Starts executing a BACKUP or RESTORE query. Returns ID of the operation.
|
||||
/// For asynchronous operations the function throws no exceptions on failure usually,
|
||||
/// call getInfo() on a returned operation id to check for errors.
|
||||
BackupOperationID start(const ASTPtr & backup_or_restore_query, ContextMutablePtr context);
|
||||
|
||||
/// Waits until the specified backup or restore operation finishes or stops.
|
||||
@ -75,6 +70,7 @@ private:
|
||||
BackupOperationID startMakingBackup(const ASTPtr & query, const ContextPtr & context);
|
||||
|
||||
void doBackup(
|
||||
BackupMutablePtr & backup,
|
||||
const std::shared_ptr<ASTBackupQuery> & backup_query,
|
||||
const BackupOperationID & backup_id,
|
||||
const String & backup_name_for_logging,
|
||||
@ -82,9 +78,7 @@ private:
|
||||
BackupSettings backup_settings,
|
||||
std::shared_ptr<IBackupCoordination> backup_coordination,
|
||||
const ContextPtr & context,
|
||||
ContextMutablePtr mutable_context,
|
||||
ThreadGroupPtr thread_group,
|
||||
bool called_async);
|
||||
ContextMutablePtr mutable_context);
|
||||
|
||||
/// Builds file infos for specified backup entries.
|
||||
void buildFileInfosForBackupEntries(const BackupPtr & backup, const BackupEntries & backup_entries, const ReadSettings & read_settings, std::shared_ptr<IBackupCoordination> backup_coordination, QueryStatusPtr process_list_element);
|
||||
@ -101,9 +95,7 @@ private:
|
||||
const BackupInfo & backup_info,
|
||||
RestoreSettings restore_settings,
|
||||
std::shared_ptr<IRestoreCoordination> restore_coordination,
|
||||
ContextMutablePtr context,
|
||||
ThreadGroupPtr thread_group,
|
||||
bool called_async);
|
||||
ContextMutablePtr context);
|
||||
|
||||
/// Run data restoring tasks which insert data to tables.
|
||||
void restoreTablesData(const BackupOperationID & restore_id, BackupPtr backup, DataRestoreTasks && tasks, ThreadPool & thread_pool, QueryStatusPtr process_list_element);
|
||||
@ -126,6 +118,7 @@ private:
|
||||
|
||||
const bool allow_concurrent_backups;
|
||||
const bool allow_concurrent_restores;
|
||||
const bool remove_backup_files_after_failure;
|
||||
const bool test_inject_sleep;
|
||||
|
||||
LoggerPtr log;
|
||||
|
@ -117,11 +117,14 @@ public:
|
||||
/// Puts a new entry to the backup.
|
||||
virtual void writeFile(const BackupFileInfo & file_info, BackupEntryPtr entry) = 0;
|
||||
|
||||
/// Whether it's possible to add new entries to the backup in multiple threads.
|
||||
virtual bool supportsWritingInMultipleThreads() const = 0;
|
||||
|
||||
/// Finalizes writing the backup, should be called after all entries have been successfully written.
|
||||
virtual void finalizeWriting() = 0;
|
||||
|
||||
/// Whether it's possible to add new entries to the backup in multiple threads.
|
||||
virtual bool supportsWritingInMultipleThreads() const = 0;
|
||||
/// Try to remove all files copied to the backup. Used after an exception or it the backup was cancelled.
|
||||
virtual void tryRemoveAllFiles() = 0;
|
||||
};
|
||||
|
||||
using BackupPtr = std::shared_ptr<const IBackup>;
|
||||
|
@ -2590,15 +2590,11 @@ BackupsWorker & Context::getBackupsWorker() const
|
||||
{
|
||||
callOnce(shared->backups_worker_initialized, [&] {
|
||||
const auto & config = getConfigRef();
|
||||
const bool allow_concurrent_backups = config.getBool("backups.allow_concurrent_backups", true);
|
||||
const bool allow_concurrent_restores = config.getBool("backups.allow_concurrent_restores", true);
|
||||
const bool test_inject_sleep = config.getBool("backups.test_inject_sleep", false);
|
||||
|
||||
const auto & settings_ref = getSettingsRef();
|
||||
UInt64 backup_threads = config.getUInt64("backup_threads", settings_ref.backup_threads);
|
||||
UInt64 restore_threads = config.getUInt64("restore_threads", settings_ref.restore_threads);
|
||||
|
||||
shared->backups_worker.emplace(getGlobalContext(), backup_threads, restore_threads, allow_concurrent_backups, allow_concurrent_restores, test_inject_sleep);
|
||||
shared->backups_worker.emplace(getGlobalContext(), backup_threads, restore_threads);
|
||||
});
|
||||
|
||||
return *shared->backups_worker;
|
||||
|
Loading…
Reference in New Issue
Block a user