mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 09:02:00 +00:00
Add backup_log table
This commit is contained in:
parent
a89d711113
commit
9c34ef899e
@ -1218,6 +1218,16 @@
|
||||
<ttl>event_date + INTERVAL 3 DAY</ttl>
|
||||
</asynchronous_insert_log>
|
||||
|
||||
<!-- Backup/restore log.
|
||||
Uncomment to write backup/restore log records into a system table.
|
||||
<backup_log>
|
||||
<database>system</database>
|
||||
<table>backup_log</table>
|
||||
<partition_by>toYYYYMM(event_date)</partition_by>
|
||||
<flush_interval_milliseconds>0</flush_interval_milliseconds>
|
||||
</backup_log>
|
||||
-->
|
||||
|
||||
<!-- <top_level_domains_path>/var/lib/clickhouse/top_level_domains/</top_level_domains_path> -->
|
||||
<!-- Custom TLD lists.
|
||||
Format: <name>/path/to/file</name>
|
||||
|
@ -14,6 +14,7 @@
|
||||
#include <Backups/RestorerFromBackup.h>
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/BackupLog.h>
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
#include <Parsers/ASTBackupQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
@ -230,6 +231,7 @@ BackupsWorker::BackupsWorker(size_t num_backup_threads, size_t num_restore_threa
|
||||
|
||||
OperationID BackupsWorker::start(const ASTPtr & backup_or_restore_query, ContextMutablePtr context)
|
||||
{
|
||||
backup_log = context->getBackupLog();
|
||||
const ASTBackupQuery & backup_query = typeid_cast<const ASTBackupQuery &>(*backup_or_restore_query);
|
||||
if (backup_query.kind == ASTBackupQuery::Kind::BACKUP)
|
||||
return startMakingBackup(backup_or_restore_query, context);
|
||||
@ -449,9 +451,10 @@ void BackupsWorker::doBackup(
|
||||
backup.reset();
|
||||
|
||||
LOG_INFO(log, "{} {} was created successfully", (backup_settings.internal ? "Internal backup" : "Backup"), backup_name_for_logging);
|
||||
setStatus(backup_id, BackupStatus::BACKUP_CREATED);
|
||||
/// NOTE: we need to update metadata again after backup->finalizeWriting(), because backup metadata is written there.
|
||||
setNumFilesAndSize(backup_id, num_files, total_size, num_entries, uncompressed_size, compressed_size, 0, 0);
|
||||
/// NOTE: setStatus is called after setNumFilesAndSize in order to have actual information in a backup log record
|
||||
setStatus(backup_id, BackupStatus::BACKUP_CREATED);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -890,6 +893,9 @@ void BackupsWorker::addInfo(const OperationID & id, const String & name, bool in
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot start a backup or restore: ID {} is already in use", id);
|
||||
}
|
||||
|
||||
if (backup_log)
|
||||
backup_log->add(BackupLogElement{info});
|
||||
|
||||
infos[id] = std::move(info);
|
||||
|
||||
num_active_backups += getNumActiveBackupsChange(status);
|
||||
@ -923,6 +929,9 @@ void BackupsWorker::setStatus(const String & id, BackupStatus status, bool throw
|
||||
info.exception = std::current_exception();
|
||||
}
|
||||
|
||||
if (backup_log)
|
||||
backup_log->add(BackupLogElement{info});
|
||||
|
||||
num_active_backups += getNumActiveBackupsChange(status) - getNumActiveBackupsChange(old_status);
|
||||
num_active_restores += getNumActiveRestoresChange(status) - getNumActiveRestoresChange(old_status);
|
||||
}
|
||||
|
@ -25,6 +25,7 @@ class IBackupEntry;
|
||||
using BackupEntries = std::vector<std::pair<String, std::shared_ptr<const IBackupEntry>>>;
|
||||
using DataRestoreTasks = std::vector<std::function<void()>>;
|
||||
struct ReadSettings;
|
||||
class BackupLog;
|
||||
|
||||
/// Manager of backups and restores: executes backups and restores' threads in the background.
|
||||
/// Keeps information about backups and restores started in this session.
|
||||
@ -91,6 +92,7 @@ private:
|
||||
std::unique_ptr<ThreadPool> restores_thread_pool;
|
||||
|
||||
std::unordered_map<BackupOperationInfo::ID, BackupOperationInfo> infos;
|
||||
std::shared_ptr<BackupLog> backup_log;
|
||||
std::condition_variable status_changed;
|
||||
std::atomic<size_t> num_active_backups = 0;
|
||||
std::atomic<size_t> num_active_restores = 0;
|
||||
|
@ -15,6 +15,7 @@
|
||||
#include <Interpreters/ZooKeeperLog.h>
|
||||
#include <Interpreters/TransactionsInfoLog.h>
|
||||
#include <Interpreters/AsynchronousInsertLog.h>
|
||||
#include <Interpreters/BackupLog.h>
|
||||
|
||||
#include <Common/MemoryTrackerBlockerInThread.h>
|
||||
#include <Common/SystemLogBase.h>
|
||||
|
@ -29,7 +29,8 @@
|
||||
M(TextLogElement) \
|
||||
M(FilesystemCacheLogElement) \
|
||||
M(FilesystemReadPrefetchesLogElement) \
|
||||
M(AsynchronousInsertLogElement)
|
||||
M(AsynchronousInsertLogElement) \
|
||||
M(BackupLogElement)
|
||||
|
||||
namespace Poco
|
||||
{
|
||||
|
59
src/Interpreters/BackupLog.cpp
Normal file
59
src/Interpreters/BackupLog.cpp
Normal file
@ -0,0 +1,59 @@
|
||||
#include <Interpreters/BackupLog.h>
|
||||
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
BackupLogElement::BackupLogElement(BackupOperationInfo info_)
|
||||
: event_time(timeInSeconds(std::chrono::system_clock::now())), info(std::move(info_))
|
||||
{
|
||||
}
|
||||
|
||||
NamesAndTypesList BackupLogElement::getNamesAndTypes()
|
||||
{
|
||||
return
|
||||
{
|
||||
{"event_date", std::make_shared<DataTypeDate>()},
|
||||
{"event_time", std::make_shared<DataTypeDateTime>()},
|
||||
{"id", std::make_shared<DataTypeString>()},
|
||||
{"name", std::make_shared<DataTypeString>()},
|
||||
{"status", std::make_shared<DataTypeEnum8>(getBackupStatusEnumValues())},
|
||||
{"error", std::make_shared<DataTypeString>()},
|
||||
{"start_time", std::make_shared<DataTypeDateTime>()},
|
||||
{"end_time", std::make_shared<DataTypeDateTime>()},
|
||||
{"num_files", std::make_shared<DataTypeUInt64>()},
|
||||
{"total_size", std::make_shared<DataTypeUInt64>()},
|
||||
{"num_entries", std::make_shared<DataTypeUInt64>()},
|
||||
{"uncompressed_size", std::make_shared<DataTypeUInt64>()},
|
||||
{"compressed_size", std::make_shared<DataTypeUInt64>()},
|
||||
{"files_read", std::make_shared<DataTypeUInt64>()},
|
||||
{"bytes_read", std::make_shared<DataTypeUInt64>()},
|
||||
};
|
||||
}
|
||||
|
||||
void BackupLogElement::appendToBlock(MutableColumns & columns) const
|
||||
{
|
||||
size_t i = 0;
|
||||
columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
|
||||
columns[i++]->insert(static_cast<UInt32>(event_time));
|
||||
columns[i++]->insert(info.id);
|
||||
columns[i++]->insert(info.name);
|
||||
columns[i++]->insert(static_cast<Int8>(info.status));
|
||||
columns[i++]->insert(info.error_message);
|
||||
columns[i++]->insert(static_cast<UInt32>(std::chrono::system_clock::to_time_t(info.start_time)));
|
||||
columns[i++]->insert(static_cast<UInt32>(std::chrono::system_clock::to_time_t(info.end_time)));
|
||||
columns[i++]->insert(info.num_files);
|
||||
columns[i++]->insert(info.total_size);
|
||||
columns[i++]->insert(info.num_entries);
|
||||
columns[i++]->insert(info.uncompressed_size);
|
||||
columns[i++]->insert(info.compressed_size);
|
||||
columns[i++]->insert(info.num_read_files);
|
||||
columns[i++]->insert(info.num_read_bytes);
|
||||
}
|
||||
|
||||
}
|
38
src/Interpreters/BackupLog.h
Normal file
38
src/Interpreters/BackupLog.h
Normal file
@ -0,0 +1,38 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/SystemLog.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Core/NamesAndAliases.h>
|
||||
#include <Backups/BackupOperationInfo.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** A struct which will be inserted as row into backup_log table.
|
||||
* Contains a record about backup or restore operation.
|
||||
*/
|
||||
struct BackupLogElement
|
||||
{
|
||||
BackupLogElement() = default;
|
||||
BackupLogElement(BackupOperationInfo info_);
|
||||
BackupLogElement(const BackupLogElement &) = default;
|
||||
BackupLogElement & operator=(const BackupLogElement &) = default;
|
||||
BackupLogElement(BackupLogElement &&) = default;
|
||||
BackupLogElement & operator=(BackupLogElement &&) = default;
|
||||
|
||||
time_t event_time;
|
||||
BackupOperationInfo info;
|
||||
|
||||
static std::string name() { return "BackupLog"; }
|
||||
static NamesAndTypesList getNamesAndTypes();
|
||||
static NamesAndAliases getNamesAndAliases() { return {}; }
|
||||
void appendToBlock(MutableColumns & columns) const;
|
||||
static const char * getCustomColumnList() { return nullptr; }
|
||||
};
|
||||
|
||||
class BackupLog : public SystemLog<BackupLogElement>
|
||||
{
|
||||
using SystemLog<BackupLogElement>::SystemLog;
|
||||
};
|
||||
|
||||
}
|
@ -3381,6 +3381,16 @@ std::shared_ptr<AsynchronousInsertLog> Context::getAsynchronousInsertLog() const
|
||||
return shared->system_logs->asynchronous_insert_log;
|
||||
}
|
||||
|
||||
std::shared_ptr<BackupLog> Context::getBackupLog() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
if (!shared->system_logs)
|
||||
return {};
|
||||
|
||||
return shared->system_logs->backup_log;
|
||||
}
|
||||
|
||||
CompressionCodecPtr Context::chooseCompressionCodec(size_t part_size, double part_size_ratio) const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
@ -103,6 +103,7 @@ class ProcessorsProfileLog;
|
||||
class FilesystemCacheLog;
|
||||
class FilesystemReadPrefetchesLog;
|
||||
class AsynchronousInsertLog;
|
||||
class BackupLog;
|
||||
class IAsynchronousReader;
|
||||
struct MergeTreeSettings;
|
||||
struct InitialAllRangesAnnouncement;
|
||||
@ -1019,6 +1020,7 @@ public:
|
||||
std::shared_ptr<FilesystemCacheLog> getFilesystemCacheLog() const;
|
||||
std::shared_ptr<FilesystemReadPrefetchesLog> getFilesystemReadPrefetchesLog() const;
|
||||
std::shared_ptr<AsynchronousInsertLog> getAsynchronousInsertLog() const;
|
||||
std::shared_ptr<BackupLog> getBackupLog() const;
|
||||
|
||||
/// Returns an object used to log operations with parts if it possible.
|
||||
/// Provide table name to make required checks.
|
||||
|
@ -36,6 +36,7 @@
|
||||
#include <Interpreters/TransactionsInfoLog.h>
|
||||
#include <Interpreters/ProcessorsProfileLog.h>
|
||||
#include <Interpreters/AsynchronousInsertLog.h>
|
||||
#include <Interpreters/BackupLog.h>
|
||||
#include <Interpreters/JIT/CompiledExpressionCache.h>
|
||||
#include <Interpreters/TransactionLog.h>
|
||||
#include <Interpreters/AsynchronousInsertQueue.h>
|
||||
@ -572,7 +573,8 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
[&] { if (auto transactions_info_log = getContext()->getTransactionsInfoLog()) transactions_info_log->flush(true); },
|
||||
[&] { if (auto processors_profile_log = getContext()->getProcessorsProfileLog()) processors_profile_log->flush(true); },
|
||||
[&] { if (auto cache_log = getContext()->getFilesystemCacheLog()) cache_log->flush(true); },
|
||||
[&] { if (auto asynchronous_insert_log = getContext()->getAsynchronousInsertLog()) asynchronous_insert_log->flush(true); }
|
||||
[&] { if (auto asynchronous_insert_log = getContext()->getAsynchronousInsertLog()) asynchronous_insert_log->flush(true); },
|
||||
[&] { if (auto backup_log = getContext()->getBackupLog()) backup_log->flush(true); }
|
||||
);
|
||||
break;
|
||||
}
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include <Interpreters/FilesystemCacheLog.h>
|
||||
#include <Interpreters/FilesystemReadPrefetchesLog.h>
|
||||
#include <Interpreters/ZooKeeperLog.h>
|
||||
#include <Interpreters/BackupLog.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIndexDeclaration.h>
|
||||
@ -286,6 +287,7 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf
|
||||
global_context, "system", "transactions_info_log", config, "transactions_info_log");
|
||||
processors_profile_log = createSystemLog<ProcessorsProfileLog>(global_context, "system", "processors_profile_log", config, "processors_profile_log");
|
||||
asynchronous_insert_log = createSystemLog<AsynchronousInsertLog>(global_context, "system", "asynchronous_insert_log", config, "asynchronous_insert_log");
|
||||
backup_log = createSystemLog<BackupLog>(global_context, "system", "backup_log", config, "backup_log");
|
||||
|
||||
if (query_log)
|
||||
logs.emplace_back(query_log.get());
|
||||
@ -324,6 +326,8 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf
|
||||
logs.emplace_back(filesystem_read_prefetches_log.get());
|
||||
if (asynchronous_insert_log)
|
||||
logs.emplace_back(asynchronous_insert_log.get());
|
||||
if (backup_log)
|
||||
logs.emplace_back(backup_log.get());
|
||||
|
||||
try
|
||||
{
|
||||
|
@ -49,6 +49,7 @@ class ProcessorsProfileLog;
|
||||
class FilesystemCacheLog;
|
||||
class FilesystemReadPrefetchesLog;
|
||||
class AsynchronousInsertLog;
|
||||
class BackupLog;
|
||||
|
||||
/// System logs should be destroyed in destructor of the last Context and before tables,
|
||||
/// because SystemLog destruction makes insert query while flushing data into underlying tables
|
||||
@ -84,6 +85,8 @@ struct SystemLogs
|
||||
/// Used to log processors profiling
|
||||
std::shared_ptr<ProcessorsProfileLog> processors_profile_log;
|
||||
std::shared_ptr<AsynchronousInsertLog> asynchronous_insert_log;
|
||||
/// Backup and restore events
|
||||
std::shared_ptr<BackupLog> backup_log;
|
||||
|
||||
std::vector<ISystemLog *> logs;
|
||||
};
|
||||
|
Loading…
Reference in New Issue
Block a user