From a04fd90cb66753ed7e460e7a58c2164fcb20ac1d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 18 Jan 2022 01:55:40 +0700 Subject: [PATCH 01/18] Make restore tasks explicit. --- src/Backups/BackupUtils.cpp | 374 +--------- src/Backups/BackupUtils.h | 16 +- src/Backups/IRestoreFromBackupTask.h | 31 + src/Backups/RestoreFromBackupUtils.cpp | 637 ++++++++++++++++++ src/Backups/RestoreFromBackupUtils.h | 23 + .../hasCompatibleDataToRestoreTable.cpp | 4 + src/Common/ErrorCodes.cpp | 1 + src/Interpreters/InterpreterBackupQuery.cpp | 4 +- src/Storages/IStorage.cpp | 2 +- src/Storages/IStorage.h | 5 +- src/Storages/MergeTree/MergeTreeData.cpp | 115 +++- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageLog.cpp | 58 +- src/Storages/StorageLog.h | 3 +- src/Storages/StorageMergeTree.cpp | 4 +- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageStripeLog.cpp | 60 +- src/Storages/StorageStripeLog.h | 3 +- 18 files changed, 926 insertions(+), 418 deletions(-) create mode 100644 src/Backups/IRestoreFromBackupTask.h create mode 100644 src/Backups/RestoreFromBackupUtils.cpp create mode 100644 src/Backups/RestoreFromBackupUtils.h diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index c26eec440e64..35c2fb336312 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -2,24 +2,17 @@ #include #include #include -#include #include #include +#include #include -#include #include -#include #include -#include -#include #include #include #include #include #include -#include - -namespace fs = std::filesystem; namespace DB @@ -29,8 +22,6 @@ namespace ErrorCodes extern const int BACKUP_ELEMENT_DUPLICATE; extern const int BACKUP_IS_EMPTY; extern const int LOGICAL_ERROR; - extern const int TABLE_ALREADY_EXISTS; - extern const int CANNOT_RESTORE_TABLE; } namespace @@ -265,28 +256,6 @@ namespace elements[database.index].except_list.emplace(table_name); } - /// Reorder the elements: databases should be before tables and dictionaries they contain. - for (auto & [database_name, database] : databases) - { - if (database.index == static_cast(-1)) - continue; - size_t min_index = std::numeric_limits::max(); - auto min_index_it = database.tables.end(); - for (auto it = database.tables.begin(); it != database.tables.end(); ++it) - { - if (min_index > it->second) - { - min_index = it->second; - min_index_it = it; - } - } - if (database.index > min_index) - { - std::swap(elements[database.index], elements[min_index]); - std::swap(database.index, min_index_it->second); - } - } - for (auto skip_index : skip_indices | boost::adaptors::reversed) elements.erase(elements.begin() + skip_index); } @@ -302,48 +271,6 @@ namespace return res; } - String getDataPathInBackup(const DatabaseAndTableName & table_name) - { - if (table_name.first.empty() || table_name.second.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name and table name must not be empty"); - assert(!table_name.first.empty() && !table_name.second.empty()); - return String{"data/"} + escapeForFileName(table_name.first) + "/" + escapeForFileName(table_name.second) + "/"; - } - - String getDataPathInBackup(const IAST & create_query) - { - const auto & create = create_query.as(); - if (!create.table) - return {}; - if (create.temporary) - return getDataPathInBackup({DatabaseCatalog::TEMPORARY_DATABASE, create.getTable()}); - return getDataPathInBackup({create.getDatabase(), create.getTable()}); - } - - String getMetadataPathInBackup(const DatabaseAndTableName & table_name) - { - if (table_name.first.empty() || table_name.second.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name and table name must not be empty"); - return String{"metadata/"} + escapeForFileName(table_name.first) + "/" + escapeForFileName(table_name.second) + ".sql"; - } - - String getMetadataPathInBackup(const String & database_name) - { - if (database_name.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name must not be empty"); - return String{"metadata/"} + escapeForFileName(database_name) + ".sql"; - } - - String getMetadataPathInBackup(const IAST & create_query) - { - const auto & create = create_query.as(); - if (!create.table) - return getMetadataPathInBackup(create.getDatabase()); - if (create.temporary) - return getMetadataPathInBackup({DatabaseCatalog::TEMPORARY_DATABASE, create.getTable()}); - return getMetadataPathInBackup({create.getDatabase(), create.getTable()}); - } - void backupCreateQuery(const IAST & create_query, BackupEntries & backup_entries) { auto metadata_entry = std::make_unique(serializeAST(create_query)); @@ -412,179 +339,6 @@ namespace backupDatabase(database, {}, context, renaming_config, backup_entries); } } - - void makeDatabaseIfNotExists(const String & database_name, ContextMutablePtr context) - { - if (DatabaseCatalog::instance().isDatabaseExist(database_name)) - return; - - /// We create and execute `create` query for the database name. - auto create_query = std::make_shared(); - create_query->setDatabase(database_name); - create_query->if_not_exists = true; - InterpreterCreateQuery create_interpreter{create_query, context}; - create_interpreter.execute(); - } - - ASTPtr readCreateQueryFromBackup(const DatabaseAndTableName & table_name, const BackupPtr & backup) - { - String create_query_path = getMetadataPathInBackup(table_name); - auto read_buffer = backup->readFile(create_query_path)->getReadBuffer(); - String create_query_str; - readStringUntilEOF(create_query_str, *read_buffer); - read_buffer.reset(); - ParserCreateQuery create_parser; - return parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); - } - - ASTPtr readCreateQueryFromBackup(const String & database_name, const BackupPtr & backup) - { - String create_query_path = getMetadataPathInBackup(database_name); - auto read_buffer = backup->readFile(create_query_path)->getReadBuffer(); - String create_query_str; - readStringUntilEOF(create_query_str, *read_buffer); - read_buffer.reset(); - ParserCreateQuery create_parser; - return parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); - } - - void restoreTable( - const DatabaseAndTableName & table_name, - const ASTs & partitions, - ContextMutablePtr context, - const BackupPtr & backup, - const BackupRenamingConfigPtr & renaming_config, - RestoreObjectsTasks & restore_tasks) - { - ASTPtr create_query = readCreateQueryFromBackup(table_name, backup); - auto new_create_query = typeid_cast>(renameInCreateQuery(create_query, renaming_config, context)); - - restore_tasks.emplace_back([table_name, new_create_query, partitions, context, backup]() -> RestoreDataTasks - { - DatabaseAndTableName new_table_name{new_create_query->getDatabase(), new_create_query->getTable()}; - if (new_create_query->temporary) - new_table_name.first = DatabaseCatalog::TEMPORARY_DATABASE; - - context->checkAccess(AccessType::INSERT, new_table_name.first, new_table_name.second); - - StoragePtr storage; - for (size_t try_index = 0; try_index != 10; ++try_index) - { - if (DatabaseCatalog::instance().isTableExist({new_table_name.first, new_table_name.second}, context)) - { - DatabasePtr existing_database; - StoragePtr existing_storage; - std::tie(existing_database, existing_storage) = DatabaseCatalog::instance().tryGetDatabaseAndTable({new_table_name.first, new_table_name.second}, context); - if (existing_storage) - { - if (auto existing_table_create_query = existing_database->tryGetCreateTableQuery(new_table_name.second, context)) - { - if (hasCompatibleDataToRestoreTable(*new_create_query, existing_table_create_query->as())) - { - storage = existing_storage; - break; - } - else - { - String error_message = (new_table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) - ? ("Temporary table " + backQuoteIfNeed(new_table_name.second) + " already exists") - : ("Table " + backQuoteIfNeed(new_table_name.first) + "." + backQuoteIfNeed(new_table_name.second) - + " already exists"); - throw Exception(error_message, ErrorCodes::CANNOT_RESTORE_TABLE); - } - } - } - } - - makeDatabaseIfNotExists(new_table_name.first, context); - - try - { - InterpreterCreateQuery create_interpreter{new_create_query, context}; - create_interpreter.execute(); - } - catch (Exception & e) - { - if (e.code() != ErrorCodes::TABLE_ALREADY_EXISTS) - throw; - } - } - - if (!storage) - { - String error_message = (new_table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) - ? ("Could not create temporary table " + backQuoteIfNeed(new_table_name.second) + " for restoring") - : ("Could not create table " + backQuoteIfNeed(new_table_name.first) + "." + backQuoteIfNeed(new_table_name.second) - + " for restoring"); - throw Exception(error_message, ErrorCodes::CANNOT_RESTORE_TABLE); - } - - String data_path_in_backup = getDataPathInBackup(table_name); - RestoreDataTasks restore_data_tasks = storage->restoreFromBackup(backup, data_path_in_backup, partitions, context); - - /// Keep `storage` alive while we're executing `restore_data_tasks`. - for (auto & restore_data_task : restore_data_tasks) - restore_data_task = [restore_data_task, storage]() { restore_data_task(); }; - - return restore_data_tasks; - }); - } - - void restoreDatabase(const String & database_name, const std::set & except_list, ContextMutablePtr context, const BackupPtr & backup, const BackupRenamingConfigPtr & renaming_config, RestoreObjectsTasks & restore_tasks) - { - ASTPtr create_query = readCreateQueryFromBackup(database_name, backup); - auto new_create_query = typeid_cast>(renameInCreateQuery(create_query, renaming_config, context)); - - restore_tasks.emplace_back([database_name, new_create_query, except_list, context, backup, renaming_config]() -> RestoreDataTasks - { - const String & new_database_name = new_create_query->getDatabase(); - context->checkAccess(AccessType::SHOW_TABLES, new_database_name); - - if (!DatabaseCatalog::instance().isDatabaseExist(new_database_name)) - { - /// We create and execute `create` query for the database name. - new_create_query->if_not_exists = true; - InterpreterCreateQuery create_interpreter{new_create_query, context}; - create_interpreter.execute(); - } - - RestoreObjectsTasks restore_objects_tasks; - Strings table_metadata_filenames = backup->listFiles("metadata/" + escapeForFileName(database_name) + "/", "/"); - for (const String & table_metadata_filename : table_metadata_filenames) - { - String table_name = unescapeForFileName(fs::path{table_metadata_filename}.stem()); - if (except_list.contains(table_name)) - continue; - restoreTable({database_name, table_name}, {}, context, backup, renaming_config, restore_objects_tasks); - } - - RestoreDataTasks restore_data_tasks; - for (auto & restore_object_task : restore_objects_tasks) - insertAtEnd(restore_data_tasks, std::move(restore_object_task)()); - return restore_data_tasks; - }); - } - - void restoreAllDatabases(const std::set & except_list, ContextMutablePtr context, const BackupPtr & backup, const BackupRenamingConfigPtr & renaming_config, RestoreObjectsTasks & restore_tasks) - { - restore_tasks.emplace_back([except_list, context, backup, renaming_config]() -> RestoreDataTasks - { - RestoreObjectsTasks restore_objects_tasks; - Strings database_metadata_filenames = backup->listFiles("metadata/", "/"); - for (const String & database_metadata_filename : database_metadata_filenames) - { - String database_name = unescapeForFileName(fs::path{database_metadata_filename}.stem()); - if (except_list.contains(database_name)) - continue; - restoreDatabase(database_name, {}, context, backup, renaming_config, restore_objects_tasks); - } - - RestoreDataTasks restore_data_tasks; - for (auto & restore_object_task : restore_objects_tasks) - insertAtEnd(restore_data_tasks, std::move(restore_object_task)()); - return restore_data_tasks; - }); - } } @@ -728,106 +482,46 @@ void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries backup->finalizeWriting(); } - -RestoreObjectsTasks makeRestoreTasks(const Elements & elements, ContextMutablePtr context, const BackupPtr & backup) +String getDataPathInBackup(const DatabaseAndTableName & table_name) { - RestoreObjectsTasks restore_tasks; - - auto elements2 = adjustElements(elements, context->getCurrentDatabase()); - auto renaming_config = std::make_shared(); - renaming_config->setFromBackupQueryElements(elements2); - - for (const auto & element : elements2) - { - switch (element.type) - { - case ElementType::TABLE: - { - const String & database_name = element.name.first; - const String & table_name = element.name.second; - restoreTable({database_name, table_name}, element.partitions, context, backup, renaming_config, restore_tasks); - break; - } - - case ElementType::DATABASE: - { - const String & database_name = element.name.first; - restoreDatabase(database_name, element.except_list, context, backup, renaming_config, restore_tasks); - break; - } - - case ElementType::ALL_DATABASES: - { - restoreAllDatabases(element.except_list, context, backup, renaming_config, restore_tasks); - break; - } - - default: - throw Exception("Unexpected element type", ErrorCodes::LOGICAL_ERROR); /// other element types have been removed in deduplicateElements() - } - } - - return restore_tasks; + if (table_name.first.empty() || table_name.second.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name and table name must not be empty"); + assert(!table_name.first.empty() && !table_name.second.empty()); + return String{"data/"} + escapeForFileName(table_name.first) + "/" + escapeForFileName(table_name.second) + "/"; } - -void executeRestoreTasks(RestoreObjectsTasks && restore_tasks, size_t num_threads) +String getDataPathInBackup(const IAST & create_query) { - if (!num_threads) - num_threads = 1; - - RestoreDataTasks restore_data_tasks; - for (auto & restore_object_task : restore_tasks) - insertAtEnd(restore_data_tasks, std::move(restore_object_task)()); - restore_tasks.clear(); - - std::vector threads; - size_t num_active_threads = 0; - std::mutex mutex; - std::condition_variable cond; - std::exception_ptr exception; - - for (auto & restore_data_task : restore_data_tasks) - { - { - std::unique_lock lock{mutex}; - if (exception) - break; - cond.wait(lock, [&] { return num_active_threads < num_threads; }); - if (exception) - break; - ++num_active_threads; - } - - threads.emplace_back([&restore_data_task, &mutex, &cond, &num_active_threads, &exception]() mutable - { - try - { - restore_data_task(); - restore_data_task = {}; - } - catch (...) - { - std::lock_guard lock{mutex}; - if (!exception) - exception = std::current_exception(); - } - - { - std::lock_guard lock{mutex}; - --num_active_threads; - cond.notify_all(); - } - }); - } + const auto & create = create_query.as(); + if (!create.table) + return {}; + if (create.temporary) + return getDataPathInBackup({DatabaseCatalog::TEMPORARY_DATABASE, create.getTable()}); + return getDataPathInBackup({create.getDatabase(), create.getTable()}); +} - for (auto & thread : threads) - thread.join(); +String getMetadataPathInBackup(const DatabaseAndTableName & table_name) +{ + if (table_name.first.empty() || table_name.second.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name and table name must not be empty"); + return String{"metadata/"} + escapeForFileName(table_name.first) + "/" + escapeForFileName(table_name.second) + ".sql"; +} - restore_data_tasks.clear(); +String getMetadataPathInBackup(const String & database_name) +{ + if (database_name.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name must not be empty"); + return String{"metadata/"} + escapeForFileName(database_name) + ".sql"; +} - if (exception) - std::rethrow_exception(exception); +String getMetadataPathInBackup(const IAST & create_query) +{ + const auto & create = create_query.as(); + if (!create.table) + return getMetadataPathInBackup(create.getDatabase()); + if (create.temporary) + return getMetadataPathInBackup({DatabaseCatalog::TEMPORARY_DATABASE, create.getTable()}); + return getMetadataPathInBackup({create.getDatabase(), create.getTable()}); } } diff --git a/src/Backups/BackupUtils.h b/src/Backups/BackupUtils.h index 70f080cf6e98..c2f56044d5d2 100644 --- a/src/Backups/BackupUtils.h +++ b/src/Backups/BackupUtils.h @@ -12,13 +12,8 @@ using BackupMutablePtr = std::shared_ptr; class IBackupEntry; using BackupEntryPtr = std::unique_ptr; using BackupEntries = std::vector>; -using RestoreDataTask = std::function; -using RestoreDataTasks = std::vector; -using RestoreObjectTask = std::function; -using RestoreObjectsTasks = std::vector; class Context; using ContextPtr = std::shared_ptr; -using ContextMutablePtr = std::shared_ptr; /// Prepares backup entries. @@ -30,10 +25,13 @@ UInt64 estimateBackupSize(const BackupEntries & backup_entries, const BackupPtr /// Write backup entries to an opened backup. void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries, size_t num_threads); -/// Prepare restore tasks. -RestoreObjectsTasks makeRestoreTasks(const ASTBackupQuery::Elements & elements, ContextMutablePtr context, const BackupPtr & backup); +/// Returns the path to metadata in backup. +String getMetadataPathInBackup(const DatabaseAndTableName & table_name); +String getMetadataPathInBackup(const String & database_name); +String getMetadataPathInBackup(const IAST & create_query); -/// Execute restore tasks. -void executeRestoreTasks(RestoreObjectsTasks && restore_tasks, size_t num_threads); +/// Returns the path to table's data in backup. +String getDataPathInBackup(const DatabaseAndTableName & table_name); +String getDataPathInBackup(const IAST & create_query); } diff --git a/src/Backups/IRestoreFromBackupTask.h b/src/Backups/IRestoreFromBackupTask.h new file mode 100644 index 000000000000..93e08b035ee9 --- /dev/null +++ b/src/Backups/IRestoreFromBackupTask.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +/// Represents a task of restoring something (database / table / table's part) from backup. +class IRestoreFromBackupTask +{ +public: + IRestoreFromBackupTask() = default; + virtual ~IRestoreFromBackupTask() = default; + + /// Perform restoring, the function also can return a list of nested tasks that should be run later. + virtual std::vector> run() = 0; + + /// Is it necessary to run this task sequentially? + /// Sequential tasks are executed first and strictly in one thread. + virtual bool isSequential() const { return false; } + + /// Reverts the effect of run(). If that's not possible, the function does nothing. + virtual void rollback() {} +}; + +using RestoreFromBackupTaskPtr = std::unique_ptr; +using RestoreFromBackupTasks = std::vector; + +} diff --git a/src/Backups/RestoreFromBackupUtils.cpp b/src/Backups/RestoreFromBackupUtils.cpp new file mode 100644 index 000000000000..af3cc926a7e2 --- /dev/null +++ b/src/Backups/RestoreFromBackupUtils.cpp @@ -0,0 +1,637 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace fs = std::filesystem; + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int CANNOT_RESTORE_TABLE; + extern const int CANNOT_RESTORE_DATABASE; +} + +namespace +{ + using Kind = ASTBackupQuery::Kind; + using Element = ASTBackupQuery::Element; + using Elements = ASTBackupQuery::Elements; + using ElementType = ASTBackupQuery::ElementType; + + + /// Replaces an empty database with the current database. + void replaceEmptyDatabaseWithCurrentDatabase(Elements & elements, const String & current_database) + { + for (auto & element : elements) + { + if (element.type == ElementType::TABLE) + { + if (element.name.first.empty() && !element.name.second.empty()) + element.name.first = current_database; + if (element.new_name.first.empty() && !element.new_name.second.empty()) + element.new_name.first = current_database; + } + } + } + + + /// Restores a database (without tables inside), should be executed before executing + /// RestoreTableFromBackupTask. + class RestoreDatabaseFromBackupTask : public IRestoreFromBackupTask + { + public: + RestoreDatabaseFromBackupTask(ContextMutablePtr context_, const ASTPtr & create_query_) + : context(context_), create_query(typeid_cast>(create_query_)) + { + } + + RestoreFromBackupTasks run() override + { + createDatabase(); + return {}; + } + + bool isSequential() const override { return true; } + + private: + void createDatabase() + { + InterpreterCreateQuery create_interpreter{create_query, context}; + create_interpreter.execute(); + } + + ContextMutablePtr context; + std::shared_ptr create_query; + }; + + + /// Restores a table and fills it with data. + class RestoreTableFromBackupTask : public IRestoreFromBackupTask + { + public: + RestoreTableFromBackupTask( + ContextMutablePtr context_, + const ASTPtr & create_query_, + const ASTs & partitions_, + const BackupPtr & backup_, + const DatabaseAndTableName & table_name_in_backup_) + : context(context_), create_query(typeid_cast>(create_query_)), + partitions(partitions_), backup(backup_), table_name_in_backup(table_name_in_backup_) + { + table_name = DatabaseAndTableName{create_query->getDatabase(), create_query->getTable()}; + if (create_query->temporary) + table_name.first = DatabaseCatalog::TEMPORARY_DATABASE; + } + + RestoreFromBackupTasks run() override + { + createStorage(); + auto storage = getStorage(); + RestoreFromBackupTasks tasks; + if (auto task = insertDataIntoStorage(storage)) + tasks.push_back(std::move(task)); + return tasks; + } + + bool isSequential() const override { return true; } + + private: + void createStorage() + { + InterpreterCreateQuery create_interpreter{create_query, context}; + create_interpreter.execute(); + } + + StoragePtr tryGetStorage() + { + if (!DatabaseCatalog::instance().isTableExist({table_name.first, table_name.second}, context)) + return nullptr; + + DatabasePtr existing_database; + StoragePtr existing_storage; + std::tie(existing_database, existing_storage) = DatabaseCatalog::instance().tryGetDatabaseAndTable({table_name.first, table_name.second}, context); + if (!existing_storage) + return nullptr; + + auto existing_table_create_query = existing_database->tryGetCreateTableQuery(table_name.second, context); + if (!existing_table_create_query) + return nullptr; + + if (!hasCompatibleDataToRestoreTable(*create_query, existing_table_create_query->as())) + throw Exception( + ErrorCodes::CANNOT_RESTORE_TABLE, + "Table {}.{} from backup is incompatible with existing table {}.{}. " + "The create query of the table from backup: {}." + "The create query of the existing table: {}", + backQuoteIfNeed(table_name_in_backup.first), + backQuoteIfNeed(table_name_in_backup.second), + backQuoteIfNeed(table_name.first), + backQuoteIfNeed(table_name.second), + serializeAST(*create_query), + serializeAST(*existing_table_create_query)); + + return existing_storage; + } + + StoragePtr getStorage() + { + if (auto storage = tryGetStorage()) + return storage; + + String error_message = (table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) + ? ("Could not create temporary table " + backQuoteIfNeed(table_name.second) + " for restoring") + : ("Could not create table " + backQuoteIfNeed(table_name.first) + "." + backQuoteIfNeed(table_name.second) + + " for restoring"); + throw Exception(error_message, ErrorCodes::CANNOT_RESTORE_TABLE); + } + + RestoreFromBackupTaskPtr insertDataIntoStorage(StoragePtr storage) + { + context->checkAccess(AccessType::INSERT, table_name.first, table_name.second); + String data_path_in_backup = getDataPathInBackup(table_name_in_backup); + return storage->restoreFromBackup(backup, data_path_in_backup, partitions, context); + } + + ContextMutablePtr context; + std::shared_ptr create_query; + DatabaseAndTableName table_name; + ASTs partitions; + BackupPtr backup; + DatabaseAndTableName table_name_in_backup; + }; + + + /// Makes tasks for restoring databases and tables according to the elements of ASTBackupQuery. + /// Keep this class consistent with BackupEntriesBuilder. + class RestoreTasksBuilder + { + public: + RestoreTasksBuilder(ContextMutablePtr context_, const BackupPtr & backup_) + : context(context_), backup(backup_) {} + + /// Prepares internal structures for making tasks for restoring. + void prepare(const ASTBackupQuery::Elements & elements) + { + auto elements2 = elements; + replaceEmptyDatabaseWithCurrentDatabase(elements2, context->getCurrentDatabase()); + + auto new_renaming_config = std::make_shared(); + new_renaming_config->setFromBackupQueryElements(elements2); + renaming_config = new_renaming_config; + + for (const auto & element : elements2) + { + switch (element.type) + { + case ElementType::TABLE: [[fallthrough]]; + case ElementType::DICTIONARY: + { + const String & database_name = element.name.first; + const String & table_name = element.name.second; + prepareToRestoreTable(DatabaseAndTableName{database_name, table_name}, element.partitions); + break; + } + + case ElementType::TEMPORARY_TABLE: + { + String database_name = DatabaseCatalog::TEMPORARY_DATABASE; + const String & table_name = element.name.second; + prepareToRestoreTable(DatabaseAndTableName{database_name, table_name}, element.partitions); + break; + } + + case ElementType::DATABASE: + { + const String & database_name = element.name.first; + prepareToRestoreDatabase(database_name, element.except_list); + break; + } + + case ElementType::ALL_TEMPORARY_TABLES: + { + prepareToRestoreDatabase(DatabaseCatalog::TEMPORARY_DATABASE, element.except_list); + break; + } + + case ElementType::ALL_DATABASES: + { + prepareToRestoreAllDatabases(element.except_list); + break; + } + + case ElementType::EVERYTHING: + { + prepareToRestoreAllDatabases({}); + break; + } + } + } + } + + /// Makes tasks for restoring, should be called after prepare(). + RestoreFromBackupTasks makeTasks() const + { + /// Check that there are not `different_create_query`. (If it's set it means error.) + for (auto & info : databases | boost::adaptors::map_values) + { + if (info.different_create_query) + throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, + "Couldn't restore a database because two different create queries were generated for it: {} and {}", + serializeAST(*info.create_query), serializeAST(*info.different_create_query)); + } + + RestoreFromBackupTasks res; + for (auto & info : databases | boost::adaptors::map_values) + res.push_back(std::make_unique(context, info.create_query)); + + /// TODO: We need to restore tables according to their dependencies. + for (auto & info : tables | boost::adaptors::map_values) + res.push_back(std::make_unique(context, info.create_query, info.partitions, backup, info.name_in_backup)); + + return res; + } + + private: + /// Prepares to restore a single table and probably its database's definition. + void prepareToRestoreTable(const DatabaseAndTableName & table_name_, const ASTs & partitions_) + { + /// Check that we are not trying to restore the same table again. + DatabaseAndTableName new_table_name = renaming_config->getNewTableName(table_name_); + if (tables.contains(new_table_name)) + { + String message; + if (new_table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) + message = fmt::format("Couldn't restore temporary table {} twice", backQuoteIfNeed(new_table_name.second)); + else + message = fmt::format("Couldn't restore table {}.{} twice", backQuoteIfNeed(new_table_name.first), backQuoteIfNeed(new_table_name.second)); + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, message); + } + + /// Make a create query for this table. + auto create_query = renameInCreateQuery(readCreateQueryFromBackup(table_name_)); + create_query->if_not_exists = true; + + CreateTableInfo info; + info.create_query = create_query; + info.name_in_backup = table_name_; + info.partitions = partitions_; + tables[new_table_name] = std::move(info); + + /// If it's not system or temporary database then probably we need to restore the database's definition too. + if (!isSystemOrTemporaryDatabase(new_table_name.first)) + { + if (!databases.contains(new_table_name.first)) + { + /// Add a create query for restoring the database if we haven't done it yet. + std::shared_ptr create_db_query; + String db_name_in_backup = table_name_.first; + if (hasCreateQueryInBackup(db_name_in_backup)) + { + create_db_query = renameInCreateQuery(readCreateQueryFromBackup(db_name_in_backup)); + } + else + { + create_db_query = std::make_shared(); + db_name_in_backup.clear(); + } + create_db_query->setDatabase(new_table_name.first); + create_db_query->if_not_exists = true; + + CreateDatabaseInfo info_db; + info_db.create_query = create_db_query; + info_db.name_in_backup = std::move(db_name_in_backup); + info_db.is_explicit = false; + databases[new_table_name.first] = std::move(info_db); + } + else + { + /// We already have added a create query for restoring the database, + /// set `different_create_query` if it's not the same. + auto & info_db = databases[new_table_name.first]; + if (!info_db.is_explicit && (info_db.name_in_backup != table_name_.first) && !info_db.different_create_query) + { + std::shared_ptr create_db_query; + if (hasCreateQueryInBackup(table_name_.first)) + create_db_query = renameInCreateQuery(readCreateQueryFromBackup(table_name_.first)); + else + create_db_query = std::make_shared(); + create_db_query->setDatabase(new_table_name.first); + create_db_query->if_not_exists = true; + if (serializeAST(*info_db.create_query) != serializeAST(*create_db_query)) + info_db.different_create_query = create_db_query; + } + } + } + } + + /// Prepares to restore a database and all tables in it. + void prepareToRestoreDatabase(const String & database_name_, const std::set & except_list_) + { + /// Check that we are not trying to restore the same database again. + String new_database_name = renaming_config->getNewDatabaseName(database_name_); + if (databases.contains(new_database_name) && databases[new_database_name].is_explicit) + throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Couldn't restore database {} twice", backQuoteIfNeed(new_database_name)); + + Strings table_metadata_filenames = backup->listFiles("metadata/" + escapeForFileName(database_name_) + "/", "/"); + + bool throw_if_no_create_database_query = table_metadata_filenames.empty(); + if (throw_if_no_create_database_query && !hasCreateQueryInBackup(database_name_)) + throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Could not restore database {} because there is no such database in the backup", backQuoteIfNeed(database_name_)); + + /// Of course we're not going to restore the definition of the system or the temporary database. + if (!isSystemOrTemporaryDatabase(new_database_name)) + { + /// Make a create query for this database. + std::shared_ptr create_db_query; + String db_name_in_backup = database_name_; + if (hasCreateQueryInBackup(db_name_in_backup)) + { + create_db_query = renameInCreateQuery(readCreateQueryFromBackup(db_name_in_backup)); + } + else + { + create_db_query = std::make_shared(); + create_db_query->setDatabase(database_name_); + db_name_in_backup.clear(); + } + + create_db_query->if_not_exists = true; + + CreateDatabaseInfo info_db; + info_db.create_query = create_db_query; + info_db.name_in_backup = std::move(db_name_in_backup); + info_db.is_explicit = true; + databases[new_database_name] = std::move(info_db); + } + + /// Restore tables in this database. + for (const String & table_metadata_filename : table_metadata_filenames) + { + String table_name = unescapeForFileName(fs::path{table_metadata_filename}.stem()); + if (except_list_.contains(table_name)) + continue; + prepareToRestoreTable(DatabaseAndTableName{database_name_, table_name}, ASTs{}); + } + } + + /// Prepares to restore all the databases contained in the backup. + void prepareToRestoreAllDatabases(const std::set & except_list_) + { + Strings database_metadata_filenames = backup->listFiles("metadata/", "/"); + for (const String & database_metadata_filename : database_metadata_filenames) + { + String database_name = unescapeForFileName(fs::path{database_metadata_filename}.stem()); + if (except_list_.contains(database_name)) + continue; + prepareToRestoreDatabase(database_name, std::set{}); + } + } + + /// Reads a create query for creating a specified table from the backup. + std::shared_ptr readCreateQueryFromBackup(const DatabaseAndTableName & table_name) const + { + String create_query_path = getMetadataPathInBackup(table_name); + if (!backup->fileExists(create_query_path)) + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Could not restore table {}.{} because there is no such table in the backup", backQuoteIfNeed(table_name.first), backQuoteIfNeed(table_name.second)); + auto read_buffer = backup->readFile(create_query_path)->getReadBuffer(); + String create_query_str; + readStringUntilEOF(create_query_str, *read_buffer); + read_buffer.reset(); + ParserCreateQuery create_parser; + return typeid_cast>(parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH)); + } + + /// Reads a create query for creating a specified database from the backup. + std::shared_ptr readCreateQueryFromBackup(const String & database_name) const + { + String create_query_path = getMetadataPathInBackup(database_name); + if (!backup->fileExists(create_query_path)) + throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Could not restore database {} because there is no such database in the backup", backQuoteIfNeed(database_name)); + auto read_buffer = backup->readFile(create_query_path)->getReadBuffer(); + String create_query_str; + readStringUntilEOF(create_query_str, *read_buffer); + read_buffer.reset(); + ParserCreateQuery create_parser; + return typeid_cast>(parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH)); + } + + /// Whether there is a create query for creating a specified database in the backup. + bool hasCreateQueryInBackup(const String & database_name) const + { + String create_query_path = getMetadataPathInBackup(database_name); + return backup->fileExists(create_query_path); + } + + /// Do renaming in the create query according to the renaming config. + std::shared_ptr renameInCreateQuery(const ASTPtr & ast) const + { + return typeid_cast>(::DB::renameInCreateQuery(ast, renaming_config, context)); + } + + static bool isSystemOrTemporaryDatabase(const String & database_name) + { + return (database_name == DatabaseCatalog::SYSTEM_DATABASE) || (database_name == DatabaseCatalog::TEMPORARY_DATABASE); + } + + /// Information which is used to make an instance of RestoreTableFromBackupTask. + struct CreateTableInfo + { + ASTPtr create_query; + DatabaseAndTableName name_in_backup; + ASTs partitions; + }; + + /// Information which is used to make an instance of RestoreDatabaseFromBackupTask. + struct CreateDatabaseInfo + { + ASTPtr create_query; + String name_in_backup; + + /// Whether the creation of this database is specified explicitly, via RESTORE DATABASE or + /// RESTORE ALL DATABASES. + /// It's false if the creation of this database is caused by creating a table contained in it. + bool is_explicit = false; + + /// If this is set it means the following error: + /// it means that for implicitly created database there were two different create query + /// generated so we cannot restore the database. + ASTPtr different_create_query; + }; + + ContextMutablePtr context; + BackupPtr backup; + BackupRenamingConfigPtr renaming_config; + std::map databases; + std::map tables; + }; + + + /// Reverts completed restore tasks (in reversed order). + void rollbackRestoreTasks(RestoreFromBackupTasks && restore_tasks) + { + for (auto & restore_task : restore_tasks | boost::adaptors::reversed) + { + try + { + std::move(restore_task)->rollback(); + } + catch (...) + { + tryLogCurrentException("Restore", "Couldn't rollback changes after failed RESTORE"); + } + } + } +} + + +RestoreFromBackupTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const Elements & elements) +{ + RestoreTasksBuilder builder{context, backup}; + builder.prepare(elements); + return builder.makeTasks(); +} + + +void executeRestoreTasks(RestoreFromBackupTasks && restore_tasks, size_t num_threads) +{ + if (!num_threads) + num_threads = 1; + + RestoreFromBackupTasks completed_tasks; + bool need_rollback_completed_tasks = true; + + SCOPE_EXIT({ + if (need_rollback_completed_tasks) + rollbackRestoreTasks(std::move(completed_tasks)); + }); + + std::deque> sequential_tasks; + std::deque> enqueued_tasks; + + /// There are two kinds of restore tasks: sequential and non-sequential ones. + /// Sequential tasks are executed first and always in one thread. + for (auto & task : restore_tasks) + { + if (task->isSequential()) + sequential_tasks.push_back(std::move(task)); + else + enqueued_tasks.push_back(std::move(task)); + } + + /// Sequential tasks. + while (!sequential_tasks.empty()) + { + auto current_task = std::move(sequential_tasks.front()); + sequential_tasks.pop_front(); + + RestoreFromBackupTasks new_tasks = current_task->run(); + + completed_tasks.push_back(std::move(current_task)); + for (auto & task : new_tasks) + { + if (task->isSequential()) + sequential_tasks.push_back(std::move(task)); + else + enqueued_tasks.push_back(std::move(task)); + } + } + + /// Non-sequential tasks. + std::unordered_map> running_tasks; + std::vector threads; + std::mutex mutex; + std::condition_variable cond; + std::exception_ptr exception; + + while (true) + { + IRestoreFromBackupTask * current_task = nullptr; + { + std::unique_lock lock{mutex}; + cond.wait(lock, [&] + { + if (exception) + return true; + if (enqueued_tasks.empty()) + return running_tasks.empty(); + return (running_tasks.size() < num_threads); + }); + + if (exception || enqueued_tasks.empty()) + break; + + auto current_task_ptr = std::move(enqueued_tasks.front()); + current_task = current_task_ptr.get(); + enqueued_tasks.pop_front(); + running_tasks[current_task] = std::move(current_task_ptr); + } + + assert(current_task); + threads.emplace_back([current_task, &mutex, &cond, &enqueued_tasks, &running_tasks, &completed_tasks, &exception]() mutable + { + { + std::lock_guard lock{mutex}; + if (exception) + return; + } + + RestoreFromBackupTasks new_tasks; + std::exception_ptr new_exception; + try + { + new_tasks = current_task->run(); + } + catch (...) + { + new_exception = std::current_exception(); + } + + { + std::lock_guard lock{mutex}; + auto current_task_it = running_tasks.find(current_task); + auto current_task_ptr = std::move(current_task_it->second); + running_tasks.erase(current_task_it); + + if (!new_exception) + { + completed_tasks.push_back(std::move(current_task_ptr)); + enqueued_tasks.insert( + enqueued_tasks.end(), std::make_move_iterator(new_tasks.begin()), std::make_move_iterator(new_tasks.end())); + } + + if (!exception) + exception = new_exception; + + cond.notify_all(); + } + }); + } + + for (auto & thread : threads) + thread.join(); + + if (exception) + std::rethrow_exception(exception); + else + need_rollback_completed_tasks = false; +} + +} diff --git a/src/Backups/RestoreFromBackupUtils.h b/src/Backups/RestoreFromBackupUtils.h new file mode 100644 index 000000000000..75fe96495d6c --- /dev/null +++ b/src/Backups/RestoreFromBackupUtils.h @@ -0,0 +1,23 @@ +#pragma once + +#include + + +namespace DB +{ + +class IBackup; +using BackupPtr = std::shared_ptr; +class IRestoreFromBackupTask; +using RestoreFromBackupTaskPtr = std::unique_ptr; +using RestoreFromBackupTasks = std::vector; +class Context; +using ContextMutablePtr = std::shared_ptr; + +/// Prepares restore tasks. +RestoreFromBackupTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const ASTBackupQuery::Elements & elements); + +/// Executes restore tasks. +void executeRestoreTasks(RestoreFromBackupTasks && tasks, size_t num_threads); + +} diff --git a/src/Backups/hasCompatibleDataToRestoreTable.cpp b/src/Backups/hasCompatibleDataToRestoreTable.cpp index 9c11d371bb08..8ac2fda93d84 100644 --- a/src/Backups/hasCompatibleDataToRestoreTable.cpp +++ b/src/Backups/hasCompatibleDataToRestoreTable.cpp @@ -16,6 +16,10 @@ bool hasCompatibleDataToRestoreTable(const ASTCreateQuery & query1, const ASTCre q1->uuid = UUIDHelpers::Nil; q2->uuid = UUIDHelpers::Nil; + /// Clear IF NOT EXISTS flag. + q1->if_not_exists = false; + q2->if_not_exists = false; + return serializeAST(*q1) == serializeAST(*q2); } diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index e991daf32091..abbbecf374a4 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -612,6 +612,7 @@ M(641, CANNOT_APPEND_TO_FILE) \ M(642, CANNOT_PACK_ARCHIVE) \ M(643, CANNOT_UNPACK_ARCHIVE) \ + M(644, CANNOT_RESTORE_DATABASE) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Interpreters/InterpreterBackupQuery.cpp b/src/Interpreters/InterpreterBackupQuery.cpp index 45eb8e485993..de8a9c273cd0 100644 --- a/src/Interpreters/InterpreterBackupQuery.cpp +++ b/src/Interpreters/InterpreterBackupQuery.cpp @@ -2,8 +2,10 @@ #include #include #include +#include #include #include +#include #include #include @@ -45,7 +47,7 @@ namespace void executeRestore(const ASTBackupQuery & query, ContextMutablePtr context) { BackupPtr backup = createBackup(query, context); - auto restore_tasks = makeRestoreTasks(query.elements, context, backup); + auto restore_tasks = makeRestoreTasks(context, backup, query.elements); executeRestoreTasks(std::move(restore_tasks), context->getSettingsRef().max_backup_threads); } } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index a923258b111c..5bd3833ebe11 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -222,7 +222,7 @@ BackupEntries IStorage::backup(const ASTs &, ContextPtr) throw Exception("Table engine " + getName() + " doesn't support backups", ErrorCodes::NOT_IMPLEMENTED); } -RestoreDataTasks IStorage::restoreFromBackup(const BackupPtr &, const String &, const ASTs &, ContextMutablePtr) +RestoreFromBackupTaskPtr IStorage::restoreFromBackup(const BackupPtr &, const String &, const ASTs &, ContextMutablePtr) { throw Exception("Table engine " + getName() + " doesn't support restoring", ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index fc79c7d174f4..68548634bed2 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -71,7 +71,8 @@ class IBackup; using BackupPtr = std::shared_ptr; class IBackupEntry; using BackupEntries = std::vector>>; -using RestoreDataTasks = std::vector>; +class IRestoreFromBackupTask; +using RestoreFromBackupTaskPtr = std::unique_ptr; struct ColumnSize { @@ -215,7 +216,7 @@ class IStorage : public std::enable_shared_from_this, public TypePromo virtual BackupEntries backup(const ASTs & partitions, ContextPtr context); /// Extract data from the backup and put it to the storage. - virtual RestoreDataTasks restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context); + virtual RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context); /// Returns whether the column is virtual - by default all columns are real. /// Initially reserved virtual column name may be shadowed by real column. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 68fa81e1df9a..a60a8a3388df 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -3633,40 +3634,79 @@ BackupEntries MergeTreeData::backupDataParts(const DataPartsVector & data_parts) } -RestoreDataTasks MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & backup, const String & data_path_in_backup, - const std::unordered_set & partition_ids, - SimpleIncrement * increment) +class MergeTreeDataRestoreTask : public IRestoreFromBackupTask { - RestoreDataTasks restore_tasks; +public: + MergeTreeDataRestoreTask( + const std::shared_ptr & storage_, + const BackupPtr & backup_, + const String & data_path_in_backup_, + const std::unordered_set & partition_ids_, + SimpleIncrement * increment_) + : storage(storage_) + , backup(backup_) + , data_path_in_backup(data_path_in_backup_) + , partition_ids(partition_ids_) + , increment(increment_) + { + } - Strings part_names = backup->listFiles(data_path_in_backup); - for (const String & part_name : part_names) + RestoreFromBackupTasks run() override { - const auto part_info = MergeTreePartInfo::tryParsePartName(part_name, format_version); + RestoreFromBackupTasks restore_part_tasks; + Strings part_names = backup->listFiles(data_path_in_backup); + for (const String & part_name : part_names) + { + const auto part_info = MergeTreePartInfo::tryParsePartName(part_name, storage->format_version); - if (!part_info) - continue; + if (!part_info) + continue; - if (!partition_ids.empty() && !partition_ids.contains(part_info->partition_id)) - continue; + if (!partition_ids.empty() && !partition_ids.contains(part_info->partition_id)) + continue; - UInt64 total_size_of_part = 0; - Strings filenames = backup->listFiles(data_path_in_backup + part_name + "/", ""); - for (const String & filename : filenames) - total_size_of_part += backup->getFileSize(data_path_in_backup + part_name + "/" + filename); + restore_part_tasks.push_back( + std::make_unique(storage, backup, data_path_in_backup, part_name, *part_info, increment)); + } + return restore_part_tasks; + } - std::shared_ptr reservation = getStoragePolicy()->reserveAndCheck(total_size_of_part); +private: + std::shared_ptr storage; + BackupPtr backup; + String data_path_in_backup; + std::unordered_set partition_ids; + SimpleIncrement * increment; - auto restore_task = [this, - backup, - data_path_in_backup, - part_name, - part_info = std::move(part_info), - filenames = std::move(filenames), - reservation, - increment]() + class RestorePartTask : public IRestoreFromBackupTask + { + public: + RestorePartTask( + const std::shared_ptr & storage_, + const BackupPtr & backup_, + const String & data_path_in_backup_, + const String & part_name_, + const MergeTreePartInfo & part_info_, + SimpleIncrement * increment_) + : storage(storage_) + , backup(backup_) + , data_path_in_backup(data_path_in_backup_) + , part_name(part_name_) + , part_info(part_info_) + , increment(increment_) { + } + + RestoreFromBackupTasks run() override + { + UInt64 total_size_of_part = 0; + Strings filenames = backup->listFiles(data_path_in_backup + part_name + "/", ""); + for (const String & filename : filenames) + total_size_of_part += backup->getFileSize(data_path_in_backup + part_name + "/" + filename); + + std::shared_ptr reservation = storage->getStoragePolicy()->reserveAndCheck(total_size_of_part); auto disk = reservation->getDisk(); + String relative_data_path = storage->getRelativeDataPath(); auto temp_part_dir_owner = std::make_shared(disk, relative_data_path + "restoring_" + part_name + "_"); String temp_part_dir = temp_part_dir_owner->getPath(); @@ -3681,18 +3721,33 @@ RestoreDataTasks MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & bac auto read_buffer = backup_entry->getReadBuffer(); auto write_buffer = disk->writeFile(temp_part_dir + "/" + filename); copyData(*read_buffer, *write_buffer); + reservation->update(reservation->getSize() - backup_entry->getSize()); } auto single_disk_volume = std::make_shared(disk->getName(), disk, 0); - auto part = createPart(part_name, *part_info, single_disk_volume, relative_temp_part_dir); + auto part = storage->createPart(part_name, part_info, single_disk_volume, relative_temp_part_dir); part->loadColumnsChecksumsIndexes(false, true); - renameTempPartAndAdd(part, increment); - }; + storage->renameTempPartAndAdd(part, increment); + return {}; + } - restore_tasks.emplace_back(std::move(restore_task)); - } + private: + std::shared_ptr storage; + BackupPtr backup; + String data_path_in_backup; + String part_name; + MergeTreePartInfo part_info; + SimpleIncrement * increment; + }; +}; - return restore_tasks; + +RestoreFromBackupTaskPtr MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & backup, const String & data_path_in_backup, + const std::unordered_set & partition_ids, + SimpleIncrement * increment) +{ + return std::make_unique( + std::static_pointer_cast(shared_from_this()), backup, data_path_in_backup, partition_ids, increment); } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 93add8d69355..6db2dc7ffe74 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -654,7 +654,7 @@ class MergeTreeData : public IStorage, public WithMutableContext static BackupEntries backupDataParts(const DataPartsVector & data_parts); /// Extract data from the backup and put it to the storage. - RestoreDataTasks restoreDataPartsFromBackup( + RestoreFromBackupTaskPtr restoreDataPartsFromBackup( const BackupPtr & backup, const String & data_path_in_backup, const std::unordered_set & partition_ids, diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 5f0bd240f647..fba48f13d6a8 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include #include @@ -948,43 +949,57 @@ BackupEntries StorageLog::backup(const ASTs & partitions, ContextPtr context) return backup_entries; } -RestoreDataTasks StorageLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) +class LogRestoreTask : public IRestoreFromBackupTask { - if (!partitions.empty()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); + using WriteLock = StorageLog::WriteLock; + using Mark = StorageLog::Mark; + +public: + LogRestoreTask( + std::shared_ptr storage_, const BackupPtr & backup_, const String & data_path_in_backup_, ContextMutablePtr context_) + : storage(storage_), backup(backup_), data_path_in_backup(data_path_in_backup_), context(context_) + { + } - auto restore_task = [this, backup, data_path_in_backup, context]() + RestoreFromBackupTasks run() override { auto lock_timeout = getLockTimeout(context); - WriteLock lock{rwlock, lock_timeout}; + WriteLock lock{storage->rwlock, lock_timeout}; if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + const auto num_data_files = storage->num_data_files; if (!num_data_files) - return; + return {}; + + auto & file_checker = storage->file_checker; /// Load the marks if not loaded yet. We have to do that now because we're going to update these marks. - loadMarks(lock); + storage->loadMarks(lock); /// If there were no files, save zero file sizes to be able to rollback in case of error. - saveFileSizes(lock); + storage->saveFileSizes(lock); try { /// Append data files. + auto & data_files = storage->data_files; for (const auto & data_file : data_files) { String file_path_in_backup = data_path_in_backup + fileName(data_file.path); auto backup_entry = backup->readFile(file_path_in_backup); + const auto & disk = storage->disk; auto in = backup_entry->getReadBuffer(); - auto out = disk->writeFile(data_file.path, max_compress_block_size, WriteMode::Append); + auto out = disk->writeFile(data_file.path, storage->max_compress_block_size, WriteMode::Append); copyData(*in, *out); } + const bool use_marks_file = storage->use_marks_file; if (use_marks_file) { /// Append marks. size_t num_extra_marks = 0; + const auto & marks_file_path = storage->marks_file_path; String file_path_in_backup = data_path_in_backup + fileName(marks_file_path); size_t file_size = backup->getFileSize(file_path_in_backup); if (file_size % (num_data_files * sizeof(Mark)) != 0) @@ -1023,19 +1038,34 @@ RestoreDataTasks StorageLog::restoreFromBackup(const BackupPtr & backup, const S } /// Finish writing. - saveMarks(lock); - saveFileSizes(lock); + storage->saveMarks(lock); + storage->saveFileSizes(lock); } catch (...) { /// Rollback partial writes. file_checker.repair(); - removeUnsavedMarks(lock); + storage->removeUnsavedMarks(lock); throw; } - }; - return {restore_task}; + return {}; + } + +private: + std::shared_ptr storage; + BackupPtr backup; + String data_path_in_backup; + ContextMutablePtr context; +}; + +RestoreFromBackupTaskPtr StorageLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) +{ + if (!partitions.empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); + + return std::make_unique( + typeid_cast>(shared_from_this()), backup, data_path_in_backup, context); } diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 8b2ef0ccac15..3d527521c376 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -23,6 +23,7 @@ class StorageLog final : public shared_ptr_helper, public IStorage { friend class LogSource; friend class LogSink; + friend class LogRestoreTask; friend struct shared_ptr_helper; public: @@ -52,7 +53,7 @@ class StorageLog final : public shared_ptr_helper, public IStorage ColumnSizeByName getColumnSizes() const override; BackupEntries backup(const ASTs & partitions, ContextPtr context) override; - RestoreDataTasks restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; + RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; protected: /** Attach the table with the appropriate name, along the appropriate path (with / at the end), diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 6cd1e2d66af5..be0416559c54 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -3,7 +3,7 @@ #include #include - +#include #include #include #include @@ -1645,7 +1645,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ } -RestoreDataTasks StorageMergeTree::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr local_context) +RestoreFromBackupTaskPtr StorageMergeTree::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr local_context) { return restoreDataPartsFromBackup(backup, data_path_in_backup, getPartitionIDsFromQuery(partitions, local_context), &increment); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index d3970449ceb0..2079bf903eb9 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -97,7 +97,7 @@ class StorageMergeTree final : public shared_ptr_helper, publi CheckResults checkData(const ASTPtr & query, ContextPtr context) override; - RestoreDataTasks restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; + RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index c401d27a8fc3..f7e63340dba9 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -38,6 +38,7 @@ #include #include #include +#include #include #include @@ -553,37 +554,51 @@ BackupEntries StorageStripeLog::backup(const ASTs & partitions, ContextPtr conte return backup_entries; } -RestoreDataTasks StorageStripeLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) +class StripeLogRestoreTask : public IRestoreFromBackupTask { - if (!partitions.empty()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); + using WriteLock = StorageStripeLog::WriteLock; - auto restore_task = [this, backup, data_path_in_backup, context]() +public: + StripeLogRestoreTask( + const std::shared_ptr storage_, + const BackupPtr & backup_, + const String & data_path_in_backup_, + ContextMutablePtr context_) + : storage(storage_), backup(backup_), data_path_in_backup(data_path_in_backup_), context(context_) { - WriteLock lock{rwlock, getLockTimeout(context)}; + } + + RestoreFromBackupTasks run() override + { + WriteLock lock{storage->rwlock, getLockTimeout(context)}; if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + auto & file_checker = storage->file_checker; + /// Load the indices if not loaded yet. We have to do that now because we're going to update these indices. - loadIndices(lock); + storage->loadIndices(lock); /// If there were no files, save zero file sizes to be able to rollback in case of error. - saveFileSizes(lock); + storage->saveFileSizes(lock); try { /// Append the data file. - auto old_data_size = file_checker.getFileSize(data_file_path); + auto old_data_size = file_checker.getFileSize(storage->data_file_path); { + const auto & data_file_path = storage->data_file_path; String file_path_in_backup = data_path_in_backup + fileName(data_file_path); auto backup_entry = backup->readFile(file_path_in_backup); + const auto & disk = storage->disk; auto in = backup_entry->getReadBuffer(); - auto out = disk->writeFile(data_file_path, max_compress_block_size, WriteMode::Append); + auto out = disk->writeFile(data_file_path, storage->max_compress_block_size, WriteMode::Append); copyData(*in, *out); } /// Append the index. { + const auto & index_file_path = storage->index_file_path; String index_path_in_backup = data_path_in_backup + fileName(index_file_path); IndexForNativeFormat extra_indices; auto backup_entry = backup->readFile(index_path_in_backup); @@ -598,23 +613,38 @@ RestoreDataTasks StorageStripeLog::restoreFromBackup(const BackupPtr & backup, c column.location.offset_in_compressed_file += old_data_size; } - insertAtEnd(indices.blocks, std::move(extra_indices.blocks)); + insertAtEnd(storage->indices.blocks, std::move(extra_indices.blocks)); } /// Finish writing. - saveIndices(lock); - saveFileSizes(lock); + storage->saveIndices(lock); + storage->saveFileSizes(lock); + return {}; } catch (...) { /// Rollback partial writes. file_checker.repair(); - removeUnsavedIndices(lock); + storage->removeUnsavedIndices(lock); throw; } + } - }; - return {restore_task}; +private: + std::shared_ptr storage; + BackupPtr backup; + String data_path_in_backup; + ContextMutablePtr context; +}; + + +RestoreFromBackupTaskPtr StorageStripeLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) +{ + if (!partitions.empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); + + return std::make_unique( + typeid_cast>(shared_from_this()), backup, data_path_in_backup, context); } diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 579e2f991e70..5266a8627e27 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -23,6 +23,7 @@ class StorageStripeLog final : public shared_ptr_helper, publi { friend class StripeLogSource; friend class StripeLogSink; + friend class StripeLogRestoreTask; friend struct shared_ptr_helper; public: @@ -51,7 +52,7 @@ class StorageStripeLog final : public shared_ptr_helper, publi void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder&) override; BackupEntries backup(const ASTs & partitions, ContextPtr context) override; - RestoreDataTasks restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; + RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; protected: StorageStripeLog( From c68b81a25fe28ec427a7c834b71340eb2a5b2aca Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 18 Jan 2022 16:44:42 +0700 Subject: [PATCH 02/18] Make BACKUP more consistent with RESTORE. --- src/Backups/BackupUtils.cpp | 531 +++++++++++++++--------------------- src/Backups/BackupUtils.h | 6 - src/Common/ErrorCodes.cpp | 5 +- 3 files changed, 221 insertions(+), 321 deletions(-) diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 35c2fb336312..820a7f86f7fb 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -10,18 +10,15 @@ #include #include #include -#include -#include -#include namespace DB { namespace ErrorCodes { - extern const int BACKUP_ELEMENT_DUPLICATE; + extern const int CANNOT_BACKUP_TABLE; + extern const int CANNOT_BACKUP_DATABASE; extern const int BACKUP_IS_EMPTY; - extern const int LOGICAL_ERROR; } namespace @@ -31,33 +28,6 @@ namespace using Elements = ASTBackupQuery::Elements; using ElementType = ASTBackupQuery::ElementType; - /// Replace elements of types DICTIONARY or EVERYTHING with elements of other types. - void replaceElementTypesWithBaseElementTypes(Elements & elements) - { - for (size_t i = 0; i != elements.size(); ++i) - { - auto & element = elements[i]; - switch (element.type) - { - case ElementType::DICTIONARY: - { - element.type = ElementType::TABLE; - break; - } - - case ElementType::EVERYTHING: - { - element.type = ElementType::ALL_DATABASES; - auto & new_element = elements.emplace_back(); - new_element.type = ElementType::ALL_TEMPORARY_TABLES; - break; - } - - default: - break; - } - } - } /// Replaces an empty database with the current database. void replaceEmptyDatabaseWithCurrentDatabase(Elements & elements, const String & current_database) @@ -74,353 +44,287 @@ namespace } } - /// Replaces elements of types TEMPORARY_TABLE or ALL_TEMPORARY_TABLES with elements of type TABLE or DATABASE. - void replaceTemporaryTablesWithTemporaryDatabase(Elements & elements) - { - for (auto & element : elements) - { - switch (element.type) - { - case ElementType::TEMPORARY_TABLE: - { - element.type = ElementType::TABLE; - element.name.first = DatabaseCatalog::TEMPORARY_DATABASE; - if (element.new_name.first.empty() && !element.new_name.second.empty()) - element.new_name.first = DatabaseCatalog::TEMPORARY_DATABASE; - break; - } - - case ElementType::ALL_TEMPORARY_TABLES: - { - element.type = ElementType::DATABASE; - element.name.first = DatabaseCatalog::TEMPORARY_DATABASE; - break; - } - - default: - break; - } - } - } - /// Set new names if they are not specified. - void setNewNamesIfNotSet(Elements & elements) + /// Makes backup entries to backup databases and tables according to the elements of ASTBackupQuery. + /// Keep this class consistent with RestoreTasksBuilder. + class BackupEntriesBuilder { - for (auto & element : elements) - { - switch (element.type) - { - case ElementType::TABLE: - { - if (element.new_name.second.empty()) - element.new_name = element.name; - break; - } + public: + BackupEntriesBuilder(ContextPtr context_) : context(context_) { } - case ElementType::DATABASE: - { - if (element.new_name.first.empty()) - element.new_name = element.name; - break; - } - - default: - break; - } - } - } - - /// Removes duplications in the elements of a backup query by removing some excessive elements and by updating except_lists. - /// This function helps deduplicate elements in queries like "BACKUP ALL DATABASES, DATABASE xxx USING NAME yyy" - /// (we need a deduplication for that query because `ALL DATABASES` includes `xxx` however we don't want - /// to backup/restore the same database twice while executing the same query). - /// Also this function slightly reorders elements: it puts databases before tables and dictionaries they contain. - void deduplicateAndReorderElements(Elements & elements) - { - std::set skip_indices; /// Indices of elements which should be removed in the end of this function. - size_t index_all_databases = static_cast(-1); /// Index of the first element of type ALL_DATABASES or -1 if not found. - - struct DatabaseInfo + /// Prepares internal structures for making backup entries. + void prepare(const ASTBackupQuery::Elements & elements) { - size_t index = static_cast(-1); - std::unordered_map tables; - }; - std::unordered_map databases; /// Found databases and tables. + auto elements2 = elements; + replaceEmptyDatabaseWithCurrentDatabase(elements2, context->getCurrentDatabase()); - for (size_t i = 0; i != elements.size(); ++i) - { - auto & element = elements[i]; - switch (element.type) + auto new_renaming_config = std::make_shared(); + new_renaming_config->setFromBackupQueryElements(elements2); + renaming_config = new_renaming_config; + + for (const auto & element : elements2) { - case ElementType::TABLE: + switch (element.type) { - auto & tables = databases.emplace(element.name.first, DatabaseInfo{}).first->second.tables; - auto it = tables.find(element.name.second); - if (it == tables.end()) - { - tables.emplace(element.name.second, i); - } - else + case ElementType::TABLE: [[fallthrough]]; + case ElementType::DICTIONARY: { - size_t prev_index = it->second; - if ((elements[i].new_name == elements[prev_index].new_name) - && (elements[i].partitions.empty() == elements[prev_index].partitions.empty())) - { - insertAtEnd(elements[prev_index].partitions, elements[i].partitions); - skip_indices.emplace(i); - } - else - { - throw Exception( - "Table " + backQuote(element.name.first) + "." + backQuote(element.name.second) + " was specified twice", - ErrorCodes::BACKUP_ELEMENT_DUPLICATE); - } + const String & database_name = element.name.first; + const String & table_name = element.name.second; + prepareToBackupTable(DatabaseAndTableName{database_name, table_name}, element.partitions); + break; } - break; - } - case ElementType::DATABASE: - { - auto it = databases.find(element.name.first); - if (it == databases.end()) + case ElementType::TEMPORARY_TABLE: { - DatabaseInfo new_db_info; - new_db_info.index = i; - databases.emplace(element.name.first, new_db_info); + String database_name = DatabaseCatalog::TEMPORARY_DATABASE; + const String & table_name = element.name.second; + prepareToBackupTable(DatabaseAndTableName{database_name, table_name}, element.partitions); + break; } - else if (it->second.index == static_cast(-1)) + + case ElementType::DATABASE: { - it->second.index = i; + const String & database_name = element.name.first; + prepareToBackupDatabase(database_name, element.except_list); + break; } - else - { - size_t prev_index = it->second.index; - if ((elements[i].new_name == elements[prev_index].new_name) - && (elements[i].except_list == elements[prev_index].except_list)) - { - skip_indices.emplace(i); - } - else - { - throw Exception("Database " + backQuote(element.name.first) + " was specified twice", ErrorCodes::BACKUP_ELEMENT_DUPLICATE); - } + case ElementType::ALL_TEMPORARY_TABLES: + { + prepareToBackupDatabase(DatabaseCatalog::TEMPORARY_DATABASE, element.except_list); + break; } - break; - } - case ElementType::ALL_DATABASES: - { - if (index_all_databases == static_cast(-1)) + case ElementType::ALL_DATABASES: { - index_all_databases = i; + prepareToBackupAllDatabases(element.except_list); + break; } - else + + case ElementType::EVERYTHING: { - size_t prev_index = index_all_databases; - if (elements[i].except_list == elements[prev_index].except_list) - skip_indices.emplace(i); - else - throw Exception("The tag ALL DATABASES was specified twice", ErrorCodes::BACKUP_ELEMENT_DUPLICATE); + prepareToBackupAllDatabases({}); + prepareToBackupDatabase(DatabaseCatalog::TEMPORARY_DATABASE, {}); + break; } - break; } - - default: - /// replaceElementTypesWithBaseElementTypes() and replaceTemporaryTablesWithTemporaryDatabase() should have removed all other element types. - throw Exception("Unexpected element type: " + std::to_string(static_cast(element.type)), ErrorCodes::LOGICAL_ERROR); } } - if (index_all_databases != static_cast(-1)) + /// Makes backup entries, should be called after prepare(). + BackupEntries makeBackupEntries() const { - for (auto & [database_name, database] : databases) + /// Check that there are not `different_create_query`. (If it's set it means error.) + for (auto & info : databases | boost::adaptors::map_values) { - elements[index_all_databases].except_list.emplace(database_name); - if (database.index == static_cast(-1)) + if (info.different_create_query) + throw Exception(ErrorCodes::CANNOT_BACKUP_DATABASE, + "Couldn't backup a database because two different create queries were generated for it: {} and {}", + serializeAST(*info.create_query), serializeAST(*info.different_create_query)); + } + + BackupEntries res; + for (auto & info : databases | boost::adaptors::map_values) + res.push_back(makeBackupEntryForMetadata(*info.create_query)); + + for (auto & info : tables | boost::adaptors::map_values) + { + res.push_back(makeBackupEntryForMetadata(*info.create_query)); + + auto data_backup = info.storage->backup(info.partitions, context); + if (!data_backup.empty()) { - auto & new_element = elements.emplace_back(); - new_element.type = ElementType::DATABASE; - new_element.name.first = database_name; - new_element.new_name = new_element.name; - database.index = elements.size() - 1; + String data_path = getDataPathInBackup(*info.create_query); + for (auto & [path_in_backup, backup_entry] : data_backup) + res.emplace_back(data_path + path_in_backup, std::move(backup_entry)); } } + + /// A backup cannot be empty. + if (res.empty()) + throw Exception("Backup must not be empty", ErrorCodes::BACKUP_IS_EMPTY); + + return res; } - for (auto & [database_name, database] : databases) + private: + /// Prepares to backup a single table and probably its database's definition. + void prepareToBackupTable(const DatabaseAndTableName & table_name_, const ASTs & partitions_) { - if (database.index == static_cast(-1)) - continue; - for (const auto & [table_name, table_index] : database.tables) - elements[database.index].except_list.emplace(table_name); + auto [database, storage] = DatabaseCatalog::instance().getDatabaseAndTable({table_name_.first, table_name_.second}, context); + prepareToBackupTable(table_name_, {database, storage}, partitions_); } - for (auto skip_index : skip_indices | boost::adaptors::reversed) - elements.erase(elements.begin() + skip_index); - } - - Elements adjustElements(const Elements & elements, const String & current_database) - { - auto res = elements; - replaceElementTypesWithBaseElementTypes(res); - replaceEmptyDatabaseWithCurrentDatabase(res, current_database); - replaceTemporaryTablesWithTemporaryDatabase(res); - setNewNamesIfNotSet(res); - deduplicateAndReorderElements(res); - return res; - } + void prepareToBackupTable(const DatabaseAndTableName & table_name_, const DatabaseAndTable & table_, const ASTs & partitions_) + { + context->checkAccess(AccessType::SELECT, table_name_.first, table_name_.second); - void backupCreateQuery(const IAST & create_query, BackupEntries & backup_entries) - { - auto metadata_entry = std::make_unique(serializeAST(create_query)); - String metadata_path = getMetadataPathInBackup(create_query); - backup_entries.emplace_back(metadata_path, std::move(metadata_entry)); - } + /// Check that we are not trying to backup the same table again. + DatabaseAndTableName new_table_name = renaming_config->getNewTableName(table_name_); + if (tables.contains(new_table_name)) + { + String message; + if (new_table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) + message = fmt::format("Couldn't backup temporary table {} twice", backQuoteIfNeed(new_table_name.second)); + else + message = fmt::format("Couldn't backup table {}.{} twice", backQuoteIfNeed(new_table_name.first), backQuoteIfNeed(new_table_name.second)); + throw Exception(ErrorCodes::CANNOT_BACKUP_TABLE, message); + } - void backupTable( - const DatabaseAndTable & database_and_table, - const String & table_name, - const ASTs & partitions, - const ContextPtr & context, - const BackupRenamingConfigPtr & renaming_config, - BackupEntries & backup_entries) - { - const auto & database = database_and_table.first; - const auto & storage = database_and_table.second; - context->checkAccess(AccessType::SELECT, database->getDatabaseName(), table_name); + /// Make a create query for this table. + auto create_query = renameInCreateQuery(table_.first->getCreateTableQuery(table_name_.second, context)); - auto create_query = database->getCreateTableQuery(table_name, context); - ASTPtr new_create_query = renameInCreateQuery(create_query, renaming_config, context); - backupCreateQuery(*new_create_query, backup_entries); + CreateTableInfo info; + info.create_query = create_query; + info.storage = table_.second; + info.name_in_backup = new_table_name; + info.partitions = partitions_; + tables[new_table_name] = std::move(info); - auto data_backup = storage->backup(partitions, context); - if (!data_backup.empty()) - { - String data_path = getDataPathInBackup(*new_create_query); - for (auto & [path_in_backup, backup_entry] : data_backup) - backup_entries.emplace_back(data_path + path_in_backup, std::move(backup_entry)); + /// If it's not system or temporary database then probably we need to backup the database's definition too. + if (!isSystemOrTemporaryDatabase(table_name_.first)) + { + if (!databases.contains(new_table_name.first)) + { + /// Add a create query to backup the database if we haven't done it yet. + auto create_db_query = renameInCreateQuery(table_.first->getCreateDatabaseQuery()); + create_db_query->setDatabase(new_table_name.first); + + CreateDatabaseInfo info_db; + info_db.create_query = create_db_query; + info_db.original_name = table_name_.first; + info_db.is_explicit = false; + databases[new_table_name.first] = std::move(info_db); + } + else + { + /// We already have added a create query to backup the database, + /// set `different_create_query` if it's not the same. + auto & info_db = databases[new_table_name.first]; + if (!info_db.is_explicit && (info_db.original_name != table_name_.first) && !info_db.different_create_query) + { + auto create_db_query = renameInCreateQuery(table_.first->getCreateDatabaseQuery()); + create_db_query->setDatabase(new_table_name.first); + if (serializeAST(*info_db.create_query) != serializeAST(*create_db_query)) + info_db.different_create_query = create_db_query; + } + } + } } - } - - void backupDatabase( - const DatabasePtr & database, - const std::set & except_list, - const ContextPtr & context, - const BackupRenamingConfigPtr & renaming_config, - BackupEntries & backup_entries) - { - context->checkAccess(AccessType::SHOW_TABLES, database->getDatabaseName()); - - auto create_query = database->getCreateDatabaseQuery(); - ASTPtr new_create_query = renameInCreateQuery(create_query, renaming_config, context); - backupCreateQuery(*new_create_query, backup_entries); - for (auto it = database->getTablesIteratorForBackup(context); it->isValid(); it->next()) + /// Prepares to restore a database and all tables in it. + void prepareToBackupDatabase(const String & database_name_, const std::set & except_list_) { - if (except_list.contains(it->name())) - continue; - backupTable({database, it->table()}, it->name(), {}, context, renaming_config, backup_entries); + auto database = DatabaseCatalog::instance().getDatabase(database_name_, context); + prepareToBackupDatabase(database_name_, database, except_list_); } - } - void backupAllDatabases( - const std::set & except_list, - const ContextPtr & context, - const BackupRenamingConfigPtr & renaming_config, - BackupEntries & backup_entries) - { - for (const auto & [database_name, database] : DatabaseCatalog::instance().getDatabases()) + void prepareToBackupDatabase(const String & database_name_, const DatabasePtr & database_, const std::set & except_list_) { - if (except_list.contains(database_name)) - continue; - if (database_name == DatabaseCatalog::SYSTEM_DATABASE || database_name == DatabaseCatalog::TEMPORARY_DATABASE) - continue; - backupDatabase(database, {}, context, renaming_config, backup_entries); - } - } -} - - -BackupEntries makeBackupEntries(const Elements & elements, const ContextPtr & context) -{ - BackupEntries backup_entries; + context->checkAccess(AccessType::SHOW_DATABASES, database_name_); - auto elements2 = adjustElements(elements, context->getCurrentDatabase()); - auto renaming_config = std::make_shared(); - renaming_config->setFromBackupQueryElements(elements2); + /// Check that we are not trying to restore the same database again. + String new_database_name = renaming_config->getNewDatabaseName(database_name_); + if (databases.contains(new_database_name) && databases[new_database_name].is_explicit) + throw Exception(ErrorCodes::CANNOT_BACKUP_DATABASE, "Couldn't backup database {} twice", backQuoteIfNeed(new_database_name)); - for (const auto & element : elements2) - { - switch (element.type) - { - case ElementType::TABLE: + /// Of course we're not going to backup the definition of the system or the temporary database. + if (!isSystemOrTemporaryDatabase(database_name_)) { - const String & database_name = element.name.first; - const String & table_name = element.name.second; - auto [database, storage] = DatabaseCatalog::instance().getDatabaseAndTable({database_name, table_name}, context); - backupTable({database, storage}, table_name, element.partitions, context, renaming_config, backup_entries); - break; + /// Make a create query for this database. + auto create_db_query = renameInCreateQuery(database_->getCreateDatabaseQuery()); + + CreateDatabaseInfo info_db; + info_db.create_query = create_db_query; + info_db.original_name = database_name_; + info_db.is_explicit = true; + databases[new_database_name] = std::move(info_db); } - case ElementType::DATABASE: + /// Backup tables in this database. + for (auto it = database_->getTablesIteratorForBackup(context); it->isValid(); it->next()) { - const String & database_name = element.name.first; - auto database = DatabaseCatalog::instance().getDatabase(database_name, context); - backupDatabase(database, element.except_list, context, renaming_config, backup_entries); - break; + if (except_list_.contains(it->name())) + continue; + prepareToBackupTable({database_name_, it->name()}, {database_, it->table()}, {}); } + } - case ElementType::ALL_DATABASES: + /// Prepares to backup all the databases contained in the backup. + void prepareToBackupAllDatabases(const std::set & except_list_) + { + for (const auto & [database_name, database] : DatabaseCatalog::instance().getDatabases()) { - backupAllDatabases(element.except_list, context, renaming_config, backup_entries); - break; + if (except_list_.contains(database_name)) + continue; + if (isSystemOrTemporaryDatabase(database_name)) + continue; + prepareToBackupDatabase(database_name, database, {}); } + } - default: - throw Exception("Unexpected element type", ErrorCodes::LOGICAL_ERROR); /// other element types have been removed in deduplicateElements() + /// Do renaming in the create query according to the renaming config. + std::shared_ptr renameInCreateQuery(const ASTPtr & ast) const + { + return typeid_cast>(::DB::renameInCreateQuery(ast, renaming_config, context)); } - } - /// A backup cannot be empty. - if (backup_entries.empty()) - throw Exception("Backup must not be empty", ErrorCodes::BACKUP_IS_EMPTY); + static bool isSystemOrTemporaryDatabase(const String & database_name) + { + return (database_name == DatabaseCatalog::SYSTEM_DATABASE) || (database_name == DatabaseCatalog::TEMPORARY_DATABASE); + } - /// Check that all backup entries are unique. - ::sort( - backup_entries.begin(), - backup_entries.end(), - [](const std::pair> & lhs, const std::pair> & rhs) + static std::pair makeBackupEntryForMetadata(const IAST & create_query) { - return lhs.first < rhs.first; - }); - auto adjacent = std::adjacent_find(backup_entries.begin(), backup_entries.end()); - if (adjacent != backup_entries.end()) - throw Exception("Cannot write multiple entries with the same name " + quoteString(adjacent->first), ErrorCodes::BACKUP_ELEMENT_DUPLICATE); + auto metadata_entry = std::make_unique(serializeAST(create_query)); + String metadata_path = getMetadataPathInBackup(create_query); + return {metadata_path, std::move(metadata_entry)}; + } - return backup_entries; + /// Information which is used to make an instance of RestoreTableFromBackupTask. + struct CreateTableInfo + { + ASTPtr create_query; + StoragePtr storage; + DatabaseAndTableName name_in_backup; + ASTs partitions; + }; + + /// Information which is used to make an instance of RestoreDatabaseFromBackupTask. + struct CreateDatabaseInfo + { + ASTPtr create_query; + String original_name; + + /// Whether the creation of this database is specified explicitly, via RESTORE DATABASE or + /// RESTORE ALL DATABASES. + /// It's false if the creation of this database is caused by creating a table contained in it. + bool is_explicit = false; + + /// If this is set it means the following error: + /// it means that for implicitly created database there were two different create query + /// generated so we cannot restore the database. + ASTPtr different_create_query; + }; + + ContextPtr context; + BackupMutablePtr backup; + BackupRenamingConfigPtr renaming_config; + std::map databases; + std::map tables; + }; } -UInt64 estimateBackupSize(const BackupEntries & backup_entries, const BackupPtr & base_backup) + +BackupEntries makeBackupEntries(const Elements & elements, const ContextPtr & context) { - UInt64 total_size = 0; - for (const auto & [name, entry] : backup_entries) - { - UInt64 data_size = entry->getSize(); - if (base_backup) - { - if (base_backup->fileExists(name) && (data_size == base_backup->getFileSize(name))) - { - auto checksum = entry->getChecksum(); - if (checksum && (*checksum == base_backup->getFileChecksum(name))) - continue; - } - } - total_size += data_size; - } - return total_size; + BackupEntriesBuilder builder{context}; + builder.prepare(elements); + return builder.makeBackupEntries(); } + void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries, size_t num_threads) { if (!num_threads || !backup->supportsWritingInMultipleThreads()) @@ -482,6 +386,7 @@ void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries backup->finalizeWriting(); } + String getDataPathInBackup(const DatabaseAndTableName & table_name) { if (table_name.first.empty() || table_name.second.empty()) diff --git a/src/Backups/BackupUtils.h b/src/Backups/BackupUtils.h index c2f56044d5d2..51e849ae6f3b 100644 --- a/src/Backups/BackupUtils.h +++ b/src/Backups/BackupUtils.h @@ -5,9 +5,7 @@ namespace DB { - class IBackup; -using BackupPtr = std::shared_ptr; using BackupMutablePtr = std::shared_ptr; class IBackupEntry; using BackupEntryPtr = std::unique_ptr; @@ -15,13 +13,9 @@ using BackupEntries = std::vector>; class Context; using ContextPtr = std::shared_ptr; - /// Prepares backup entries. BackupEntries makeBackupEntries(const ASTBackupQuery::Elements & elements, const ContextPtr & context); -/// Estimate total size of the backup which would be written from the specified entries. -UInt64 estimateBackupSize(const BackupEntries & backup_entries, const BackupPtr & base_backup); - /// Write backup entries to an opened backup. void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries, size_t num_threads); diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index abbbecf374a4..dacccaa3470e 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -575,7 +575,7 @@ M(604, BACKUP_ENTRY_ALREADY_EXISTS) \ M(605, BACKUP_ENTRY_NOT_FOUND) \ M(606, BACKUP_IS_EMPTY) \ - M(607, BACKUP_ELEMENT_DUPLICATE) \ + M(607, CANNOT_RESTORE_DATABASE) \ M(608, CANNOT_RESTORE_TABLE) \ M(609, FUNCTION_ALREADY_EXISTS) \ M(610, CANNOT_DROP_FUNCTION) \ @@ -612,7 +612,8 @@ M(641, CANNOT_APPEND_TO_FILE) \ M(642, CANNOT_PACK_ARCHIVE) \ M(643, CANNOT_UNPACK_ARCHIVE) \ - M(644, CANNOT_RESTORE_DATABASE) \ + M(644, CANNOT_BACKUP_DATABASE) \ + M(645, CANNOT_BACKUP_TABLE) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ From 1fbbaa43beaff5b01d43de0436871cf8f69f90da Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 18 Jan 2022 17:28:27 +0700 Subject: [PATCH 03/18] Fixed checking access rights while producing backup, add IStorage::hasHollowBackup(). --- src/Backups/BackupUtils.cpp | 49 ++++++++++++++++++-------- src/Backups/RestoreFromBackupUtils.cpp | 2 ++ src/Databases/IDatabase.h | 10 +++--- src/Storages/IStorage.h | 3 ++ 4 files changed, 44 insertions(+), 20 deletions(-) diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 820a7f86f7fb..5d2b7362cb4c 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -131,13 +131,15 @@ namespace for (auto & info : tables | boost::adaptors::map_values) { res.push_back(makeBackupEntryForMetadata(*info.create_query)); - - auto data_backup = info.storage->backup(info.partitions, context); - if (!data_backup.empty()) + if (info.has_data) { - String data_path = getDataPathInBackup(*info.create_query); - for (auto & [path_in_backup, backup_entry] : data_backup) - res.emplace_back(data_path + path_in_backup, std::move(backup_entry)); + auto data_backup = info.storage->backup(info.partitions, context); + if (!data_backup.empty()) + { + String data_path = getDataPathInBackup(*info.create_query); + for (auto & [path_in_backup, backup_entry] : data_backup) + res.emplace_back(data_path + path_in_backup, std::move(backup_entry)); + } } } @@ -158,7 +160,17 @@ namespace void prepareToBackupTable(const DatabaseAndTableName & table_name_, const DatabaseAndTable & table_, const ASTs & partitions_) { - context->checkAccess(AccessType::SELECT, table_name_.first, table_name_.second); + context->checkAccess(AccessType::SHOW_TABLES, table_name_.first, table_name_.second); + + const auto & database = table_.first; + const auto & storage = table_.second; + + if (database->hasHollowBackup()) + throw Exception( + ErrorCodes::CANNOT_BACKUP_TABLE, + "Couldn't backup table {}.{} because of the database's engine {} is hollow", + backQuoteIfNeed(table_name_.first), backQuoteIfNeed(table_name_.second), + database->getEngineName()); /// Check that we are not trying to backup the same table again. DatabaseAndTableName new_table_name = renaming_config->getNewTableName(table_name_); @@ -173,13 +185,18 @@ namespace } /// Make a create query for this table. - auto create_query = renameInCreateQuery(table_.first->getCreateTableQuery(table_name_.second, context)); + auto create_query = renameInCreateQuery(database->getCreateTableQuery(table_name_.second, context)); + + bool has_data = !storage->hasHollowBackup(); + if (has_data) + context->checkAccess(AccessType::SELECT, table_name_.first, table_name_.second); CreateTableInfo info; info.create_query = create_query; - info.storage = table_.second; + info.storage = storage; info.name_in_backup = new_table_name; info.partitions = partitions_; + info.has_data = has_data; tables[new_table_name] = std::move(info); /// If it's not system or temporary database then probably we need to backup the database's definition too. @@ -188,7 +205,7 @@ namespace if (!databases.contains(new_table_name.first)) { /// Add a create query to backup the database if we haven't done it yet. - auto create_db_query = renameInCreateQuery(table_.first->getCreateDatabaseQuery()); + auto create_db_query = renameInCreateQuery(database->getCreateDatabaseQuery()); create_db_query->setDatabase(new_table_name.first); CreateDatabaseInfo info_db; @@ -243,11 +260,14 @@ namespace } /// Backup tables in this database. - for (auto it = database_->getTablesIteratorForBackup(context); it->isValid(); it->next()) + if (!database_->hasHollowBackup()) { - if (except_list_.contains(it->name())) - continue; - prepareToBackupTable({database_name_, it->name()}, {database_, it->table()}, {}); + for (auto it = database_->getTablesIterator(context); it->isValid(); it->next()) + { + if (except_list_.contains(it->name())) + continue; + prepareToBackupTable({database_name_, it->name()}, {database_, it->table()}, {}); + } } } @@ -289,6 +309,7 @@ namespace StoragePtr storage; DatabaseAndTableName name_in_backup; ASTs partitions; + bool has_data = false; }; /// Information which is used to make an instance of RestoreDatabaseFromBackupTask. diff --git a/src/Backups/RestoreFromBackupUtils.cpp b/src/Backups/RestoreFromBackupUtils.cpp index af3cc926a7e2..70543e50c582 100644 --- a/src/Backups/RestoreFromBackupUtils.cpp +++ b/src/Backups/RestoreFromBackupUtils.cpp @@ -165,6 +165,8 @@ namespace RestoreFromBackupTaskPtr insertDataIntoStorage(StoragePtr storage) { + if (storage->hasHollowBackup()) + return {}; context->checkAccess(AccessType::INSERT, table_name.first, table_name.second); String data_path_in_backup = getDataPathInBackup(table_name_in_backup); return storage->restoreFromBackup(backup, data_path_in_backup, partitions, context); diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index b1aa4eb1aaeb..652014487a0f 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -284,12 +284,6 @@ class IDatabase : public std::enable_shared_from_this throw Exception(getEngineName() + ": RENAME DATABASE is not supported", ErrorCodes::NOT_IMPLEMENTED); } - /// Whether the contained tables should be written to a backup. - virtual DatabaseTablesIteratorPtr getTablesIteratorForBackup(ContextPtr context) const - { - return getTablesIterator(context); /// By default we backup each table. - } - /// Returns path for persistent data storage if the database supports it, empty string otherwise virtual String getDataPath() const { return {}; } @@ -330,6 +324,10 @@ class IDatabase : public std::enable_shared_from_this throw Exception(ErrorCodes::LOGICAL_ERROR, "Database engine {} does not run a replication thread!", getEngineName()); } + /// Returns true if the backup of the database is hollow, which means it doesn't contain + /// any tables which can be stored to a backup. + virtual bool hasHollowBackup() const { return false; } + virtual ~IDatabase() = default; protected: diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 68548634bed2..8da6639c81bb 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -212,6 +212,9 @@ class IStorage : public std::enable_shared_from_this, public TypePromo NameDependencies getDependentViewsByColumn(ContextPtr context) const; + /// Returns true if the backup is hollow, which means it doesn't contain any data. + virtual bool hasHollowBackup() const { return false; } + /// Prepares entries to backup data of the storage. virtual BackupEntries backup(const ASTs & partitions, ContextPtr context); From 02f478fb66b2e71fd5b4ebad9ad82fb80dcb9184 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 18 Jan 2022 18:45:22 +0700 Subject: [PATCH 04/18] Simplify elements of ASTBackupQuery. --- src/Backups/BackupRenamingConfig.cpp | 58 ++++----- src/Backups/BackupRenamingConfig.h | 8 +- src/Backups/BackupUtils.cpp | 53 +------- src/Backups/RestoreFromBackupUtils.cpp | 52 +------- src/Backups/renameInCreateQuery.cpp | 44 ++++--- src/Parsers/ASTBackupQuery.cpp | 75 +++++++---- src/Parsers/ASTBackupQuery.h | 22 ++-- src/Parsers/ParserBackupQuery.cpp | 173 +++++++++++++++++++------ src/Parsers/ParserBackupQuery.h | 14 +- 9 files changed, 271 insertions(+), 228 deletions(-) diff --git a/src/Backups/BackupRenamingConfig.cpp b/src/Backups/BackupRenamingConfig.cpp index ff510d82a32b..e89cdfd51ea8 100644 --- a/src/Backups/BackupRenamingConfig.cpp +++ b/src/Backups/BackupRenamingConfig.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB @@ -17,47 +18,52 @@ void BackupRenamingConfig::setNewDatabaseName(const String & old_database_name, old_to_new_database_names[old_database_name] = new_database_name; } -void BackupRenamingConfig::setNewTemporaryTableName(const String & old_temporary_table_name, const String & new_temporary_table_name) +void BackupRenamingConfig::setFromBackupQuery(const ASTBackupQuery & backup_query, const String & current_database) { - old_to_new_temporary_table_names[old_temporary_table_name] = new_temporary_table_name; + setFromBackupQueryElements(backup_query.elements, current_database); } -void BackupRenamingConfig::setFromBackupQuery(const ASTBackupQuery & backup_query) -{ - setFromBackupQueryElements(backup_query.elements); -} - -void BackupRenamingConfig::setFromBackupQueryElements(const ASTBackupQuery::Elements & backup_query_elements) +void BackupRenamingConfig::setFromBackupQueryElements(const ASTBackupQuery::Elements & backup_query_elements, const String & current_database) { for (const auto & element : backup_query_elements) { switch (element.type) { - case ElementType::TABLE: [[fallthrough]]; - case ElementType::DICTIONARY: + case ElementType::TABLE: { - const auto & new_name = element.new_name.second.empty() ? element.name : element.new_name; - setNewTableName(element.name, new_name); + const String & table_name = element.name.second; + String database_name = element.name.first; + if (element.name_is_in_temp_db) + database_name = DatabaseCatalog::TEMPORARY_DATABASE; + else if (database_name.empty()) + database_name = current_database; + + const String & new_table_name = element.new_name.second; + String new_database_name = element.new_name.first; + if (element.new_name_is_in_temp_db) + new_database_name = DatabaseCatalog::TEMPORARY_DATABASE; + else if (new_database_name.empty()) + new_database_name = current_database; + + setNewTableName({database_name, table_name}, {new_database_name, new_table_name}); break; } case ASTBackupQuery::DATABASE: { - const auto & new_name = element.new_name.first.empty() ? element.name.first : element.new_name.first; - setNewDatabaseName(element.name.first, new_name); - break; - } + String database_name = element.name.first; + if (element.name_is_in_temp_db) + database_name = DatabaseCatalog::TEMPORARY_DATABASE; - case ASTBackupQuery::TEMPORARY_TABLE: - { - const auto & new_name = element.new_name.second.empty() ? element.name.second : element.new_name.second; - setNewTemporaryTableName(element.name.second, new_name); + String new_database_name = element.new_name.first; + if (element.new_name_is_in_temp_db) + new_database_name = DatabaseCatalog::TEMPORARY_DATABASE; + + setNewDatabaseName(database_name, new_database_name); break; } case ASTBackupQuery::ALL_DATABASES: break; - case ASTBackupQuery::ALL_TEMPORARY_TABLES: break; - case ASTBackupQuery::EVERYTHING: break; } } } @@ -78,12 +84,4 @@ const String & BackupRenamingConfig::getNewDatabaseName(const String & old_datab return old_database_name; } -const String & BackupRenamingConfig::getNewTemporaryTableName(const String & old_temporary_table_name) const -{ - auto it = old_to_new_temporary_table_names.find(old_temporary_table_name); - if (it != old_to_new_temporary_table_names.end()) - return it->second; - return old_temporary_table_name; -} - } diff --git a/src/Backups/BackupRenamingConfig.h b/src/Backups/BackupRenamingConfig.h index 740781c9c9f0..17329dfaccd1 100644 --- a/src/Backups/BackupRenamingConfig.h +++ b/src/Backups/BackupRenamingConfig.h @@ -19,19 +19,17 @@ class BackupRenamingConfig void setNewTableName(const DatabaseAndTableName & old_table_name, const DatabaseAndTableName & new_table_name); void setNewDatabaseName(const String & old_database_name, const String & new_database_name); - void setNewTemporaryTableName(const String & old_temporary_table_name, const String & new_temporary_table_name); - void setFromBackupQuery(const ASTBackupQuery & backup_query); - void setFromBackupQueryElements(const ASTBackupQuery::Elements & backup_query_elements); + + void setFromBackupQuery(const ASTBackupQuery & backup_query, const String & current_database); + void setFromBackupQueryElements(const ASTBackupQuery::Elements & backup_query_elements, const String & current_database); /// Changes names according to the renaming. DatabaseAndTableName getNewTableName(const DatabaseAndTableName & old_table_name) const; const String & getNewDatabaseName(const String & old_database_name) const; - const String & getNewTemporaryTableName(const String & old_temporary_table_name) const; private: std::map old_to_new_table_names; std::unordered_map old_to_new_database_names; - std::unordered_map old_to_new_temporary_table_names; }; using BackupRenamingConfigPtr = std::shared_ptr; diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 5d2b7362cb4c..800c6322f7fd 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -28,23 +28,6 @@ namespace using Elements = ASTBackupQuery::Elements; using ElementType = ASTBackupQuery::ElementType; - - /// Replaces an empty database with the current database. - void replaceEmptyDatabaseWithCurrentDatabase(Elements & elements, const String & current_database) - { - for (auto & element : elements) - { - if (element.type == ElementType::TABLE) - { - if (element.name.first.empty() && !element.name.second.empty()) - element.name.first = current_database; - if (element.new_name.first.empty() && !element.new_name.second.empty()) - element.new_name.first = current_database; - } - } - } - - /// Makes backup entries to backup databases and tables according to the elements of ASTBackupQuery. /// Keep this class consistent with RestoreTasksBuilder. class BackupEntriesBuilder @@ -55,30 +38,21 @@ namespace /// Prepares internal structures for making backup entries. void prepare(const ASTBackupQuery::Elements & elements) { - auto elements2 = elements; - replaceEmptyDatabaseWithCurrentDatabase(elements2, context->getCurrentDatabase()); - auto new_renaming_config = std::make_shared(); - new_renaming_config->setFromBackupQueryElements(elements2); + String current_database = context->getCurrentDatabase(); + new_renaming_config->setFromBackupQueryElements(elements, current_database); renaming_config = new_renaming_config; - for (const auto & element : elements2) + for (const auto & element : elements) { switch (element.type) { - case ElementType::TABLE: [[fallthrough]]; - case ElementType::DICTIONARY: + case ElementType::TABLE: { - const String & database_name = element.name.first; - const String & table_name = element.name.second; - prepareToBackupTable(DatabaseAndTableName{database_name, table_name}, element.partitions); - break; - } - - case ElementType::TEMPORARY_TABLE: - { - String database_name = DatabaseCatalog::TEMPORARY_DATABASE; const String & table_name = element.name.second; + String database_name = element.name.first; + if (database_name.empty()) + database_name = current_database; prepareToBackupTable(DatabaseAndTableName{database_name, table_name}, element.partitions); break; } @@ -90,24 +64,11 @@ namespace break; } - case ElementType::ALL_TEMPORARY_TABLES: - { - prepareToBackupDatabase(DatabaseCatalog::TEMPORARY_DATABASE, element.except_list); - break; - } - case ElementType::ALL_DATABASES: { prepareToBackupAllDatabases(element.except_list); break; } - - case ElementType::EVERYTHING: - { - prepareToBackupAllDatabases({}); - prepareToBackupDatabase(DatabaseCatalog::TEMPORARY_DATABASE, {}); - break; - } } } } diff --git a/src/Backups/RestoreFromBackupUtils.cpp b/src/Backups/RestoreFromBackupUtils.cpp index 70543e50c582..2f4ed370a06e 100644 --- a/src/Backups/RestoreFromBackupUtils.cpp +++ b/src/Backups/RestoreFromBackupUtils.cpp @@ -36,23 +36,6 @@ namespace using Elements = ASTBackupQuery::Elements; using ElementType = ASTBackupQuery::ElementType; - - /// Replaces an empty database with the current database. - void replaceEmptyDatabaseWithCurrentDatabase(Elements & elements, const String & current_database) - { - for (auto & element : elements) - { - if (element.type == ElementType::TABLE) - { - if (element.name.first.empty() && !element.name.second.empty()) - element.name.first = current_database; - if (element.new_name.first.empty() && !element.new_name.second.empty()) - element.new_name.first = current_database; - } - } - } - - /// Restores a database (without tables inside), should be executed before executing /// RestoreTableFromBackupTask. class RestoreDatabaseFromBackupTask : public IRestoreFromBackupTask @@ -192,30 +175,21 @@ namespace /// Prepares internal structures for making tasks for restoring. void prepare(const ASTBackupQuery::Elements & elements) { - auto elements2 = elements; - replaceEmptyDatabaseWithCurrentDatabase(elements2, context->getCurrentDatabase()); - auto new_renaming_config = std::make_shared(); - new_renaming_config->setFromBackupQueryElements(elements2); + String current_database = context->getCurrentDatabase(); + new_renaming_config->setFromBackupQueryElements(elements, current_database); renaming_config = new_renaming_config; - for (const auto & element : elements2) + for (const auto & element : elements) { switch (element.type) { - case ElementType::TABLE: [[fallthrough]]; - case ElementType::DICTIONARY: + case ElementType::TABLE: { - const String & database_name = element.name.first; - const String & table_name = element.name.second; - prepareToRestoreTable(DatabaseAndTableName{database_name, table_name}, element.partitions); - break; - } - - case ElementType::TEMPORARY_TABLE: - { - String database_name = DatabaseCatalog::TEMPORARY_DATABASE; const String & table_name = element.name.second; + String database_name = element.name.first; + if (database_name.empty()) + database_name = current_database; prepareToRestoreTable(DatabaseAndTableName{database_name, table_name}, element.partitions); break; } @@ -227,23 +201,11 @@ namespace break; } - case ElementType::ALL_TEMPORARY_TABLES: - { - prepareToRestoreDatabase(DatabaseCatalog::TEMPORARY_DATABASE, element.except_list); - break; - } - case ElementType::ALL_DATABASES: { prepareToRestoreAllDatabases(element.except_list); break; } - - case ElementType::EVERYTHING: - { - prepareToRestoreAllDatabases({}); - break; - } } } } diff --git a/src/Backups/renameInCreateQuery.cpp b/src/Backups/renameInCreateQuery.cpp index 5b2492b3be44..fe661868040a 100644 --- a/src/Backups/renameInCreateQuery.cpp +++ b/src/Backups/renameInCreateQuery.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -47,26 +48,39 @@ namespace /// CREATE DICTIONARY or CREATE VIEW or CREATE TEMPORARY TABLE or CREATE DATABASE query. static void visitCreateQuery(ASTCreateQuery & create, const Data & data) { - if (create.temporary) + if (create.table) { - if (!create.table) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Table name specified in the CREATE TEMPORARY TABLE query must not be empty"); - create.setTable(data.renaming_config->getNewTemporaryTableName(create.getTable())); + DatabaseAndTableName table_name; + table_name.second = create.getTable(); + if (create.temporary) + table_name.first = DatabaseCatalog::TEMPORARY_DATABASE; + else if (create.database) + table_name.first = create.getDatabase(); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name specified in the CREATE TABLE query must not be empty"); + + table_name = data.renaming_config->getNewTableName(table_name); + + if (table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) + { + create.temporary = true; + create.setDatabase(""); + } + else + { + create.temporary = false; + create.setDatabase(table_name.first); + } + create.setTable(table_name.second); } - else if (!create.table) + else if (create.database) { - if (!create.database) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name specified in the CREATE DATABASE query must not be empty"); - create.setDatabase(data.renaming_config->getNewDatabaseName(create.getDatabase())); + String database_name = create.getDatabase(); + database_name = data.renaming_config->getNewDatabaseName(database_name); + create.setDatabase(database_name); } else - { - if (!create.database) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name specified in the CREATE TABLE query must not be empty"); - auto table_and_database_name = data.renaming_config->getNewTableName({create.getDatabase(), create.getTable()}); - create.setDatabase(table_and_database_name.first); - create.setTable(table_and_database_name.second); - } + throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name specified in the CREATE DATABASE query must not be empty"); create.uuid = UUIDHelpers::Nil; diff --git a/src/Parsers/ASTBackupQuery.cpp b/src/Parsers/ASTBackupQuery.cpp index adc6bb979853..f8fcbd988724 100644 --- a/src/Parsers/ASTBackupQuery.cpp +++ b/src/Parsers/ASTBackupQuery.cpp @@ -11,31 +11,50 @@ namespace using Element = ASTBackupQuery::Element; using ElementType = ASTBackupQuery::ElementType; - void formatName(const DatabaseAndTableName & name, ElementType type, const IAST::FormatSettings & format) + void formatTypeWithName(const DatabaseAndTableName & name, bool name_is_in_temp_db, ElementType type, bool show_type, const IAST::FormatSettings & format) { switch (type) { - case ElementType::TABLE: [[fallthrough]]; - case ElementType::DICTIONARY: + case ElementType::TABLE: { + if (show_type) + { + format.ostr << (format.hilite ? IAST::hilite_keyword : ""); + if (name_is_in_temp_db) + format.ostr << " TEMPORARY TABLE"; + else + format.ostr << " TABLE"; + format.ostr << (format.hilite ? IAST::hilite_none : ""); + } + format.ostr << " "; - if (!name.first.empty()) + if (!name_is_in_temp_db && !name.first.empty()) format.ostr << backQuoteIfNeed(name.first) << "."; format.ostr << backQuoteIfNeed(name.second); break; } case ElementType::DATABASE: { - format.ostr << " " << backQuoteIfNeed(name.first); + if (show_type) + { + format.ostr << (format.hilite ? IAST::hilite_keyword : ""); + if (name_is_in_temp_db) + format.ostr << " ALL TEMPORARY TABLES"; + else + format.ostr << " DATABASE"; + format.ostr << (format.hilite ? IAST::hilite_none : ""); + } + + if (!name_is_in_temp_db) + format.ostr << " " << backQuoteIfNeed(name.first); break; } - case ElementType::TEMPORARY_TABLE: + case ElementType::ALL_DATABASES: { - format.ostr << " " << backQuoteIfNeed(name.second); + if (show_type) + format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " ALL DATABASES" << (format.hilite ? IAST::hilite_none : ""); break; } - default: - break; } } @@ -55,32 +74,36 @@ namespace } } - void formatElement(const Element & element, Kind kind, const IAST::FormatSettings & format) + void formatExceptList(const std::set & except_list, const IAST::FormatSettings & format) { - format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " "; - switch (element.type) + if (except_list.empty()) + return; + format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " EXCEPT " + << (format.hilite ? IAST::hilite_none : ""); + bool need_comma = false; + for (const auto & item : except_list) { - case ElementType::TABLE: format.ostr << "TABLE"; break; - case ElementType::DICTIONARY: format.ostr << "DICTIONARY"; break; - case ElementType::DATABASE: format.ostr << "DATABASE"; break; - case ElementType::ALL_DATABASES: format.ostr << "ALL DATABASES"; break; - case ElementType::TEMPORARY_TABLE: format.ostr << "TEMPORARY TABLE"; break; - case ElementType::ALL_TEMPORARY_TABLES: format.ostr << "ALL TEMPORARY TABLES"; break; - case ElementType::EVERYTHING: format.ostr << "EVERYTHING"; break; + if (std::exchange(need_comma, true)) + format.ostr << ","; + format.ostr << " " << backQuoteIfNeed(item); } - format.ostr << (format.hilite ? IAST::hilite_none : ""); + } - formatName(element.name, element.type, format); + void formatElement(const Element & element, Kind kind, const IAST::FormatSettings & format) + { + formatTypeWithName(element.name, element.name_is_in_temp_db, element.type, true, format); - bool under_another_name = !element.new_name.first.empty() || !element.new_name.second.empty(); - if (under_another_name) + formatPartitions(element.partitions, format); + formatExceptList(element.except_list, format); + + bool new_name_is_different = (element.new_name != element.name) || (element.new_name_is_in_temp_db != element.name_is_in_temp_db); + if (new_name_is_different) { format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " " << ((kind == Kind::BACKUP) ? "AS" : "INTO") << (format.hilite ? IAST::hilite_none : ""); - formatName(element.new_name, element.type, format); + bool show_type = (element.new_name_is_in_temp_db != element.name_is_in_temp_db); + formatTypeWithName(element.new_name, element.new_name_is_in_temp_db, element.type, show_type, format); } - - formatPartitions(element.partitions, format); } void formatElements(const std::vector & elements, Kind kind, const IAST::FormatSettings & format) diff --git a/src/Parsers/ASTBackupQuery.h b/src/Parsers/ASTBackupQuery.h index 0042fca558f6..648bcf27bce3 100644 --- a/src/Parsers/ASTBackupQuery.h +++ b/src/Parsers/ASTBackupQuery.h @@ -11,22 +11,20 @@ using DatabaseAndTableName = std::pair; /** BACKUP { TABLE [db.]table_name [AS [db.]table_name_in_backup] [PARTITION[S] partition_expr [,...]] | * DICTIONARY [db.]dictionary_name [AS [db.]dictionary_name_in_backup] | - * DATABASE database_name [AS database_name_in_backup] | - * ALL DATABASES | - * TEMPORARY TABLE table_name [AS table_name_in_backup] - * ALL TEMPORARY TABLES | - * EVERYTHING } [,...] + * TEMPORARY TABLE table_name [AS table_name_in_backup] | + * ALL TEMPORARY TABLES [EXCEPT ...] | + * DATABASE database_name [EXCEPT ...] [AS database_name_in_backup] | + * ALL DATABASES [EXCEPT ...] } [,...] * TO { File('path/') | * Disk('disk_name', 'path/') * [SETTINGS base_backup = {File(...) | Disk(...)}] * * RESTORE { TABLE [db.]table_name_in_backup [INTO [db.]table_name] [PARTITION[S] partition_expr [,...]] | * DICTIONARY [db.]dictionary_name_in_backup [INTO [db.]dictionary_name] | - * DATABASE database_name_in_backup [INTO database_name] | - * ALL DATABASES | * TEMPORARY TABLE table_name_in_backup [INTO table_name] | - * ALL TEMPORARY TABLES | - * EVERYTHING } [,...] + * ALL TEMPORARY TABLES [EXCEPT ...] | + * DATABASE database_name_in_backup [EXCEPT ...] [INTO database_name] | + * ALL DATABASES [EXCEPT ...] } [,...] * FROM {File(...) | Disk(...)} * * Notes: @@ -57,12 +55,8 @@ class ASTBackupQuery : public IAST enum ElementType { TABLE, - DICTIONARY, DATABASE, ALL_DATABASES, - TEMPORARY_TABLE, - ALL_TEMPORARY_TABLES, - EVERYTHING, }; struct Element @@ -70,6 +64,8 @@ class ASTBackupQuery : public IAST ElementType type; DatabaseAndTableName name; DatabaseAndTableName new_name; + bool name_is_in_temp_db = false; + bool new_name_is_in_temp_db = false; ASTs partitions; std::set except_list; }; diff --git a/src/Parsers/ParserBackupQuery.cpp b/src/Parsers/ParserBackupQuery.cpp index 666600b58de4..844a91fa515f 100644 --- a/src/Parsers/ParserBackupQuery.cpp +++ b/src/Parsers/ParserBackupQuery.cpp @@ -18,38 +18,109 @@ namespace using Element = ASTBackupQuery::Element; using ElementType = ASTBackupQuery::ElementType; - bool parseName(IParser::Pos & pos, Expected & expected, ElementType type, DatabaseAndTableName & name) + bool parseType(IParser::Pos & pos, Expected & expected, ElementType & type, bool & name_is_in_temp_db) { + name_is_in_temp_db = false; + if (ParserKeyword{"TABLE"}.ignore(pos, expected) || ParserKeyword{"DICTIONARY"}.ignore(pos, expected)) + { + type = ElementType::TABLE; + return true; + } + if (ParserKeyword{"TEMPORARY TABLE"}.ignore(pos, expected)) + { + type = ElementType::TABLE; + name_is_in_temp_db = true; + return true; + } + if (ParserKeyword{"DATABASE"}.ignore(pos, expected)) + { + type = ElementType::DATABASE; + return true; + } + if (ParserKeyword{"ALL TEMPORARY TABLES"}.ignore(pos, expected)) + { + type = ElementType::DATABASE; + name_is_in_temp_db = true; + return true; + } + if (ParserKeyword{"ALL DATABASES"}.ignore(pos, expected)) + { + type = ElementType::ALL_DATABASES; + return true; + } + return false; + } + + bool parseTempDBFlag(IParser::Pos & pos, Expected & expected, ElementType type, bool & temp_db_flag) + { + temp_db_flag = false; switch (type) { - case ElementType::TABLE: [[fallthrough]]; - case ElementType::DICTIONARY: + case ElementType::TABLE: { - return parseDatabaseAndTableName(pos, expected, name.first, name.second); + if (ParserKeyword{"TABLE"}.ignore(pos, expected) || ParserKeyword{"DICTIONARY"}.ignore(pos, expected)) + { + return true; + } + if (ParserKeyword{"TEMPORARY TABLE"}.ignore(pos, expected)) + { + temp_db_flag = true; + return true; + } + return false; } case ElementType::DATABASE: { - ASTPtr ast; - if (!ParserIdentifier{}.parse(pos, ast, expected)) - return false; - name.first = getIdentifierName(ast); - name.second.clear(); - return true; + if (ParserKeyword{"DATABASE"}.ignore(pos, expected)) + { + return true; + } + if (ParserKeyword{"ALL TEMPORARY TABLES"}.ignore(pos, expected)) + { + temp_db_flag = true; + return true; + } + return false; } - case ElementType::TEMPORARY_TABLE: + default: + return false; + } + } + + bool parseName(IParser::Pos & pos, Expected & expected, ElementType type, bool name_is_in_temp_db, DatabaseAndTableName & name) + { + name.first.clear(); + name.second.clear(); + switch (type) + { + case ElementType::TABLE: { + if (name_is_in_temp_db) + { + ASTPtr ast; + if (!ParserIdentifier{}.parse(pos, ast, expected)) + return false; + name.second = getIdentifierName(ast); + return true; + } + return parseDatabaseAndTableName(pos, expected, name.first, name.second); + } + + case ElementType::DATABASE: + { + if (name_is_in_temp_db) + return false; ASTPtr ast; if (!ParserIdentifier{}.parse(pos, ast, expected)) return false; - name.second = getIdentifierName(ast); - name.first.clear(); + name.first = getIdentifierName(ast); return true; } default: - return true; + return false; } } @@ -64,7 +135,7 @@ namespace ASTPtr ast; if (!ParserPartition{}.parse(pos, ast, expected)) return false; - result.emplace_back(ast); + result.push_back(ast); return true; }; if (!ParserList::parseUtil(pos, expected, parse_list_element, false)) @@ -74,50 +145,72 @@ namespace return true; } + bool parseExceptList(IParser::Pos & pos, Expected & expected, std::set & except_list) + { + if (!ParserKeyword{"EXCEPT"}.ignore(pos, expected)) + return false; + + std::set result; + auto parse_list_element = [&] + { + ASTPtr ast; + if (!ParserIdentifier{}.parse(pos, ast, expected)) + return false; + result.insert(getIdentifierName(ast)); + return true; + }; + if (!ParserList::parseUtil(pos, expected, parse_list_element, false)) + return false; + + except_list = std::move(result); + return true; + } + bool parseElement(IParser::Pos & pos, Expected & expected, Element & entry) { return IParserBase::wrapParseImpl(pos, [&] { ElementType type; - if (ParserKeyword{"TABLE"}.ignore(pos, expected)) - type = ElementType::TABLE; - else if (ParserKeyword{"DICTIONARY"}.ignore(pos, expected)) - type = ElementType::DICTIONARY; - else if (ParserKeyword{"DATABASE"}.ignore(pos, expected)) - type = ElementType::DATABASE; - else if (ParserKeyword{"ALL DATABASES"}.ignore(pos, expected)) - type = ElementType::ALL_DATABASES; - else if (ParserKeyword{"TEMPORARY TABLE"}.ignore(pos, expected)) - type = ElementType::TEMPORARY_TABLE; - else if (ParserKeyword{"ALL TEMPORARY TABLES"}.ignore(pos, expected)) - type = ElementType::ALL_TEMPORARY_TABLES; - else if (ParserKeyword{"EVERYTHING"}.ignore(pos, expected)) - type = ElementType::EVERYTHING; - else + bool name_is_in_temp_db = false; + if (!parseType(pos, expected, type, name_is_in_temp_db)) return false; DatabaseAndTableName name; - if (!parseName(pos, expected, type, name)) - return false; - - ASTs partitions; - if (type == ElementType::TABLE) - parsePartitions(pos, expected, partitions); + if ((type == ElementType::TABLE) || (type == ElementType::DATABASE && !name_is_in_temp_db)) + { + if (!parseName(pos, expected, type, name_is_in_temp_db, name)) + return false; + } - DatabaseAndTableName new_name; + bool new_name_is_in_temp_db = name_is_in_temp_db; + DatabaseAndTableName new_name = name; if (ParserKeyword{"AS"}.ignore(pos, expected) || ParserKeyword{"INTO"}.ignore(pos, expected)) { - if (!parseName(pos, expected, type, new_name)) - return false; + if (!parseTempDBFlag(pos, expected, type, new_name_is_in_temp_db)) + new_name_is_in_temp_db = name_is_in_temp_db; + + if ((type == ElementType::TABLE) || (type == ElementType::DATABASE && !new_name_is_in_temp_db)) + { + if (!parseName(pos, expected, type, new_name_is_in_temp_db, new_name)) + new_name = name; + } } - if ((type == ElementType::TABLE) && partitions.empty()) + ASTs partitions; + if (type == ElementType::TABLE) parsePartitions(pos, expected, partitions); + std::set except_list; + if (type != ElementType::TABLE) + parseExceptList(pos, expected, except_list); + entry.type = type; entry.name = std::move(name); entry.new_name = std::move(new_name); + entry.name_is_in_temp_db = name_is_in_temp_db; + entry.new_name_is_in_temp_db = new_name_is_in_temp_db; entry.partitions = std::move(partitions); + entry.except_list = std::move(except_list); return true; }); } diff --git a/src/Parsers/ParserBackupQuery.h b/src/Parsers/ParserBackupQuery.h index e42326c25908..b01c149601c7 100644 --- a/src/Parsers/ParserBackupQuery.h +++ b/src/Parsers/ParserBackupQuery.h @@ -8,22 +8,20 @@ namespace DB /** Parses queries like * BACKUP { TABLE [db.]table_name [AS [db.]table_name_in_backup] [PARTITION[S] partition_expr [,...]] | * DICTIONARY [db.]dictionary_name [AS [db.]dictionary_name_in_backup] | - * DATABASE database_name [AS database_name_in_backup] | - * ALL DATABASES | * TEMPORARY TABLE table_name [AS table_name_in_backup] - * ALL TEMPORARY TABLES | - * EVERYTHING } [,...] + * ALL TEMPORARY TABLES [EXCEPT ...] | + * DATABASE database_name [AS database_name_in_backup] [EXCEPT ...] | + * ALL DATABASES [EXCEPT ...] } [,...] * TO { File('path/') | * Disk('disk_name', 'path/') * [SETTINGS base_backup = {FILE(...) | DISK(...)}] * * RESTORE { TABLE [db.]table_name_in_backup [INTO [db.]table_name] [PARTITION[S] partition_expr [,...]] | * DICTIONARY [db.]dictionary_name_in_backup [INTO [db.]dictionary_name] | - * DATABASE database_name_in_backup [INTO database_name] | - * ALL DATABASES | * TEMPORARY TABLE table_name_in_backup [INTO table_name] | - * ALL TEMPORARY TABLES | - * EVERYTHING } [,...] + * ALL TEMPORARY TABLES [EXCEPT ...] | + * DATABASE database_name_in_backup [EXCEPT ...] [INTO database_name] | + * ALL DATABASES [EXCEPT ...] } [,...] * FROM {File(...) | Disk(...)} */ class ParserBackupQuery : public IParserBase From dc73903737a1ad81738dbb74ea893091c4f3fd5f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 18 Jan 2022 20:05:13 +0700 Subject: [PATCH 05/18] Move renameInCreateQuery() to a better place. --- src/Backups/BackupRenamingConfig.cpp | 87 ---- src/Backups/BackupUtils.cpp | 15 +- src/Backups/DDLRenamingVisitor.cpp | 384 ++++++++++++++++++ ...pRenamingConfig.h => DDLRenamingVisitor.h} | 34 +- src/Backups/RestoreFromBackupUtils.cpp | 15 +- src/Backups/renameInCreateQuery.cpp | 296 -------------- src/Backups/renameInCreateQuery.h | 16 - src/Common/ErrorCodes.cpp | 1 + 8 files changed, 426 insertions(+), 422 deletions(-) delete mode 100644 src/Backups/BackupRenamingConfig.cpp create mode 100644 src/Backups/DDLRenamingVisitor.cpp rename src/Backups/{BackupRenamingConfig.h => DDLRenamingVisitor.h} (51%) delete mode 100644 src/Backups/renameInCreateQuery.cpp delete mode 100644 src/Backups/renameInCreateQuery.h diff --git a/src/Backups/BackupRenamingConfig.cpp b/src/Backups/BackupRenamingConfig.cpp deleted file mode 100644 index e89cdfd51ea8..000000000000 --- a/src/Backups/BackupRenamingConfig.cpp +++ /dev/null @@ -1,87 +0,0 @@ -#include -#include -#include - - -namespace DB -{ -using Kind = ASTBackupQuery::Kind; -using ElementType = ASTBackupQuery::ElementType; - -void BackupRenamingConfig::setNewTableName(const DatabaseAndTableName & old_table_name, const DatabaseAndTableName & new_table_name) -{ - old_to_new_table_names[old_table_name] = new_table_name; -} - -void BackupRenamingConfig::setNewDatabaseName(const String & old_database_name, const String & new_database_name) -{ - old_to_new_database_names[old_database_name] = new_database_name; -} - -void BackupRenamingConfig::setFromBackupQuery(const ASTBackupQuery & backup_query, const String & current_database) -{ - setFromBackupQueryElements(backup_query.elements, current_database); -} - -void BackupRenamingConfig::setFromBackupQueryElements(const ASTBackupQuery::Elements & backup_query_elements, const String & current_database) -{ - for (const auto & element : backup_query_elements) - { - switch (element.type) - { - case ElementType::TABLE: - { - const String & table_name = element.name.second; - String database_name = element.name.first; - if (element.name_is_in_temp_db) - database_name = DatabaseCatalog::TEMPORARY_DATABASE; - else if (database_name.empty()) - database_name = current_database; - - const String & new_table_name = element.new_name.second; - String new_database_name = element.new_name.first; - if (element.new_name_is_in_temp_db) - new_database_name = DatabaseCatalog::TEMPORARY_DATABASE; - else if (new_database_name.empty()) - new_database_name = current_database; - - setNewTableName({database_name, table_name}, {new_database_name, new_table_name}); - break; - } - - case ASTBackupQuery::DATABASE: - { - String database_name = element.name.first; - if (element.name_is_in_temp_db) - database_name = DatabaseCatalog::TEMPORARY_DATABASE; - - String new_database_name = element.new_name.first; - if (element.new_name_is_in_temp_db) - new_database_name = DatabaseCatalog::TEMPORARY_DATABASE; - - setNewDatabaseName(database_name, new_database_name); - break; - } - - case ASTBackupQuery::ALL_DATABASES: break; - } - } -} - -DatabaseAndTableName BackupRenamingConfig::getNewTableName(const DatabaseAndTableName & old_table_name) const -{ - auto it = old_to_new_table_names.find(old_table_name); - if (it != old_to_new_table_names.end()) - return it->second; - return {getNewDatabaseName(old_table_name.first), old_table_name.second}; -} - -const String & BackupRenamingConfig::getNewDatabaseName(const String & old_database_name) const -{ - auto it = old_to_new_database_names.find(old_database_name); - if (it != old_to_new_database_names.end()) - return it->second; - return old_database_name; -} - -} diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 800c6322f7fd..0ad5cf973933 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -1,8 +1,7 @@ #include #include -#include +#include #include -#include #include #include #include @@ -38,10 +37,8 @@ namespace /// Prepares internal structures for making backup entries. void prepare(const ASTBackupQuery::Elements & elements) { - auto new_renaming_config = std::make_shared(); String current_database = context->getCurrentDatabase(); - new_renaming_config->setFromBackupQueryElements(elements, current_database); - renaming_config = new_renaming_config; + renaming_settings.setFromBackupQuery(elements, current_database); for (const auto & element : elements) { @@ -134,7 +131,7 @@ namespace database->getEngineName()); /// Check that we are not trying to backup the same table again. - DatabaseAndTableName new_table_name = renaming_config->getNewTableName(table_name_); + DatabaseAndTableName new_table_name = renaming_settings.getNewTableName(table_name_); if (tables.contains(new_table_name)) { String message; @@ -203,7 +200,7 @@ namespace context->checkAccess(AccessType::SHOW_DATABASES, database_name_); /// Check that we are not trying to restore the same database again. - String new_database_name = renaming_config->getNewDatabaseName(database_name_); + String new_database_name = renaming_settings.getNewDatabaseName(database_name_); if (databases.contains(new_database_name) && databases[new_database_name].is_explicit) throw Exception(ErrorCodes::CANNOT_BACKUP_DATABASE, "Couldn't backup database {} twice", backQuoteIfNeed(new_database_name)); @@ -248,7 +245,7 @@ namespace /// Do renaming in the create query according to the renaming config. std::shared_ptr renameInCreateQuery(const ASTPtr & ast) const { - return typeid_cast>(::DB::renameInCreateQuery(ast, renaming_config, context)); + return typeid_cast>(::DB::renameInCreateQuery(ast, context, renaming_settings)); } static bool isSystemOrTemporaryDatabase(const String & database_name) @@ -292,7 +289,7 @@ namespace ContextPtr context; BackupMutablePtr backup; - BackupRenamingConfigPtr renaming_config; + DDLRenamingSettings renaming_settings; std::map databases; std::map tables; }; diff --git a/src/Backups/DDLRenamingVisitor.cpp b/src/Backups/DDLRenamingVisitor.cpp new file mode 100644 index 000000000000..9d87d18e142b --- /dev/null +++ b/src/Backups/DDLRenamingVisitor.cpp @@ -0,0 +1,384 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int WRONG_DDL_RENAMING_SETTINGS; + extern const int LOGICAL_ERROR; +} + +namespace +{ + /// Replaces names of tables and databases used in a CREATE query, which can be either CREATE TABLE or + /// CREATE DICTIONARY or CREATE VIEW or CREATE TEMPORARY TABLE or CREATE DATABASE query. + void visitCreateQuery(ASTCreateQuery & create, const DDLRenamingVisitor::Data & data) + { + if (create.table) + { + DatabaseAndTableName table_name; + table_name.second = create.getTable(); + if (create.temporary) + table_name.first = DatabaseCatalog::TEMPORARY_DATABASE; + else if (create.database) + table_name.first = create.getDatabase(); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name specified in the CREATE TABLE query must not be empty"); + + table_name = data.renaming_settings.getNewTableName(table_name); + + if (table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) + { + create.temporary = true; + create.setDatabase(""); + } + else + { + create.temporary = false; + create.setDatabase(table_name.first); + } + create.setTable(table_name.second); + } + else if (create.database) + { + String database_name = create.getDatabase(); + database_name = data.renaming_settings.getNewDatabaseName(database_name); + create.setDatabase(database_name); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name specified in the CREATE DATABASE query must not be empty"); + + create.uuid = UUIDHelpers::Nil; + + if (!create.as_table.empty() && !create.as_database.empty()) + std::tie(create.as_database, create.as_table) = data.renaming_settings.getNewTableName({create.as_database, create.as_table}); + + if (!create.to_table_id.table_name.empty() && !create.to_table_id.database_name.empty()) + { + auto to_table = data.renaming_settings.getNewTableName({create.to_table_id.database_name, create.to_table_id.table_name}); + create.to_table_id = StorageID{to_table.first, to_table.second}; + } + } + + /// Replaces names of a database and a table in a expression like `db`.`table` + void visitTableExpression(ASTTableExpression & expr, const DDLRenamingVisitor::Data & data) + { + if (!expr.database_and_table_name) + return; + + ASTIdentifier * id = expr.database_and_table_name->as(); + if (!id) + return; + + auto table_id = id->createTable(); + if (!table_id) + return; + + const String & db_name = table_id->getDatabaseName(); + const String & table_name = table_id->shortName(); + if (db_name.empty() || table_name.empty()) + return; + + String new_db_name, new_table_name; + std::tie(new_db_name, new_table_name) = data.renaming_settings.getNewTableName({db_name, table_name}); + if ((new_db_name == db_name) && (new_table_name == table_name)) + return; + + expr.database_and_table_name = std::make_shared(Strings{new_db_name, new_table_name}); + expr.children.push_back(expr.database_and_table_name); + } + + /// Replaces a database's name passed via an argument of the function merge() or the table engine Merge. + void visitFunctionMerge(ASTFunction & function, const DDLRenamingVisitor::Data & data) + { + if (!function.arguments) + return; + + /// The first argument is a database's name and we can rename it. + /// The second argument is a regular expression and we can do nothing about it. + auto & args = function.arguments->as().children; + size_t db_name_arg_index = 0; + if (args.size() <= db_name_arg_index) + return; + + String db_name = evaluateConstantExpressionForDatabaseName(args[db_name_arg_index], data.context)->as().value.safeGet(); + if (db_name.empty()) + return; + + String new_db_name = data.renaming_settings.getNewDatabaseName(db_name); + if (new_db_name == db_name) + return; + args[db_name_arg_index] = std::make_shared(new_db_name); + } + + /// Replaces names of a table and a database passed via arguments of the function remote() or cluster() or the table engine Distributed. + void visitFunctionRemote(ASTFunction & function, const DDLRenamingVisitor::Data & data) + { + if (!function.arguments) + return; + + /// The first argument is an address or cluster's name, so we skip it. + /// The second argument can be either 'db.name' or just 'db' followed by the third argument 'table'. + auto & args = function.arguments->as().children; + + const auto * second_arg_as_function = args[1]->as(); + if (second_arg_as_function && TableFunctionFactory::instance().isTableFunctionName(second_arg_as_function->name)) + return; + + size_t db_name_index = 1; + if (args.size() <= db_name_index) + return; + + String name = evaluateConstantExpressionForDatabaseName(args[db_name_index], data.context)->as().value.safeGet(); + + size_t table_name_index = static_cast(-1); + + QualifiedTableName qualified_name; + + if (function.name == "Distributed") + qualified_name.table = name; + else + qualified_name = QualifiedTableName::parseFromString(name); + + if (qualified_name.database.empty()) + { + std::swap(qualified_name.database, qualified_name.table); + table_name_index = 2; + if (args.size() <= table_name_index) + return; + qualified_name.table = evaluateConstantExpressionForDatabaseName(args[table_name_index], data.context)->as().value.safeGet(); + } + + const String & db_name = qualified_name.database; + const String & table_name = qualified_name.table; + + if (db_name.empty() || table_name.empty()) + return; + + String new_db_name, new_table_name; + std::tie(new_db_name, new_table_name) = data.renaming_settings.getNewTableName({db_name, table_name}); + if ((new_db_name == db_name) && (new_table_name == table_name)) + return; + + if (table_name_index != static_cast(-1)) + { + if (new_db_name != db_name) + args[db_name_index] = std::make_shared(new_db_name); + if (new_table_name != table_name) + args[table_name_index] = std::make_shared(new_table_name); + } + else + { + args[db_name_index] = std::make_shared(new_db_name); + args.insert(args.begin() + db_name_index + 1, std::make_shared(new_table_name)); + } + } + + /// Replaces names of tables and databases used in arguments of a table function or a table engine. + void visitFunction(ASTFunction & function, const DDLRenamingVisitor::Data & data) + { + if ((function.name == "merge") || (function.name == "Merge")) + { + visitFunctionMerge(function, data); + } + else if ((function.name == "remote") || (function.name == "remoteSecure") || (function.name == "cluster") || + (function.name == "clusterAllReplicas") || (function.name == "Distributed")) + { + visitFunctionRemote(function, data); + } + } + + /// Replaces names of a table and a database used in source parameters of a dictionary. + void visitDictionary(ASTDictionary & dictionary, const DDLRenamingVisitor::Data & data) + { + if (!dictionary.source || dictionary.source->name != "clickhouse" || !dictionary.source->elements) + return; + + auto & elements = dictionary.source->elements->as().children; + String db_name, table_name; + size_t db_name_index = static_cast(-1); + size_t table_name_index = static_cast(-1); + + for (size_t i = 0; i != elements.size(); ++i) + { + auto & pair = elements[i]->as(); + if (pair.first == "db") + { + if (db_name_index != static_cast(-1)) + return; + db_name = pair.second->as().value.safeGet(); + db_name_index = i; + } + else if (pair.first == "table") + { + if (table_name_index != static_cast(-1)) + return; + table_name = pair.second->as().value.safeGet(); + table_name_index = i; + } + } + + if (db_name.empty() || table_name.empty()) + return; + + String new_db_name, new_table_name; + std::tie(new_db_name, new_table_name) = data.renaming_settings.getNewTableName({db_name, table_name}); + if ((new_db_name == db_name) && (new_table_name == table_name)) + return; + + if (new_db_name != db_name) + { + auto & pair = elements[db_name_index]->as(); + pair.replace(pair.second, std::make_shared(new_db_name)); + } + if (new_table_name != table_name) + { + auto & pair = elements[table_name_index]->as(); + pair.replace(pair.second, std::make_shared(new_table_name)); + } + } +} + + +void DDLRenamingSettings::setNewTableName(const DatabaseAndTableName & old_table_name, const DatabaseAndTableName & new_table_name) +{ + auto it = old_to_new_table_names.find(old_table_name); + if ((it != old_to_new_table_names.end())) + { + if (it->second == new_table_name) + return; + throw Exception(ErrorCodes::WRONG_DDL_RENAMING_SETTINGS, "Wrong renaming: it's specified that table {}.{} should be renamed to {}.{} and to {}.{} at the same time", + backQuoteIfNeed(old_table_name.first), backQuoteIfNeed(old_table_name.second), + backQuoteIfNeed(it->second.first), backQuoteIfNeed(it->second.second), + backQuoteIfNeed(new_table_name.first), backQuoteIfNeed(new_table_name.second)); + } + old_to_new_table_names[old_table_name] = new_table_name; +} + +void DDLRenamingSettings::setNewDatabaseName(const String & old_database_name, const String & new_database_name) +{ + auto it = old_to_new_database_names.find(old_database_name); + if ((it != old_to_new_database_names.end())) + { + if (it->second == new_database_name) + return; + throw Exception(ErrorCodes::WRONG_DDL_RENAMING_SETTINGS, "Wrong renaming: it's specified that database {} should be renamed to {} and to {} at the same time", + backQuoteIfNeed(old_database_name), backQuoteIfNeed(it->second), backQuoteIfNeed(new_database_name)); + } + old_to_new_database_names[old_database_name] = new_database_name; +} + +void DDLRenamingSettings::setFromBackupQuery(const ASTBackupQuery & backup_query, const String & current_database) +{ + setFromBackupQuery(backup_query.elements, current_database); +} + +void DDLRenamingSettings::setFromBackupQuery(const ASTBackupQuery::Elements & backup_query_elements, const String & current_database) +{ + old_to_new_table_names.clear(); + old_to_new_database_names.clear(); + + using ElementType = ASTBackupQuery::ElementType; + + for (const auto & element : backup_query_elements) + { + switch (element.type) + { + case ElementType::TABLE: + { + const String & table_name = element.name.second; + String database_name = element.name.first; + if (element.name_is_in_temp_db) + database_name = DatabaseCatalog::TEMPORARY_DATABASE; + else if (database_name.empty()) + database_name = current_database; + + const String & new_table_name = element.new_name.second; + String new_database_name = element.new_name.first; + if (element.new_name_is_in_temp_db) + new_database_name = DatabaseCatalog::TEMPORARY_DATABASE; + else if (new_database_name.empty()) + new_database_name = current_database; + + setNewTableName({database_name, table_name}, {new_database_name, new_table_name}); + break; + } + + case ASTBackupQuery::DATABASE: + { + String database_name = element.name.first; + if (element.name_is_in_temp_db) + database_name = DatabaseCatalog::TEMPORARY_DATABASE; + + String new_database_name = element.new_name.first; + if (element.new_name_is_in_temp_db) + new_database_name = DatabaseCatalog::TEMPORARY_DATABASE; + + setNewDatabaseName(database_name, new_database_name); + break; + } + + case ASTBackupQuery::ALL_DATABASES: break; + } + } +} + +DatabaseAndTableName DDLRenamingSettings::getNewTableName(const DatabaseAndTableName & old_table_name) const +{ + auto it = old_to_new_table_names.find(old_table_name); + if (it != old_to_new_table_names.end()) + return it->second; + return {getNewDatabaseName(old_table_name.first), old_table_name.second}; +} + +const String & DDLRenamingSettings::getNewDatabaseName(const String & old_database_name) const +{ + auto it = old_to_new_database_names.find(old_database_name); + if (it != old_to_new_database_names.end()) + return it->second; + return old_database_name; +} + + +bool DDLRenamingVisitor::needChildVisit(ASTPtr &, const ASTPtr &) { return true; } + +void DDLRenamingVisitor::visit(ASTPtr & ast, const Data & data) +{ + if (auto * create = ast->as()) + visitCreateQuery(*create, data); + else if (auto * expr = ast->as()) + visitTableExpression(*expr, data); + else if (auto * function = ast->as()) + visitFunction(*function, data); + else if (auto * dictionary = ast->as()) + visitDictionary(*dictionary, data); +} + +ASTPtr renameInCreateQuery(const ASTPtr & ast, const ContextPtr & global_context, const DDLRenamingSettings & renaming_settings) +{ + auto new_ast = ast->clone(); + try + { + DDLRenamingVisitor::Data data{renaming_settings, global_context}; + DDLRenamingVisitor::Visitor{data}.visit(new_ast); + return new_ast; + } + catch (...) + { + tryLogCurrentException("Backup", "Error while renaming in AST"); + return ast; + } +} + +} diff --git a/src/Backups/BackupRenamingConfig.h b/src/Backups/DDLRenamingVisitor.h similarity index 51% rename from src/Backups/BackupRenamingConfig.h rename to src/Backups/DDLRenamingVisitor.h index 17329dfaccd1..e74f68a40334 100644 --- a/src/Backups/BackupRenamingConfig.h +++ b/src/Backups/DDLRenamingVisitor.h @@ -1,27 +1,33 @@ #pragma once -#include #include +#include +#include #include +#include #include namespace DB { using DatabaseAndTableName = std::pair; +class IAST; +using ASTPtr = std::shared_ptr; +class Context; +using ContextPtr = std::shared_ptr; /// Keeps information about renamings of databases or tables being processed /// while we're making a backup or while we're restoring from a backup. -class BackupRenamingConfig +class DDLRenamingSettings { public: - BackupRenamingConfig() = default; + DDLRenamingSettings() = default; void setNewTableName(const DatabaseAndTableName & old_table_name, const DatabaseAndTableName & new_table_name); void setNewDatabaseName(const String & old_database_name, const String & new_database_name); void setFromBackupQuery(const ASTBackupQuery & backup_query, const String & current_database); - void setFromBackupQueryElements(const ASTBackupQuery::Elements & backup_query_elements, const String & current_database); + void setFromBackupQuery(const ASTBackupQuery::Elements & backup_query_elements, const String & current_database); /// Changes names according to the renaming. DatabaseAndTableName getNewTableName(const DatabaseAndTableName & old_table_name) const; @@ -32,6 +38,24 @@ class BackupRenamingConfig std::unordered_map old_to_new_database_names; }; -using BackupRenamingConfigPtr = std::shared_ptr; + +/// Changes names in AST according to the renaming settings. +ASTPtr renameInCreateQuery(const ASTPtr & ast, const ContextPtr & global_context, const DDLRenamingSettings & config); + +/// Visits ASTCreateQuery and changes names of tables and databases according to passed DDLRenamingConfig. +class DDLRenamingVisitor +{ +public: + struct Data + { + const DDLRenamingSettings & renaming_settings; + ContextPtr context; + }; + + using Visitor = InDepthNodeVisitor; + + static bool needChildVisit(ASTPtr &, const ASTPtr &); + static void visit(ASTPtr & ast, const Data & data); +}; } diff --git a/src/Backups/RestoreFromBackupUtils.cpp b/src/Backups/RestoreFromBackupUtils.cpp index 2f4ed370a06e..121cd21f2267 100644 --- a/src/Backups/RestoreFromBackupUtils.cpp +++ b/src/Backups/RestoreFromBackupUtils.cpp @@ -1,10 +1,9 @@ #include -#include +#include #include #include #include #include -#include #include #include #include @@ -175,10 +174,8 @@ namespace /// Prepares internal structures for making tasks for restoring. void prepare(const ASTBackupQuery::Elements & elements) { - auto new_renaming_config = std::make_shared(); String current_database = context->getCurrentDatabase(); - new_renaming_config->setFromBackupQueryElements(elements, current_database); - renaming_config = new_renaming_config; + renaming_settings.setFromBackupQuery(elements, current_database); for (const auto & element : elements) { @@ -238,7 +235,7 @@ namespace void prepareToRestoreTable(const DatabaseAndTableName & table_name_, const ASTs & partitions_) { /// Check that we are not trying to restore the same table again. - DatabaseAndTableName new_table_name = renaming_config->getNewTableName(table_name_); + DatabaseAndTableName new_table_name = renaming_settings.getNewTableName(table_name_); if (tables.contains(new_table_name)) { String message; @@ -310,7 +307,7 @@ namespace void prepareToRestoreDatabase(const String & database_name_, const std::set & except_list_) { /// Check that we are not trying to restore the same database again. - String new_database_name = renaming_config->getNewDatabaseName(database_name_); + String new_database_name = renaming_settings.getNewDatabaseName(database_name_); if (databases.contains(new_database_name) && databases[new_database_name].is_explicit) throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Couldn't restore database {} twice", backQuoteIfNeed(new_database_name)); @@ -407,7 +404,7 @@ namespace /// Do renaming in the create query according to the renaming config. std::shared_ptr renameInCreateQuery(const ASTPtr & ast) const { - return typeid_cast>(::DB::renameInCreateQuery(ast, renaming_config, context)); + return typeid_cast>(::DB::renameInCreateQuery(ast, context, renaming_settings)); } static bool isSystemOrTemporaryDatabase(const String & database_name) @@ -442,7 +439,7 @@ namespace ContextMutablePtr context; BackupPtr backup; - BackupRenamingConfigPtr renaming_config; + DDLRenamingSettings renaming_settings; std::map databases; std::map tables; }; diff --git a/src/Backups/renameInCreateQuery.cpp b/src/Backups/renameInCreateQuery.cpp deleted file mode 100644 index fe661868040a..000000000000 --- a/src/Backups/renameInCreateQuery.cpp +++ /dev/null @@ -1,296 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -namespace -{ - class RenameInCreateQueryTransformMatcher - { - public: - struct Data - { - BackupRenamingConfigPtr renaming_config; - ContextPtr context; - }; - - static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; } - - static void visit(ASTPtr & ast, const Data & data) - { - if (auto * create = ast->as()) - visitCreateQuery(*create, data); - else if (auto * expr = ast->as()) - visitTableExpression(*expr, data); - else if (auto * function = ast->as()) - visitFunction(*function, data); - else if (auto * dictionary = ast->as()) - visitDictionary(*dictionary, data); - } - - private: - /// Replaces names of tables and databases used in a CREATE query, which can be either CREATE TABLE or - /// CREATE DICTIONARY or CREATE VIEW or CREATE TEMPORARY TABLE or CREATE DATABASE query. - static void visitCreateQuery(ASTCreateQuery & create, const Data & data) - { - if (create.table) - { - DatabaseAndTableName table_name; - table_name.second = create.getTable(); - if (create.temporary) - table_name.first = DatabaseCatalog::TEMPORARY_DATABASE; - else if (create.database) - table_name.first = create.getDatabase(); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name specified in the CREATE TABLE query must not be empty"); - - table_name = data.renaming_config->getNewTableName(table_name); - - if (table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) - { - create.temporary = true; - create.setDatabase(""); - } - else - { - create.temporary = false; - create.setDatabase(table_name.first); - } - create.setTable(table_name.second); - } - else if (create.database) - { - String database_name = create.getDatabase(); - database_name = data.renaming_config->getNewDatabaseName(database_name); - create.setDatabase(database_name); - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name specified in the CREATE DATABASE query must not be empty"); - - create.uuid = UUIDHelpers::Nil; - - if (!create.as_table.empty() && !create.as_database.empty()) - std::tie(create.as_database, create.as_table) = data.renaming_config->getNewTableName({create.as_database, create.as_table}); - - if (!create.to_table_id.table_name.empty() && !create.to_table_id.database_name.empty()) - { - auto to_table = data.renaming_config->getNewTableName({create.to_table_id.database_name, create.to_table_id.table_name}); - create.to_table_id = StorageID{to_table.first, to_table.second}; - } - } - - /// Replaces names of a database and a table in a expression like `db`.`table` - static void visitTableExpression(ASTTableExpression & expr, const Data & data) - { - if (!expr.database_and_table_name) - return; - - ASTIdentifier * id = expr.database_and_table_name->as(); - if (!id) - return; - - auto table_id = id->createTable(); - if (!table_id) - return; - - const String & db_name = table_id->getDatabaseName(); - const String & table_name = table_id->shortName(); - if (db_name.empty() || table_name.empty()) - return; - - String new_db_name, new_table_name; - std::tie(new_db_name, new_table_name) = data.renaming_config->getNewTableName({db_name, table_name}); - if ((new_db_name == db_name) && (new_table_name == table_name)) - return; - - expr.database_and_table_name = std::make_shared(Strings{new_db_name, new_table_name}); - expr.children.push_back(expr.database_and_table_name); - } - - /// Replaces names of tables and databases used in arguments of a table function or a table engine. - static void visitFunction(ASTFunction & function, const Data & data) - { - if ((function.name == "merge") || (function.name == "Merge")) - { - visitFunctionMerge(function, data); - } - else if ((function.name == "remote") || (function.name == "remoteSecure") || (function.name == "cluster") || - (function.name == "clusterAllReplicas") || (function.name == "Distributed")) - { - visitFunctionRemote(function, data); - } - } - - /// Replaces a database's name passed via an argument of the function merge() or the table engine Merge. - static void visitFunctionMerge(ASTFunction & function, const Data & data) - { - if (!function.arguments) - return; - - /// The first argument is a database's name and we can rename it. - /// The second argument is a regular expression and we can do nothing about it. - auto & args = function.arguments->as().children; - size_t db_name_arg_index = 0; - if (args.size() <= db_name_arg_index) - return; - - String db_name = evaluateConstantExpressionForDatabaseName(args[db_name_arg_index], data.context)->as().value.safeGet(); - if (db_name.empty()) - return; - - String new_db_name = data.renaming_config->getNewDatabaseName(db_name); - if (new_db_name == db_name) - return; - args[db_name_arg_index] = std::make_shared(new_db_name); - } - - /// Replaces names of a table and a database passed via arguments of the function remote() or cluster() or the table engine Distributed. - static void visitFunctionRemote(ASTFunction & function, const Data & data) - { - if (!function.arguments) - return; - - /// The first argument is an address or cluster's name, so we skip it. - /// The second argument can be either 'db.name' or just 'db' followed by the third argument 'table'. - auto & args = function.arguments->as().children; - - const auto * second_arg_as_function = args[1]->as(); - if (second_arg_as_function && TableFunctionFactory::instance().isTableFunctionName(second_arg_as_function->name)) - return; - - size_t db_name_index = 1; - if (args.size() <= db_name_index) - return; - - String name = evaluateConstantExpressionForDatabaseName(args[db_name_index], data.context)->as().value.safeGet(); - - size_t table_name_index = static_cast(-1); - - QualifiedTableName qualified_name; - - if (function.name == "Distributed") - qualified_name.table = name; - else - qualified_name = QualifiedTableName::parseFromString(name); - - if (qualified_name.database.empty()) - { - std::swap(qualified_name.database, qualified_name.table); - table_name_index = 2; - if (args.size() <= table_name_index) - return; - qualified_name.table = evaluateConstantExpressionForDatabaseName(args[table_name_index], data.context)->as().value.safeGet(); - } - - const String & db_name = qualified_name.database; - const String & table_name = qualified_name.table; - - if (db_name.empty() || table_name.empty()) - return; - - String new_db_name, new_table_name; - std::tie(new_db_name, new_table_name) = data.renaming_config->getNewTableName({db_name, table_name}); - if ((new_db_name == db_name) && (new_table_name == table_name)) - return; - - if (table_name_index != static_cast(-1)) - { - if (new_db_name != db_name) - args[db_name_index] = std::make_shared(new_db_name); - if (new_table_name != table_name) - args[table_name_index] = std::make_shared(new_table_name); - } - else - { - args[db_name_index] = std::make_shared(new_db_name); - args.insert(args.begin() + db_name_index + 1, std::make_shared(new_table_name)); - } - } - - /// Replaces names of a table and a database used in source parameters of a dictionary. - static void visitDictionary(ASTDictionary & dictionary, const Data & data) - { - if (!dictionary.source || dictionary.source->name != "clickhouse" || !dictionary.source->elements) - return; - - auto & elements = dictionary.source->elements->as().children; - String db_name, table_name; - size_t db_name_index = static_cast(-1); - size_t table_name_index = static_cast(-1); - - for (size_t i = 0; i != elements.size(); ++i) - { - auto & pair = elements[i]->as(); - if (pair.first == "db") - { - if (db_name_index != static_cast(-1)) - return; - db_name = pair.second->as().value.safeGet(); - db_name_index = i; - } - else if (pair.first == "table") - { - if (table_name_index != static_cast(-1)) - return; - table_name = pair.second->as().value.safeGet(); - table_name_index = i; - } - } - - if (db_name.empty() || table_name.empty()) - return; - - String new_db_name, new_table_name; - std::tie(new_db_name, new_table_name) = data.renaming_config->getNewTableName({db_name, table_name}); - if ((new_db_name == db_name) && (new_table_name == table_name)) - return; - - if (new_db_name != db_name) - { - auto & pair = elements[db_name_index]->as(); - pair.replace(pair.second, std::make_shared(new_db_name)); - } - if (new_table_name != table_name) - { - auto & pair = elements[table_name_index]->as(); - pair.replace(pair.second, std::make_shared(new_table_name)); - } - } - }; - - using RenameInCreateQueryTransformVisitor = InDepthNodeVisitor; -} - - -ASTPtr renameInCreateQuery(const ASTPtr & ast, const BackupRenamingConfigPtr & renaming_config, const ContextPtr & context) -{ - auto new_ast = ast->clone(); - try - { - RenameInCreateQueryTransformVisitor::Data data{renaming_config, context}; - RenameInCreateQueryTransformVisitor{data}.visit(new_ast); - return new_ast; - } - catch (...) - { - tryLogCurrentException("Backup", "Error while renaming in AST"); - return ast; - } -} - -} diff --git a/src/Backups/renameInCreateQuery.h b/src/Backups/renameInCreateQuery.h deleted file mode 100644 index 9c62d07e5c69..000000000000 --- a/src/Backups/renameInCreateQuery.h +++ /dev/null @@ -1,16 +0,0 @@ -#pragma once - -#include - -namespace DB -{ -class IAST; -using ASTPtr = std::shared_ptr; -class Context; -using ContextPtr = std::shared_ptr; -class BackupRenamingConfig; -using BackupRenamingConfigPtr = std::shared_ptr; - -/// Changes names in AST according to the renaming settings. -ASTPtr renameInCreateQuery(const ASTPtr & ast, const BackupRenamingConfigPtr & renaming_config, const ContextPtr & context); -} diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index dacccaa3470e..0ffb8815a750 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -614,6 +614,7 @@ M(643, CANNOT_UNPACK_ARCHIVE) \ M(644, CANNOT_BACKUP_DATABASE) \ M(645, CANNOT_BACKUP_TABLE) \ + M(646, WRONG_DDL_RENAMING_SETTINGS) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ From 0430c1851d9253663d8cf5170a7aa0b7d7c9a324 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 20 Jan 2022 01:56:08 +0700 Subject: [PATCH 06/18] Shorter names: rename IRestoreFromBackupTask -> IRestoreTask. --- ...RestoreFromBackupTask.h => IRestoreTask.h} | 12 ++--- src/Backups/RestoreFromBackupUtils.h | 23 -------- ...reFromBackupUtils.cpp => RestoreUtils.cpp} | 53 ++++++++++--------- src/Backups/RestoreUtils.h | 23 ++++++++ src/Interpreters/InterpreterBackupQuery.cpp | 10 ++-- src/Storages/IStorage.cpp | 2 +- src/Storages/IStorage.h | 6 +-- src/Storages/MergeTree/MergeTreeData.cpp | 14 ++--- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageLog.cpp | 8 +-- src/Storages/StorageLog.h | 2 +- src/Storages/StorageMergeTree.cpp | 4 +- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageStripeLog.cpp | 8 +-- src/Storages/StorageStripeLog.h | 2 +- 15 files changed, 86 insertions(+), 85 deletions(-) rename src/Backups/{IRestoreFromBackupTask.h => IRestoreTask.h} (63%) delete mode 100644 src/Backups/RestoreFromBackupUtils.h rename src/Backups/{RestoreFromBackupUtils.cpp => RestoreUtils.cpp} (93%) create mode 100644 src/Backups/RestoreUtils.h diff --git a/src/Backups/IRestoreFromBackupTask.h b/src/Backups/IRestoreTask.h similarity index 63% rename from src/Backups/IRestoreFromBackupTask.h rename to src/Backups/IRestoreTask.h index 93e08b035ee9..10046ff18070 100644 --- a/src/Backups/IRestoreFromBackupTask.h +++ b/src/Backups/IRestoreTask.h @@ -8,14 +8,14 @@ namespace DB { /// Represents a task of restoring something (database / table / table's part) from backup. -class IRestoreFromBackupTask +class IRestoreTask { public: - IRestoreFromBackupTask() = default; - virtual ~IRestoreFromBackupTask() = default; + IRestoreTask() = default; + virtual ~IRestoreTask() = default; /// Perform restoring, the function also can return a list of nested tasks that should be run later. - virtual std::vector> run() = 0; + virtual std::vector> run() = 0; /// Is it necessary to run this task sequentially? /// Sequential tasks are executed first and strictly in one thread. @@ -25,7 +25,7 @@ class IRestoreFromBackupTask virtual void rollback() {} }; -using RestoreFromBackupTaskPtr = std::unique_ptr; -using RestoreFromBackupTasks = std::vector; +using RestoreTaskPtr = std::unique_ptr; +using RestoreTasks = std::vector; } diff --git a/src/Backups/RestoreFromBackupUtils.h b/src/Backups/RestoreFromBackupUtils.h deleted file mode 100644 index 75fe96495d6c..000000000000 --- a/src/Backups/RestoreFromBackupUtils.h +++ /dev/null @@ -1,23 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -class IBackup; -using BackupPtr = std::shared_ptr; -class IRestoreFromBackupTask; -using RestoreFromBackupTaskPtr = std::unique_ptr; -using RestoreFromBackupTasks = std::vector; -class Context; -using ContextMutablePtr = std::shared_ptr; - -/// Prepares restore tasks. -RestoreFromBackupTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const ASTBackupQuery::Elements & elements); - -/// Executes restore tasks. -void executeRestoreTasks(RestoreFromBackupTasks && tasks, size_t num_threads); - -} diff --git a/src/Backups/RestoreFromBackupUtils.cpp b/src/Backups/RestoreUtils.cpp similarity index 93% rename from src/Backups/RestoreFromBackupUtils.cpp rename to src/Backups/RestoreUtils.cpp index 121cd21f2267..c2dac978caa0 100644 --- a/src/Backups/RestoreFromBackupUtils.cpp +++ b/src/Backups/RestoreUtils.cpp @@ -1,8 +1,9 @@ +#include #include #include #include #include -#include +#include #include #include #include @@ -36,16 +37,16 @@ namespace using ElementType = ASTBackupQuery::ElementType; /// Restores a database (without tables inside), should be executed before executing - /// RestoreTableFromBackupTask. - class RestoreDatabaseFromBackupTask : public IRestoreFromBackupTask + /// RestoreTableTask. + class RestoreDatabaseTask : public IRestoreTask { public: - RestoreDatabaseFromBackupTask(ContextMutablePtr context_, const ASTPtr & create_query_) + RestoreDatabaseTask(ContextMutablePtr context_, const ASTPtr & create_query_) : context(context_), create_query(typeid_cast>(create_query_)) { } - RestoreFromBackupTasks run() override + RestoreTasks run() override { createDatabase(); return {}; @@ -66,10 +67,10 @@ namespace /// Restores a table and fills it with data. - class RestoreTableFromBackupTask : public IRestoreFromBackupTask + class RestoreTableTask : public IRestoreTask { public: - RestoreTableFromBackupTask( + RestoreTableTask( ContextMutablePtr context_, const ASTPtr & create_query_, const ASTs & partitions_, @@ -83,11 +84,11 @@ namespace table_name.first = DatabaseCatalog::TEMPORARY_DATABASE; } - RestoreFromBackupTasks run() override + RestoreTasks run() override { createStorage(); auto storage = getStorage(); - RestoreFromBackupTasks tasks; + RestoreTasks tasks; if (auto task = insertDataIntoStorage(storage)) tasks.push_back(std::move(task)); return tasks; @@ -145,7 +146,7 @@ namespace throw Exception(error_message, ErrorCodes::CANNOT_RESTORE_TABLE); } - RestoreFromBackupTaskPtr insertDataIntoStorage(StoragePtr storage) + RestoreTaskPtr insertDataIntoStorage(StoragePtr storage) { if (storage->hasHollowBackup()) return {}; @@ -208,7 +209,7 @@ namespace } /// Makes tasks for restoring, should be called after prepare(). - RestoreFromBackupTasks makeTasks() const + RestoreTasks makeTasks() const { /// Check that there are not `different_create_query`. (If it's set it means error.) for (auto & info : databases | boost::adaptors::map_values) @@ -219,13 +220,13 @@ namespace serializeAST(*info.create_query), serializeAST(*info.different_create_query)); } - RestoreFromBackupTasks res; + RestoreTasks res; for (auto & info : databases | boost::adaptors::map_values) - res.push_back(std::make_unique(context, info.create_query)); + res.push_back(std::make_unique(context, info.create_query)); /// TODO: We need to restore tables according to their dependencies. for (auto & info : tables | boost::adaptors::map_values) - res.push_back(std::make_unique(context, info.create_query, info.partitions, backup, info.name_in_backup)); + res.push_back(std::make_unique(context, info.create_query, info.partitions, backup, info.name_in_backup)); return res; } @@ -412,7 +413,7 @@ namespace return (database_name == DatabaseCatalog::SYSTEM_DATABASE) || (database_name == DatabaseCatalog::TEMPORARY_DATABASE); } - /// Information which is used to make an instance of RestoreTableFromBackupTask. + /// Information which is used to make an instance of RestoreTableTask. struct CreateTableInfo { ASTPtr create_query; @@ -420,7 +421,7 @@ namespace ASTs partitions; }; - /// Information which is used to make an instance of RestoreDatabaseFromBackupTask. + /// Information which is used to make an instance of RestoreDatabaseTask. struct CreateDatabaseInfo { ASTPtr create_query; @@ -446,7 +447,7 @@ namespace /// Reverts completed restore tasks (in reversed order). - void rollbackRestoreTasks(RestoreFromBackupTasks && restore_tasks) + void rollbackRestoreTasks(RestoreTasks && restore_tasks) { for (auto & restore_task : restore_tasks | boost::adaptors::reversed) { @@ -463,7 +464,7 @@ namespace } -RestoreFromBackupTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const Elements & elements) +RestoreTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const Elements & elements) { RestoreTasksBuilder builder{context, backup}; builder.prepare(elements); @@ -471,12 +472,12 @@ RestoreFromBackupTasks makeRestoreTasks(ContextMutablePtr context, const BackupP } -void executeRestoreTasks(RestoreFromBackupTasks && restore_tasks, size_t num_threads) +void executeRestoreTasks(RestoreTasks && restore_tasks, size_t num_threads) { if (!num_threads) num_threads = 1; - RestoreFromBackupTasks completed_tasks; + RestoreTasks completed_tasks; bool need_rollback_completed_tasks = true; SCOPE_EXIT({ @@ -484,8 +485,8 @@ void executeRestoreTasks(RestoreFromBackupTasks && restore_tasks, size_t num_thr rollbackRestoreTasks(std::move(completed_tasks)); }); - std::deque> sequential_tasks; - std::deque> enqueued_tasks; + std::deque> sequential_tasks; + std::deque> enqueued_tasks; /// There are two kinds of restore tasks: sequential and non-sequential ones. /// Sequential tasks are executed first and always in one thread. @@ -503,7 +504,7 @@ void executeRestoreTasks(RestoreFromBackupTasks && restore_tasks, size_t num_thr auto current_task = std::move(sequential_tasks.front()); sequential_tasks.pop_front(); - RestoreFromBackupTasks new_tasks = current_task->run(); + RestoreTasks new_tasks = current_task->run(); completed_tasks.push_back(std::move(current_task)); for (auto & task : new_tasks) @@ -516,7 +517,7 @@ void executeRestoreTasks(RestoreFromBackupTasks && restore_tasks, size_t num_thr } /// Non-sequential tasks. - std::unordered_map> running_tasks; + std::unordered_map> running_tasks; std::vector threads; std::mutex mutex; std::condition_variable cond; @@ -524,7 +525,7 @@ void executeRestoreTasks(RestoreFromBackupTasks && restore_tasks, size_t num_thr while (true) { - IRestoreFromBackupTask * current_task = nullptr; + IRestoreTask * current_task = nullptr; { std::unique_lock lock{mutex}; cond.wait(lock, [&] @@ -554,7 +555,7 @@ void executeRestoreTasks(RestoreFromBackupTasks && restore_tasks, size_t num_thr return; } - RestoreFromBackupTasks new_tasks; + RestoreTasks new_tasks; std::exception_ptr new_exception; try { diff --git a/src/Backups/RestoreUtils.h b/src/Backups/RestoreUtils.h new file mode 100644 index 000000000000..df03dffa28ca --- /dev/null +++ b/src/Backups/RestoreUtils.h @@ -0,0 +1,23 @@ +#pragma once + +#include + + +namespace DB +{ + +class IBackup; +using BackupPtr = std::shared_ptr; +class IRestoreTask; +using RestoreTaskPtr = std::unique_ptr; +using RestoreTasks = std::vector; +class Context; +using ContextMutablePtr = std::shared_ptr; + +/// Prepares restore tasks. +RestoreTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const ASTBackupQuery::Elements & elements); + +/// Executes restore tasks. +void executeRestoreTasks(RestoreTasks && tasks, size_t num_threads); + +} diff --git a/src/Interpreters/InterpreterBackupQuery.cpp b/src/Interpreters/InterpreterBackupQuery.cpp index de8a9c273cd0..29cf77c7f527 100644 --- a/src/Interpreters/InterpreterBackupQuery.cpp +++ b/src/Interpreters/InterpreterBackupQuery.cpp @@ -1,13 +1,13 @@ #include +#include +#include +#include #include #include #include -#include -#include -#include -#include -#include +#include #include +#include namespace DB diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 5bd3833ebe11..891c3ee053aa 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -222,7 +222,7 @@ BackupEntries IStorage::backup(const ASTs &, ContextPtr) throw Exception("Table engine " + getName() + " doesn't support backups", ErrorCodes::NOT_IMPLEMENTED); } -RestoreFromBackupTaskPtr IStorage::restoreFromBackup(const BackupPtr &, const String &, const ASTs &, ContextMutablePtr) +RestoreTaskPtr IStorage::restoreFromBackup(const BackupPtr &, const String &, const ASTs &, ContextMutablePtr) { throw Exception("Table engine " + getName() + " doesn't support restoring", ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 8da6639c81bb..0ace0ed2886d 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -71,8 +71,8 @@ class IBackup; using BackupPtr = std::shared_ptr; class IBackupEntry; using BackupEntries = std::vector>>; -class IRestoreFromBackupTask; -using RestoreFromBackupTaskPtr = std::unique_ptr; +class IRestoreTask; +using RestoreTaskPtr = std::unique_ptr; struct ColumnSize { @@ -219,7 +219,7 @@ class IStorage : public std::enable_shared_from_this, public TypePromo virtual BackupEntries backup(const ASTs & partitions, ContextPtr context); /// Extract data from the backup and put it to the storage. - virtual RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context); + virtual RestoreTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context); /// Returns whether the column is virtual - by default all columns are real. /// Initially reserved virtual column name may be shadowed by real column. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a60a8a3388df..576e84d45d5e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include @@ -3634,7 +3634,7 @@ BackupEntries MergeTreeData::backupDataParts(const DataPartsVector & data_parts) } -class MergeTreeDataRestoreTask : public IRestoreFromBackupTask +class MergeTreeDataRestoreTask : public IRestoreTask { public: MergeTreeDataRestoreTask( @@ -3651,9 +3651,9 @@ class MergeTreeDataRestoreTask : public IRestoreFromBackupTask { } - RestoreFromBackupTasks run() override + RestoreTasks run() override { - RestoreFromBackupTasks restore_part_tasks; + RestoreTasks restore_part_tasks; Strings part_names = backup->listFiles(data_path_in_backup); for (const String & part_name : part_names) { @@ -3678,7 +3678,7 @@ class MergeTreeDataRestoreTask : public IRestoreFromBackupTask std::unordered_set partition_ids; SimpleIncrement * increment; - class RestorePartTask : public IRestoreFromBackupTask + class RestorePartTask : public IRestoreTask { public: RestorePartTask( @@ -3697,7 +3697,7 @@ class MergeTreeDataRestoreTask : public IRestoreFromBackupTask { } - RestoreFromBackupTasks run() override + RestoreTasks run() override { UInt64 total_size_of_part = 0; Strings filenames = backup->listFiles(data_path_in_backup + part_name + "/", ""); @@ -3742,7 +3742,7 @@ class MergeTreeDataRestoreTask : public IRestoreFromBackupTask }; -RestoreFromBackupTaskPtr MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & backup, const String & data_path_in_backup, +RestoreTaskPtr MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const std::unordered_set & partition_ids, SimpleIncrement * increment) { diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 6db2dc7ffe74..0a1570b195b3 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -654,7 +654,7 @@ class MergeTreeData : public IStorage, public WithMutableContext static BackupEntries backupDataParts(const DataPartsVector & data_parts); /// Extract data from the backup and put it to the storage. - RestoreFromBackupTaskPtr restoreDataPartsFromBackup( + RestoreTaskPtr restoreDataPartsFromBackup( const BackupPtr & backup, const String & data_path_in_backup, const std::unordered_set & partition_ids, diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index fba48f13d6a8..77c5cce51c44 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -28,7 +28,7 @@ #include #include #include -#include +#include #include #include @@ -949,7 +949,7 @@ BackupEntries StorageLog::backup(const ASTs & partitions, ContextPtr context) return backup_entries; } -class LogRestoreTask : public IRestoreFromBackupTask +class LogRestoreTask : public IRestoreTask { using WriteLock = StorageLog::WriteLock; using Mark = StorageLog::Mark; @@ -961,7 +961,7 @@ class LogRestoreTask : public IRestoreFromBackupTask { } - RestoreFromBackupTasks run() override + RestoreTasks run() override { auto lock_timeout = getLockTimeout(context); WriteLock lock{storage->rwlock, lock_timeout}; @@ -1059,7 +1059,7 @@ class LogRestoreTask : public IRestoreFromBackupTask ContextMutablePtr context; }; -RestoreFromBackupTaskPtr StorageLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) +RestoreTaskPtr StorageLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) { if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 3d527521c376..dcea3917c0f6 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -53,7 +53,7 @@ class StorageLog final : public shared_ptr_helper, public IStorage ColumnSizeByName getColumnSizes() const override; BackupEntries backup(const ASTs & partitions, ContextPtr context) override; - RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; + RestoreTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; protected: /** Attach the table with the appropriate name, along the appropriate path (with / at the end), diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index be0416559c54..8495ebfdc40b 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -3,7 +3,7 @@ #include #include -#include +#include #include #include #include @@ -1645,7 +1645,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ } -RestoreFromBackupTaskPtr StorageMergeTree::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr local_context) +RestoreTaskPtr StorageMergeTree::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr local_context) { return restoreDataPartsFromBackup(backup, data_path_in_backup, getPartitionIDsFromQuery(partitions, local_context), &increment); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 2079bf903eb9..f3e482db5f9d 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -97,7 +97,7 @@ class StorageMergeTree final : public shared_ptr_helper, publi CheckResults checkData(const ASTPtr & query, ContextPtr context) override; - RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; + RestoreTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index f7e63340dba9..d31483c69bc7 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -38,7 +38,7 @@ #include #include #include -#include +#include #include #include @@ -554,7 +554,7 @@ BackupEntries StorageStripeLog::backup(const ASTs & partitions, ContextPtr conte return backup_entries; } -class StripeLogRestoreTask : public IRestoreFromBackupTask +class StripeLogRestoreTask : public IRestoreTask { using WriteLock = StorageStripeLog::WriteLock; @@ -568,7 +568,7 @@ class StripeLogRestoreTask : public IRestoreFromBackupTask { } - RestoreFromBackupTasks run() override + RestoreTasks run() override { WriteLock lock{storage->rwlock, getLockTimeout(context)}; if (!lock) @@ -638,7 +638,7 @@ class StripeLogRestoreTask : public IRestoreFromBackupTask }; -RestoreFromBackupTaskPtr StorageStripeLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) +RestoreTaskPtr StorageStripeLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) { if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 5266a8627e27..8e3a70e6158d 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -52,7 +52,7 @@ class StorageStripeLog final : public shared_ptr_helper, publi void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder&) override; BackupEntries backup(const ASTs & partitions, ContextPtr context) override; - RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; + RestoreTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; protected: StorageStripeLog( From 32714522ce40573d0e16b627519b96b1c8fc0918 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 20 Jan 2022 05:13:16 +0700 Subject: [PATCH 07/18] Rework BackupSettings and RestoreSettings a little, pass StorageRestoreSettings to storages. --- src/Backups/BackupSettings.cpp | 16 +++++++- src/Backups/BackupSettings.h | 16 ++++---- src/Backups/BackupUtils.cpp | 9 ++--- src/Backups/BackupUtils.h | 3 +- src/Backups/RestoreSettings.cpp | 20 +++++++++ src/Backups/RestoreSettings.h | 22 ++++++++++ src/Backups/RestoreUtils.cpp | 25 +++++++----- src/Backups/RestoreUtils.h | 3 +- src/Interpreters/InterpreterBackupQuery.cpp | 45 +++++++++------------ src/Storages/IStorage.cpp | 4 +- src/Storages/IStorage.h | 5 ++- src/Storages/MergeTree/MergeTreeData.cpp | 8 ++-- src/Storages/MergeTree/MergeTreeData.h | 4 +- src/Storages/StorageLog.cpp | 4 +- src/Storages/StorageLog.h | 4 +- src/Storages/StorageMergeTree.cpp | 4 +- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageStripeLog.cpp | 4 +- src/Storages/StorageStripeLog.h | 4 +- 19 files changed, 130 insertions(+), 72 deletions(-) create mode 100644 src/Backups/RestoreSettings.cpp create mode 100644 src/Backups/RestoreSettings.h diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index f383330f246f..a73b6974631f 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -1,6 +1,20 @@ #include +#include +#include +#include + namespace DB { -IMPLEMENT_SETTINGS_TRAITS(BackupSettingsTraits, LIST_OF_BACKUP_SETTINGS) + +BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) +{ + BackupSettings res; + + if (query.base_backup_name) + res.base_backup_info = std::make_shared(BackupInfo::fromAST(*query.base_backup_name)); + + return res; +} + } diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index 05fbda2598ec..81d74d2813b7 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -1,16 +1,18 @@ #pragma once -#include +#include namespace DB { +class ASTBackupQuery; +struct BackupInfo; -#define LIST_OF_BACKUP_SETTINGS(M) \ - M(Bool, dummy, false, "", 0) \ - -DECLARE_SETTINGS_TRAITS_ALLOW_CUSTOM_SETTINGS(BackupSettingsTraits, LIST_OF_BACKUP_SETTINGS) - -struct BackupSettings : public BaseSettings {}; +/// Settings specified in the "SETTINGS" clause of a BACKUP query. +struct BackupSettings +{ + std::shared_ptr base_backup_info; + static BackupSettings fromBackupQuery(const ASTBackupQuery & query); +}; } diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 0ad5cf973933..5fff6eb3d374 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -32,7 +32,7 @@ namespace class BackupEntriesBuilder { public: - BackupEntriesBuilder(ContextPtr context_) : context(context_) { } + BackupEntriesBuilder(const ContextPtr & context_, const BackupSettings &) : context(context_) { } /// Prepares internal structures for making backup entries. void prepare(const ASTBackupQuery::Elements & elements) @@ -91,7 +91,7 @@ namespace res.push_back(makeBackupEntryForMetadata(*info.create_query)); if (info.has_data) { - auto data_backup = info.storage->backup(info.partitions, context); + auto data_backup = info.storage->backup(context, info.partitions); if (!data_backup.empty()) { String data_path = getDataPathInBackup(*info.create_query); @@ -288,7 +288,6 @@ namespace }; ContextPtr context; - BackupMutablePtr backup; DDLRenamingSettings renaming_settings; std::map databases; std::map tables; @@ -296,9 +295,9 @@ namespace } -BackupEntries makeBackupEntries(const Elements & elements, const ContextPtr & context) +BackupEntries makeBackupEntries(const ContextPtr & context, const Elements & elements, const BackupSettings & backup_settings) { - BackupEntriesBuilder builder{context}; + BackupEntriesBuilder builder{context, backup_settings}; builder.prepare(elements); return builder.makeBackupEntries(); } diff --git a/src/Backups/BackupUtils.h b/src/Backups/BackupUtils.h index 51e849ae6f3b..d001d5a4bec9 100644 --- a/src/Backups/BackupUtils.h +++ b/src/Backups/BackupUtils.h @@ -10,11 +10,12 @@ using BackupMutablePtr = std::shared_ptr; class IBackupEntry; using BackupEntryPtr = std::unique_ptr; using BackupEntries = std::vector>; +struct BackupSettings; class Context; using ContextPtr = std::shared_ptr; /// Prepares backup entries. -BackupEntries makeBackupEntries(const ASTBackupQuery::Elements & elements, const ContextPtr & context); +BackupEntries makeBackupEntries(const ContextPtr & context, const ASTBackupQuery::Elements & elements, const BackupSettings & backup_settings); /// Write backup entries to an opened backup. void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries, size_t num_threads); diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp new file mode 100644 index 000000000000..2aede5faf881 --- /dev/null +++ b/src/Backups/RestoreSettings.cpp @@ -0,0 +1,20 @@ +#include +#include +#include +#include + + +namespace DB +{ + +RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) +{ + RestoreSettings res; + + if (query.base_backup_name) + res.base_backup_info = std::make_shared(BackupInfo::fromAST(*query.base_backup_name)); + + return res; +} + +} diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h new file mode 100644 index 000000000000..9184a741066b --- /dev/null +++ b/src/Backups/RestoreSettings.h @@ -0,0 +1,22 @@ +#pragma once + +#include + + +namespace DB +{ +class ASTBackupQuery; +struct BackupInfo; + +struct StorageRestoreSettings +{ +}; + +/// Settings specified in the "SETTINGS" clause of a RESTORE query. +struct RestoreSettings : public StorageRestoreSettings +{ + std::shared_ptr base_backup_info; + static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query); +}; + +} diff --git a/src/Backups/RestoreUtils.cpp b/src/Backups/RestoreUtils.cpp index c2dac978caa0..f788742522b1 100644 --- a/src/Backups/RestoreUtils.cpp +++ b/src/Backups/RestoreUtils.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -41,7 +42,7 @@ namespace class RestoreDatabaseTask : public IRestoreTask { public: - RestoreDatabaseTask(ContextMutablePtr context_, const ASTPtr & create_query_) + RestoreDatabaseTask(ContextMutablePtr context_, const ASTPtr & create_query_, const RestoreSettings &) : context(context_), create_query(typeid_cast>(create_query_)) { } @@ -75,9 +76,11 @@ namespace const ASTPtr & create_query_, const ASTs & partitions_, const BackupPtr & backup_, - const DatabaseAndTableName & table_name_in_backup_) + const DatabaseAndTableName & table_name_in_backup_, + const RestoreSettings & restore_settings_) : context(context_), create_query(typeid_cast>(create_query_)), - partitions(partitions_), backup(backup_), table_name_in_backup(table_name_in_backup_) + partitions(partitions_), backup(backup_), table_name_in_backup(table_name_in_backup_), + restore_settings(restore_settings_) { table_name = DatabaseAndTableName{create_query->getDatabase(), create_query->getTable()}; if (create_query->temporary) @@ -152,7 +155,7 @@ namespace return {}; context->checkAccess(AccessType::INSERT, table_name.first, table_name.second); String data_path_in_backup = getDataPathInBackup(table_name_in_backup); - return storage->restoreFromBackup(backup, data_path_in_backup, partitions, context); + return storage->restoreFromBackup(context, partitions, backup, data_path_in_backup, restore_settings); } ContextMutablePtr context; @@ -161,6 +164,7 @@ namespace ASTs partitions; BackupPtr backup; DatabaseAndTableName table_name_in_backup; + RestoreSettings restore_settings; }; @@ -169,8 +173,8 @@ namespace class RestoreTasksBuilder { public: - RestoreTasksBuilder(ContextMutablePtr context_, const BackupPtr & backup_) - : context(context_), backup(backup_) {} + RestoreTasksBuilder(ContextMutablePtr context_, const BackupPtr & backup_, const RestoreSettings & restore_settings_) + : context(context_), backup(backup_), restore_settings(restore_settings_) {} /// Prepares internal structures for making tasks for restoring. void prepare(const ASTBackupQuery::Elements & elements) @@ -222,11 +226,11 @@ namespace RestoreTasks res; for (auto & info : databases | boost::adaptors::map_values) - res.push_back(std::make_unique(context, info.create_query)); + res.push_back(std::make_unique(context, info.create_query, restore_settings)); /// TODO: We need to restore tables according to their dependencies. for (auto & info : tables | boost::adaptors::map_values) - res.push_back(std::make_unique(context, info.create_query, info.partitions, backup, info.name_in_backup)); + res.push_back(std::make_unique(context, info.create_query, info.partitions, backup, info.name_in_backup, restore_settings)); return res; } @@ -440,6 +444,7 @@ namespace ContextMutablePtr context; BackupPtr backup; + RestoreSettings restore_settings; DDLRenamingSettings renaming_settings; std::map databases; std::map tables; @@ -464,9 +469,9 @@ namespace } -RestoreTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const Elements & elements) +RestoreTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const Elements & elements, const RestoreSettings & restore_settings) { - RestoreTasksBuilder builder{context, backup}; + RestoreTasksBuilder builder{context, backup, restore_settings}; builder.prepare(elements); return builder.makeTasks(); } diff --git a/src/Backups/RestoreUtils.h b/src/Backups/RestoreUtils.h index df03dffa28ca..33d2f7ff527a 100644 --- a/src/Backups/RestoreUtils.h +++ b/src/Backups/RestoreUtils.h @@ -11,11 +11,12 @@ using BackupPtr = std::shared_ptr; class IRestoreTask; using RestoreTaskPtr = std::unique_ptr; using RestoreTasks = std::vector; +struct RestoreSettings; class Context; using ContextMutablePtr = std::shared_ptr; /// Prepares restore tasks. -RestoreTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const ASTBackupQuery::Elements & elements); +RestoreTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const ASTBackupQuery::Elements & elements, const RestoreSettings & restore_settings); /// Executes restore tasks. void executeRestoreTasks(RestoreTasks && tasks, size_t num_threads); diff --git a/src/Interpreters/InterpreterBackupQuery.cpp b/src/Interpreters/InterpreterBackupQuery.cpp index 29cf77c7f527..ac930e1f6a72 100644 --- a/src/Interpreters/InterpreterBackupQuery.cpp +++ b/src/Interpreters/InterpreterBackupQuery.cpp @@ -5,49 +5,42 @@ #include #include #include +#include #include #include -#include namespace DB { namespace { - BackupMutablePtr createBackup(const ASTBackupQuery & query, const ContextPtr & context) + BackupMutablePtr createBackup( + const BackupInfo & backup_info, + IBackup::OpenMode open_mode, + const BackupInfo * base_backup_info, + const ContextPtr & context) { BackupFactory::CreateParams params; - params.open_mode = (query.kind == ASTBackupQuery::BACKUP) ? IBackup::OpenMode::WRITE : IBackup::OpenMode::READ; + params.open_mode = open_mode; params.context = context; - - params.backup_info = BackupInfo::fromAST(*query.backup_name); - if (query.base_backup_name) - params.base_backup_info = BackupInfo::fromAST(*query.base_backup_name); - + params.backup_info = backup_info; + params.base_backup_info = base_backup_info ? *base_backup_info : std::optional{}; return BackupFactory::instance().createBackup(params); } -#if 0 - void getBackupSettings(const ASTBackupQuery & query, BackupSettings & settings, std::optional & base_backup) - { - settings = {}; - if (query.settings) - settings.applyChanges(query.settings->as().changes); - return settings; - } -#endif - - void executeBackup(const ASTBackupQuery & query, const ContextPtr & context) + void executeBackup(const ContextPtr & context, const ASTBackupQuery & query) { - BackupMutablePtr backup = createBackup(query, context); - auto backup_entries = makeBackupEntries(query.elements, context); + auto backup_settings = BackupSettings::fromBackupQuery(query); + BackupMutablePtr backup = createBackup(BackupInfo::fromAST(*query.backup_name), IBackup::OpenMode::WRITE, backup_settings.base_backup_info.get(), context); + auto backup_entries = makeBackupEntries(context, query.elements, backup_settings); writeBackupEntries(backup, std::move(backup_entries), context->getSettingsRef().max_backup_threads); } - void executeRestore(const ASTBackupQuery & query, ContextMutablePtr context) + void executeRestore(ContextMutablePtr context, const ASTBackupQuery & query) { - BackupPtr backup = createBackup(query, context); - auto restore_tasks = makeRestoreTasks(context, backup, query.elements); + auto restore_settings = RestoreSettings::fromRestoreQuery(query); + BackupPtr backup = createBackup(BackupInfo::fromAST(*query.backup_name), IBackup::OpenMode::READ, restore_settings.base_backup_info.get(), context); + auto restore_tasks = makeRestoreTasks(context, backup, query.elements, restore_settings); executeRestoreTasks(std::move(restore_tasks), context->getSettingsRef().max_backup_threads); } } @@ -56,9 +49,9 @@ BlockIO InterpreterBackupQuery::execute() { const auto & query = query_ptr->as(); if (query.kind == ASTBackupQuery::BACKUP) - executeBackup(query, context); + executeBackup(context, query); else if (query.kind == ASTBackupQuery::RESTORE) - executeRestore(query, context); + executeRestore(context, query); return {}; } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 891c3ee053aa..142e548773a9 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -217,12 +217,12 @@ bool IStorage::isStaticStorage() const return false; } -BackupEntries IStorage::backup(const ASTs &, ContextPtr) +BackupEntries IStorage::backup(ContextPtr, const ASTs &) { throw Exception("Table engine " + getName() + " doesn't support backups", ErrorCodes::NOT_IMPLEMENTED); } -RestoreTaskPtr IStorage::restoreFromBackup(const BackupPtr &, const String &, const ASTs &, ContextMutablePtr) +RestoreTaskPtr IStorage::restoreFromBackup(ContextMutablePtr, const ASTs &, const BackupPtr &, const String &, const StorageRestoreSettings &) { throw Exception("Table engine " + getName() + " doesn't support restoring", ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 0ace0ed2886d..c0223bf1813e 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -73,6 +73,7 @@ class IBackupEntry; using BackupEntries = std::vector>>; class IRestoreTask; using RestoreTaskPtr = std::unique_ptr; +struct StorageRestoreSettings; struct ColumnSize { @@ -216,10 +217,10 @@ class IStorage : public std::enable_shared_from_this, public TypePromo virtual bool hasHollowBackup() const { return false; } /// Prepares entries to backup data of the storage. - virtual BackupEntries backup(const ASTs & partitions, ContextPtr context); + virtual BackupEntries backup(ContextPtr context, const ASTs & partitions); /// Extract data from the backup and put it to the storage. - virtual RestoreTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context); + virtual RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings); /// Returns whether the column is virtual - by default all columns are real. /// Initially reserved virtual column name may be shadowed by real column. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 576e84d45d5e..43d42f24eaed 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3582,7 +3582,7 @@ Pipe MergeTreeData::alterPartition( } -BackupEntries MergeTreeData::backup(const ASTs & partitions, ContextPtr local_context) +BackupEntries MergeTreeData::backup(ContextPtr local_context, const ASTs & partitions) { DataPartsVector data_parts; if (partitions.empty()) @@ -3742,9 +3742,9 @@ class MergeTreeDataRestoreTask : public IRestoreTask }; -RestoreTaskPtr MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & backup, const String & data_path_in_backup, - const std::unordered_set & partition_ids, - SimpleIncrement * increment) +RestoreTaskPtr MergeTreeData::restoreDataPartsFromBackup(const std::unordered_set & partition_ids, + const BackupPtr & backup, const String & data_path_in_backup, + SimpleIncrement * increment) { return std::make_unique( std::static_pointer_cast(shared_from_this()), backup, data_path_in_backup, partition_ids, increment); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 0a1570b195b3..a8c26bfa4bd6 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -650,14 +650,14 @@ class MergeTreeData : public IStorage, public WithMutableContext TableLockHolder & table_lock_holder); /// Prepares entries to backup data of the storage. - BackupEntries backup(const ASTs & partitions, ContextPtr context) override; + BackupEntries backup(ContextPtr context, const ASTs & partitions) override; static BackupEntries backupDataParts(const DataPartsVector & data_parts); /// Extract data from the backup and put it to the storage. RestoreTaskPtr restoreDataPartsFromBackup( + const std::unordered_set & partition_ids, const BackupPtr & backup, const String & data_path_in_backup, - const std::unordered_set & partition_ids, SimpleIncrement * increment); /// Moves partition to specified Disk diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 77c5cce51c44..277be91b9770 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -884,7 +884,7 @@ IStorage::ColumnSizeByName StorageLog::getColumnSizes() const } -BackupEntries StorageLog::backup(const ASTs & partitions, ContextPtr context) +BackupEntries StorageLog::backup(ContextPtr context, const ASTs & partitions) { if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); @@ -1059,7 +1059,7 @@ class LogRestoreTask : public IRestoreTask ContextMutablePtr context; }; -RestoreTaskPtr StorageLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) +RestoreTaskPtr StorageLog::restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &) { if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index dcea3917c0f6..93c292dc485b 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -52,8 +52,8 @@ class StorageLog final : public shared_ptr_helper, public IStorage bool supportsSubcolumns() const override { return true; } ColumnSizeByName getColumnSizes() const override; - BackupEntries backup(const ASTs & partitions, ContextPtr context) override; - RestoreTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; + BackupEntries backup(ContextPtr context, const ASTs & partitions) override; + RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override; protected: /** Attach the table with the appropriate name, along the appropriate path (with / at the end), diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 8495ebfdc40b..cf27a4e907e6 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1645,9 +1645,9 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ } -RestoreTaskPtr StorageMergeTree::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr local_context) +RestoreTaskPtr StorageMergeTree::restoreFromBackup(ContextMutablePtr local_context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &) { - return restoreDataPartsFromBackup(backup, data_path_in_backup, getPartitionIDsFromQuery(partitions, local_context), &increment); + return restoreDataPartsFromBackup(getPartitionIDsFromQuery(partitions, local_context), backup, data_path_in_backup, &increment); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index f3e482db5f9d..ad9381dbc492 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -97,7 +97,7 @@ class StorageMergeTree final : public shared_ptr_helper, publi CheckResults checkData(const ASTPtr & query, ContextPtr context) override; - RestoreTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; + RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override; bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index d31483c69bc7..6317f08b3c76 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -492,7 +492,7 @@ void StorageStripeLog::saveFileSizes(const WriteLock & /* already locked for wri } -BackupEntries StorageStripeLog::backup(const ASTs & partitions, ContextPtr context) +BackupEntries StorageStripeLog::backup(ContextPtr context, const ASTs & partitions) { if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); @@ -638,7 +638,7 @@ class StripeLogRestoreTask : public IRestoreTask }; -RestoreTaskPtr StorageStripeLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) +RestoreTaskPtr StorageStripeLog::restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &) { if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 8e3a70e6158d..60adc0f5a20c 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -51,8 +51,8 @@ class StorageStripeLog final : public shared_ptr_helper, publi void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder&) override; - BackupEntries backup(const ASTs & partitions, ContextPtr context) override; - RestoreTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; + BackupEntries backup(ContextPtr context, const ASTs & partitions) override; + RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override; protected: StorageStripeLog( From 57fad4caaddfdcefed1fe14b8e55e169bae3c4f9 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 20 Jan 2022 17:02:50 +0700 Subject: [PATCH 08/18] Add setting "structure_only." --- src/Backups/BackupSettings.cpp | 16 ++++++++++++++++ src/Backups/BackupSettings.h | 6 ++++++ src/Backups/BackupUtils.cpp | 9 +++++++-- src/Backups/RestoreSettings.cpp | 16 ++++++++++++++++ src/Backups/RestoreSettings.h | 8 ++++++++ src/Backups/RestoreUtils.cpp | 2 +- 6 files changed, 54 insertions(+), 3 deletions(-) diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index a73b6974631f..602cf67cc2b7 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -6,6 +6,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) { @@ -14,6 +18,18 @@ BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) if (query.base_backup_name) res.base_backup_info = std::make_shared(BackupInfo::fromAST(*query.base_backup_name)); + if (query.settings) + { + const auto & settings = query.settings->as().changes; + for (const auto & setting : settings) + { + if (setting.name == "structure_only") + res.structure_only = setting.value.safeGet(); + else + throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting {}", setting.name); + } + } + return res; } diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index 81d74d2813b7..ce911382c782 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -11,7 +11,13 @@ struct BackupInfo; /// Settings specified in the "SETTINGS" clause of a BACKUP query. struct BackupSettings { + /// Base backup, if it's set an incremental backup will be built. std::shared_ptr base_backup_info; + + /// If this is set to true then only create queries will be written to backup, + /// without the data of tables. + bool structure_only = false; + static BackupSettings fromBackupQuery(const ASTBackupQuery & query); }; diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 5fff6eb3d374..19287019a3e8 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -32,7 +33,10 @@ namespace class BackupEntriesBuilder { public: - BackupEntriesBuilder(const ContextPtr & context_, const BackupSettings &) : context(context_) { } + BackupEntriesBuilder(const ContextPtr & context_, const BackupSettings & backup_settings_) + : context(context_), backup_settings(backup_settings_) + { + } /// Prepares internal structures for making backup entries. void prepare(const ASTBackupQuery::Elements & elements) @@ -145,7 +149,7 @@ namespace /// Make a create query for this table. auto create_query = renameInCreateQuery(database->getCreateTableQuery(table_name_.second, context)); - bool has_data = !storage->hasHollowBackup(); + bool has_data = !storage->hasHollowBackup() && !backup_settings.structure_only; if (has_data) context->checkAccess(AccessType::SELECT, table_name_.first, table_name_.second); @@ -288,6 +292,7 @@ namespace }; ContextPtr context; + BackupSettings backup_settings; DDLRenamingSettings renaming_settings; std::map databases; std::map tables; diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 2aede5faf881..f7ff0a5ac051 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -6,6 +6,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) { @@ -14,6 +18,18 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) if (query.base_backup_name) res.base_backup_info = std::make_shared(BackupInfo::fromAST(*query.base_backup_name)); + if (query.settings) + { + const auto & settings = query.settings->as().changes; + for (const auto & setting : settings) + { + if (setting.name == "structure_only") + res.structure_only = setting.value.safeGet(); + else + throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting {}", setting.name); + } + } + return res; } diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 9184a741066b..41d323d165de 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -15,7 +15,15 @@ struct StorageRestoreSettings /// Settings specified in the "SETTINGS" clause of a RESTORE query. struct RestoreSettings : public StorageRestoreSettings { + /// Base backup, with this setting we can override the location of the base backup while restoring. + /// Any incremental backup keeps inside the information about its base backup, + /// so using this setting is optional. std::shared_ptr base_backup_info; + + /// If this is set to true then only create queries will be read from backup, + /// without the data of tables. + bool structure_only = false; + static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query); }; diff --git a/src/Backups/RestoreUtils.cpp b/src/Backups/RestoreUtils.cpp index f788742522b1..f7bac6609448 100644 --- a/src/Backups/RestoreUtils.cpp +++ b/src/Backups/RestoreUtils.cpp @@ -151,7 +151,7 @@ namespace RestoreTaskPtr insertDataIntoStorage(StoragePtr storage) { - if (storage->hasHollowBackup()) + if (storage->hasHollowBackup() || restore_settings.structure_only) return {}; context->checkAccess(AccessType::INSERT, table_name.first, table_name.second); String data_path_in_backup = getDataPathInBackup(table_name_in_backup); From d4e89420087a7fb972dcf71a79f89b7e5196e797 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 20 Jan 2022 20:10:54 +0700 Subject: [PATCH 09/18] Add settings "throw_if_database_exists" and "throw_if_table_exists". --- src/Backups/RestoreSettings.cpp | 4 ++++ src/Backups/RestoreSettings.h | 6 ++++++ src/Backups/RestoreUtils.cpp | 4 ++-- tests/integration/test_backup_restore_new/test.py | 4 ++-- 4 files changed, 14 insertions(+), 4 deletions(-) diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index f7ff0a5ac051..f27522446017 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -25,6 +25,10 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) { if (setting.name == "structure_only") res.structure_only = setting.value.safeGet(); + else if (setting.name == "throw_if_database_exists") + res.throw_if_database_exists = setting.value.safeGet(); + else if (setting.name == "throw_if_table_exists") + res.throw_if_table_exists = setting.value.safeGet(); else throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting {}", setting.name); } diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 41d323d165de..8981ad0fac06 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -24,6 +24,12 @@ struct RestoreSettings : public StorageRestoreSettings /// without the data of tables. bool structure_only = false; + /// Whether RESTORE DATABASE will throw an exception if a destination database already exists. + bool throw_if_database_exists = true; + + /// Whether RESTORE TABLE will throw an exception if a destination table already exists. + bool throw_if_table_exists = true; + static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query); }; diff --git a/src/Backups/RestoreUtils.cpp b/src/Backups/RestoreUtils.cpp index f7bac6609448..5378666187c9 100644 --- a/src/Backups/RestoreUtils.cpp +++ b/src/Backups/RestoreUtils.cpp @@ -253,7 +253,7 @@ namespace /// Make a create query for this table. auto create_query = renameInCreateQuery(readCreateQueryFromBackup(table_name_)); - create_query->if_not_exists = true; + create_query->if_not_exists = !restore_settings.throw_if_table_exists; CreateTableInfo info; info.create_query = create_query; @@ -339,7 +339,7 @@ namespace db_name_in_backup.clear(); } - create_db_query->if_not_exists = true; + create_db_query->if_not_exists = !restore_settings.throw_if_database_exists; CreateDatabaseInfo info_db; info_db.create_query = create_db_query; diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index f9bfababadcc..0d51a284e492 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -60,10 +60,10 @@ def test_restore_table_into_existing_table(engine): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" instance.query(f"BACKUP TABLE test.table TO {backup_name}") - instance.query(f"RESTORE TABLE test.table INTO test.table FROM {backup_name}") + instance.query(f"RESTORE TABLE test.table INTO test.table FROM {backup_name} SETTINGS throw_if_table_exists=0") assert instance.query("SELECT count(), sum(x) FROM test.table") == "200\t9900\n" - instance.query(f"RESTORE TABLE test.table INTO test.table FROM {backup_name}") + instance.query(f"RESTORE TABLE test.table INTO test.table FROM {backup_name} SETTINGS throw_if_table_exists=0") assert instance.query("SELECT count(), sum(x) FROM test.table") == "300\t14850\n" From b16395fe3c3825fac0917bb3c35cb06af1a352d3 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 23 Jan 2022 20:35:11 +0700 Subject: [PATCH 10/18] Add settings "throw_if_table_has_different_definition" and "throw_if_database_has_different_definition". --- src/Backups/BackupUtils.cpp | 24 +-- src/Backups/DDLCompareUtils.cpp | 83 ++++++++ src/Backups/DDLCompareUtils.h | 17 ++ src/Backups/RestoreSettings.cpp | 4 + src/Backups/RestoreSettings.h | 8 + src/Backups/RestoreUtils.cpp | 191 ++++++++++++------ .../formatTableNameOrTemporaryTableName.cpp | 17 ++ .../formatTableNameOrTemporaryTableName.h | 13 ++ .../hasCompatibleDataToRestoreTable.cpp | 26 --- src/Backups/hasCompatibleDataToRestoreTable.h | 11 - 10 files changed, 286 insertions(+), 108 deletions(-) create mode 100644 src/Backups/DDLCompareUtils.cpp create mode 100644 src/Backups/DDLCompareUtils.h create mode 100644 src/Backups/formatTableNameOrTemporaryTableName.cpp create mode 100644 src/Backups/formatTableNameOrTemporaryTableName.h delete mode 100644 src/Backups/hasCompatibleDataToRestoreTable.cpp delete mode 100644 src/Backups/hasCompatibleDataToRestoreTable.h diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 19287019a3e8..5e40e1dbb696 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -1,8 +1,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -82,7 +84,7 @@ namespace { if (info.different_create_query) throw Exception(ErrorCodes::CANNOT_BACKUP_DATABASE, - "Couldn't backup a database because two different create queries were generated for it: {} and {}", + "Cannot backup a database because two different create queries were generated for it: {} and {}", serializeAST(*info.create_query), serializeAST(*info.different_create_query)); } @@ -130,28 +132,24 @@ namespace if (database->hasHollowBackup()) throw Exception( ErrorCodes::CANNOT_BACKUP_TABLE, - "Couldn't backup table {}.{} because of the database's engine {} is hollow", - backQuoteIfNeed(table_name_.first), backQuoteIfNeed(table_name_.second), + "Cannot backup the {} because it's contained in a hollow database (engine: {})", + formatTableNameOrTemporaryTableName(table_name_), database->getEngineName()); /// Check that we are not trying to backup the same table again. DatabaseAndTableName new_table_name = renaming_settings.getNewTableName(table_name_); if (tables.contains(new_table_name)) - { - String message; - if (new_table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) - message = fmt::format("Couldn't backup temporary table {} twice", backQuoteIfNeed(new_table_name.second)); - else - message = fmt::format("Couldn't backup table {}.{} twice", backQuoteIfNeed(new_table_name.first), backQuoteIfNeed(new_table_name.second)); - throw Exception(ErrorCodes::CANNOT_BACKUP_TABLE, message); - } + throw Exception(ErrorCodes::CANNOT_BACKUP_TABLE, "Cannot backup the {} twice", formatTableNameOrTemporaryTableName(new_table_name)); /// Make a create query for this table. auto create_query = renameInCreateQuery(database->getCreateTableQuery(table_name_.second, context)); bool has_data = !storage->hasHollowBackup() && !backup_settings.structure_only; if (has_data) + { + /// We check for SELECT privilege only if we're going to read data from the table. context->checkAccess(AccessType::SELECT, table_name_.first, table_name_.second); + } CreateTableInfo info; info.create_query = create_query; @@ -185,7 +183,7 @@ namespace { auto create_db_query = renameInCreateQuery(table_.first->getCreateDatabaseQuery()); create_db_query->setDatabase(new_table_name.first); - if (serializeAST(*info_db.create_query) != serializeAST(*create_db_query)) + if (!areDatabaseDefinitionsSame(*info_db.create_query, *create_db_query)) info_db.different_create_query = create_db_query; } } @@ -206,7 +204,7 @@ namespace /// Check that we are not trying to restore the same database again. String new_database_name = renaming_settings.getNewDatabaseName(database_name_); if (databases.contains(new_database_name) && databases[new_database_name].is_explicit) - throw Exception(ErrorCodes::CANNOT_BACKUP_DATABASE, "Couldn't backup database {} twice", backQuoteIfNeed(new_database_name)); + throw Exception(ErrorCodes::CANNOT_BACKUP_DATABASE, "Cannot backup the database {} twice", backQuoteIfNeed(new_database_name)); /// Of course we're not going to backup the definition of the system or the temporary database. if (!isSystemOrTemporaryDatabase(database_name_)) diff --git a/src/Backups/DDLCompareUtils.cpp b/src/Backups/DDLCompareUtils.cpp new file mode 100644 index 000000000000..7c857cff7d59 --- /dev/null +++ b/src/Backups/DDLCompareUtils.cpp @@ -0,0 +1,83 @@ +#include +#include +#include + + +namespace DB +{ +namespace +{ + std::shared_ptr prepareDDLToCompare(const ASTCreateQuery & ast) + { + auto res = typeid_cast>(ast.shared_from_this()); + + std::shared_ptr clone; + auto get_clone = [&] + { + if (!clone) + { + clone = typeid_cast>(res->clone()); + res = clone; + } + return clone; + }; + + /// Remove UUID. + if (res->uuid != UUIDHelpers::Nil) + get_clone()->uuid = UUIDHelpers::Nil; + + /// Clear IF NOT EXISTS flag. + if (res->if_not_exists) + get_clone()->if_not_exists = false; + + return res; + } +} + + +bool areTableDefinitionsSame(const IAST & table1, const IAST & table2) +{ + auto ast1 = typeid_cast>(table1.shared_from_this()); + if (!ast1 || !ast1->table) + return false; + + auto ast2 = typeid_cast>(table2.shared_from_this()); + if (!ast2 || !ast2->table) + return false; + + if ((ast1->uuid != ast2->uuid) || (ast1->if_not_exists != ast2->if_not_exists)) + { + ast1 = prepareDDLToCompare(*ast1); + ast2 = prepareDDLToCompare(*ast2); + } + + return serializeAST(*ast1) == serializeAST(*ast1); +} + + +bool areDatabaseDefinitionsSame(const IAST & database1, const IAST & database2) +{ + auto ast1 = typeid_cast>(database1.shared_from_this()); + if (!ast1 || ast1->table || !ast1->database) + return false; + + auto ast2 = typeid_cast>(database2.shared_from_this()); + if (!ast2 || ast2->table || !ast2->database) + return false; + + if ((ast1->uuid != ast2->uuid) || (ast1->if_not_exists != ast2->if_not_exists)) + { + ast1 = prepareDDLToCompare(*ast1); + ast2 = prepareDDLToCompare(*ast2); + } + + return serializeAST(*ast1) == serializeAST(*ast1); +} + + +bool areTableDataCompatible(const IAST & src_table, const IAST & dest_table) +{ + return areTableDefinitionsSame(src_table, dest_table); +} + +} diff --git a/src/Backups/DDLCompareUtils.h b/src/Backups/DDLCompareUtils.h new file mode 100644 index 000000000000..acb99c243ea3 --- /dev/null +++ b/src/Backups/DDLCompareUtils.h @@ -0,0 +1,17 @@ +#pragma once + + +namespace DB +{ +class IAST; + +/// Checks that two table definitions are actually the same. +bool areTableDefinitionsSame(const IAST & table1, const IAST & table2); + +/// Checks that two database definitions are actually the same. +bool areDatabaseDefinitionsSame(const IAST & database1, const IAST & database2); + +/// Whether the data from the first table can be attached to the second table. +bool areTableDataCompatible(const IAST & src_table, const IAST & dest_table); + +} diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index f27522446017..6d96c6f22ebb 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -29,6 +29,10 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) res.throw_if_database_exists = setting.value.safeGet(); else if (setting.name == "throw_if_table_exists") res.throw_if_table_exists = setting.value.safeGet(); + else if (setting.name == "throw_if_database_has_different_definition") + res.throw_if_database_has_different_definition = setting.value.safeGet(); + else if (setting.name == "throw_if_table_has_different_definition") + res.throw_if_table_has_different_definition = setting.value.safeGet(); else throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting {}", setting.name); } diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 8981ad0fac06..ad046dfa1cfb 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -30,6 +30,14 @@ struct RestoreSettings : public StorageRestoreSettings /// Whether RESTORE TABLE will throw an exception if a destination table already exists. bool throw_if_table_exists = true; + /// Whether RESTORE DATABASE will throw an exception if a destination database has + /// a different definition comparing with the definition read from backup. + bool throw_if_database_has_different_definition = true; + + /// Whether RESTORE TABLE will throw an exception if a destination table has + /// a different definition comparing with the definition read from backup. + bool throw_if_table_has_different_definition = true; + static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query); }; diff --git a/src/Backups/RestoreUtils.cpp b/src/Backups/RestoreUtils.cpp index 5378666187c9..3c8c1dcba09a 100644 --- a/src/Backups/RestoreUtils.cpp +++ b/src/Backups/RestoreUtils.cpp @@ -1,11 +1,12 @@ #include #include +#include #include #include #include #include #include -#include +#include #include #include #include @@ -37,19 +38,29 @@ namespace using Elements = ASTBackupQuery::Elements; using ElementType = ASTBackupQuery::ElementType; + /// Restores a database (without tables inside), should be executed before executing /// RestoreTableTask. class RestoreDatabaseTask : public IRestoreTask { public: - RestoreDatabaseTask(ContextMutablePtr context_, const ASTPtr & create_query_, const RestoreSettings &) - : context(context_), create_query(typeid_cast>(create_query_)) + RestoreDatabaseTask( + ContextMutablePtr context_, + const ASTPtr & create_query_, + const RestoreSettings & restore_settings_, + bool skip_same_definition_check_) + : context(context_) + , create_query(typeid_cast>(create_query_)) + , restore_settings(restore_settings_) + , skip_same_definition_check(skip_same_definition_check_) { } RestoreTasks run() override { createDatabase(); + getDatabase(); + checkDatabaseCreateQuery(); return {}; } @@ -58,12 +69,50 @@ namespace private: void createDatabase() { - InterpreterCreateQuery create_interpreter{create_query, context}; + /// We need to call clone() for `create_query` because the interpreter can decide + /// to change a passed AST a little bit. + InterpreterCreateQuery create_interpreter{create_query->clone(), context}; create_interpreter.execute(); } + DatabasePtr getDatabase() + { + if (!database) + database = DatabaseCatalog::instance().getDatabase(create_query->getDatabase()); + return database; + } + + ASTPtr getDatabaseCreateQuery() + { + if (!database_create_query) + database_create_query = getDatabase()->getCreateDatabaseQuery(); + return database_create_query; + } + + void checkDatabaseCreateQuery() + { + if (skip_same_definition_check || !restore_settings.throw_if_database_has_different_definition) + return; + + getDatabaseCreateQuery(); + if (areDatabaseDefinitionsSame(*create_query, *database_create_query)) + return; + + throw Exception( + ErrorCodes::CANNOT_RESTORE_DATABASE, + "The database {} already exists but has a different definition: {}, " + "compare to its definition in the backup: {}", + backQuoteIfNeed(create_query->getDatabase()), + serializeAST(*database_create_query), + serializeAST(*create_query)); + } + ContextMutablePtr context; std::shared_ptr create_query; + RestoreSettings restore_settings; + bool skip_same_definition_check = false; + DatabasePtr database; + ASTPtr database_create_query; }; @@ -90,9 +139,10 @@ namespace RestoreTasks run() override { createStorage(); - auto storage = getStorage(); + getStorage(); + checkStorageCreateQuery(); RestoreTasks tasks; - if (auto task = insertDataIntoStorage(storage)) + if (auto task = insertData()) tasks.push_back(std::move(task)); return tasks; } @@ -102,59 +152,84 @@ namespace private: void createStorage() { - InterpreterCreateQuery create_interpreter{create_query, context}; + /// We need to call clone() for `create_query` because the interpreter can decide + /// to change a passed AST a little bit. + InterpreterCreateQuery create_interpreter{create_query->clone(), context}; create_interpreter.execute(); } - StoragePtr tryGetStorage() + StoragePtr getStorage() { - if (!DatabaseCatalog::instance().isTableExist({table_name.first, table_name.second}, context)) - return nullptr; + if (!storage) + std::tie(database, storage) = DatabaseCatalog::instance().getDatabaseAndTable({table_name.first, table_name.second}, context); + return storage; + } - DatabasePtr existing_database; - StoragePtr existing_storage; - std::tie(existing_database, existing_storage) = DatabaseCatalog::instance().tryGetDatabaseAndTable({table_name.first, table_name.second}, context); - if (!existing_storage) - return nullptr; + ASTPtr getStorageCreateQuery() + { + if (!storage_create_query) + { + getStorage(); + storage_create_query = database->getCreateTableQuery(table_name.second, context); + } + return storage_create_query; + } + + void checkStorageCreateQuery() + { + if (!restore_settings.throw_if_table_has_different_definition) + return; + + getStorageCreateQuery(); + if (areTableDefinitionsSame(*create_query, *storage_create_query)) + return; + + throw Exception( + ErrorCodes::CANNOT_RESTORE_TABLE, + "The {} already exists but has a different definition: {}, " + "compare to its definition in the backup: {}", + formatTableNameOrTemporaryTableName(table_name), + serializeAST(*storage_create_query), + serializeAST(*create_query)); + } + + bool hasData() + { + if (has_data) + return *has_data; + + has_data = false; + if (restore_settings.structure_only) + return false; - auto existing_table_create_query = existing_database->tryGetCreateTableQuery(table_name.second, context); - if (!existing_table_create_query) - return nullptr; + data_path_in_backup = getDataPathInBackup(table_name_in_backup); + if (backup->listFiles(data_path_in_backup).empty()) + return false; - if (!hasCompatibleDataToRestoreTable(*create_query, existing_table_create_query->as())) + getStorageCreateQuery(); + if (!areTableDataCompatible(*create_query, *storage_create_query)) throw Exception( ErrorCodes::CANNOT_RESTORE_TABLE, - "Table {}.{} from backup is incompatible with existing table {}.{}. " - "The create query of the table from backup: {}." - "The create query of the existing table: {}", - backQuoteIfNeed(table_name_in_backup.first), - backQuoteIfNeed(table_name_in_backup.second), - backQuoteIfNeed(table_name.first), - backQuoteIfNeed(table_name.second), + "Cannot attach data of the {} in the backup to the existing {} because of they are not compatible. " + "Here is the definition of the {} in the backup: {}, and here is the definition of the existing {}: {}", + formatTableNameOrTemporaryTableName(table_name_in_backup), + formatTableNameOrTemporaryTableName(table_name), + formatTableNameOrTemporaryTableName(table_name_in_backup), serializeAST(*create_query), - serializeAST(*existing_table_create_query)); + formatTableNameOrTemporaryTableName(table_name), + serializeAST(*storage_create_query)); - return existing_storage; - } + /// We check for INSERT privilege only if we're going to write into table. + context->checkAccess(AccessType::INSERT, table_name.first, table_name.second); - StoragePtr getStorage() - { - if (auto storage = tryGetStorage()) - return storage; - - String error_message = (table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) - ? ("Could not create temporary table " + backQuoteIfNeed(table_name.second) + " for restoring") - : ("Could not create table " + backQuoteIfNeed(table_name.first) + "." + backQuoteIfNeed(table_name.second) - + " for restoring"); - throw Exception(error_message, ErrorCodes::CANNOT_RESTORE_TABLE); + has_data = true; + return true; } - RestoreTaskPtr insertDataIntoStorage(StoragePtr storage) + RestoreTaskPtr insertData() { - if (storage->hasHollowBackup() || restore_settings.structure_only) + if (!hasData()) return {}; - context->checkAccess(AccessType::INSERT, table_name.first, table_name.second); - String data_path_in_backup = getDataPathInBackup(table_name_in_backup); return storage->restoreFromBackup(context, partitions, backup, data_path_in_backup, restore_settings); } @@ -165,6 +240,11 @@ namespace BackupPtr backup; DatabaseAndTableName table_name_in_backup; RestoreSettings restore_settings; + DatabasePtr database; + StoragePtr storage; + ASTPtr storage_create_query; + std::optional has_data; + String data_path_in_backup; }; @@ -220,13 +300,14 @@ namespace { if (info.different_create_query) throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, - "Couldn't restore a database because two different create queries were generated for it: {} and {}", + "Cannot restore a database because two different create queries were generated for it: {} and {}", serializeAST(*info.create_query), serializeAST(*info.different_create_query)); } RestoreTasks res; for (auto & info : databases | boost::adaptors::map_values) - res.push_back(std::make_unique(context, info.create_query, restore_settings)); + res.push_back(std::make_unique(context, info.create_query, restore_settings, + /* skip_same_definition_check = */ !info.is_explicit)); /// TODO: We need to restore tables according to their dependencies. for (auto & info : tables | boost::adaptors::map_values) @@ -242,14 +323,7 @@ namespace /// Check that we are not trying to restore the same table again. DatabaseAndTableName new_table_name = renaming_settings.getNewTableName(table_name_); if (tables.contains(new_table_name)) - { - String message; - if (new_table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) - message = fmt::format("Couldn't restore temporary table {} twice", backQuoteIfNeed(new_table_name.second)); - else - message = fmt::format("Couldn't restore table {}.{} twice", backQuoteIfNeed(new_table_name.first), backQuoteIfNeed(new_table_name.second)); - throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, message); - } + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore the {} twice", formatTableNameOrTemporaryTableName(new_table_name)); /// Make a create query for this table. auto create_query = renameInCreateQuery(readCreateQueryFromBackup(table_name_)); @@ -301,7 +375,7 @@ namespace create_db_query = std::make_shared(); create_db_query->setDatabase(new_table_name.first); create_db_query->if_not_exists = true; - if (serializeAST(*info_db.create_query) != serializeAST(*create_db_query)) + if (!areDatabaseDefinitionsSame(*info_db.create_query, *create_db_query)) info_db.different_create_query = create_db_query; } } @@ -314,13 +388,13 @@ namespace /// Check that we are not trying to restore the same database again. String new_database_name = renaming_settings.getNewDatabaseName(database_name_); if (databases.contains(new_database_name) && databases[new_database_name].is_explicit) - throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Couldn't restore database {} twice", backQuoteIfNeed(new_database_name)); + throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Cannot restore the database {} twice", backQuoteIfNeed(new_database_name)); Strings table_metadata_filenames = backup->listFiles("metadata/" + escapeForFileName(database_name_) + "/", "/"); bool throw_if_no_create_database_query = table_metadata_filenames.empty(); if (throw_if_no_create_database_query && !hasCreateQueryInBackup(database_name_)) - throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Could not restore database {} because there is no such database in the backup", backQuoteIfNeed(database_name_)); + throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Cannot restore the database {} because there is no such database in the backup", backQuoteIfNeed(database_name_)); /// Of course we're not going to restore the definition of the system or the temporary database. if (!isSystemOrTemporaryDatabase(new_database_name)) @@ -376,7 +450,8 @@ namespace { String create_query_path = getMetadataPathInBackup(table_name); if (!backup->fileExists(create_query_path)) - throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Could not restore table {}.{} because there is no such table in the backup", backQuoteIfNeed(table_name.first), backQuoteIfNeed(table_name.second)); + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore the {} because there is no such table in the backup", + formatTableNameOrTemporaryTableName(table_name)); auto read_buffer = backup->readFile(create_query_path)->getReadBuffer(); String create_query_str; readStringUntilEOF(create_query_str, *read_buffer); @@ -390,7 +465,7 @@ namespace { String create_query_path = getMetadataPathInBackup(database_name); if (!backup->fileExists(create_query_path)) - throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Could not restore database {} because there is no such database in the backup", backQuoteIfNeed(database_name)); + throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Cannot restore the database {} because there is no such database in the backup", backQuoteIfNeed(database_name)); auto read_buffer = backup->readFile(create_query_path)->getReadBuffer(); String create_query_str; readStringUntilEOF(create_query_str, *read_buffer); diff --git a/src/Backups/formatTableNameOrTemporaryTableName.cpp b/src/Backups/formatTableNameOrTemporaryTableName.cpp new file mode 100644 index 000000000000..7338e1dab230 --- /dev/null +++ b/src/Backups/formatTableNameOrTemporaryTableName.cpp @@ -0,0 +1,17 @@ +#include +#include +#include + + +namespace DB +{ + +String formatTableNameOrTemporaryTableName(const DatabaseAndTableName & table_name) +{ + if (table_name.first == DatabaseCatalog::TEMPORARY_DATABASE) + return "temporary table " + backQuoteIfNeed(table_name.second); + else + return "table " + backQuoteIfNeed(table_name.first) + "." + backQuoteIfNeed(table_name.second); +} + +} diff --git a/src/Backups/formatTableNameOrTemporaryTableName.h b/src/Backups/formatTableNameOrTemporaryTableName.h new file mode 100644 index 000000000000..a6b94cd4077d --- /dev/null +++ b/src/Backups/formatTableNameOrTemporaryTableName.h @@ -0,0 +1,13 @@ +#pragma once + +#include + + +namespace DB +{ +using DatabaseAndTableName = std::pair; + +/// Outputs either "table db_name.table_name" or "temporary table table_name". +String formatTableNameOrTemporaryTableName(const DatabaseAndTableName & table_name); + +} diff --git a/src/Backups/hasCompatibleDataToRestoreTable.cpp b/src/Backups/hasCompatibleDataToRestoreTable.cpp deleted file mode 100644 index 8ac2fda93d84..000000000000 --- a/src/Backups/hasCompatibleDataToRestoreTable.cpp +++ /dev/null @@ -1,26 +0,0 @@ -#include -#include -#include - - -namespace DB -{ - -bool hasCompatibleDataToRestoreTable(const ASTCreateQuery & query1, const ASTCreateQuery & query2) -{ - /// TODO: Write more subtle condition here. - auto q1 = typeid_cast>(query1.clone()); - auto q2 = typeid_cast>(query2.clone()); - - /// Remove UUIDs. - q1->uuid = UUIDHelpers::Nil; - q2->uuid = UUIDHelpers::Nil; - - /// Clear IF NOT EXISTS flag. - q1->if_not_exists = false; - q2->if_not_exists = false; - - return serializeAST(*q1) == serializeAST(*q2); -} - -} diff --git a/src/Backups/hasCompatibleDataToRestoreTable.h b/src/Backups/hasCompatibleDataToRestoreTable.h deleted file mode 100644 index 92aab8b45798..000000000000 --- a/src/Backups/hasCompatibleDataToRestoreTable.h +++ /dev/null @@ -1,11 +0,0 @@ -#pragma once - - -namespace DB -{ -class ASTCreateQuery; - -/// Whether the data of the first table can be inserted to the second table. -bool hasCompatibleDataToRestoreTable(const ASTCreateQuery & query1, const ASTCreateQuery & query2); - -} From 5f30aaa7f5eece9cbf4678bc1ed9387327dc6dcc Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 25 Jan 2022 00:41:13 +0700 Subject: [PATCH 11/18] Implement BACKUP & RESTORE for the Memory table engine. --- src/Common/FileChecker.cpp | 46 +++-- src/Common/FileChecker.h | 8 +- src/Storages/StorageMemory.cpp | 157 ++++++++++++++++++ src/Storages/StorageMemory.h | 4 + src/Storages/StorageStripeLog.h | 1 + .../test_backup_restore_new/test.py | 4 +- 6 files changed, 204 insertions(+), 16 deletions(-) diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index 4de5a92a1b84..ef90e31982ce 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -1,13 +1,16 @@ -#include +#include +#include +#include #include #include #include #include #include -#include +#include +#include -#include +namespace fs = std::filesystem; namespace DB { @@ -19,6 +22,10 @@ namespace ErrorCodes } +FileChecker::FileChecker(const String & file_info_path_) : FileChecker(nullptr, file_info_path_) +{ +} + FileChecker::FileChecker(DiskPtr disk_, const String & file_info_path_) : disk(std::move(disk_)) { setPath(file_info_path_); @@ -45,8 +52,8 @@ String FileChecker::getPath() const void FileChecker::update(const String & full_file_path) { - bool exists = disk->exists(full_file_path); - auto real_size = exists ? disk->getFileSize(full_file_path) : 0; /// No race condition assuming no one else is working with these files. + bool exists = fileReallyExists(full_file_path); + auto real_size = exists ? getRealFileSize(full_file_path) : 0; /// No race condition assuming no one else is working with these files. map[fileName(full_file_path)] = real_size; } @@ -74,8 +81,8 @@ CheckResults FileChecker::check() const { const String & name = name_size.first; String path = parentPath(files_info_path) + name; - bool exists = disk->exists(path); - auto real_size = exists ? disk->getFileSize(path) : 0; /// No race condition assuming no one else is working with these files. + bool exists = fileReallyExists(path); + auto real_size = exists ? getRealFileSize(path) : 0; /// No race condition assuming no one else is working with these files. if (real_size != name_size.second) { @@ -99,8 +106,8 @@ void FileChecker::repair() const String & name = name_size.first; size_t expected_size = name_size.second; String path = parentPath(files_info_path) + name; - bool exists = disk->exists(path); - auto real_size = exists ? disk->getFileSize(path) : 0; /// No race condition assuming no one else is working with these files. + bool exists = fileReallyExists(path); + auto real_size = exists ? getRealFileSize(path) : 0; /// No race condition assuming no one else is working with these files. if (real_size < expected_size) throw Exception(ErrorCodes::UNEXPECTED_END_OF_FILE, "Size of {} is less than expected. Size is {} but should be {}.", @@ -119,7 +126,7 @@ void FileChecker::save() const std::string tmp_files_info_path = parentPath(files_info_path) + "tmp_" + fileName(files_info_path); { - std::unique_ptr out = disk->writeFile(tmp_files_info_path); + std::unique_ptr out = disk ? disk->writeFile(tmp_files_info_path) : std::make_unique(tmp_files_info_path); /// So complex JSON structure - for compatibility with the old format. writeCString("{\"clickhouse\":{", *out); @@ -141,17 +148,20 @@ void FileChecker::save() const out->next(); } - disk->replaceFile(tmp_files_info_path, files_info_path); + if (disk) + disk->replaceFile(tmp_files_info_path, files_info_path); + else + fs::rename(tmp_files_info_path, files_info_path); } void FileChecker::load() { map.clear(); - if (!disk->exists(files_info_path)) + if (!fileReallyExists(files_info_path)) return; - std::unique_ptr in = disk->readFile(files_info_path); + std::unique_ptr in = disk ? disk->readFile(files_info_path) : createReadBufferFromFileBase(files_info_path, {}); WriteBufferFromOwnString out; /// The JSON library does not support whitespace. We delete them. Inefficient. @@ -169,4 +179,14 @@ void FileChecker::load() map[unescapeForFileName(file.getName())] = file.getValue()["size"].toUInt(); } +bool FileChecker::fileReallyExists(const String & path_) const +{ + return disk ? disk->exists(path_) : fs::exists(path_); +} + +size_t FileChecker::getRealFileSize(const String & path_) const +{ + return disk ? disk->getFileSize(path_) : fs::file_size(path_); +} + } diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index a0ea449393ef..98e7837db758 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -2,16 +2,19 @@ #include #include -#include namespace DB { +class IDisk; +using DiskPtr = std::shared_ptr; + /// Stores the sizes of all columns, and can check whether the columns are corrupted. class FileChecker { public: + FileChecker(const String & file_info_path_); FileChecker(DiskPtr disk_, const String & file_info_path_); void setPath(const String & file_info_path_); @@ -36,6 +39,9 @@ class FileChecker private: void load(); + bool fileReallyExists(const String & path_) const; + size_t getRealFileSize(const String & path_) const; + const DiskPtr disk; const Poco::Logger * log = &Poco::Logger::get("FileChecker"); diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 72851472b797..17431b0e9dc0 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -13,6 +13,17 @@ #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + namespace DB { @@ -327,6 +338,152 @@ void StorageMemory::truncate( total_size_rows.store(0, std::memory_order_relaxed); } + +BackupEntries StorageMemory::backup(ContextPtr context, const ASTs & partitions) +{ + if (!partitions.empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); + + auto blocks = data.get(); + + /// We store our data in the StripeLog format. + BackupEntries backup_entries; + auto temp_dir_owner = std::make_shared(); + auto temp_dir = temp_dir_owner->path(); + fs::create_directories(temp_dir); + + /// Writing data.bin + constexpr char data_file_name[] = "data.bin"; + String data_file_path = temp_dir + "/" + data_file_name; + IndexForNativeFormat index; + { + auto data_out_compressed = std::make_unique(data_file_path); + CompressedWriteBuffer data_out{*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), context->getSettingsRef().max_compress_block_size}; + NativeWriter block_out{data_out, 0, getInMemoryMetadataPtr()->getSampleBlock(), false, &index}; + for (const auto & block : *blocks) + block_out.write(block); + } + + /// Writing index.mrk + constexpr char index_file_name[] = "index.mrk"; + String index_file_path = temp_dir + "/" + index_file_name; + { + auto index_out_compressed = std::make_unique(index_file_path); + CompressedWriteBuffer index_out{*index_out_compressed}; + index.write(index_out); + } + + /// Writing sizes.json + constexpr char sizes_file_name[] = "sizes.json"; + String sizes_file_path = temp_dir + "/" + sizes_file_name; + FileChecker file_checker{sizes_file_path}; + file_checker.update(data_file_path); + file_checker.update(index_file_path); + file_checker.save(); + + /// Prepare backup entries. + backup_entries.emplace_back( + data_file_name, + std::make_unique( + data_file_path, file_checker.getFileSize(data_file_path), std::nullopt, temp_dir_owner)); + + backup_entries.emplace_back( + index_file_name, + std::make_unique( + index_file_path, file_checker.getFileSize(index_file_path), std::nullopt, temp_dir_owner)); + + backup_entries.emplace_back( + sizes_file_name, + std::make_unique( + sizes_file_path, std::nullopt, std::nullopt, temp_dir_owner)); + + return backup_entries; +} + + +class MemoryRestoreTask : public IRestoreTask +{ +public: + MemoryRestoreTask( + std::shared_ptr storage_, const BackupPtr & backup_, const String & data_path_in_backup_, ContextMutablePtr context_) + : storage(storage_), backup(backup_), data_path_in_backup(data_path_in_backup_), context(context_) + { + } + + RestoreTasks run() override + { + /// Our data are in the StripeLog format. + + /// Reading index.mrk + IndexForNativeFormat index; + { + String index_file_path = data_path_in_backup + "index.mrk"; + auto backup_entry = backup->readFile(index_file_path); + auto in = backup_entry->getReadBuffer(); + CompressedReadBuffer compressed_in{*in}; + index.read(compressed_in); + } + + /// Reading data.bin + Blocks new_blocks; + size_t new_bytes = 0; + size_t new_rows = 0; + { + String data_file_path = data_path_in_backup + "data.bin"; + auto backup_entry = backup->readFile(data_file_path); + std::unique_ptr in = backup_entry->getReadBuffer(); + std::optional temp_data_copy; + if (!typeid_cast(in.get())) + { + temp_data_copy.emplace(); + auto temp_data_copy_out = std::make_unique(temp_data_copy->path()); + copyData(*in, *temp_data_copy_out); + temp_data_copy_out.reset(); + in = createReadBufferFromFileBase(temp_data_copy->path(), {}); + } + std::unique_ptr in_from_file{static_cast(in.release())}; + CompressedReadBufferFromFile compressed_in{std::move(in_from_file)}; + NativeReader block_in{compressed_in, 0, index.blocks.begin(), index.blocks.end()}; + + while (auto block = block_in.read()) + { + new_bytes += block.bytes(); + new_rows += block.rows(); + new_blocks.push_back(std::move(block)); + } + } + + /// Append old blocks with the new ones. + auto old_blocks = storage->data.get(); + Blocks old_and_new_blocks = *old_blocks; + old_and_new_blocks.insert(old_and_new_blocks.end(), std::make_move_iterator(new_blocks.begin()), std::make_move_iterator(new_blocks.end())); + + /// Finish restoring. + storage->data.set(std::make_unique(std::move(old_and_new_blocks))); + storage->total_size_bytes += new_bytes; + storage->total_size_rows += new_rows; + + return {}; + } + +private: + std::shared_ptr storage; + BackupPtr backup; + String data_path_in_backup; + ContextMutablePtr context; +}; + + +RestoreTaskPtr StorageMemory::restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &) +{ + if (!partitions.empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); + + return std::make_unique( + typeid_cast>(shared_from_this()), backup, data_path_in_backup, context); +} + + std::optional StorageMemory::totalRows(const Settings &) const { /// All modifications of these counters are done under mutex which automatically guarantees synchronization/consistency diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index 063802faf1a0..b9c9cc32a587 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -22,6 +22,7 @@ namespace DB class StorageMemory final : public shared_ptr_helper, public IStorage { friend class MemorySink; +friend class MemoryRestoreTask; friend struct shared_ptr_helper; public: @@ -55,6 +56,9 @@ friend struct shared_ptr_helper; void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; + BackupEntries backup(ContextPtr context, const ASTs & partitions) override; + RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override; + std::optional totalRows(const Settings &) const override; std::optional totalBytes(const Settings &) const override; diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 60adc0f5a20c..8ba2857dbc23 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 0d51a284e492..ad1190baac06 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -37,7 +37,7 @@ def new_backup_name(): return f"Disk('backups', '{backup_id_counter}/')" -@pytest.mark.parametrize("engine", ["MergeTree", "Log", "TinyLog", "StripeLog"]) +@pytest.mark.parametrize("engine", ["MergeTree", "Log", "TinyLog", "StripeLog", "Memory"]) def test_restore_table(engine): backup_name = new_backup_name() create_and_fill_table(engine=engine) @@ -52,7 +52,7 @@ def test_restore_table(engine): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" -@pytest.mark.parametrize("engine", ["MergeTree", "Log", "TinyLog", "StripeLog"]) +@pytest.mark.parametrize("engine", ["MergeTree", "Log", "TinyLog", "StripeLog", "Memory"]) def test_restore_table_into_existing_table(engine): backup_name = new_backup_name() create_and_fill_table(engine=engine) From ee48021d6b8f041b79d472d36b14dd43d4cc5c3c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 22 Feb 2022 07:59:41 +0300 Subject: [PATCH 12/18] Improve zip archive utilities. --- src/IO/Archives/IArchiveReader.h | 1 - src/IO/Archives/IArchiveWriter.h | 2 +- src/IO/Archives/ZipArchiveReader.cpp | 23 +++++- src/IO/Archives/ZipArchiveReader.h | 13 ++-- src/IO/Archives/ZipArchiveWriter.cpp | 75 +++++++++++-------- src/IO/Archives/ZipArchiveWriter.h | 25 +++---- .../hasRegisteredArchiveFileExtension.cpp | 12 +++ .../hasRegisteredArchiveFileExtension.h | 12 +++ 8 files changed, 105 insertions(+), 58 deletions(-) create mode 100644 src/IO/Archives/hasRegisteredArchiveFileExtension.cpp create mode 100644 src/IO/Archives/hasRegisteredArchiveFileExtension.h diff --git a/src/IO/Archives/IArchiveReader.h b/src/IO/Archives/IArchiveReader.h index 584e80a7d090..b5c474977bf5 100644 --- a/src/IO/Archives/IArchiveReader.h +++ b/src/IO/Archives/IArchiveReader.h @@ -23,7 +23,6 @@ class IArchiveReader : public std::enable_shared_from_this { UInt64 uncompressed_size; UInt64 compressed_size; - int compression_method; bool is_encrypted; }; diff --git a/src/IO/Archives/IArchiveWriter.h b/src/IO/Archives/IArchiveWriter.h index 6879d470b62a..c6330509f548 100644 --- a/src/IO/Archives/IArchiveWriter.h +++ b/src/IO/Archives/IArchiveWriter.h @@ -29,7 +29,7 @@ class IArchiveWriter : public std::enable_shared_from_this /// Sets compression method and level. /// Changing them will affect next file in the archive. - virtual void setCompression(int /* compression_method */, int /* compression_level */ = kDefaultCompressionLevel) {} + virtual void setCompression(const String & /* compression_method */, int /* compression_level */ = kDefaultCompressionLevel) {} /// Sets password. If the password is not empty it will enable encryption in the archive. virtual void setPassword(const String & /* password */) {} diff --git a/src/IO/Archives/ZipArchiveReader.cpp b/src/IO/Archives/ZipArchiveReader.cpp index 16604da62dcc..45d037a505d0 100644 --- a/src/IO/Archives/ZipArchiveReader.cpp +++ b/src/IO/Archives/ZipArchiveReader.cpp @@ -1,6 +1,7 @@ #include #if USE_MINIZIP +#include #include #include #include @@ -18,6 +19,20 @@ namespace ErrorCodes using RawHandle = unzFile; +namespace +{ + void checkCompressionMethodIsEnabled(int compression_method_) + { + ZipArchiveWriter::checkCompressionMethodIsEnabled(compression_method_); + } + + void checkEncryptionIsEnabled() + { + ZipArchiveWriter::checkEncryptionIsEnabled(); + } +} + + /// Holds a raw handle, calls acquireRawHandle() in the constructor and releaseRawHandle() in the destructor. class ZipArchiveReader::HandleHolder { @@ -108,7 +123,7 @@ class ZipArchiveReader::HandleHolder return *file_name; } - const FileInfo & getFileInfo() const + const FileInfoImpl & getFileInfo() const { if (!file_info) retrieveFileInfo(); @@ -161,7 +176,7 @@ class ZipArchiveReader::HandleHolder std::shared_ptr reader; RawHandle raw_handle = nullptr; mutable std::optional file_name; - mutable std::optional file_info; + mutable std::optional file_info; }; @@ -174,7 +189,7 @@ class ZipArchiveReader::ReadBufferFromZipArchive : public ReadBufferFromFileBase , handle(std::move(handle_)) { const auto & file_info = handle.getFileInfo(); - checkCompressionMethodIsEnabled(static_cast(file_info.compression_method)); + checkCompressionMethodIsEnabled(file_info.compression_method); const char * password_cstr = nullptr; if (file_info.is_encrypted) @@ -227,7 +242,7 @@ class ZipArchiveReader::ReadBufferFromZipArchive : public ReadBufferFromFileBase if (new_pos > static_cast(file_info.uncompressed_size)) throw Exception("Seek position is out of bound", ErrorCodes::SEEK_POSITION_OUT_OF_BOUND); - if (file_info.compression_method == static_cast(CompressionMethod::kStore)) + if (file_info.compression_method == MZ_COMPRESS_METHOD_STORE) { /// unzSeek64() works only for non-compressed files. checkResult(unzSeek64(raw_handle, off, whence)); diff --git a/src/IO/Archives/ZipArchiveReader.h b/src/IO/Archives/ZipArchiveReader.h index 6932a93e23f2..7236b0b660c4 100644 --- a/src/IO/Archives/ZipArchiveReader.h +++ b/src/IO/Archives/ZipArchiveReader.h @@ -4,7 +4,6 @@ #if USE_MINIZIP #include -#include #include #include #include @@ -20,8 +19,6 @@ class SeekableReadBuffer; class ZipArchiveReader : public shared_ptr_helper, public IArchiveReader { public: - using CompressionMethod = ZipArchiveWriter::CompressionMethod; - ~ZipArchiveReader() override; /// Returns true if there is a specified file in the archive. @@ -45,11 +42,6 @@ class ZipArchiveReader : public shared_ptr_helper, public IArc /// Sets password used to decrypt the contents of the files in the archive. void setPassword(const String & password_) override; - /// Utility functions. - static CompressionMethod parseCompressionMethod(const String & str) { return ZipArchiveWriter::parseCompressionMethod(str); } - static void checkCompressionMethodIsEnabled(CompressionMethod method) { ZipArchiveWriter::checkCompressionMethodIsEnabled(method); } - static void checkEncryptionIsEnabled() { ZipArchiveWriter::checkEncryptionIsEnabled(); } - private: /// Constructs an archive's reader that will read from a file in the local filesystem. explicit ZipArchiveReader(const String & path_to_archive_); @@ -66,6 +58,11 @@ class ZipArchiveReader : public shared_ptr_helper, public IArc void init(); + struct FileInfoImpl : public FileInfo + { + int compression_method; + }; + HandleHolder acquireHandle(); RawHandle acquireRawHandle(); void releaseRawHandle(RawHandle handle_); diff --git a/src/IO/Archives/ZipArchiveWriter.cpp b/src/IO/Archives/ZipArchiveWriter.cpp index f5ecea5e5aaa..414b853975da 100644 --- a/src/IO/Archives/ZipArchiveWriter.cpp +++ b/src/IO/Archives/ZipArchiveWriter.cpp @@ -80,7 +80,7 @@ class ZipArchiveWriter::WriteBufferFromZipArchive : public WriteBufferFromFileBa { auto compress_method = handle.getWriter()->compression_method; auto compress_level = handle.getWriter()->compression_level; - checkCompressionMethodIsEnabled(static_cast(compress_method)); + checkCompressionMethodIsEnabled(compress_method); const char * password_cstr = nullptr; const String & password_str = handle.getWriter()->password; @@ -238,7 +238,7 @@ ZipArchiveWriter::ZipArchiveWriter(const String & path_to_archive_) } ZipArchiveWriter::ZipArchiveWriter(const String & path_to_archive_, std::unique_ptr archive_write_buffer_) - : path_to_archive(path_to_archive_) + : path_to_archive(path_to_archive_), compression_method(MZ_COMPRESS_METHOD_DEFLATE) { if (archive_write_buffer_) handle = StreamFromWriteBuffer::open(std::move(archive_write_buffer_)); @@ -246,6 +246,7 @@ ZipArchiveWriter::ZipArchiveWriter(const String & path_to_archive_, std::unique_ handle = zipOpen64(path_to_archive.c_str(), /* append= */ false); if (!handle) throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Couldn't create zip archive {}", quoteString(path_to_archive)); + } ZipArchiveWriter::~ZipArchiveWriter() @@ -274,10 +275,10 @@ bool ZipArchiveWriter::isWritingFile() const return !handle; } -void ZipArchiveWriter::setCompression(int compression_method_, int compression_level_) +void ZipArchiveWriter::setCompression(const String & compression_method_, int compression_level_) { std::lock_guard lock{mutex}; - compression_method = compression_method_; + compression_method = compressionMethodToInt(compression_method_); compression_level = compression_level_; } @@ -287,48 +288,62 @@ void ZipArchiveWriter::setPassword(const String & password_) password = password_; } -ZipArchiveWriter::CompressionMethod ZipArchiveWriter::parseCompressionMethod(const String & str) +int ZipArchiveWriter::compressionMethodToInt(const String & compression_method_) { - if (str.empty()) - return CompressionMethod::kDeflate; /// Default compression method is DEFLATE. - else if (boost::iequals(str, "store")) - return CompressionMethod::kStore; - else if (boost::iequals(str, "deflate")) - return CompressionMethod::kDeflate; - else if (boost::iequals(str, "bzip2")) - return CompressionMethod::kBzip2; - else if (boost::iequals(str, "lzma")) - return CompressionMethod::kLzma; - else if (boost::iequals(str, "zstd")) - return CompressionMethod::kZstd; - else if (boost::iequals(str, "xz")) - return CompressionMethod::kXz; + if (compression_method_.empty()) + return MZ_COMPRESS_METHOD_DEFLATE; /// By default the compression method is "deflate". + else if (compression_method_ == kStore) + return MZ_COMPRESS_METHOD_STORE; + else if (compression_method_ == kDeflate) + return MZ_COMPRESS_METHOD_DEFLATE; + else if (compression_method_ == kBzip2) + return MZ_COMPRESS_METHOD_BZIP2; + else if (compression_method_ == kLzma) + return MZ_COMPRESS_METHOD_LZMA; + else if (compression_method_ == kZstd) + return MZ_COMPRESS_METHOD_ZSTD; + else if (compression_method_ == kXz) + return MZ_COMPRESS_METHOD_XZ; else - throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Unknown compression method specified for a zip archive: {}", str); + throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Unknown compression method specified for a zip archive: {}", compression_method_); +} + +String ZipArchiveWriter::intToCompressionMethod(int compression_method_) +{ + switch (compression_method_) + { + case MZ_COMPRESS_METHOD_STORE: return kStore; + case MZ_COMPRESS_METHOD_DEFLATE: return kDeflate; + case MZ_COMPRESS_METHOD_BZIP2: return kBzip2; + case MZ_COMPRESS_METHOD_LZMA: return kLzma; + case MZ_COMPRESS_METHOD_ZSTD: return kZstd; + case MZ_COMPRESS_METHOD_XZ: return kXz; + } + throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Unknown compression method specified for a zip archive: {}", compression_method_); } /// Checks that a passed compression method can be used. -void ZipArchiveWriter::checkCompressionMethodIsEnabled(CompressionMethod method) +void ZipArchiveWriter::checkCompressionMethodIsEnabled(int compression_method_) { - switch (method) + switch (compression_method_) { - case CompressionMethod::kStore: [[fallthrough]]; - case CompressionMethod::kDeflate: - case CompressionMethod::kLzma: - case CompressionMethod::kXz: - case CompressionMethod::kZstd: + case MZ_COMPRESS_METHOD_STORE: [[fallthrough]]; + case MZ_COMPRESS_METHOD_DEFLATE: + case MZ_COMPRESS_METHOD_LZMA: + case MZ_COMPRESS_METHOD_ZSTD: + case MZ_COMPRESS_METHOD_XZ: return; - case CompressionMethod::kBzip2: + case MZ_COMPRESS_METHOD_BZIP2: { #if USE_BZIP2 return; #else - throw Exception("BZIP2 compression method is disabled", ErrorCodes::SUPPORT_IS_DISABLED); + throw Exception("bzip2 compression method is disabled", ErrorCodes::SUPPORT_IS_DISABLED); #endif } } - throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Unknown compression method specified for a zip archive: {}", static_cast(method)); + throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Unknown compression method specified for a zip archive: {}", compression_method_); } /// Checks that encryption is enabled. diff --git a/src/IO/Archives/ZipArchiveWriter.h b/src/IO/Archives/ZipArchiveWriter.h index 76f8dd8e9e53..58df4902434d 100644 --- a/src/IO/Archives/ZipArchiveWriter.h +++ b/src/IO/Archives/ZipArchiveWriter.h @@ -31,16 +31,12 @@ class ZipArchiveWriter : public shared_ptr_helper, public IArc bool isWritingFile() const override; /// Supported compression methods. - enum class CompressionMethod - { - /// See mz.h - kStore = 0, - kDeflate = 8, - kBzip2 = 12, - kLzma = 14, - kZstd = 93, - kXz = 95, - }; + static constexpr const char kStore[] = "store"; + static constexpr const char kDeflate[] = "deflate"; + static constexpr const char kBzip2[] = "bzip2"; + static constexpr const char kLzma[] = "lzma"; + static constexpr const char kZstd[] = "zstd"; + static constexpr const char kXz[] = "xz"; /// Some compression levels. enum class CompressionLevels @@ -53,7 +49,7 @@ class ZipArchiveWriter : public shared_ptr_helper, public IArc /// Sets compression method and level. /// Changing them will affect next file in the archive. - void setCompression(int compression_method_, int compression_level_) override; + void setCompression(const String & compression_method_, int compression_level_) override; /// Sets password. Only contents of the files are encrypted, /// names of files are not encrypted. @@ -61,8 +57,9 @@ class ZipArchiveWriter : public shared_ptr_helper, public IArc void setPassword(const String & password_) override; /// Utility functions. - static CompressionMethod parseCompressionMethod(const String & str); - static void checkCompressionMethodIsEnabled(CompressionMethod method); + static int compressionMethodToInt(const String & compression_method_); + static String intToCompressionMethod(int compression_method_); + static void checkCompressionMethodIsEnabled(int compression_method_); static void checkEncryptionIsEnabled(); private: @@ -85,7 +82,7 @@ class ZipArchiveWriter : public shared_ptr_helper, public IArc [[noreturn]] void showError(const String & message) const; const String path_to_archive; - int compression_method = static_cast(CompressionMethod::kDeflate); + int compression_method; /// By default the compression method is "deflate". int compression_level = kDefaultCompressionLevel; String password; RawHandle handle = nullptr; diff --git a/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp b/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp new file mode 100644 index 000000000000..6b2ef29d0547 --- /dev/null +++ b/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp @@ -0,0 +1,12 @@ +#include + + +namespace DB +{ + +bool hasRegisteredArchiveFileExtension(const String & path) +{ + return path.ends_with(".zip") || path.ends_with(".zipx"); +} + +} diff --git a/src/IO/Archives/hasRegisteredArchiveFileExtension.h b/src/IO/Archives/hasRegisteredArchiveFileExtension.h new file mode 100644 index 000000000000..cff2a0a8bec8 --- /dev/null +++ b/src/IO/Archives/hasRegisteredArchiveFileExtension.h @@ -0,0 +1,12 @@ +#pragma once + +#include + + +namespace DB +{ + +/// Returns true if a specified path has one of the registed file extensions for an archive. +bool hasRegisteredArchiveFileExtension(const String & path); + +} From f751c095d26f631339bdb9bac5d452b769fd51ee Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 29 Jan 2022 00:33:35 +0700 Subject: [PATCH 13/18] Implement writing backups as zip archive. --- src/Backups/ArchiveBackup.cpp | 105 +++++++++++ src/Backups/ArchiveBackup.h | 52 ++++++ src/Backups/BackupFactory.cpp | 11 +- src/Backups/BackupFactory.h | 3 + src/Backups/BackupImpl.cpp | 80 ++++++--- src/Backups/BackupImpl.h | 37 ++-- src/Backups/BackupInDirectory.cpp | 160 ----------------- src/Backups/BackupSettings.cpp | 11 +- src/Backups/BackupSettings.h | 8 + src/Backups/DirectoryBackup.cpp | 82 +++++++++ ...{BackupInDirectory.h => DirectoryBackup.h} | 11 +- src/Backups/IBackup.h | 16 +- src/Backups/RestoreSettings.cpp | 15 +- src/Backups/RestoreSettings.h | 4 + src/Backups/registerBackupEngines.cpp | 14 -- .../registerBackupEnginesFileAndDisk.cpp | 168 ++++++++++++++++++ src/Interpreters/InterpreterBackupQuery.cpp | 28 ++- .../test_backup_restore_new/test.py | 28 +++ 18 files changed, 584 insertions(+), 249 deletions(-) create mode 100644 src/Backups/ArchiveBackup.cpp create mode 100644 src/Backups/ArchiveBackup.h delete mode 100644 src/Backups/BackupInDirectory.cpp create mode 100644 src/Backups/DirectoryBackup.cpp rename src/Backups/{BackupInDirectory.h => DirectoryBackup.h} (80%) delete mode 100644 src/Backups/registerBackupEngines.cpp create mode 100644 src/Backups/registerBackupEnginesFileAndDisk.cpp diff --git a/src/Backups/ArchiveBackup.cpp b/src/Backups/ArchiveBackup.cpp new file mode 100644 index 000000000000..36148430f68b --- /dev/null +++ b/src/Backups/ArchiveBackup.cpp @@ -0,0 +1,105 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +ArchiveBackup::ArchiveBackup( + const String & backup_name_, + const DiskPtr & disk_, + const String & path_, + const ContextPtr & context_, + const std::optional & base_backup_info_) + : BackupImpl(backup_name_, context_, base_backup_info_), disk(disk_), path(path_) +{ +} + +ArchiveBackup::~ArchiveBackup() +{ + close(); +} + +bool ArchiveBackup::backupExists() const +{ + return disk ? disk->exists(path) : fs::exists(path); +} + +void ArchiveBackup::openImpl(OpenMode open_mode_) +{ + /// mutex is already locked + if (open_mode_ == OpenMode::WRITE) + { + if (disk) + writer = createArchiveWriter(path, disk->writeFile(path)); + else + writer = createArchiveWriter(path); + + writer->setCompression(compression_method, compression_level); + writer->setPassword(password); + } + else if (open_mode_ == OpenMode::READ) + { + if (disk) + { + auto archive_read_function = [d = disk, p = path]() -> std::unique_ptr { return d->readFile(p); }; + size_t archive_size = disk->getFileSize(path); + reader = createArchiveReader(path, archive_read_function, archive_size); + } + else + reader = createArchiveReader(path); + + reader->setPassword(password); + } +} + +void ArchiveBackup::closeImpl(bool writing_finalized_) +{ + /// mutex is already locked + if (writer && writer->isWritingFile()) + throw Exception("There is some writing unfinished on close", ErrorCodes::LOGICAL_ERROR); + + writer.reset(); + reader.reset(); + + if ((getOpenModeNoLock() == OpenMode::WRITE) && !writing_finalized_) + fs::remove(path); +} + +std::unique_ptr ArchiveBackup::readFileImpl(const String & file_name) const +{ + /// mutex is already locked + return reader->readFile(file_name); +} + +std::unique_ptr ArchiveBackup::addFileImpl(const String & file_name) +{ + /// mutex is already locked + return writer->writeFile(file_name); +} + +void ArchiveBackup::setCompression(const String & compression_method_, int compression_level_) +{ + std::lock_guard lock{mutex}; + compression_method = compression_method_; + compression_level = compression_level_; + if (writer) + writer->setCompression(compression_method, compression_level); +} + +void ArchiveBackup::setPassword(const String & password_) +{ + std::lock_guard lock{mutex}; + password = password_; + if (writer) + writer->setPassword(password); + if (reader) + reader->setPassword(password); +} + +} diff --git a/src/Backups/ArchiveBackup.h b/src/Backups/ArchiveBackup.h new file mode 100644 index 000000000000..8c76830977aa --- /dev/null +++ b/src/Backups/ArchiveBackup.h @@ -0,0 +1,52 @@ +#pragma once + +#include + + +namespace DB +{ +class IDisk; +using DiskPtr = std::shared_ptr; +class IArchiveReader; +class IArchiveWriter; + +/// Stores a backup as a single .zip file. +class ArchiveBackup : public BackupImpl +{ +public: + /// `disk`_ is allowed to be nullptr and that means the `path_` is a path in the local filesystem. + ArchiveBackup( + const String & backup_name_, + const DiskPtr & disk_, + const String & path_, + const ContextPtr & context_, + const std::optional & base_backup_info_ = {}); + + ~ArchiveBackup() override; + + static constexpr const int kDefaultCompressionLevel = -1; + + /// Sets compression method and level. + void setCompression(const String & compression_method_, int compression_level_ = kDefaultCompressionLevel); + + /// Sets password. + void setPassword(const String & password_); + +private: + bool backupExists() const override; + void openImpl(OpenMode open_mode_) override; + void closeImpl(bool writing_finalized_) override; + bool supportsWritingInMultipleThreads() const override { return false; } + std::unique_ptr readFileImpl(const String & file_name) const override; + std::unique_ptr addFileImpl(const String & file_name) override; + + const DiskPtr disk; + const String path; + std::shared_ptr reader; + std::shared_ptr writer; + String compression_method; + int compression_level = kDefaultCompressionLevel; + String password; +}; + +} diff --git a/src/Backups/BackupFactory.cpp b/src/Backups/BackupFactory.cpp index 490bfb002db1..d64c2bd0318b 100644 --- a/src/Backups/BackupFactory.cpp +++ b/src/Backups/BackupFactory.cpp @@ -21,7 +21,9 @@ BackupMutablePtr BackupFactory::createBackup(const CreateParams & params) const auto it = creators.find(engine_name); if (it == creators.end()) throw Exception(ErrorCodes::BACKUP_ENGINE_NOT_FOUND, "Not found backup engine {}", engine_name); - return (it->second)(params); + BackupMutablePtr backup = (it->second)(params); + backup->open(params.open_mode); + return backup; } void BackupFactory::registerBackupEngine(const String & engine_name, const CreatorFn & creator_fn) @@ -31,7 +33,12 @@ void BackupFactory::registerBackupEngine(const String & engine_name, const Creat creators[engine_name] = creator_fn; } -void registerBackupEngines(BackupFactory & factory); +void registerBackupEnginesFileAndDisk(BackupFactory &); + +void registerBackupEngines(BackupFactory & factory) +{ + registerBackupEnginesFileAndDisk(factory); +} BackupFactory::BackupFactory() { diff --git a/src/Backups/BackupFactory.h b/src/Backups/BackupFactory.h index 51d70c61f54f..d3ebcfe23695 100644 --- a/src/Backups/BackupFactory.h +++ b/src/Backups/BackupFactory.h @@ -26,6 +26,9 @@ class BackupFactory : boost::noncopyable OpenMode open_mode = OpenMode::WRITE; BackupInfo backup_info; std::optional base_backup_info; + String compression_method; + int compression_level = -1; + String password; ContextPtr context; }; diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 8615f6b610fd..71bc29b73de0 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -47,34 +47,44 @@ namespace } } -BackupImpl::BackupImpl(const String & backup_name_, OpenMode open_mode_, const ContextPtr & context_, const std::optional & base_backup_info_) - : backup_name(backup_name_), open_mode(open_mode_), context(context_), base_backup_info(base_backup_info_) +BackupImpl::BackupImpl(const String & backup_name_, const ContextPtr & context_, const std::optional & base_backup_info_) + : backup_name(backup_name_), context(context_), base_backup_info_param(base_backup_info_) { } BackupImpl::~BackupImpl() = default; -void BackupImpl::open() +void BackupImpl::open(OpenMode open_mode_) { - if (open_mode == OpenMode::WRITE) + std::lock_guard lock{mutex}; + if (open_mode == open_mode_) + return; + + if (open_mode != OpenMode::NONE) + throw Exception("Backup is already opened", ErrorCodes::LOGICAL_ERROR); + + if (open_mode_ == OpenMode::WRITE) { if (backupExists()) throw Exception(ErrorCodes::BACKUP_ALREADY_EXISTS, "Backup {} already exists", getName()); timestamp = std::time(nullptr); uuid = UUIDHelpers::generateV4(); - - startWriting(); - writing_started = true; + writing_finalized = false; } - if (open_mode == OpenMode::READ) + if (open_mode_ == OpenMode::READ) { if (!backupExists()) throw Exception(ErrorCodes::BACKUP_NOT_FOUND, "Backup {} not found", getName()); - readBackupMetadata(); } + openImpl(open_mode_); + + base_backup_info = base_backup_info_param; + if (open_mode_ == OpenMode::READ) + readBackupMetadata(); + if (base_backup_info) { BackupFactory::CreateParams params; @@ -83,25 +93,43 @@ void BackupImpl::open() params.context = context; base_backup = BackupFactory::instance().createBackup(params); - if (open_mode == OpenMode::WRITE) + if (open_mode_ == OpenMode::WRITE) base_backup_uuid = base_backup->getUUID(); else if (base_backup_uuid != base_backup->getUUID()) throw Exception(ErrorCodes::WRONG_BASE_BACKUP, "Backup {}: The base backup {} has different UUID ({} != {})", getName(), base_backup->getName(), toString(base_backup->getUUID()), (base_backup_uuid ? toString(*base_backup_uuid) : "")); } + + open_mode = open_mode_; } void BackupImpl::close() { - if (open_mode == OpenMode::WRITE) - { - if (writing_started && !writing_finalized) - { - /// Creating of the backup wasn't finished correctly, - /// so the backup cannot be used and it's better to remove its files. - removeAllFilesAfterFailure(); - } - } + std::lock_guard lock{mutex}; + if (open_mode == OpenMode::NONE) + return; + + closeImpl(writing_finalized); + + uuid = UUIDHelpers::Nil; + timestamp = 0; + base_backup_info.reset(); + base_backup.reset(); + base_backup_uuid.reset(); + file_infos.clear(); + open_mode = OpenMode::NONE; +} + +IBackup::OpenMode BackupImpl::getOpenMode() const +{ + std::lock_guard lock{mutex}; + return open_mode; +} + +time_t BackupImpl::getTimestamp() const +{ + std::lock_guard lock{mutex}; + return timestamp; } void BackupImpl::writeBackupMetadata() @@ -244,6 +272,9 @@ UInt128 BackupImpl::getFileChecksum(const String & file_name) const BackupEntryPtr BackupImpl::readFile(const String & file_name) const { std::lock_guard lock{mutex}; + if (open_mode != OpenMode::READ) + throw Exception("Backup is not opened for reading", ErrorCodes::LOGICAL_ERROR); + auto it = file_infos.find(file_name); if (it == file_infos.end()) throw Exception( @@ -329,7 +360,7 @@ void BackupImpl::addFile(const String & file_name, BackupEntryPtr entry) { std::lock_guard lock{mutex}; if (open_mode != OpenMode::WRITE) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal operation: Cannot write to a backup opened for reading"); + throw Exception("Backup is not opened for writing", ErrorCodes::LOGICAL_ERROR); if (file_infos.contains(file_name)) throw Exception( @@ -467,8 +498,13 @@ void BackupImpl::addFile(const String & file_name, BackupEntryPtr entry) void BackupImpl::finalizeWriting() { + std::lock_guard lock{mutex}; + if (writing_finalized) + return; + if (open_mode != OpenMode::WRITE) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal operation: Cannot write to a backup opened for reading"); + throw Exception("Backup is not opened for writing", ErrorCodes::LOGICAL_ERROR); + writeBackupMetadata(); writing_finalized = true; } diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index 5a0269cd6623..ac2771cfc6c2 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -20,14 +20,15 @@ class BackupImpl : public IBackup public: BackupImpl( const String & backup_name_, - OpenMode open_mode_, const ContextPtr & context_, const std::optional & base_backup_info_ = {}); ~BackupImpl() override; const String & getName() const override { return backup_name; } - OpenMode getOpenMode() const override { return open_mode; } - time_t getTimestamp() const override { return timestamp; } + void open(OpenMode open_mode_) override; + OpenMode getOpenMode() const override; + void close() override; + time_t getTimestamp() const override; UUID getUUID() const override { return uuid; } Strings listFiles(const String & prefix, const String & terminator) const override; bool fileExists(const String & file_name) const override; @@ -38,11 +39,13 @@ class BackupImpl : public IBackup void finalizeWriting() override; protected: - /// Should be called in the constructor of a derived class. - void open(); + /// Checks if this backup exists. + virtual bool backupExists() const = 0; + + virtual void openImpl(OpenMode open_mode_) = 0; + OpenMode getOpenModeNoLock() const { return open_mode; } - /// Should be called in the destructor of a derived class. - void close(); + virtual void closeImpl(bool writing_finalized_) = 0; /// Read a file from the backup. /// Low level: the function doesn't check base backup or checksums. @@ -52,16 +55,7 @@ class BackupImpl : public IBackup /// Low level: the function doesn't check base backup or checksums. virtual std::unique_ptr addFileImpl(const String & file_name) = 0; - /// Checks if this backup exists. - virtual bool backupExists() const = 0; - - /// Starts writing of this backup, only used if `open_mode == OpenMode::WRITE`. - /// After calling this function `backupExists()` should return true. - virtual void startWriting() = 0; - - /// Removes all the backup files, called if something goes wrong while we're writing the backup. - /// This function is called by `close()` if `startWriting()` was called and `finalizeWriting()` wasn't. - virtual void removeAllFilesAfterFailure() = 0; + mutable std::mutex mutex; private: void writeBackupMetadata(); @@ -78,17 +72,16 @@ class BackupImpl : public IBackup }; const String backup_name; - const OpenMode open_mode; - UUID uuid; - time_t timestamp = 0; ContextPtr context; + const std::optional base_backup_info_param; + OpenMode open_mode = OpenMode::NONE; + UUID uuid = {}; + time_t timestamp = 0; std::optional base_backup_info; std::shared_ptr base_backup; std::optional base_backup_uuid; std::map file_infos; - bool writing_started = false; bool writing_finalized = false; - mutable std::mutex mutex; }; } diff --git a/src/Backups/BackupInDirectory.cpp b/src/Backups/BackupInDirectory.cpp deleted file mode 100644 index f071a21618cd..000000000000 --- a/src/Backups/BackupInDirectory.cpp +++ /dev/null @@ -1,160 +0,0 @@ -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} - -namespace -{ - /// Checks multiple keys "key", "key[1]", "key[2]", and so on in the configuration - /// and find out if some of them have matching value. - bool findConfigKeyWithMatchingValue(const Poco::Util::AbstractConfiguration & config, const String & key, const std::function & match_function) - { - String current_key = key; - size_t counter = 0; - while (config.has(current_key)) - { - if (match_function(config.getString(current_key))) - return true; - current_key = key + "[" + std::to_string(++counter) + "]"; - } - return false; - } - - bool isDiskAllowed(const String & disk_name, const Poco::Util::AbstractConfiguration & config) - { - return findConfigKeyWithMatchingValue(config, "backups.allowed_disk", [&](const String & value) { return value == disk_name; }); - } - - bool isPathAllowed(const String & path, const Poco::Util::AbstractConfiguration & config) - { - return findConfigKeyWithMatchingValue(config, "backups.allowed_path", [&](const String & value) { return path.starts_with(value); }); - } -} - - -BackupInDirectory::BackupInDirectory( - const String & backup_name_, - OpenMode open_mode_, - const DiskPtr & disk_, - const String & path_, - const ContextPtr & context_, - const std::optional & base_backup_info_) - : BackupImpl(backup_name_, open_mode_, context_, base_backup_info_) - , disk(disk_), path(path_) -{ - /// Path to backup must end with '/' - if (path.back() != '/') - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Backup {}: Path to backup must end with '/', but {} doesn't.", getName(), quoteString(path)); - dir_path = fs::path(path).parent_path(); /// get path without terminating slash - - /// If `disk` is not specified, we create an internal instance of `DiskLocal` here. - if (!disk) - { - auto fspath = fs::path{dir_path}; - if (!fspath.has_filename()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Backup {}: Path to a backup must be a directory path.", getName(), quoteString(path)); - path = fspath.filename() / ""; - dir_path = fs::path(path).parent_path(); /// get path without terminating slash - String disk_path = fspath.remove_filename(); - disk = std::make_shared(disk_path, disk_path, 0); - } - - open(); -} - - -BackupInDirectory::~BackupInDirectory() -{ - close(); -} - -bool BackupInDirectory::backupExists() const -{ - return disk->isDirectory(dir_path); -} - -void BackupInDirectory::startWriting() -{ - disk->createDirectories(dir_path); -} - -void BackupInDirectory::removeAllFilesAfterFailure() -{ - if (disk->isDirectory(dir_path)) - disk->removeRecursive(dir_path); -} - -std::unique_ptr BackupInDirectory::readFileImpl(const String & file_name) const -{ - String file_path = path + file_name; - return disk->readFile(file_path); -} - -std::unique_ptr BackupInDirectory::addFileImpl(const String & file_name) -{ - String file_path = path + file_name; - disk->createDirectories(fs::path(file_path).parent_path()); - return disk->writeFile(file_path); -} - - -void registerBackupEngineFile(BackupFactory & factory) -{ - auto creator_fn = [](const BackupFactory::CreateParams & params) - { - String backup_name = params.backup_info.toString(); - const String & engine_name = params.backup_info.backup_engine_name; - const auto & args = params.backup_info.args; - - DiskPtr disk; - String path; - if (engine_name == "File") - { - if (args.size() != 1) - { - throw Exception( - "Backup engine 'File' requires 1 argument (path)", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - } - - path = args[0].safeGet(); - - if (!isPathAllowed(path, params.context->getConfigRef())) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path {} is not allowed for backups", path); - } - else if (engine_name == "Disk") - { - if (args.size() != 2) - { - throw Exception( - "Backup engine 'Disk' requires 2 arguments (disk_name, path)", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - } - - String disk_name = args[0].safeGet(); - disk = params.context->getDisk(disk_name); - path = args[1].safeGet(); - - if (!isDiskAllowed(disk_name, params.context->getConfigRef())) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk {} is not allowed for backups", disk_name); - } - - return std::make_shared(backup_name, params.open_mode, disk, path, params.context, params.base_backup_info); - }; - - factory.registerBackupEngine("File", creator_fn); - factory.registerBackupEngine("Disk", creator_fn); -} - -} diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 602cf67cc2b7..2189ac11a282 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -23,8 +24,14 @@ BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) const auto & settings = query.settings->as().changes; for (const auto & setting : settings) { - if (setting.name == "structure_only") - res.structure_only = setting.value.safeGet(); + if (setting.name == "compression_method") + res.compression_method = SettingFieldString{setting.value}; + else if (setting.name == "compression_level") + res.compression_level = SettingFieldInt64{setting.value}; + else if (setting.name == "password") + res.password = SettingFieldString{setting.value}; + else if (setting.name == "structure_only") + res.structure_only = SettingFieldBool{setting.value}; else throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting {}", setting.name); } diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index ce911382c782..f46e65353fb1 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -1,5 +1,6 @@ #pragma once +#include #include @@ -14,6 +15,13 @@ struct BackupSettings /// Base backup, if it's set an incremental backup will be built. std::shared_ptr base_backup_info; + /// Compression method and level for writing the backup (when applicable). + String compression_method; /// "" means default method + int compression_level = -1; /// -1 means default level + + /// Password used to encrypt the backup. + String password; + /// If this is set to true then only create queries will be written to backup, /// without the data of tables. bool structure_only = false; diff --git a/src/Backups/DirectoryBackup.cpp b/src/Backups/DirectoryBackup.cpp new file mode 100644 index 000000000000..2adee1601802 --- /dev/null +++ b/src/Backups/DirectoryBackup.cpp @@ -0,0 +1,82 @@ +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + + +DirectoryBackup::DirectoryBackup( + const String & backup_name_, + const DiskPtr & disk_, + const String & path_, + const ContextPtr & context_, + const std::optional & base_backup_info_) + : BackupImpl(backup_name_, context_, base_backup_info_) + , disk(disk_), path(path_) +{ + /// Path to backup must end with '/' + if (!path.ends_with("/")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Backup {}: Path to backup must end with '/', but {} doesn't.", getName(), quoteString(path)); + dir_path = fs::path(path).parent_path(); /// get path without terminating slash + + /// If `disk` is not specified, we create an internal instance of `DiskLocal` here. + if (!disk) + { + auto fspath = fs::path{dir_path}; + if (!fspath.has_filename()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Backup {}: Path to a backup must be a directory path.", getName(), quoteString(path)); + path = fspath.filename() / ""; + dir_path = fs::path(path).parent_path(); /// get path without terminating slash + String disk_path = fspath.remove_filename(); + disk = std::make_shared(disk_path, disk_path, 0); + } +} + + +DirectoryBackup::~DirectoryBackup() +{ + close(); +} + +bool DirectoryBackup::backupExists() const +{ + return disk->isDirectory(dir_path); +} + +void DirectoryBackup::openImpl(OpenMode open_mode_) +{ + if (open_mode_ == OpenMode::WRITE) + disk->createDirectories(dir_path); +} + +void DirectoryBackup::closeImpl(bool writing_finalized_) +{ + if ((getOpenModeNoLock() == OpenMode::WRITE) && !writing_finalized_ && disk->isDirectory(dir_path)) + { + /// Creating of the backup wasn't finished correctly, + /// so the backup cannot be used and it's better to remove its files. + disk->removeRecursive(dir_path); + } +} + +std::unique_ptr DirectoryBackup::readFileImpl(const String & file_name) const +{ + String file_path = path + file_name; + return disk->readFile(file_path); +} + +std::unique_ptr DirectoryBackup::addFileImpl(const String & file_name) +{ + String file_path = path + file_name; + disk->createDirectories(fs::path(file_path).parent_path()); + return disk->writeFile(file_path); +} + +} diff --git a/src/Backups/BackupInDirectory.h b/src/Backups/DirectoryBackup.h similarity index 80% rename from src/Backups/BackupInDirectory.h rename to src/Backups/DirectoryBackup.h index dd6a39d019a7..0b6dd1361d1d 100644 --- a/src/Backups/BackupInDirectory.h +++ b/src/Backups/DirectoryBackup.h @@ -10,23 +10,22 @@ using DiskPtr = std::shared_ptr; /// Represents a backup stored on a disk. /// A backup is stored as a directory, each entry is stored as a file in that directory. -class BackupInDirectory : public BackupImpl +class DirectoryBackup : public BackupImpl { public: /// `disk`_ is allowed to be nullptr and that means the `path_` is a path in the local filesystem. - BackupInDirectory( + DirectoryBackup( const String & backup_name_, - OpenMode open_mode_, const DiskPtr & disk_, const String & path_, const ContextPtr & context_, const std::optional & base_backup_info_ = {}); - ~BackupInDirectory() override; + ~DirectoryBackup() override; private: bool backupExists() const override; - void startWriting() override; - void removeAllFilesAfterFailure() override; + void openImpl(OpenMode open_mode_) override; + void closeImpl(bool writing_finalized_) override; std::unique_ptr readFileImpl(const String & file_name) const override; std::unique_ptr addFileImpl(const String & file_name) override; diff --git a/src/Backups/IBackup.h b/src/Backups/IBackup.h index 2dbd0b6320b7..7759879ecfe1 100644 --- a/src/Backups/IBackup.h +++ b/src/Backups/IBackup.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include @@ -13,7 +12,7 @@ using BackupEntryPtr = std::unique_ptr; /// Represents a backup, i.e. a storage of BackupEntries which can be accessed by their names. /// A backup can be either incremental or non-incremental. An incremental backup doesn't store /// the data of the entries which are not changed compared to its base backup. -class IBackup : public std::enable_shared_from_this, public TypePromotion +class IBackup : public std::enable_shared_from_this { public: IBackup() {} @@ -24,13 +23,18 @@ class IBackup : public std::enable_shared_from_this, public TypePromoti enum class OpenMode { + NONE, READ, WRITE, }; - /// A backup can be open either in READ or WRITE mode. + /// Opens the backup and start its reading or writing depending on `open_mode`. + virtual void open(OpenMode open_mode) = 0; virtual OpenMode getOpenMode() const = 0; + /// Closes the backup and ends its reading or writing. + virtual void close() = 0; + /// Returns the time point when this backup was created. virtual time_t getTimestamp() const = 0; @@ -63,11 +67,11 @@ class IBackup : public std::enable_shared_from_this, public TypePromoti /// Puts a new entry to the backup. virtual void addFile(const String & file_name, BackupEntryPtr entry) = 0; - /// Whether it's possible to add new entries to the backup in multiple threads. - virtual bool supportsWritingInMultipleThreads() const { return true; } - /// 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 { return true; } }; using BackupPtr = std::shared_ptr; diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 6d96c6f22ebb..546f3007df58 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -23,16 +24,18 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) const auto & settings = query.settings->as().changes; for (const auto & setting : settings) { - if (setting.name == "structure_only") - res.structure_only = setting.value.safeGet(); + if (setting.name == "password") + res.password = SettingFieldString{setting.value}; + else if (setting.name == "structure_only") + res.structure_only = SettingFieldBool{setting.value}; else if (setting.name == "throw_if_database_exists") - res.throw_if_database_exists = setting.value.safeGet(); + res.throw_if_database_exists = SettingFieldBool{setting.value}; else if (setting.name == "throw_if_table_exists") - res.throw_if_table_exists = setting.value.safeGet(); + res.throw_if_table_exists = SettingFieldBool{setting.value}; else if (setting.name == "throw_if_database_has_different_definition") - res.throw_if_database_has_different_definition = setting.value.safeGet(); + res.throw_if_database_has_different_definition = SettingFieldBool{setting.value}; else if (setting.name == "throw_if_table_has_different_definition") - res.throw_if_table_has_different_definition = setting.value.safeGet(); + res.throw_if_table_has_different_definition = SettingFieldBool{setting.value}; else throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting {}", setting.name); } diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index ad046dfa1cfb..dcb55973d85b 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -1,5 +1,6 @@ #pragma once +#include #include @@ -20,6 +21,9 @@ struct RestoreSettings : public StorageRestoreSettings /// so using this setting is optional. std::shared_ptr base_backup_info; + /// Password used to decrypt the backup. + String password; + /// If this is set to true then only create queries will be read from backup, /// without the data of tables. bool structure_only = false; diff --git a/src/Backups/registerBackupEngines.cpp b/src/Backups/registerBackupEngines.cpp deleted file mode 100644 index 33bc2c8fdd93..000000000000 --- a/src/Backups/registerBackupEngines.cpp +++ /dev/null @@ -1,14 +0,0 @@ - - -namespace DB -{ -class BackupFactory; - -void registerBackupEngineFile(BackupFactory &); - -void registerBackupEngines(BackupFactory & factory) -{ - registerBackupEngineFile(factory); -} - -} diff --git a/src/Backups/registerBackupEnginesFileAndDisk.cpp b/src/Backups/registerBackupEnginesFileAndDisk.cpp new file mode 100644 index 000000000000..6a34d67115dc --- /dev/null +++ b/src/Backups/registerBackupEnginesFileAndDisk.cpp @@ -0,0 +1,168 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int INVALID_CONFIG_PARAMETER; +} + + +namespace +{ + namespace fs = std::filesystem; + + [[noreturn]] void throwDiskIsAllowed(const String & disk_name) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk {} is not allowed for backups", disk_name); + } + + [[noreturn]] void throwPathNotAllowed(const fs::path & path) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path {} is not allowed for backups", quoteString(String{path})); + } + + void checkAllowedPathInConfigIsValid(const String & key, const fs::path & value) + { + if (value.empty() || value.is_relative()) + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Configuration parameter {} has a wrong value {}", key, String{value}); + } + + /// Checks that a disk name and a path specified as parameters of Disk() are valid. + void checkDiskNameAndPath(const String & disk_name, fs::path & path, const Poco::Util::AbstractConfiguration & config) + { + String key = "backups.allowed_disk"; + bool disk_name_found = false; + size_t counter = 0; + while (config.has(key)) + { + if (config.getString(key) == disk_name) + { + disk_name_found = true; + break; + } + key = "backups.allowed_disk[" + std::to_string(++counter) + "]"; + } + + if (!disk_name_found) + throwDiskIsAllowed(disk_name); + + path = path.lexically_normal(); + if (!path.is_relative() || path.empty() || (*path.begin() == "..")) + throwPathNotAllowed(path); + } + + /// Checks that a path specified as a parameter of File() is valid. + void checkPath(fs::path & path, const Poco::Util::AbstractConfiguration & config) + { + String key = "backups.allowed_path"; + + path = path.lexically_normal(); + if (path.empty()) + throwPathNotAllowed(path); + + if (path.is_relative()) + { + if (*path.begin() == "..") + throwPathNotAllowed(path); + + auto base = fs::path(config.getString(key, "")); + checkAllowedPathInConfigIsValid(key, base); + path = base / path; + return; + } + + bool path_found_in_config = false; + size_t counter = 0; + while (config.has(key)) + { + auto base = fs::path(config.getString(key)); + checkAllowedPathInConfigIsValid(key, base); + auto rel = path.lexically_relative(base); + if (!rel.empty() && (*rel.begin() != "..")) + { + path_found_in_config = true; + break; + } + key = "backups.allowed_path[" + std::to_string(++counter) + "]"; + } + + if (!path_found_in_config) + throwPathNotAllowed(path); + } +} + + +void registerBackupEnginesFileAndDisk(BackupFactory & factory) +{ + auto creator_fn = [](const BackupFactory::CreateParams & params) -> std::unique_ptr + { + String backup_name = params.backup_info.toString(); + const String & engine_name = params.backup_info.backup_engine_name; + const auto & args = params.backup_info.args; + + DiskPtr disk; + fs::path path; + if (engine_name == "File") + { + if (args.size() != 1) + { + throw Exception( + "Backup engine 'File' requires 1 argument (path)", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + path = args[0].safeGet(); + checkPath(path, params.context->getConfigRef()); + } + else if (engine_name == "Disk") + { + if (args.size() != 2) + { + throw Exception( + "Backup engine 'Disk' requires 2 arguments (disk_name, path)", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + String disk_name = args[0].safeGet(); + path = args[1].safeGet(); + checkDiskNameAndPath(disk_name, path, params.context->getConfigRef()); + disk = params.context->getDisk(disk_name); + } + + std::unique_ptr backup; + + if (!path.has_filename() && !path.empty()) + { + if (!params.password.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Password is not applicable, backup cannot be encrypted"); + backup = std::make_unique(backup_name, disk, path, params.context, params.base_backup_info); + } + else if (hasRegisteredArchiveFileExtension(path)) + { + auto archive_backup = std::make_unique(backup_name, disk, path, params.context, params.base_backup_info); + archive_backup->setCompression(params.compression_method, params.compression_level); + archive_backup->setPassword(params.password); + backup = std::move(archive_backup); + } + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path to backup must be either a directory or a path to an archive"); + + return backup; + }; + + factory.registerBackupEngine("File", creator_fn); + factory.registerBackupEngine("Disk", creator_fn); +} + +} diff --git a/src/Interpreters/InterpreterBackupQuery.cpp b/src/Interpreters/InterpreterBackupQuery.cpp index ac930e1f6a72..390399e04708 100644 --- a/src/Interpreters/InterpreterBackupQuery.cpp +++ b/src/Interpreters/InterpreterBackupQuery.cpp @@ -14,24 +14,34 @@ namespace DB { namespace { - BackupMutablePtr createBackup( - const BackupInfo & backup_info, - IBackup::OpenMode open_mode, - const BackupInfo * base_backup_info, - const ContextPtr & context) + BackupMutablePtr createBackup(const BackupInfo & backup_info, const BackupSettings & backup_settings, const ContextPtr & context) { BackupFactory::CreateParams params; - params.open_mode = open_mode; + params.open_mode = IBackup::OpenMode::WRITE; params.context = context; params.backup_info = backup_info; - params.base_backup_info = base_backup_info ? *base_backup_info : std::optional{}; + params.base_backup_info = backup_settings.base_backup_info ? *backup_settings.base_backup_info : std::optional{}; + params.compression_method = backup_settings.compression_method; + params.compression_level = backup_settings.compression_level; + params.password = backup_settings.password; + return BackupFactory::instance().createBackup(params); + } + + BackupMutablePtr openBackup(const BackupInfo & backup_info, const RestoreSettings & restore_settings, const ContextPtr & context) + { + BackupFactory::CreateParams params; + params.open_mode = IBackup::OpenMode::READ; + params.context = context; + params.backup_info = backup_info; + params.base_backup_info = restore_settings.base_backup_info ? *restore_settings.base_backup_info : std::optional{}; + params.password = restore_settings.password; return BackupFactory::instance().createBackup(params); } void executeBackup(const ContextPtr & context, const ASTBackupQuery & query) { auto backup_settings = BackupSettings::fromBackupQuery(query); - BackupMutablePtr backup = createBackup(BackupInfo::fromAST(*query.backup_name), IBackup::OpenMode::WRITE, backup_settings.base_backup_info.get(), context); + BackupMutablePtr backup = createBackup(BackupInfo::fromAST(*query.backup_name), backup_settings, context); auto backup_entries = makeBackupEntries(context, query.elements, backup_settings); writeBackupEntries(backup, std::move(backup_entries), context->getSettingsRef().max_backup_threads); } @@ -39,7 +49,7 @@ namespace void executeRestore(ContextMutablePtr context, const ASTBackupQuery & query) { auto restore_settings = RestoreSettings::fromRestoreQuery(query); - BackupPtr backup = createBackup(BackupInfo::fromAST(*query.backup_name), IBackup::OpenMode::READ, restore_settings.base_backup_info.get(), context); + BackupPtr backup = openBackup(BackupInfo::fromAST(*query.backup_name), restore_settings, context); auto restore_tasks = makeRestoreTasks(context, backup, query.elements, restore_settings); executeRestoreTasks(std::move(restore_tasks), context->getSettingsRef().max_backup_threads); } diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index ad1190baac06..dcd6c241801b 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -147,3 +147,31 @@ def test_database(): instance.query(f"RESTORE DATABASE test FROM {backup_name}") assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + + +def test_zip_archive(): + backup_name = f"File('/backups/archive.zip')" + create_and_fill_table() + + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + instance.query(f"BACKUP TABLE test.table TO {backup_name}") + + instance.query("DROP TABLE test.table") + assert instance.query("EXISTS test.table") == "0\n" + + instance.query(f"RESTORE TABLE test.table FROM {backup_name}") + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + + +def test_zip_archive_with_settings(): + backup_name = f"File('/backups/archive_with_settings.zip')" + create_and_fill_table() + + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + instance.query(f"BACKUP TABLE test.table TO {backup_name} SETTINGS compression_method='lzma', compression_level=3, password='qwerty'") + + instance.query("DROP TABLE test.table") + assert instance.query("EXISTS test.table") == "0\n" + + instance.query(f"RESTORE TABLE test.table FROM {backup_name} SETTINGS password='qwerty'") + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" From 87576e603b935a968da507384204d44ff1df85ed Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 31 Jan 2022 13:35:07 +0700 Subject: [PATCH 14/18] Use BackupEntryFromAppendOnlyFile for Log engine, and use a BackupEntriesBatch for Memort engine to improve performance. A lot of minor corrections. --- src/Backups/BackupEntryConcat.cpp | 28 ---- src/Backups/BackupEntryConcat.h | 30 ---- src/Backups/BackupEntryFromAppendOnlyFile.cpp | 2 +- src/Backups/BackupEntryFromCallback.h | 31 ---- src/Backups/BackupImpl.cpp | 53 +++++-- src/Backups/BackupImpl.h | 2 + src/Backups/BackupSettings.cpp | 2 +- src/Backups/BackupSettings.h | 7 +- src/Backups/IBackupEntriesBatch.cpp | 37 +++++ src/Backups/IBackupEntriesBatch.h | 29 ++++ src/Backups/RestoreSettings.cpp | 10 +- src/Backups/RestoreSettings.h | 19 ++- src/Backups/RestoreUtils.cpp | 31 ++-- src/IO/ConcatReadBuffer.h | 6 + src/Interpreters/InterpreterBackupQuery.cpp | 4 +- src/Storages/StorageLog.cpp | 18 +-- src/Storages/StorageMemory.cpp | 143 +++++++++++------- src/Storages/StorageStripeLog.cpp | 18 +-- 18 files changed, 266 insertions(+), 204 deletions(-) delete mode 100644 src/Backups/BackupEntryConcat.cpp delete mode 100644 src/Backups/BackupEntryConcat.h delete mode 100644 src/Backups/BackupEntryFromCallback.h create mode 100644 src/Backups/IBackupEntriesBatch.cpp create mode 100644 src/Backups/IBackupEntriesBatch.h diff --git a/src/Backups/BackupEntryConcat.cpp b/src/Backups/BackupEntryConcat.cpp deleted file mode 100644 index 1075b8be2eca..000000000000 --- a/src/Backups/BackupEntryConcat.cpp +++ /dev/null @@ -1,28 +0,0 @@ -#include -#include - - -namespace DB -{ -BackupEntryConcat::BackupEntryConcat( - BackupEntryPtr first_source_, - BackupEntryPtr second_source_, - const std::optional & checksum_) - : first_source(std::move(first_source_)) - , second_source(std::move(second_source_)) - , checksum(checksum_) -{ -} - -UInt64 BackupEntryConcat::getSize() const -{ - if (!size) - size = first_source->getSize() + second_source->getSize(); - return *size; -} - -std::unique_ptr BackupEntryConcat::getReadBuffer() const -{ - return std::make_unique(*first_source->getReadBuffer(), *second_source->getReadBuffer()); -} -} diff --git a/src/Backups/BackupEntryConcat.h b/src/Backups/BackupEntryConcat.h deleted file mode 100644 index cb38fb9b1635..000000000000 --- a/src/Backups/BackupEntryConcat.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -/// Concatenates data of two backup entries. -class BackupEntryConcat : public IBackupEntry -{ -public: - /// The constructor is allowed to not set `checksum_`, in that case it will be calculated from the data. - BackupEntryConcat( - BackupEntryPtr first_source_, - BackupEntryPtr second_source_, - const std::optional & checksum_ = {}); - - UInt64 getSize() const override; - std::optional getChecksum() const override { return checksum; } - std::unique_ptr getReadBuffer() const override; - -private: - BackupEntryPtr first_source; - BackupEntryPtr second_source; - mutable std::optional size; - std::optional checksum; -}; - -} diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.cpp b/src/Backups/BackupEntryFromAppendOnlyFile.cpp index 32d5713952f4..d7f9d5624c8e 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.cpp +++ b/src/Backups/BackupEntryFromAppendOnlyFile.cpp @@ -29,7 +29,7 @@ BackupEntryFromAppendOnlyFile::BackupEntryFromAppendOnlyFile( std::unique_ptr BackupEntryFromAppendOnlyFile::getReadBuffer() const { auto buf = BackupEntryFromImmutableFile::getReadBuffer(); - return std::make_unique(std::move(buf), limit, true); + return std::make_unique(std::move(buf), limit, false); } } diff --git a/src/Backups/BackupEntryFromCallback.h b/src/Backups/BackupEntryFromCallback.h deleted file mode 100644 index e8df2d99f7c6..000000000000 --- a/src/Backups/BackupEntryFromCallback.h +++ /dev/null @@ -1,31 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -/// Represents small preloaded data to be included in a backup. -class BackupEntryFromCallback : public IBackupEntry -{ -public: - using ReadBufferCreator = std::function()>; - - /// The constructor is allowed to not set `checksum_`, in that case it will be calculated from the data. - BackupEntryFromCallback(const ReadBufferCreator & callback_, size_t size_, const std::optional & checksum_ = {}) - : callback(callback_), size(size_), checksum(checksum_) - { - } - - UInt64 getSize() const override { return size; } - std::optional getChecksum() const override { return checksum; } - std::unique_ptr getReadBuffer() const override { return callback(); } - -private: - const ReadBufferCreator callback; - const size_t size; - const std::optional checksum; -}; - -} diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 71bc29b73de0..71eea2223e8c 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -1,13 +1,11 @@ #include #include -#include -#include #include #include #include #include -#include #include +#include #include #include #include @@ -47,6 +45,44 @@ namespace } } + +class BackupImpl::BackupEntryFromBackupImpl : public IBackupEntry +{ +public: + BackupEntryFromBackupImpl( + const std::shared_ptr & backup_, + const String & file_name_, + UInt64 size_, + const std::optional checksum_, + BackupEntryPtr base_backup_entry_ = {}) + : backup(backup_), file_name(file_name_), size(size_), checksum(checksum_), + base_backup_entry(std::move(base_backup_entry_)) + { + } + + std::unique_ptr getReadBuffer() const override + { + auto read_buffer = backup->readFileImpl(file_name); + if (base_backup_entry) + { + auto base_backup_read_buffer = base_backup_entry->getReadBuffer(); + read_buffer = std::make_unique(std::move(base_backup_read_buffer), std::move(read_buffer)); + } + return read_buffer; + } + + UInt64 getSize() const override { return size; } + std::optional getChecksum() const override { return checksum; } + +private: + const std::shared_ptr backup; + const String file_name; + const UInt64 size; + const std::optional checksum; + BackupEntryPtr base_backup_entry; +}; + + BackupImpl::BackupImpl(const String & backup_name_, const ContextPtr & context_, const std::optional & base_backup_info_) : backup_name(backup_name_), context(context_), base_backup_info_param(base_backup_info_) { @@ -295,7 +331,8 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const if (!info.base_size) { /// Data goes completely from this backup, the base backup isn't used. - return std::make_unique(read_callback, info.size, info.checksum); + return std::make_unique( + std::static_pointer_cast(shared_from_this()), file_name, info.size, info.checksum); } if (info.size < info.base_size) @@ -349,10 +386,8 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const /// The beginning of the data goes from the base backup, /// and the ending goes from this backup. - return std::make_unique( - std::move(base_entry), - std::make_unique(read_callback, info.size - info.base_size), - info.checksum); + return std::make_unique( + static_pointer_cast(shared_from_this()), file_name, info.size, info.checksum, std::move(base_entry)); } @@ -406,7 +441,7 @@ void BackupImpl::addFile(const String & file_name, BackupEntryPtr entry) checksum = hashing_read_buffer.getHash(); } if (checksum == base_checksum) - use_base = true; /// The data has not been changed. + use_base = true; /// The data have not been changed. } else if (size > base_size) { diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index ac2771cfc6c2..af31b71f4601 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -71,6 +71,8 @@ class BackupImpl : public IBackup UInt128 base_checksum{0, 0}; }; + class BackupEntryFromBackupImpl; + const String backup_name; ContextPtr context; const std::optional base_backup_info_param; diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 2189ac11a282..817e0e7e1a7d 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -17,7 +17,7 @@ BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) BackupSettings res; if (query.base_backup_name) - res.base_backup_info = std::make_shared(BackupInfo::fromAST(*query.base_backup_name)); + res.base_backup_info = BackupInfo::fromAST(*query.base_backup_name); if (query.settings) { diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index f46e65353fb1..ca95a08da8f8 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -1,19 +1,18 @@ #pragma once -#include -#include +#include +#include namespace DB { class ASTBackupQuery; -struct BackupInfo; /// Settings specified in the "SETTINGS" clause of a BACKUP query. struct BackupSettings { /// Base backup, if it's set an incremental backup will be built. - std::shared_ptr base_backup_info; + std::optional base_backup_info; /// Compression method and level for writing the backup (when applicable). String compression_method; /// "" means default method diff --git a/src/Backups/IBackupEntriesBatch.cpp b/src/Backups/IBackupEntriesBatch.cpp new file mode 100644 index 000000000000..bf6bc6cce83a --- /dev/null +++ b/src/Backups/IBackupEntriesBatch.cpp @@ -0,0 +1,37 @@ +#include +#include + + +namespace DB +{ + +class IBackupEntriesBatch::BackupEntryFromBatch : public IBackupEntry +{ +public: + BackupEntryFromBatch(const std::shared_ptr & generator_, size_t index_) : batch(generator_), index(index_) + { + assert(batch); + } + + UInt64 getSize() const override { return batch->getSize(index); } + std::optional getChecksum() const override { return batch->getChecksum(index); } + std::unique_ptr getReadBuffer() const override { return batch->getReadBuffer(index); } + +private: + const std::shared_ptr batch; + const size_t index; +}; + + +BackupEntries IBackupEntriesBatch::getBackupEntries() +{ + BackupEntries res; + res.reserve(entry_names.size()); + for (size_t i = 0; i != entry_names.size(); ++i) + { + res.emplace_back(entry_names[i], std::make_unique(shared_from_this(), i)); + } + return res; +} + +} diff --git a/src/Backups/IBackupEntriesBatch.h b/src/Backups/IBackupEntriesBatch.h new file mode 100644 index 000000000000..0d8c8d5aa265 --- /dev/null +++ b/src/Backups/IBackupEntriesBatch.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/// Helper class designed to generate multiple backup entries from one source. +class IBackupEntriesBatch : public std::enable_shared_from_this +{ +public: + BackupEntries getBackupEntries(); + + virtual ~IBackupEntriesBatch() = default; + +protected: + IBackupEntriesBatch(const Strings & entry_names_) : entry_names(entry_names_) {} + + virtual std::unique_ptr getReadBuffer(size_t index) = 0; + virtual UInt64 getSize(size_t index) = 0; + virtual std::optional getChecksum(size_t) { return {}; } + +private: + class BackupEntryFromBatch; + const Strings entry_names; +}; + +} diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 546f3007df58..485650e39f01 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -17,7 +17,7 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) RestoreSettings res; if (query.base_backup_name) - res.base_backup_info = std::make_shared(BackupInfo::fromAST(*query.base_backup_name)); + res.base_backup_info = BackupInfo::fromAST(*query.base_backup_name); if (query.settings) { @@ -32,10 +32,10 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) res.throw_if_database_exists = SettingFieldBool{setting.value}; else if (setting.name == "throw_if_table_exists") res.throw_if_table_exists = SettingFieldBool{setting.value}; - else if (setting.name == "throw_if_database_has_different_definition") - res.throw_if_database_has_different_definition = SettingFieldBool{setting.value}; - else if (setting.name == "throw_if_table_has_different_definition") - res.throw_if_table_has_different_definition = SettingFieldBool{setting.value}; + else if (setting.name == "throw_if_database_def_differs") + res.throw_if_database_def_differs = SettingFieldBool{setting.value}; + else if (setting.name == "throw_if_table_def_differs") + res.throw_if_table_def_differs = SettingFieldBool{setting.value}; else throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting {}", setting.name); } diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index dcb55973d85b..b129224943b2 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -1,13 +1,12 @@ #pragma once -#include -#include +#include +#include namespace DB { class ASTBackupQuery; -struct BackupInfo; struct StorageRestoreSettings { @@ -19,7 +18,7 @@ struct RestoreSettings : public StorageRestoreSettings /// Base backup, with this setting we can override the location of the base backup while restoring. /// Any incremental backup keeps inside the information about its base backup, /// so using this setting is optional. - std::shared_ptr base_backup_info; + std::optional base_backup_info; /// Password used to decrypt the backup. String password; @@ -28,19 +27,19 @@ struct RestoreSettings : public StorageRestoreSettings /// without the data of tables. bool structure_only = false; - /// Whether RESTORE DATABASE will throw an exception if a destination database already exists. + /// Whether RESTORE DATABASE must throw an exception if a destination database already exists. bool throw_if_database_exists = true; - /// Whether RESTORE TABLE will throw an exception if a destination table already exists. + /// Whether RESTORE TABLE must throw an exception if a destination table already exists. bool throw_if_table_exists = true; - /// Whether RESTORE DATABASE will throw an exception if a destination database has + /// Whether RESTORE DATABASE must throw an exception if a destination database has /// a different definition comparing with the definition read from backup. - bool throw_if_database_has_different_definition = true; + bool throw_if_database_def_differs = true; - /// Whether RESTORE TABLE will throw an exception if a destination table has + /// Whether RESTORE TABLE must throw an exception if a destination table has /// a different definition comparing with the definition read from backup. - bool throw_if_table_has_different_definition = true; + bool throw_if_table_def_differs = true; static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query); }; diff --git a/src/Backups/RestoreUtils.cpp b/src/Backups/RestoreUtils.cpp index 3c8c1dcba09a..a94690ea881e 100644 --- a/src/Backups/RestoreUtils.cpp +++ b/src/Backups/RestoreUtils.cpp @@ -37,6 +37,7 @@ namespace using Element = ASTBackupQuery::Element; using Elements = ASTBackupQuery::Elements; using ElementType = ASTBackupQuery::ElementType; + using RestoreSettingsPtr = std::shared_ptr; /// Restores a database (without tables inside), should be executed before executing @@ -47,12 +48,12 @@ namespace RestoreDatabaseTask( ContextMutablePtr context_, const ASTPtr & create_query_, - const RestoreSettings & restore_settings_, - bool skip_same_definition_check_) + const RestoreSettingsPtr & restore_settings_, + bool ignore_if_database_def_differs_) : context(context_) , create_query(typeid_cast>(create_query_)) , restore_settings(restore_settings_) - , skip_same_definition_check(skip_same_definition_check_) + , ignore_if_database_def_differs(ignore_if_database_def_differs_) { } @@ -91,7 +92,7 @@ namespace void checkDatabaseCreateQuery() { - if (skip_same_definition_check || !restore_settings.throw_if_database_has_different_definition) + if (ignore_if_database_def_differs || !restore_settings->throw_if_database_def_differs) return; getDatabaseCreateQuery(); @@ -109,8 +110,8 @@ namespace ContextMutablePtr context; std::shared_ptr create_query; - RestoreSettings restore_settings; - bool skip_same_definition_check = false; + RestoreSettingsPtr restore_settings; + bool ignore_if_database_def_differs = false; DatabasePtr database; ASTPtr database_create_query; }; @@ -126,7 +127,7 @@ namespace const ASTs & partitions_, const BackupPtr & backup_, const DatabaseAndTableName & table_name_in_backup_, - const RestoreSettings & restore_settings_) + const RestoreSettingsPtr & restore_settings_) : context(context_), create_query(typeid_cast>(create_query_)), partitions(partitions_), backup(backup_), table_name_in_backup(table_name_in_backup_), restore_settings(restore_settings_) @@ -177,7 +178,7 @@ namespace void checkStorageCreateQuery() { - if (!restore_settings.throw_if_table_has_different_definition) + if (!restore_settings->throw_if_table_def_differs) return; getStorageCreateQuery(); @@ -199,7 +200,7 @@ namespace return *has_data; has_data = false; - if (restore_settings.structure_only) + if (restore_settings->structure_only) return false; data_path_in_backup = getDataPathInBackup(table_name_in_backup); @@ -230,7 +231,7 @@ namespace { if (!hasData()) return {}; - return storage->restoreFromBackup(context, partitions, backup, data_path_in_backup, restore_settings); + return storage->restoreFromBackup(context, partitions, backup, data_path_in_backup, *restore_settings); } ContextMutablePtr context; @@ -239,7 +240,7 @@ namespace ASTs partitions; BackupPtr backup; DatabaseAndTableName table_name_in_backup; - RestoreSettings restore_settings; + RestoreSettingsPtr restore_settings; DatabasePtr database; StoragePtr storage; ASTPtr storage_create_query; @@ -304,14 +305,16 @@ namespace serializeAST(*info.create_query), serializeAST(*info.different_create_query)); } + auto restore_settings_ptr = std::make_shared(restore_settings); + RestoreTasks res; for (auto & info : databases | boost::adaptors::map_values) - res.push_back(std::make_unique(context, info.create_query, restore_settings, - /* skip_same_definition_check = */ !info.is_explicit)); + res.push_back(std::make_unique(context, info.create_query, restore_settings_ptr, + /* ignore_if_database_def_differs = */ !info.is_explicit)); /// TODO: We need to restore tables according to their dependencies. for (auto & info : tables | boost::adaptors::map_values) - res.push_back(std::make_unique(context, info.create_query, info.partitions, backup, info.name_in_backup, restore_settings)); + res.push_back(std::make_unique(context, info.create_query, info.partitions, backup, info.name_in_backup, restore_settings_ptr)); return res; } diff --git a/src/IO/ConcatReadBuffer.h b/src/IO/ConcatReadBuffer.h index 4ef8d04d4c9c..3f44181a6e99 100644 --- a/src/IO/ConcatReadBuffer.h +++ b/src/IO/ConcatReadBuffer.h @@ -23,6 +23,12 @@ class ConcatReadBuffer : public ReadBuffer assert(!buffers.empty()); } + ConcatReadBuffer(std::unique_ptr buf1, std::unique_ptr buf2) : ConcatReadBuffer() + { + appendBuffer(std::move(buf1)); + appendBuffer(std::move(buf2)); + } + ConcatReadBuffer(ReadBuffer & buf1, ReadBuffer & buf2) : ConcatReadBuffer() { appendBuffer(wrapReadBufferReference(buf1)); diff --git a/src/Interpreters/InterpreterBackupQuery.cpp b/src/Interpreters/InterpreterBackupQuery.cpp index 390399e04708..01970bc5cc2b 100644 --- a/src/Interpreters/InterpreterBackupQuery.cpp +++ b/src/Interpreters/InterpreterBackupQuery.cpp @@ -20,7 +20,7 @@ namespace params.open_mode = IBackup::OpenMode::WRITE; params.context = context; params.backup_info = backup_info; - params.base_backup_info = backup_settings.base_backup_info ? *backup_settings.base_backup_info : std::optional{}; + params.base_backup_info = backup_settings.base_backup_info; params.compression_method = backup_settings.compression_method; params.compression_level = backup_settings.compression_level; params.password = backup_settings.password; @@ -33,7 +33,7 @@ namespace params.open_mode = IBackup::OpenMode::READ; params.context = context; params.backup_info = backup_info; - params.base_backup_info = restore_settings.base_backup_info ? *restore_settings.base_backup_info : std::optional{}; + params.base_backup_info = restore_settings.base_backup_info; params.password = restore_settings.password; return BackupFactory::instance().createBackup(params); } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 277be91b9770..502bc5750b0c 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -25,7 +25,7 @@ #include #include -#include +#include #include #include #include @@ -910,12 +910,12 @@ BackupEntries StorageLog::backup(ContextPtr context, const ASTs & partitions) { /// We make a copy of the data file because it can be changed later in write() or in truncate(). String data_file_name = fileName(data_file.path); - String temp_file_path = temp_dir + "/" + data_file_name; - disk->copy(data_file.path, disk, temp_file_path); + String hardlink_file_path = temp_dir + "/" + data_file_name; + disk->createHardLink(data_file.path, hardlink_file_path); backup_entries.emplace_back( data_file_name, - std::make_unique( - disk, temp_file_path, file_checker.getFileSize(data_file.path), std::nullopt, temp_dir_owner)); + std::make_unique( + disk, hardlink_file_path, file_checker.getFileSize(data_file.path), std::nullopt, temp_dir_owner)); } /// __marks.mrk @@ -923,12 +923,12 @@ BackupEntries StorageLog::backup(ContextPtr context, const ASTs & partitions) { /// We make a copy of the data file because it can be changed later in write() or in truncate(). String marks_file_name = fileName(marks_file_path); - String temp_file_path = temp_dir + "/" + marks_file_name; - disk->copy(marks_file_path, disk, temp_file_path); + String hardlink_file_path = temp_dir + "/" + marks_file_name; + disk->createHardLink(marks_file_path, hardlink_file_path); backup_entries.emplace_back( marks_file_name, - std::make_unique( - disk, temp_file_path, file_checker.getFileSize(marks_file_path), std::nullopt, temp_dir_owner)); + std::make_unique( + disk, hardlink_file_path, file_checker.getFileSize(marks_file_path), std::nullopt, temp_dir_owner)); } /// sizes.json diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 17431b0e9dc0..c99884c92d3c 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -17,8 +17,8 @@ #include #include #include -#include #include +#include #include #include #include @@ -339,65 +339,106 @@ void StorageMemory::truncate( } -BackupEntries StorageMemory::backup(ContextPtr context, const ASTs & partitions) +class MemoryBackupEntriesBatch : public shared_ptr_helper, public IBackupEntriesBatch { - if (!partitions.empty()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); +private: + friend struct shared_ptr_helper; + + MemoryBackupEntriesBatch( + const StorageMetadataPtr & metadata_snapshot_, const std::shared_ptr blocks_, UInt64 max_compress_block_size_) + : IBackupEntriesBatch({"data.bin", "index.mrk", "sizes.json"}) + , metadata_snapshot(metadata_snapshot_) + , blocks(blocks_) + , max_compress_block_size(max_compress_block_size_) + { + } + + static constexpr const size_t kDataBinPos = 0; + static constexpr const size_t kIndexMrkPos = 1; + static constexpr const size_t kSizesJsonPos = 2; + static constexpr const size_t kSize = 3; + + void initialize() + { + std::call_once(initialized_flag, [this]() + { + temp_dir_owner.emplace(); + auto temp_dir = temp_dir_owner->path(); + fs::create_directories(temp_dir); + + /// Writing data.bin + constexpr char data_file_name[] = "data.bin"; + String data_file_path = temp_dir + "/" + data_file_name; + IndexForNativeFormat index; + { + auto data_out_compressed = std::make_unique(data_file_path); + CompressedWriteBuffer data_out{*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), max_compress_block_size}; + NativeWriter block_out{data_out, 0, metadata_snapshot->getSampleBlock(), false, &index}; + for (const auto & block : *blocks) + block_out.write(block); + } + + /// Writing index.mrk + constexpr char index_file_name[] = "index.mrk"; + String index_file_path = temp_dir + "/" + index_file_name; + { + auto index_out_compressed = std::make_unique(index_file_path); + CompressedWriteBuffer index_out{*index_out_compressed}; + index.write(index_out); + } + + /// Writing sizes.json + constexpr char sizes_file_name[] = "sizes.json"; + String sizes_file_path = temp_dir + "/" + sizes_file_name; + FileChecker file_checker{sizes_file_path}; + file_checker.update(data_file_path); + file_checker.update(index_file_path); + file_checker.save(); + + file_paths[kDataBinPos] = data_file_path; + file_sizes[kDataBinPos] = file_checker.getFileSize(data_file_path); - auto blocks = data.get(); + file_paths[kIndexMrkPos] = index_file_path; + file_sizes[kIndexMrkPos] = file_checker.getFileSize(index_file_path); - /// We store our data in the StripeLog format. - BackupEntries backup_entries; - auto temp_dir_owner = std::make_shared(); - auto temp_dir = temp_dir_owner->path(); - fs::create_directories(temp_dir); + file_paths[kSizesJsonPos] = sizes_file_path; + file_sizes[kSizesJsonPos] = fs::file_size(sizes_file_path); - /// Writing data.bin - constexpr char data_file_name[] = "data.bin"; - String data_file_path = temp_dir + "/" + data_file_name; - IndexForNativeFormat index; + /// We don't need to keep `blocks` any longer. + blocks.reset(); + metadata_snapshot.reset(); + }); + } + + std::unique_ptr getReadBuffer(size_t index) override { - auto data_out_compressed = std::make_unique(data_file_path); - CompressedWriteBuffer data_out{*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), context->getSettingsRef().max_compress_block_size}; - NativeWriter block_out{data_out, 0, getInMemoryMetadataPtr()->getSampleBlock(), false, &index}; - for (const auto & block : *blocks) - block_out.write(block); + initialize(); + return createReadBufferFromFileBase(file_paths[index], {}); } - /// Writing index.mrk - constexpr char index_file_name[] = "index.mrk"; - String index_file_path = temp_dir + "/" + index_file_name; + UInt64 getSize(size_t index) override { - auto index_out_compressed = std::make_unique(index_file_path); - CompressedWriteBuffer index_out{*index_out_compressed}; - index.write(index_out); + initialize(); + return file_sizes[index]; } - /// Writing sizes.json - constexpr char sizes_file_name[] = "sizes.json"; - String sizes_file_path = temp_dir + "/" + sizes_file_name; - FileChecker file_checker{sizes_file_path}; - file_checker.update(data_file_path); - file_checker.update(index_file_path); - file_checker.save(); - - /// Prepare backup entries. - backup_entries.emplace_back( - data_file_name, - std::make_unique( - data_file_path, file_checker.getFileSize(data_file_path), std::nullopt, temp_dir_owner)); - - backup_entries.emplace_back( - index_file_name, - std::make_unique( - index_file_path, file_checker.getFileSize(index_file_path), std::nullopt, temp_dir_owner)); - - backup_entries.emplace_back( - sizes_file_name, - std::make_unique( - sizes_file_path, std::nullopt, std::nullopt, temp_dir_owner)); - - return backup_entries; + StorageMetadataPtr metadata_snapshot; + std::shared_ptr blocks; + UInt64 max_compress_block_size; + std::once_flag initialized_flag; + std::optional temp_dir_owner; + std::array file_paths; + std::array file_sizes; +}; + + +BackupEntries StorageMemory::backup(ContextPtr context, const ASTs & partitions) +{ + if (!partitions.empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); + + return MemoryBackupEntriesBatch::create(getInMemoryMetadataPtr(), data.get(), context->getSettingsRef().max_compress_block_size) + ->getBackupEntries(); } @@ -433,7 +474,7 @@ class MemoryRestoreTask : public IRestoreTask auto backup_entry = backup->readFile(data_file_path); std::unique_ptr in = backup_entry->getReadBuffer(); std::optional temp_data_copy; - if (!typeid_cast(in.get())) + if (!dynamic_cast(in.get())) { temp_data_copy.emplace(); auto temp_data_copy_out = std::make_unique(temp_data_copy->path()); diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 6317f08b3c76..008a2aeabeb8 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -35,7 +35,7 @@ #include #include -#include +#include #include #include #include @@ -517,24 +517,24 @@ BackupEntries StorageStripeLog::backup(ContextPtr context, const ASTs & partitio { /// We make a copy of the data file because it can be changed later in write() or in truncate(). String data_file_name = fileName(data_file_path); - String temp_file_path = temp_dir + "/" + data_file_name; - disk->copy(data_file_path, disk, temp_file_path); + String hardlink_file_path = temp_dir + "/" + data_file_name; + disk->createHardLink(data_file_path, hardlink_file_path); backup_entries.emplace_back( data_file_name, - std::make_unique( - disk, temp_file_path, file_checker.getFileSize(data_file_path), std::nullopt, temp_dir_owner)); + std::make_unique( + disk, hardlink_file_path, file_checker.getFileSize(data_file_path), std::nullopt, temp_dir_owner)); } /// index.mrk { /// We make a copy of the data file because it can be changed later in write() or in truncate(). String index_file_name = fileName(index_file_path); - String temp_file_path = temp_dir + "/" + index_file_name; - disk->copy(index_file_path, disk, temp_file_path); + String hardlink_file_path = temp_dir + "/" + index_file_name; + disk->createHardLink(index_file_path, hardlink_file_path); backup_entries.emplace_back( index_file_name, - std::make_unique( - disk, temp_file_path, file_checker.getFileSize(index_file_path), std::nullopt, temp_dir_owner)); + std::make_unique( + disk, hardlink_file_path, file_checker.getFileSize(index_file_path), std::nullopt, temp_dir_owner)); } /// sizes.json From 7a75673e4f3da3d8b2bc4ba62b6df131f4918a39 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 22 Feb 2022 15:39:11 +0300 Subject: [PATCH 15/18] A corresponding file in base backup can be searched by checksum too, not only by name. --- src/Backups/ArchiveBackup.cpp | 2 +- src/Backups/ArchiveBackup.h | 2 +- src/Backups/BackupImpl.cpp | 217 ++++++++++-------- src/Backups/BackupImpl.h | 9 +- src/Backups/BackupUtils.cpp | 2 +- src/Backups/DirectoryBackup.cpp | 2 +- src/Backups/DirectoryBackup.h | 2 +- src/Backups/IBackup.h | 5 +- .../test_backup_restore_new/test.py | 28 +++ 9 files changed, 158 insertions(+), 111 deletions(-) diff --git a/src/Backups/ArchiveBackup.cpp b/src/Backups/ArchiveBackup.cpp index 36148430f68b..2a1008678725 100644 --- a/src/Backups/ArchiveBackup.cpp +++ b/src/Backups/ArchiveBackup.cpp @@ -77,7 +77,7 @@ std::unique_ptr ArchiveBackup::readFileImpl(const String & file_name return reader->readFile(file_name); } -std::unique_ptr ArchiveBackup::addFileImpl(const String & file_name) +std::unique_ptr ArchiveBackup::writeFileImpl(const String & file_name) { /// mutex is already locked return writer->writeFile(file_name); diff --git a/src/Backups/ArchiveBackup.h b/src/Backups/ArchiveBackup.h index 8c76830977aa..9649c0c1843e 100644 --- a/src/Backups/ArchiveBackup.h +++ b/src/Backups/ArchiveBackup.h @@ -38,7 +38,7 @@ class ArchiveBackup : public BackupImpl void closeImpl(bool writing_finalized_) override; bool supportsWritingInMultipleThreads() const override { return false; } std::unique_ptr readFileImpl(const String & file_name) const override; - std::unique_ptr addFileImpl(const String & file_name) override; + std::unique_ptr writeFileImpl(const String & file_name) override; const DiskPtr disk; const String path; diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 71eea2223e8c..620c18d6a97f 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -176,9 +176,20 @@ void BackupImpl::writeBackupMetadata() config->setString("uuid", toString(uuid)); if (base_backup_info) - config->setString("base_backup", base_backup_info->toString()); - if (base_backup_uuid) - config->setString("base_backup_uuid", toString(*base_backup_uuid)); + { + bool base_backup_in_use = false; + for (const auto & [name, info] : file_infos) + { + if (info.base_size) + base_backup_in_use = true; + } + + if (base_backup_in_use) + { + config->setString("base_backup", base_backup_info->toString()); + config->setString("base_backup_uuid", toString(*base_backup_uuid)); + } + } size_t index = 0; for (const auto & [name, info] : file_infos) @@ -192,7 +203,7 @@ void BackupImpl::writeBackupMetadata() if (info.base_size) { config->setUInt(prefix + "base_size", info.base_size); - if (info.base_size != info.size) + if (info.base_checksum != info.checksum) config->setString(prefix + "base_checksum", getHexUIntLowercase(info.base_checksum)); } } @@ -202,7 +213,7 @@ void BackupImpl::writeBackupMetadata() std::ostringstream stream; // STYLE_CHECK_ALLOW_STD_STRING_STREAM config->save(stream); String str = stream.str(); - auto out = addFileImpl(".backup"); + auto out = writeFileImpl(".backup"); out->write(str.data(), str.size()); } @@ -225,7 +236,7 @@ void BackupImpl::readBackupMetadata() if (config->has("base_backup") && !base_backup_info) base_backup_info = BackupInfo::fromString(config->getString("base_backup")); - if (config->has("base_backup_uuid") && !base_backup_uuid) + if (config->has("base_backup_uuid")) base_backup_uuid = parse(config->getString("base_backup_uuid")); file_infos.clear(); @@ -237,20 +248,22 @@ void BackupImpl::readBackupMetadata() { String prefix = "contents." + key + "."; String name = config->getString(prefix + "name"); - FileInfo & info = file_infos.emplace(name, FileInfo{}).first->second; + FileInfo info; info.size = config->getUInt(prefix + "size"); if (info.size) { info.checksum = unhexChecksum(config->getString(prefix + "checksum")); - if (config->has(prefix + "base_size")) + info.base_size = config->getUInt(prefix + "base_size", 0); + if (info.base_size) { - info.base_size = config->getUInt(prefix + "base_size"); - if (info.base_size == info.size) - info.base_checksum = info.checksum; - else + if (config->has(prefix + "base_checksum")) info.base_checksum = unhexChecksum(config->getString(prefix + "base_checksum")); + else + info.base_checksum = info.checksum; } } + file_infos.emplace(name, info); + file_checksums.emplace(info.checksum, name); } } } @@ -304,6 +317,15 @@ UInt128 BackupImpl::getFileChecksum(const String & file_name) const return it->second.checksum; } +std::optional BackupImpl::findFileByChecksum(const UInt128 & checksum) const +{ + std::lock_guard lock{mutex}; + auto it = file_checksums.find(checksum); + if (it == file_checksums.end()) + return std::nullopt; + return it->second; +} + BackupEntryPtr BackupImpl::readFile(const String & file_name) const { @@ -351,7 +373,8 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const getName(), quoteString(file_name)); } - if (!base_backup->fileExists(file_name)) + auto base_file_name = base_backup->findFileByChecksum(info.base_checksum); + if (!base_file_name) { throw Exception( ErrorCodes::WRONG_BASE_BACKUP, @@ -359,7 +382,7 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const getName(), quoteString(file_name)); } - auto base_entry = base_backup->readFile(file_name); + auto base_entry = base_backup->readFile(*base_file_name); auto base_size = base_entry->getSize(); if (base_size != info.base_size) { @@ -369,15 +392,6 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const getName(), quoteString(file_name), base_backup->getName(), base_size, info.base_size); } - auto base_checksum = base_entry->getChecksum(); - if (base_checksum && (*base_checksum != info.base_checksum)) - { - throw Exception( - ErrorCodes::WRONG_BASE_BACKUP, - "Backup {}: Entry {} has unexpected checksum in the base backup {}", - getName(), quoteString(file_name), base_backup->getName()); - } - if (info.size == info.base_size) { /// Data goes completely from the base backup (nothing goes from this backup). @@ -391,7 +405,7 @@ BackupEntryPtr BackupImpl::readFile(const String & file_name) const } -void BackupImpl::addFile(const String & file_name, BackupEntryPtr entry) +void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) { std::lock_guard lock{mutex}; if (open_mode != OpenMode::WRITE) @@ -422,44 +436,61 @@ void BackupImpl::addFile(const String & file_name, BackupEntryPtr entry) } std::unique_ptr read_buffer; /// We'll set that later. - UInt64 read_pos = 0; /// Current position in read_buffer. + std::optional hashing_read_buffer; + UInt64 hashing_pos = 0; /// Current position in `hashing_read_buffer`. /// Determine whether it's possible to receive this entry's data from the base backup completely or partly. bool use_base = false; - if (base_exists && base_size) + if (base_exists && base_size && (size >= base_size)) { - if (size == base_size) + if (checksum && (size == base_size)) { /// The size is the same, we need to compare checksums to find out - /// if the entry's data has not been changed since the base backup. - if (!checksum) - { - read_buffer = entry->getReadBuffer(); - HashingReadBuffer hashing_read_buffer{*read_buffer}; - hashing_read_buffer.ignore(size); - read_pos = size; - checksum = hashing_read_buffer.getHash(); - } - if (checksum == base_checksum) - use_base = true; /// The data have not been changed. + /// if the entry's data has not changed since the base backup. + use_base = (*checksum == base_checksum); } - else if (size > base_size) + else { - /// The size has been increased, we need to calculate a partial checksum to find out - /// if the entry's data has been only appended since the base backup. + /// The size has increased, we need to calculate a partial checksum to find out + /// if the entry's data has only appended since the base backup. read_buffer = entry->getReadBuffer(); - HashingReadBuffer hashing_read_buffer{*read_buffer}; - hashing_read_buffer.ignore(base_size); - UInt128 partial_checksum = hashing_read_buffer.getHash(); - read_pos = base_size; - if (!checksum) + hashing_read_buffer.emplace(*read_buffer); + hashing_read_buffer->ignore(base_size); + hashing_pos = base_size; + UInt128 partial_checksum = hashing_read_buffer->getHash(); + if (size == base_size) + checksum = partial_checksum; + if (partial_checksum == base_checksum) + use_base = true; + } + } + + /// Finish calculating the checksum. + if (!checksum) + { + if (!read_buffer) + read_buffer = entry->getReadBuffer(); + if (!hashing_read_buffer) + hashing_read_buffer.emplace(*read_buffer); + hashing_read_buffer->ignore(size - hashing_pos); + checksum = hashing_read_buffer->getHash(); + } + hashing_read_buffer.reset(); + + /// Check if a entry with the same checksum exists in the base backup. + if (base_backup && !use_base) + { + if (auto base_file_name = base_backup->findFileByChecksum(*checksum)) + { + if (size == base_backup->getFileSize(*base_file_name)) { - hashing_read_buffer.ignore(size - base_size); - checksum = hashing_read_buffer.getHash(); - read_pos = size; + /// The entry's data has not changed since the base backup, + /// but the entry itself has been moved or renamed. + base_size = size; + base_checksum = *checksum; + base_exists = true; + use_base = true; } - if (partial_checksum == base_checksum) - use_base = true; /// The data has been appended. } } @@ -467,70 +498,52 @@ void BackupImpl::addFile(const String & file_name, BackupEntryPtr entry) { /// The entry's data has not been changed since the base backup. FileInfo info; - info.size = base_size; - info.checksum = base_checksum; + info.size = size; + info.checksum = *checksum; info.base_size = base_size; info.base_checksum = base_checksum; file_infos.emplace(file_name, info); + file_checksums.emplace(*checksum, file_name); return; } - { - /// Either the entry wasn't exist in the base backup - /// or the entry has data appended to the end of the data from the base backup. - /// In both those cases we have to copy data to this backup. - - /// Find out where the start position to copy data is. - auto copy_pos = use_base ? base_size : 0; - - /// Move the current read position to the start position to copy data. - /// If `read_buffer` is seekable it's easier, otherwise we can use ignore(). - if (auto * seekable_buffer = dynamic_cast(read_buffer.get())) - { - if (read_pos != copy_pos) - seekable_buffer->seek(copy_pos, SEEK_SET); - } - else - { - if (read_pos > copy_pos) - { - read_buffer.reset(); - read_pos = 0; - } + /// Either the entry wasn't exist in the base backup + /// or the entry has data appended to the end of the data from the base backup. + /// In both those cases we have to copy data to this backup. - if (!read_buffer) - read_buffer = entry->getReadBuffer(); - - if (read_pos < copy_pos) - read_buffer->ignore(copy_pos - read_pos); - } + /// Find out where the start position to copy data is. + auto copy_pos = use_base ? base_size : 0; - /// If we haven't received or calculated a checksum yet, calculate it now. - ReadBuffer * maybe_hashing_read_buffer = read_buffer.get(); - std::optional hashing_read_buffer; - if (!checksum) - maybe_hashing_read_buffer = &hashing_read_buffer.emplace(*read_buffer); - - /// Copy the entry's data after `copy_pos`. - auto out = addFileImpl(file_name); - copyData(*maybe_hashing_read_buffer, *out); + /// Move the current read position to the start position to copy data. + /// If `read_buffer` is seekable it's easier, otherwise we can use ignore(). + if (auto * seekable_buffer = dynamic_cast(read_buffer.get())) + { + seekable_buffer->seek(copy_pos, SEEK_SET); + } + else + { + read_buffer = entry->getReadBuffer(); + read_buffer->ignore(copy_pos); + } - if (hashing_read_buffer) - checksum = hashing_read_buffer->getHash(); + /// Copy the entry's data after `copy_pos`. + auto out = writeFileImpl(file_name); + copyData(*read_buffer, *out); - /// Done! - FileInfo info; - info.size = size; - info.checksum = *checksum; - if (use_base) - { - info.base_size = base_size; - info.base_checksum = base_checksum; - } - file_infos.emplace(file_name, info); + /// Done! + FileInfo info; + info.size = size; + info.checksum = *checksum; + if (use_base) + { + info.base_size = base_size; + info.base_checksum = base_checksum; } + file_infos.emplace(file_name, info); + file_checksums.emplace(*checksum, file_name); } + void BackupImpl::finalizeWriting() { std::lock_guard lock{mutex}; diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index af31b71f4601..d1fc3c3248cf 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -34,8 +35,9 @@ class BackupImpl : public IBackup bool fileExists(const String & file_name) const override; size_t getFileSize(const String & file_name) const override; UInt128 getFileChecksum(const String & file_name) const override; + std::optional findFileByChecksum(const UInt128 & checksum) const override; BackupEntryPtr readFile(const String & file_name) const override; - void addFile(const String & file_name, BackupEntryPtr entry) override; + void writeFile(const String & file_name, BackupEntryPtr entry) override; void finalizeWriting() override; protected: @@ -53,7 +55,7 @@ class BackupImpl : public IBackup /// Add a file to the backup. /// Low level: the function doesn't check base backup or checksums. - virtual std::unique_ptr addFileImpl(const String & file_name) = 0; + virtual std::unique_ptr writeFileImpl(const String & file_name) = 0; mutable std::mutex mutex; @@ -82,7 +84,8 @@ class BackupImpl : public IBackup std::optional base_backup_info; std::shared_ptr base_backup; std::optional base_backup_uuid; - std::map file_infos; + std::map file_infos; /// Should be ordered alphabetically, see listFiles(). + std::unordered_map file_checksums; bool writing_finalized = false; }; diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 5e40e1dbb696..f9f9722c8d55 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -335,7 +335,7 @@ void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries { try { - backup->addFile(name, std::move(entry)); + backup->writeFile(name, std::move(entry)); } catch (...) { diff --git a/src/Backups/DirectoryBackup.cpp b/src/Backups/DirectoryBackup.cpp index 2adee1601802..dc4d098dbe9b 100644 --- a/src/Backups/DirectoryBackup.cpp +++ b/src/Backups/DirectoryBackup.cpp @@ -72,7 +72,7 @@ std::unique_ptr DirectoryBackup::readFileImpl(const String & file_na return disk->readFile(file_path); } -std::unique_ptr DirectoryBackup::addFileImpl(const String & file_name) +std::unique_ptr DirectoryBackup::writeFileImpl(const String & file_name) { String file_path = path + file_name; disk->createDirectories(fs::path(file_path).parent_path()); diff --git a/src/Backups/DirectoryBackup.h b/src/Backups/DirectoryBackup.h index 0b6dd1361d1d..7d9b5cc45573 100644 --- a/src/Backups/DirectoryBackup.h +++ b/src/Backups/DirectoryBackup.h @@ -27,7 +27,7 @@ class DirectoryBackup : public BackupImpl void openImpl(OpenMode open_mode_) override; void closeImpl(bool writing_finalized_) override; std::unique_ptr readFileImpl(const String & file_name) const override; - std::unique_ptr addFileImpl(const String & file_name) override; + std::unique_ptr writeFileImpl(const String & file_name) override; DiskPtr disk; String path; diff --git a/src/Backups/IBackup.h b/src/Backups/IBackup.h index 7759879ecfe1..31a6e0dd286d 100644 --- a/src/Backups/IBackup.h +++ b/src/Backups/IBackup.h @@ -61,11 +61,14 @@ class IBackup : public std::enable_shared_from_this /// This function does the same as `read(file_name)->getCheckum()` but faster. virtual UInt128 getFileChecksum(const String & file_name) const = 0; + /// Finds a file by its checksum, returns nullopt if not found. + virtual std::optional findFileByChecksum(const UInt128 & checksum) const = 0; + /// Reads an entry from the backup. virtual BackupEntryPtr readFile(const String & file_name) const = 0; /// Puts a new entry to the backup. - virtual void addFile(const String & file_name, BackupEntryPtr entry) = 0; + virtual void writeFile(const String & file_name, BackupEntryPtr entry) = 0; /// Finalizes writing the backup, should be called after all entries have been successfully written. virtual void finalizeWriting() = 0; diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index dcd6c241801b..e3f0ab178cce 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -1,5 +1,6 @@ import pytest import re +import os.path from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) @@ -37,6 +38,11 @@ def new_backup_name(): return f"Disk('backups', '{backup_id_counter}/')" +def get_backup_dir(backup_name): + counter = int(backup_name.split(',')[1].strip("')/ ")) + return os.path.join(instance.path, f'backups/{counter}') + + @pytest.mark.parametrize("engine", ["MergeTree", "Log", "TinyLog", "StripeLog", "Memory"]) def test_restore_table(engine): backup_name = new_backup_name() @@ -110,6 +116,27 @@ def test_incremental_backup(): assert instance.query("SELECT count(), sum(x) FROM test.table2") == "102\t5081\n" +def test_incremental_backup_after_renaming_table(): + backup_name = new_backup_name() + incremental_backup_name = new_backup_name() + create_and_fill_table() + + instance.query(f"BACKUP TABLE test.table TO {backup_name}") + instance.query("RENAME TABLE test.table TO test.table2") + instance.query(f"BACKUP TABLE test.table2 TO {incremental_backup_name} SETTINGS base_backup = {backup_name}") + + # Files in a base backup can be searched by checksum, so an incremental backup with a renamed table actually + # contains only its changed metadata. + assert os.path.isdir(os.path.join(get_backup_dir(backup_name), 'metadata')) == True + assert os.path.isdir(os.path.join(get_backup_dir(backup_name), 'data')) == True + assert os.path.isdir(os.path.join(get_backup_dir(incremental_backup_name), 'metadata')) == True + assert os.path.isdir(os.path.join(get_backup_dir(incremental_backup_name), 'data')) == False + + instance.query("DROP TABLE test.table2") + instance.query(f"RESTORE TABLE test.table2 FROM {incremental_backup_name}") + assert instance.query("SELECT count(), sum(x) FROM test.table2") == "100\t4950\n" + + def test_backup_not_found_or_already_exists(): backup_name = new_backup_name() @@ -155,6 +182,7 @@ def test_zip_archive(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" instance.query(f"BACKUP TABLE test.table TO {backup_name}") + assert os.path.isfile(os.path.join(os.path.join(instance.path, 'backups/archive.zip'))) instance.query("DROP TABLE test.table") assert instance.query("EXISTS test.table") == "0\n" From e2eb03f2c8a261d300cffd17479e28f7a6a0f87f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 22 Feb 2022 16:31:50 +0300 Subject: [PATCH 16/18] Storages and databases are hollow by default now. --- src/Backups/BackupUtils.cpp | 8 ++++---- src/Backups/RestoreUtils.cpp | 2 +- src/Databases/DatabaseMemory.h | 3 +++ src/Databases/DatabaseOrdinary.h | 3 +++ src/Databases/IDatabase.h | 2 +- src/Storages/IStorage.cpp | 6 +++--- src/Storages/IStorage.h | 6 +++--- src/Storages/MergeTree/MergeTreeData.cpp | 8 ++++---- src/Storages/MergeTree/MergeTreeData.h | 7 +++++-- src/Storages/StorageLog.cpp | 4 ++-- src/Storages/StorageLog.h | 5 +++-- src/Storages/StorageMemory.cpp | 4 ++-- src/Storages/StorageMemory.h | 5 +++-- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageStripeLog.cpp | 4 ++-- src/Storages/StorageStripeLog.h | 5 +++-- 17 files changed, 45 insertions(+), 33 deletions(-) diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index f9f9722c8d55..1d78be53ac61 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -97,7 +97,7 @@ namespace res.push_back(makeBackupEntryForMetadata(*info.create_query)); if (info.has_data) { - auto data_backup = info.storage->backup(context, info.partitions); + auto data_backup = info.storage->backupData(context, info.partitions); if (!data_backup.empty()) { String data_path = getDataPathInBackup(*info.create_query); @@ -129,7 +129,7 @@ namespace const auto & database = table_.first; const auto & storage = table_.second; - if (database->hasHollowBackup()) + if (!database->hasTablesToBackup()) throw Exception( ErrorCodes::CANNOT_BACKUP_TABLE, "Cannot backup the {} because it's contained in a hollow database (engine: {})", @@ -144,7 +144,7 @@ namespace /// Make a create query for this table. auto create_query = renameInCreateQuery(database->getCreateTableQuery(table_name_.second, context)); - bool has_data = !storage->hasHollowBackup() && !backup_settings.structure_only; + bool has_data = storage->hasDataToBackup() && !backup_settings.structure_only; if (has_data) { /// We check for SELECT privilege only if we're going to read data from the table. @@ -220,7 +220,7 @@ namespace } /// Backup tables in this database. - if (!database_->hasHollowBackup()) + if (database_->hasTablesToBackup()) { for (auto it = database_->getTablesIterator(context); it->isValid(); it->next()) { diff --git a/src/Backups/RestoreUtils.cpp b/src/Backups/RestoreUtils.cpp index a94690ea881e..0878fbbf2336 100644 --- a/src/Backups/RestoreUtils.cpp +++ b/src/Backups/RestoreUtils.cpp @@ -231,7 +231,7 @@ namespace { if (!hasData()) return {}; - return storage->restoreFromBackup(context, partitions, backup, data_path_in_backup, *restore_settings); + return storage->restoreData(context, partitions, backup, data_path_in_backup, *restore_settings); } ContextMutablePtr context; diff --git a/src/Databases/DatabaseMemory.h b/src/Databases/DatabaseMemory.h index b854d9be1f34..87fae115b598 100644 --- a/src/Databases/DatabaseMemory.h +++ b/src/Databases/DatabaseMemory.h @@ -50,6 +50,9 @@ class DatabaseMemory final : public DatabaseWithOwnTablesBase void alterTable(ContextPtr local_context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) override; + /// This database can contain tables to backup. + bool hasTablesToBackup() const override { return true; } + private: String data_path; using NameToASTCreate = std::unordered_map; diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index 982be2024ce8..2144f874b03c 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -36,6 +36,9 @@ class DatabaseOrdinary : public DatabaseOnDisk const StorageID & table_id, const StorageInMemoryMetadata & metadata) override; + /// This database can contain tables to backup. + bool hasTablesToBackup() const override { return true; } + protected: virtual void commitAlterTable( const StorageID & table_id, diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 652014487a0f..5bfc59350107 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -326,7 +326,7 @@ class IDatabase : public std::enable_shared_from_this /// Returns true if the backup of the database is hollow, which means it doesn't contain /// any tables which can be stored to a backup. - virtual bool hasHollowBackup() const { return false; } + virtual bool hasTablesToBackup() const { return false; } virtual ~IDatabase() = default; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 142e548773a9..ae4008f63e10 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -217,14 +217,14 @@ bool IStorage::isStaticStorage() const return false; } -BackupEntries IStorage::backup(ContextPtr, const ASTs &) +BackupEntries IStorage::backupData(ContextPtr, const ASTs &) { throw Exception("Table engine " + getName() + " doesn't support backups", ErrorCodes::NOT_IMPLEMENTED); } -RestoreTaskPtr IStorage::restoreFromBackup(ContextMutablePtr, const ASTs &, const BackupPtr &, const String &, const StorageRestoreSettings &) +RestoreTaskPtr IStorage::restoreData(ContextMutablePtr, const ASTs &, const BackupPtr &, const String &, const StorageRestoreSettings &) { - throw Exception("Table engine " + getName() + " doesn't support restoring", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("Table engine " + getName() + " doesn't support backups", ErrorCodes::NOT_IMPLEMENTED); } std::string PrewhereInfo::dump() const diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index c0223bf1813e..6732d5980406 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -214,13 +214,13 @@ class IStorage : public std::enable_shared_from_this, public TypePromo NameDependencies getDependentViewsByColumn(ContextPtr context) const; /// Returns true if the backup is hollow, which means it doesn't contain any data. - virtual bool hasHollowBackup() const { return false; } + virtual bool hasDataToBackup() const { return false; } /// Prepares entries to backup data of the storage. - virtual BackupEntries backup(ContextPtr context, const ASTs & partitions); + virtual BackupEntries backupData(ContextPtr context, const ASTs & partitions); /// Extract data from the backup and put it to the storage. - virtual RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings); + virtual RestoreTaskPtr restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings); /// Returns whether the column is virtual - by default all columns are real. /// Initially reserved virtual column name may be shadowed by real column. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 43d42f24eaed..3bb8c6fcb29e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3582,7 +3582,7 @@ Pipe MergeTreeData::alterPartition( } -BackupEntries MergeTreeData::backup(ContextPtr local_context, const ASTs & partitions) +BackupEntries MergeTreeData::backupData(ContextPtr local_context, const ASTs & partitions) { DataPartsVector data_parts; if (partitions.empty()) @@ -3742,9 +3742,9 @@ class MergeTreeDataRestoreTask : public IRestoreTask }; -RestoreTaskPtr MergeTreeData::restoreDataPartsFromBackup(const std::unordered_set & partition_ids, - const BackupPtr & backup, const String & data_path_in_backup, - SimpleIncrement * increment) +RestoreTaskPtr MergeTreeData::restoreDataParts(const std::unordered_set & partition_ids, + const BackupPtr & backup, const String & data_path_in_backup, + SimpleIncrement * increment) { return std::make_unique( std::static_pointer_cast(shared_from_this()), backup, data_path_in_backup, partition_ids, increment); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index a8c26bfa4bd6..2424b5101562 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -649,12 +649,15 @@ class MergeTreeData : public IStorage, public WithMutableContext ContextPtr context, TableLockHolder & table_lock_holder); + /// Storage has data to backup. + bool hasDataToBackup() const override { return true; } + /// Prepares entries to backup data of the storage. - BackupEntries backup(ContextPtr context, const ASTs & partitions) override; + BackupEntries backupData(ContextPtr context, const ASTs & partitions) override; static BackupEntries backupDataParts(const DataPartsVector & data_parts); /// Extract data from the backup and put it to the storage. - RestoreTaskPtr restoreDataPartsFromBackup( + RestoreTaskPtr restoreDataParts( const std::unordered_set & partition_ids, const BackupPtr & backup, const String & data_path_in_backup, diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 502bc5750b0c..56fa09f2182a 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -884,7 +884,7 @@ IStorage::ColumnSizeByName StorageLog::getColumnSizes() const } -BackupEntries StorageLog::backup(ContextPtr context, const ASTs & partitions) +BackupEntries StorageLog::backupData(ContextPtr context, const ASTs & partitions) { if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); @@ -1059,7 +1059,7 @@ class LogRestoreTask : public IRestoreTask ContextMutablePtr context; }; -RestoreTaskPtr StorageLog::restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &) +RestoreTaskPtr StorageLog::restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &) { if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 93c292dc485b..d7d265df7cc6 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -52,8 +52,9 @@ class StorageLog final : public shared_ptr_helper, public IStorage bool supportsSubcolumns() const override { return true; } ColumnSizeByName getColumnSizes() const override; - BackupEntries backup(ContextPtr context, const ASTs & partitions) override; - RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override; + bool hasDataToBackup() const override { return true; } + BackupEntries backupData(ContextPtr context, const ASTs & partitions) override; + RestoreTaskPtr restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override; protected: /** Attach the table with the appropriate name, along the appropriate path (with / at the end), diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index c99884c92d3c..8817737a0fa2 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -432,7 +432,7 @@ class MemoryBackupEntriesBatch : public shared_ptr_helper; void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; - BackupEntries backup(ContextPtr context, const ASTs & partitions) override; - RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override; + bool hasDataToBackup() const override { return true; } + BackupEntries backupData(ContextPtr context, const ASTs & partitions) override; + RestoreTaskPtr restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override; std::optional totalRows(const Settings &) const override; std::optional totalBytes(const Settings &) const override; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index cf27a4e907e6..888c217bfb88 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1645,9 +1645,9 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ } -RestoreTaskPtr StorageMergeTree::restoreFromBackup(ContextMutablePtr local_context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &) +RestoreTaskPtr StorageMergeTree::restoreData(ContextMutablePtr local_context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &) { - return restoreDataPartsFromBackup(getPartitionIDsFromQuery(partitions, local_context), backup, data_path_in_backup, &increment); + return restoreDataParts(getPartitionIDsFromQuery(partitions, local_context), backup, data_path_in_backup, &increment); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index ad9381dbc492..49db818b012e 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -97,7 +97,7 @@ class StorageMergeTree final : public shared_ptr_helper, publi CheckResults checkData(const ASTPtr & query, ContextPtr context) override; - RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override; + RestoreTaskPtr restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override; bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 008a2aeabeb8..d735c7fc47e0 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -492,7 +492,7 @@ void StorageStripeLog::saveFileSizes(const WriteLock & /* already locked for wri } -BackupEntries StorageStripeLog::backup(ContextPtr context, const ASTs & partitions) +BackupEntries StorageStripeLog::backupData(ContextPtr context, const ASTs & partitions) { if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); @@ -638,7 +638,7 @@ class StripeLogRestoreTask : public IRestoreTask }; -RestoreTaskPtr StorageStripeLog::restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &) +RestoreTaskPtr StorageStripeLog::restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &) { if (!partitions.empty()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 8ba2857dbc23..6c3a9a2767ae 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -52,8 +52,9 @@ class StorageStripeLog final : public shared_ptr_helper, publi void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder&) override; - BackupEntries backup(ContextPtr context, const ASTs & partitions) override; - RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override; + bool hasDataToBackup() const override { return true; } + BackupEntries backupData(ContextPtr context, const ASTs & partitions) override; + RestoreTaskPtr restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override; protected: StorageStripeLog( From bd947cc5784d527486e2c3091063d80629c97ef8 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 22 Feb 2022 20:05:52 +0300 Subject: [PATCH 17/18] Implement BACKUP and RESTORE for MatViews. --- src/Backups/BackupUtils.cpp | 17 +++++++++++------ src/Backups/DDLCompareUtils.cpp | 8 ++++++-- src/Backups/DDLRenamingVisitor.cpp | 9 ++------- src/Backups/DDLRenamingVisitor.h | 2 +- src/Backups/RestoreUtils.cpp | 5 ++++- src/Storages/StorageMaterializedView.cpp | 17 +++++++++++++++++ src/Storages/StorageMaterializedView.h | 4 ++++ .../integration/test_backup_restore_new/test.py | 12 ++++++++++++ 8 files changed, 57 insertions(+), 17 deletions(-) diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index 1d78be53ac61..6c549a3c848c 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -142,7 +142,7 @@ namespace throw Exception(ErrorCodes::CANNOT_BACKUP_TABLE, "Cannot backup the {} twice", formatTableNameOrTemporaryTableName(new_table_name)); /// Make a create query for this table. - auto create_query = renameInCreateQuery(database->getCreateTableQuery(table_name_.second, context)); + auto create_query = prepareCreateQueryForBackup(database->getCreateTableQuery(table_name_.second, context)); bool has_data = storage->hasDataToBackup() && !backup_settings.structure_only; if (has_data) @@ -165,7 +165,7 @@ namespace if (!databases.contains(new_table_name.first)) { /// Add a create query to backup the database if we haven't done it yet. - auto create_db_query = renameInCreateQuery(database->getCreateDatabaseQuery()); + auto create_db_query = prepareCreateQueryForBackup(database->getCreateDatabaseQuery()); create_db_query->setDatabase(new_table_name.first); CreateDatabaseInfo info_db; @@ -181,7 +181,7 @@ namespace auto & info_db = databases[new_table_name.first]; if (!info_db.is_explicit && (info_db.original_name != table_name_.first) && !info_db.different_create_query) { - auto create_db_query = renameInCreateQuery(table_.first->getCreateDatabaseQuery()); + auto create_db_query = prepareCreateQueryForBackup(table_.first->getCreateDatabaseQuery()); create_db_query->setDatabase(new_table_name.first); if (!areDatabaseDefinitionsSame(*info_db.create_query, *create_db_query)) info_db.different_create_query = create_db_query; @@ -210,7 +210,7 @@ namespace if (!isSystemOrTemporaryDatabase(database_name_)) { /// Make a create query for this database. - auto create_db_query = renameInCreateQuery(database_->getCreateDatabaseQuery()); + auto create_db_query = prepareCreateQueryForBackup(database_->getCreateDatabaseQuery()); CreateDatabaseInfo info_db; info_db.create_query = create_db_query; @@ -245,9 +245,14 @@ namespace } /// Do renaming in the create query according to the renaming config. - std::shared_ptr renameInCreateQuery(const ASTPtr & ast) const + std::shared_ptr prepareCreateQueryForBackup(const ASTPtr & ast) const { - return typeid_cast>(::DB::renameInCreateQuery(ast, context, renaming_settings)); + ASTPtr query = ast; + ::DB::renameInCreateQuery(query, context, renaming_settings); + auto create_query = typeid_cast>(query); + create_query->uuid = UUIDHelpers::Nil; + create_query->to_inner_uuid = UUIDHelpers::Nil; + return create_query; } static bool isSystemOrTemporaryDatabase(const String & database_name) diff --git a/src/Backups/DDLCompareUtils.cpp b/src/Backups/DDLCompareUtils.cpp index 7c857cff7d59..625a0befe637 100644 --- a/src/Backups/DDLCompareUtils.cpp +++ b/src/Backups/DDLCompareUtils.cpp @@ -22,10 +22,13 @@ namespace return clone; }; - /// Remove UUID. + /// Remove UUIDs. if (res->uuid != UUIDHelpers::Nil) get_clone()->uuid = UUIDHelpers::Nil; + if (res->to_inner_uuid != UUIDHelpers::Nil) + get_clone()->to_inner_uuid = UUIDHelpers::Nil; + /// Clear IF NOT EXISTS flag. if (res->if_not_exists) get_clone()->if_not_exists = false; @@ -45,7 +48,8 @@ bool areTableDefinitionsSame(const IAST & table1, const IAST & table2) if (!ast2 || !ast2->table) return false; - if ((ast1->uuid != ast2->uuid) || (ast1->if_not_exists != ast2->if_not_exists)) + if ((ast1->uuid != ast2->uuid) || (ast1->to_inner_uuid != ast2->to_inner_uuid) || + (ast1->if_not_exists != ast2->if_not_exists)) { ast1 = prepareDDLToCompare(*ast1); ast2 = prepareDDLToCompare(*ast2); diff --git a/src/Backups/DDLRenamingVisitor.cpp b/src/Backups/DDLRenamingVisitor.cpp index 9d87d18e142b..bcdfb32c29f6 100644 --- a/src/Backups/DDLRenamingVisitor.cpp +++ b/src/Backups/DDLRenamingVisitor.cpp @@ -59,8 +59,6 @@ namespace else throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name specified in the CREATE DATABASE query must not be empty"); - create.uuid = UUIDHelpers::Nil; - if (!create.as_table.empty() && !create.as_database.empty()) std::tie(create.as_database, create.as_table) = data.renaming_settings.getNewTableName({create.as_database, create.as_table}); @@ -365,19 +363,16 @@ void DDLRenamingVisitor::visit(ASTPtr & ast, const Data & data) visitDictionary(*dictionary, data); } -ASTPtr renameInCreateQuery(const ASTPtr & ast, const ContextPtr & global_context, const DDLRenamingSettings & renaming_settings) +void renameInCreateQuery(ASTPtr & ast, const ContextPtr & global_context, const DDLRenamingSettings & renaming_settings) { - auto new_ast = ast->clone(); try { DDLRenamingVisitor::Data data{renaming_settings, global_context}; - DDLRenamingVisitor::Visitor{data}.visit(new_ast); - return new_ast; + DDLRenamingVisitor::Visitor{data}.visit(ast); } catch (...) { tryLogCurrentException("Backup", "Error while renaming in AST"); - return ast; } } diff --git a/src/Backups/DDLRenamingVisitor.h b/src/Backups/DDLRenamingVisitor.h index e74f68a40334..a614f11002aa 100644 --- a/src/Backups/DDLRenamingVisitor.h +++ b/src/Backups/DDLRenamingVisitor.h @@ -40,7 +40,7 @@ class DDLRenamingSettings /// Changes names in AST according to the renaming settings. -ASTPtr renameInCreateQuery(const ASTPtr & ast, const ContextPtr & global_context, const DDLRenamingSettings & config); +void renameInCreateQuery(ASTPtr & ast, const ContextPtr & global_context, const DDLRenamingSettings & config); /// Visits ASTCreateQuery and changes names of tables and databases according to passed DDLRenamingConfig. class DDLRenamingVisitor diff --git a/src/Backups/RestoreUtils.cpp b/src/Backups/RestoreUtils.cpp index 0878fbbf2336..60a82579a337 100644 --- a/src/Backups/RestoreUtils.cpp +++ b/src/Backups/RestoreUtils.cpp @@ -487,7 +487,10 @@ namespace /// Do renaming in the create query according to the renaming config. std::shared_ptr renameInCreateQuery(const ASTPtr & ast) const { - return typeid_cast>(::DB::renameInCreateQuery(ast, context, renaming_settings)); + ASTPtr query = ast; + ::DB::renameInCreateQuery(query, context, renaming_settings); + auto create_query = typeid_cast>(query); + return create_query; } static bool isSystemOrTemporaryDatabase(const String & database_name) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 7c5ef5ac04cb..f8976e8b54c1 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -25,6 +25,9 @@ #include #include +#include +#include + namespace DB { @@ -428,6 +431,20 @@ Strings StorageMaterializedView::getDataPaths() const return {}; } +BackupEntries StorageMaterializedView::backupData(ContextPtr context_, const ASTs & partitions_) +{ + if (!hasInnerTable()) + return {}; + return getTargetTable()->backupData(context_, partitions_); +} + +RestoreTaskPtr StorageMaterializedView::restoreData(ContextMutablePtr context_, const ASTs & partitions_, const BackupPtr & backup_, const String & data_path_in_backup_, const StorageRestoreSettings & restore_settings_) +{ + if (!hasInnerTable()) + return {}; + return getTargetTable()->restoreData(context_, partitions_, backup_, data_path_in_backup_, restore_settings_); +} + ActionLock StorageMaterializedView::getActionLock(StorageActionBlockType type) { if (has_inner_table) diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 395560c1ca73..04191731b55e 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -97,6 +97,10 @@ class StorageMaterializedView final : public shared_ptr_helper Date: Thu, 24 Feb 2022 15:33:33 +0300 Subject: [PATCH 18/18] m6 --- src/Storages/MergeTree/MergeTreeData.cpp | 35 ++++++++++++++++++++---- 1 file changed, 29 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3bb8c6fcb29e..f2e63e15d9a0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3655,6 +3655,8 @@ class MergeTreeDataRestoreTask : public IRestoreTask { RestoreTasks restore_part_tasks; Strings part_names = backup->listFiles(data_path_in_backup); + auto lock = lockParts(); + UInt64 max_block_num = increment ? increment->get() : 0; for (const String & part_name : part_names) { const auto part_info = MergeTreePartInfo::tryParsePartName(part_name, storage->format_version); @@ -3665,9 +3667,33 @@ class MergeTreeDataRestoreTask : public IRestoreTask if (!partition_ids.empty() && !partition_ids.contains(part_info->partition_id)) continue; + auto containing_part = getActiveContainingPart(*part_info); + if (containing_part) + { + if (containing_part->info == *part_info) + { + if (containing_part->info.checksums.getTotalChecksumUInt128() == part_info->checksums.getTotalChecksumUInt128()) + continue; + } + else + continue; + } + + if (increment) + max_block_num = std::max({max_block_num, part->info.max_block, part->info.mutation}); + restore_part_tasks.push_back( - std::make_unique(storage, backup, data_path_in_backup, part_name, *part_info, increment)); + std::make_unique(storage, backup, data_path_in_backup, part_name, *part_info)); } + + if (increment) + { + UInt64 old_max_block_num = increment->get(); + while (old_max_block_num < max_block_num) + increment->value.exchange(old_max_block_num, max_block_num); + increment->set(max_block_num); + } + return restore_part_tasks; } @@ -3686,14 +3712,12 @@ class MergeTreeDataRestoreTask : public IRestoreTask const BackupPtr & backup_, const String & data_path_in_backup_, const String & part_name_, - const MergeTreePartInfo & part_info_, - SimpleIncrement * increment_) + const MergeTreePartInfo & part_info_) : storage(storage_) , backup(backup_) , data_path_in_backup(data_path_in_backup_) , part_name(part_name_) , part_info(part_info_) - , increment(increment_) { } @@ -3727,7 +3751,7 @@ class MergeTreeDataRestoreTask : public IRestoreTask auto single_disk_volume = std::make_shared(disk->getName(), disk, 0); auto part = storage->createPart(part_name, part_info, single_disk_volume, relative_temp_part_dir); part->loadColumnsChecksumsIndexes(false, true); - storage->renameTempPartAndAdd(part, increment); + storage->renameTempPartAndReplace(part); return {}; } @@ -3737,7 +3761,6 @@ class MergeTreeDataRestoreTask : public IRestoreTask String data_path_in_backup; String part_name; MergeTreePartInfo part_info; - SimpleIncrement * increment; }; };