mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Remove column "internal" from system.backups
This commit is contained in:
parent
16a60b5e93
commit
afd0982187
@ -115,7 +115,7 @@ BackupsWorker::BackupsWorker(size_t num_backup_threads, size_t num_restore_threa
|
||||
}
|
||||
|
||||
|
||||
std::pair<UUID, bool> BackupsWorker::start(const ASTPtr & backup_or_restore_query, ContextMutablePtr context)
|
||||
UUID BackupsWorker::start(const ASTPtr & backup_or_restore_query, ContextMutablePtr context)
|
||||
{
|
||||
const ASTBackupQuery & backup_query = typeid_cast<const ASTBackupQuery &>(*backup_or_restore_query);
|
||||
if (backup_query.kind == ASTBackupQuery::Kind::BACKUP)
|
||||
@ -125,7 +125,7 @@ std::pair<UUID, bool> BackupsWorker::start(const ASTPtr & backup_or_restore_quer
|
||||
}
|
||||
|
||||
|
||||
std::pair<UUID, bool> BackupsWorker::startMakingBackup(const ASTPtr & query, const ContextPtr & context)
|
||||
UUID BackupsWorker::startMakingBackup(const ASTPtr & query, const ContextPtr & context)
|
||||
{
|
||||
auto backup_query = std::static_pointer_cast<ASTBackupQuery>(query->clone());
|
||||
auto backup_settings = BackupSettings::fromBackupQuery(*backup_query);
|
||||
@ -147,7 +147,9 @@ std::pair<UUID, bool> BackupsWorker::startMakingBackup(const ASTPtr & query, con
|
||||
try
|
||||
{
|
||||
auto backup_info = BackupInfo::fromAST(*backup_query->backup_name);
|
||||
addInfo(backup_uuid, backup_settings.internal, backup_info.toString(), BackupStatus::MAKING_BACKUP);
|
||||
|
||||
if (!backup_settings.internal)
|
||||
addInfo(backup_uuid, backup_info.toString(), BackupStatus::MAKING_BACKUP);
|
||||
|
||||
/// Prepare context to use.
|
||||
ContextPtr context_in_use = context;
|
||||
@ -188,12 +190,13 @@ std::pair<UUID, bool> BackupsWorker::startMakingBackup(const ASTPtr & query, con
|
||||
/* called_async= */ false);
|
||||
}
|
||||
|
||||
return {backup_uuid, backup_settings.internal};
|
||||
return backup_uuid;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Something bad happened, the backup has not built.
|
||||
setStatus(backup_uuid, backup_settings.internal, BackupStatus::FAILED_TO_BACKUP);
|
||||
if (!backup_settings.internal)
|
||||
setStatus(backup_uuid, BackupStatus::FAILED_TO_BACKUP);
|
||||
sendCurrentExceptionToCoordination(backup_coordination, backup_settings.host_id);
|
||||
throw;
|
||||
}
|
||||
@ -305,7 +308,8 @@ void BackupsWorker::doBackup(
|
||||
backup.reset();
|
||||
|
||||
LOG_INFO(log, "{} {} was created successfully", (backup_settings.internal ? "Internal backup" : "Backup"), backup_info.toString());
|
||||
setStatus(backup_uuid, backup_settings.internal, BackupStatus::BACKUP_COMPLETE);
|
||||
if (!backup_settings.internal)
|
||||
setStatus(backup_uuid, BackupStatus::BACKUP_COMPLETE);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -313,7 +317,8 @@ void BackupsWorker::doBackup(
|
||||
if (called_async)
|
||||
{
|
||||
tryLogCurrentException(log, fmt::format("Failed to make {} {}", (backup_settings.internal ? "internal backup" : "backup"), backup_info.toString()));
|
||||
setStatus(backup_uuid, backup_settings.internal, BackupStatus::FAILED_TO_BACKUP);
|
||||
if (!backup_settings.internal)
|
||||
setStatus(backup_uuid, BackupStatus::FAILED_TO_BACKUP);
|
||||
sendCurrentExceptionToCoordination(backup_coordination, backup_settings.host_id);
|
||||
}
|
||||
else
|
||||
@ -325,7 +330,7 @@ void BackupsWorker::doBackup(
|
||||
}
|
||||
|
||||
|
||||
std::pair<UUID, bool> BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePtr context)
|
||||
UUID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePtr context)
|
||||
{
|
||||
auto restore_query = std::static_pointer_cast<ASTBackupQuery>(query->clone());
|
||||
auto restore_settings = RestoreSettings::fromRestoreQuery(*restore_query);
|
||||
@ -344,7 +349,8 @@ std::pair<UUID, bool> BackupsWorker::startRestoring(const ASTPtr & query, Contex
|
||||
try
|
||||
{
|
||||
auto backup_info = BackupInfo::fromAST(*restore_query->backup_name);
|
||||
addInfo(restore_uuid, restore_settings.internal, backup_info.toString(), BackupStatus::RESTORING);
|
||||
if (!restore_settings.internal)
|
||||
addInfo(restore_uuid, backup_info.toString(), BackupStatus::RESTORING);
|
||||
|
||||
/// Prepare context to use.
|
||||
ContextMutablePtr context_in_use = context;
|
||||
@ -382,12 +388,13 @@ std::pair<UUID, bool> BackupsWorker::startRestoring(const ASTPtr & query, Contex
|
||||
/* called_async= */ false);
|
||||
}
|
||||
|
||||
return {restore_uuid, restore_settings.internal};
|
||||
return restore_uuid;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Something bad happened, the backup has not built.
|
||||
setStatus(restore_uuid, restore_settings.internal, BackupStatus::FAILED_TO_RESTORE);
|
||||
if (!restore_settings.internal)
|
||||
setStatus(restore_uuid, BackupStatus::FAILED_TO_RESTORE);
|
||||
sendCurrentExceptionToCoordination(restore_coordination, restore_settings.host_id);
|
||||
throw;
|
||||
}
|
||||
@ -500,7 +507,8 @@ void BackupsWorker::doRestore(
|
||||
}
|
||||
|
||||
LOG_INFO(log, "Restored from {} {} successfully", (restore_settings.internal ? "internal backup" : "backup"), backup_info.toString());
|
||||
setStatus(restore_uuid, restore_settings.internal, BackupStatus::RESTORED);
|
||||
if (!restore_settings.internal)
|
||||
setStatus(restore_uuid, BackupStatus::RESTORED);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -508,7 +516,8 @@ void BackupsWorker::doRestore(
|
||||
if (called_async)
|
||||
{
|
||||
tryLogCurrentException(log, fmt::format("Failed to restore from {} {}", (restore_settings.internal ? "internal backup" : "backup"), backup_info.toString()));
|
||||
setStatus(restore_uuid, restore_settings.internal, BackupStatus::FAILED_TO_RESTORE);
|
||||
if (!restore_settings.internal)
|
||||
setStatus(restore_uuid, BackupStatus::FAILED_TO_RESTORE);
|
||||
sendCurrentExceptionToCoordination(restore_coordination, restore_settings.host_id);
|
||||
}
|
||||
else
|
||||
@ -520,31 +529,30 @@ void BackupsWorker::doRestore(
|
||||
}
|
||||
|
||||
|
||||
void BackupsWorker::addInfo(const UUID & uuid, bool internal, const String & backup_name, BackupStatus status)
|
||||
void BackupsWorker::addInfo(const UUID & uuid, const String & backup_name, BackupStatus status)
|
||||
{
|
||||
Info info;
|
||||
info.uuid = uuid;
|
||||
info.backup_name = backup_name;
|
||||
info.status = status;
|
||||
info.status_changed_time = time(nullptr);
|
||||
info.internal = internal;
|
||||
|
||||
std::lock_guard lock{infos_mutex};
|
||||
bool inserted = infos.try_emplace({uuid, internal}, std::move(info)).second;
|
||||
bool inserted = infos.try_emplace(uuid, std::move(info)).second;
|
||||
if (!inserted)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Pair of UUID={} and internal={} is already in use", uuid, internal);
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Backup's UUID={} is already in use", uuid);
|
||||
|
||||
num_active_backups += getNumActiveBackupsChange(status);
|
||||
num_active_restores += getNumActiveRestoresChange(status);
|
||||
}
|
||||
|
||||
|
||||
void BackupsWorker::setStatus(const UUID & uuid, bool internal, BackupStatus status)
|
||||
void BackupsWorker::setStatus(const UUID & uuid, BackupStatus status)
|
||||
{
|
||||
std::lock_guard lock{infos_mutex};
|
||||
auto it = infos.find({uuid, internal});
|
||||
auto it = infos.find(uuid);
|
||||
if (it == infos.end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown pair of UUID={} and internal={}", uuid, internal);
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown backup's UUID={}", uuid);
|
||||
|
||||
auto & info = it->second;
|
||||
auto old_status = info.status;
|
||||
@ -555,14 +563,14 @@ void BackupsWorker::setStatus(const UUID & uuid, bool internal, BackupStatus sta
|
||||
}
|
||||
|
||||
|
||||
void BackupsWorker::wait(const UUID & backup_or_restore_uuid, bool internal, bool rethrow_exception)
|
||||
void BackupsWorker::wait(const UUID & backup_or_restore_uuid, bool rethrow_exception)
|
||||
{
|
||||
std::unique_lock lock{infos_mutex};
|
||||
status_changed.wait(lock, [&]
|
||||
{
|
||||
auto it = infos.find({backup_or_restore_uuid, internal});
|
||||
auto it = infos.find(backup_or_restore_uuid);
|
||||
if (it == infos.end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown pair of UUID={} and internal={}", backup_or_restore_uuid, internal);
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown backup's UUID={}", backup_or_restore_uuid);
|
||||
const auto & info = it->second;
|
||||
auto current_status = info.status;
|
||||
if (rethrow_exception && ((current_status == BackupStatus::FAILED_TO_BACKUP) || (current_status == BackupStatus::FAILED_TO_RESTORE)))
|
||||
@ -571,12 +579,12 @@ void BackupsWorker::wait(const UUID & backup_or_restore_uuid, bool internal, boo
|
||||
});
|
||||
}
|
||||
|
||||
BackupsWorker::Info BackupsWorker::getInfo(const UUID & backup_or_restore_uuid, bool internal) const
|
||||
BackupsWorker::Info BackupsWorker::getInfo(const UUID & backup_or_restore_uuid) const
|
||||
{
|
||||
std::lock_guard lock{infos_mutex};
|
||||
auto it = infos.find({backup_or_restore_uuid, internal});
|
||||
auto it = infos.find(backup_or_restore_uuid);
|
||||
if (it == infos.end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown pair of UUID={} and internal={}", backup_or_restore_uuid, internal);
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown backup's UUID={}", backup_or_restore_uuid);
|
||||
return it->second;
|
||||
}
|
||||
|
||||
|
@ -29,11 +29,11 @@ public:
|
||||
void shutdown();
|
||||
|
||||
/// Starts executing a BACKUP or RESTORE query. Returns UUID of the operation.
|
||||
std::pair<UUID, bool> start(const ASTPtr & backup_or_restore_query, ContextMutablePtr context);
|
||||
UUID start(const ASTPtr & backup_or_restore_query, ContextMutablePtr context);
|
||||
|
||||
/// Waits until a BACKUP or RESTORE query started by start() is finished.
|
||||
/// The function returns immediately if the operation is already finished.
|
||||
void wait(const UUID & backup_or_restore_uuid, bool internal, bool rethrow_exception = true);
|
||||
void wait(const UUID & backup_or_restore_uuid, bool rethrow_exception = true);
|
||||
|
||||
/// Information about executing a BACKUP or RESTORE query started by calling start().
|
||||
struct Info
|
||||
@ -48,35 +48,31 @@ public:
|
||||
|
||||
String error_message;
|
||||
std::exception_ptr exception;
|
||||
|
||||
/// Whether this operation is internal, i.e. caused by another BACKUP or RESTORE operation.
|
||||
/// For example BACKUP ON CLUSTER executes an internal BACKUP commands per each node.
|
||||
bool internal = false;
|
||||
};
|
||||
|
||||
Info getInfo(const UUID & backup_or_restore_uuid, bool internal) const;
|
||||
Info getInfo(const UUID & backup_or_restore_uuid) const;
|
||||
std::vector<Info> getAllInfos() const;
|
||||
|
||||
private:
|
||||
std::pair<UUID, bool> startMakingBackup(const ASTPtr & query, const ContextPtr & context);
|
||||
UUID startMakingBackup(const ASTPtr & query, const ContextPtr & context);
|
||||
|
||||
void doBackup(const UUID & backup_uuid, const std::shared_ptr<ASTBackupQuery> & backup_query, BackupSettings backup_settings,
|
||||
const BackupInfo & backup_info, std::shared_ptr<IBackupCoordination> backup_coordination, const ContextPtr & context,
|
||||
ContextMutablePtr mutable_context, bool called_async);
|
||||
|
||||
std::pair<UUID, bool> startRestoring(const ASTPtr & query, ContextMutablePtr context);
|
||||
UUID startRestoring(const ASTPtr & query, ContextMutablePtr context);
|
||||
|
||||
void doRestore(const UUID & restore_uuid, const std::shared_ptr<ASTBackupQuery> & restore_query, RestoreSettings restore_settings,
|
||||
const BackupInfo & backup_info, std::shared_ptr<IRestoreCoordination> restore_coordination, ContextMutablePtr context,
|
||||
bool called_async);
|
||||
|
||||
void addInfo(const UUID & uuid, bool internal, const String & backup_name, BackupStatus status);
|
||||
void setStatus(const UUID & uuid, bool internal, BackupStatus status);
|
||||
void addInfo(const UUID & uuid, const String & backup_name, BackupStatus status);
|
||||
void setStatus(const UUID & uuid, BackupStatus status);
|
||||
|
||||
ThreadPool backups_thread_pool;
|
||||
ThreadPool restores_thread_pool;
|
||||
|
||||
std::map<std::pair<UUID, bool>, Info> infos;
|
||||
std::unordered_map<UUID, Info> infos;
|
||||
std::condition_variable status_changed;
|
||||
std::atomic<size_t> num_active_backups = 0;
|
||||
std::atomic<size_t> num_active_restores = 0;
|
||||
|
@ -39,9 +39,9 @@ namespace
|
||||
BlockIO InterpreterBackupQuery::execute()
|
||||
{
|
||||
auto & backups_worker = context->getBackupsWorker();
|
||||
auto [uuid, internal] = backups_worker.start(query_ptr, context);
|
||||
auto uuid = backups_worker.start(query_ptr, context);
|
||||
BlockIO res_io;
|
||||
res_io.pipeline = QueryPipeline(std::make_shared<SourceFromSingleChunk>(getResultRow(backups_worker.getInfo(uuid, internal))));
|
||||
res_io.pipeline = QueryPipeline(std::make_shared<SourceFromSingleChunk>(getResultRow(backups_worker.getInfo(uuid))));
|
||||
return res_io;
|
||||
}
|
||||
|
||||
|
@ -20,7 +20,6 @@ NamesAndTypesList StorageSystemBackups::getNamesAndTypes()
|
||||
{"status", std::make_shared<DataTypeEnum8>(getBackupStatusEnumValues())},
|
||||
{"status_changed_time", std::make_shared<DataTypeDateTime>()},
|
||||
{"error", std::make_shared<DataTypeString>()},
|
||||
{"internal", std::make_shared<DataTypeUInt8>()},
|
||||
};
|
||||
return names_and_types;
|
||||
}
|
||||
@ -34,7 +33,6 @@ void StorageSystemBackups::fillData(MutableColumns & res_columns, ContextPtr con
|
||||
auto & column_status = assert_cast<ColumnInt8 &>(*res_columns[column_index++]);
|
||||
auto & column_status_changed_time = assert_cast<ColumnUInt32 &>(*res_columns[column_index++]);
|
||||
auto & column_error = assert_cast<ColumnString &>(*res_columns[column_index++]);
|
||||
auto & column_internal = assert_cast<ColumnUInt8 &>(*res_columns[column_index++]);
|
||||
|
||||
auto add_row = [&](const BackupsWorker::Info & info)
|
||||
{
|
||||
@ -43,7 +41,6 @@ void StorageSystemBackups::fillData(MutableColumns & res_columns, ContextPtr con
|
||||
column_status.insertValue(static_cast<Int8>(info.status));
|
||||
column_status_changed_time.insertValue(info.status_changed_time);
|
||||
column_error.insertData(info.error_message.data(), info.error_message.size());
|
||||
column_internal.insertValue(info.internal);
|
||||
};
|
||||
|
||||
for (const auto & entry : context->getBackupsWorker().getAllInfos())
|
||||
|
Loading…
Reference in New Issue
Block a user