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:
Vitaly Baranov 2024-02-23 12:42:55 +01:00 committed by GitHub
commit db119eed49
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
7 changed files with 333 additions and 349 deletions

View File

@ -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.

View File

@ -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)

View File

@ -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();

View File

@ -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)

View File

@ -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;

View File

@ -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>;

View File

@ -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;