diff --git a/src/Access/AccessBackup.cpp b/src/Access/AccessBackup.cpp index 0322ad7457b..bd1344a6f14 100644 --- a/src/Access/AccessBackup.cpp +++ b/src/Access/AccessBackup.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -25,6 +26,7 @@ namespace DB namespace ErrorCodes { + extern const int CANNOT_RESTORE_TABLE; extern const int LOGICAL_ERROR; } @@ -139,7 +141,7 @@ namespace } catch (Exception & e) { - e.addMessage("While parsing " + file_path); + e.addMessage("While parsing " + file_path + " from backup"); throw; } } @@ -225,7 +227,7 @@ namespace } } - AccessRightsElements getRequiredAccessToRestore(const std::unordered_map & entities) + AccessRightsElements getRequiredAccessToRestore(const std::vector> & entities) { AccessRightsElements res; for (const auto & entity : entities | boost::adaptors::map_values) @@ -294,65 +296,78 @@ namespace } } -void backupAccessEntities( - BackupEntriesCollector & backup_entries_collector, + +std::pair makeBackupEntryForAccess( + const std::vector> access_entities, const String & data_path_in_backup, - const AccessControl & access_control, - AccessEntityType type) + size_t counter, + const AccessControl & access_control) { - auto entities = access_control.readAllForBackup(type, backup_entries_collector.getBackupSettings()); - auto dependencies = readDependenciesNamesAndTypes(findDependencies(entities), access_control); + auto dependencies = readDependenciesNamesAndTypes(findDependencies(access_entities), access_control); AccessEntitiesInBackup ab; - boost::range::copy(entities, std::inserter(ab.entities, ab.entities.end())); + boost::range::copy(access_entities, std::inserter(ab.entities, ab.entities.end())); ab.dependencies = std::move(dependencies); - backup_entries_collector.addBackupEntry(fs::path{data_path_in_backup} / "access.txt", ab.toBackupEntry()); + String filename = fmt::format("access{:02}.txt", counter + 1); /// access01.txt, access02.txt, ... + String file_path_in_backup = fs::path{data_path_in_backup} / filename; + return {file_path_in_backup, ab.toBackupEntry()}; } -AccessRestoreTask::AccessRestoreTask( - const BackupPtr & backup_, const RestoreSettings & restore_settings_, std::shared_ptr restore_coordination_) - : backup(backup_), restore_settings(restore_settings_), restore_coordination(restore_coordination_) +AccessRestorerFromBackup::AccessRestorerFromBackup( + const BackupPtr & backup_, const RestoreSettings & restore_settings_) + : backup(backup_), allow_unresolved_access_dependencies(restore_settings_.allow_unresolved_access_dependencies) { } -AccessRestoreTask::~AccessRestoreTask() = default; +AccessRestorerFromBackup::~AccessRestorerFromBackup() = default; -void AccessRestoreTask::addDataPath(const String & data_path) +void AccessRestorerFromBackup::addDataPath(const String & data_path, const QualifiedTableName & table_name_for_logs) { if (!data_paths.emplace(data_path).second) return; - String file_path = fs::path{data_path} / "access.txt"; - auto backup_entry = backup->readFile(file_path); - auto ab = AccessEntitiesInBackup::fromBackupEntry(*backup_entry, file_path); + fs::path data_path_in_backup_fs = data_path; + Strings filenames = backup->listFiles(data_path); + if (filenames.empty()) + return; + + for (const String & filename : filenames) + { + if (!filename.starts_with("access") || !filename.ends_with(".txt")) + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File name {} doesn't match the wildcard \"access*.txt\"", + table_name_for_logs.getFullName(), String{data_path_in_backup_fs / filename}); + } + + ::sort(filenames.begin(), filenames.end()); + + for (const String & filename : filenames) + { + String filepath_in_backup = data_path_in_backup_fs / filename; + auto backup_entry = backup->readFile(filepath_in_backup); + auto ab = AccessEntitiesInBackup::fromBackupEntry(*backup_entry, filepath_in_backup); + + boost::range::copy(ab.entities, std::back_inserter(entities)); + boost::range::copy(ab.dependencies, std::inserter(dependencies, dependencies.end())); + } - boost::range::copy(ab.entities, std::inserter(entities, entities.end())); - boost::range::copy(ab.dependencies, std::inserter(dependencies, dependencies.end())); for (const auto & id : entities | boost::adaptors::map_keys) dependencies.erase(id); } -bool AccessRestoreTask::hasDataPath(const String & data_path) const -{ - return data_paths.contains(data_path); -} - -AccessRightsElements AccessRestoreTask::getRequiredAccess() const +AccessRightsElements AccessRestorerFromBackup::getRequiredAccess() const { return getRequiredAccessToRestore(entities); } -void AccessRestoreTask::restore(AccessControl & access_control) const +std::vector> AccessRestorerFromBackup::getAccessEntities(const AccessControl & access_control) const { - auto old_to_new_ids = resolveDependencies(dependencies, access_control, restore_settings.allow_unresolved_access_dependencies); + auto new_entities = entities; - std::vector> new_entities; - boost::range::copy(entities, std::back_inserter(new_entities)); + auto old_to_new_ids = resolveDependencies(dependencies, access_control, allow_unresolved_access_dependencies); generateRandomIDs(new_entities, old_to_new_ids); - replaceDependencies(new_entities, old_to_new_ids); - access_control.insertFromBackup(new_entities, restore_settings, restore_coordination); + return new_entities; } } diff --git a/src/Access/AccessBackup.h b/src/Access/AccessBackup.h index 5c70e268eae..74f889e2c00 100644 --- a/src/Access/AccessBackup.h +++ b/src/Access/AccessBackup.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -9,46 +9,45 @@ namespace DB { class AccessControl; enum class AccessEntityType; -class BackupEntriesCollector; -class RestorerFromBackup; -class IBackup; -using BackupPtr = std::shared_ptr; -class IRestoreCoordination; struct IAccessEntity; using AccessEntityPtr = std::shared_ptr; class AccessRightsElements; +class IBackup; +using BackupPtr = std::shared_ptr; +class IBackupEntry; +using BackupEntryPtr = std::shared_ptr; +struct RestoreSettings; +struct QualifiedTableName; /// Makes a backup of access entities of a specified type. -void backupAccessEntities( - BackupEntriesCollector & backup_entries_collector, +std::pair makeBackupEntryForAccess( + const std::vector> access_entities, const String & data_path_in_backup, - const AccessControl & access_control, - AccessEntityType type); + size_t counter, + const AccessControl & access_control); + /// Restores access entities from a backup. -class AccessRestoreTask +class AccessRestorerFromBackup { public: - AccessRestoreTask( - const BackupPtr & backup_, const RestoreSettings & restore_settings_, std::shared_ptr restore_coordination_); - ~AccessRestoreTask(); + AccessRestorerFromBackup(const BackupPtr & backup_, const RestoreSettings & restore_settings_); + ~AccessRestorerFromBackup(); /// Adds a data path to loads access entities from. - void addDataPath(const String & data_path); - bool hasDataPath(const String & data_path) const; + void addDataPath(const String & data_path, const QualifiedTableName & table_name_for_logs); /// Checks that the current user can do restoring. AccessRightsElements getRequiredAccess() const; /// Inserts all access entities loaded from all the paths added by addDataPath(). - void restore(AccessControl & access_control) const; + std::vector> getAccessEntities(const AccessControl & access_control) const; private: BackupPtr backup; - RestoreSettings restore_settings; - std::shared_ptr restore_coordination; - std::unordered_map entities; + bool allow_unresolved_access_dependencies = false; + std::vector> entities; std::unordered_map> dependencies; std::unordered_set data_paths; }; diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index aa58044a6b0..7152820b5bc 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -459,20 +459,9 @@ UUID AccessControl::authenticate(const Credentials & credentials, const Poco::Ne } } -void AccessControl::backup(BackupEntriesCollector & backup_entries_collector, AccessEntityType type, const String & data_path_in_backup) const +void AccessControl::restoreFromBackup(RestorerFromBackup & restorer) { - backupAccessEntities(backup_entries_collector, data_path_in_backup, *this, type); -} - -void AccessControl::restore(RestorerFromBackup & restorer, const String & data_path_in_backup) -{ - /// The restorer must already know about `data_path_in_backup`, but let's check. - restorer.checkPathInBackupToRestoreAccess(data_path_in_backup); -} - -void AccessControl::insertFromBackup(const std::vector> & entities_from_backup, const RestoreSettings & restore_settings, std::shared_ptr restore_coordination) -{ - MultipleAccessStorage::insertFromBackup(entities_from_backup, restore_settings, restore_coordination); + MultipleAccessStorage::restoreFromBackup(restorer); changes_notifier->sendNotifications(); } diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index 90ad2895122..22ff0a488f7 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -42,8 +42,6 @@ class ClientInfo; class ExternalAuthenticators; class AccessChangesNotifier; struct Settings; -class BackupEntriesCollector; -class RestorerFromBackup; /// Manages access control entities. @@ -121,8 +119,7 @@ public: UUID authenticate(const Credentials & credentials, const Poco::Net::IPAddress & address) const; /// Makes a backup of access entities. - void backup(BackupEntriesCollector & backup_entries_collector, AccessEntityType type, const String & data_path_in_backup) const; - static void restore(RestorerFromBackup & restorer, const String & data_path_in_backup); + void restoreFromBackup(RestorerFromBackup & restorer) override; void setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config); @@ -198,8 +195,6 @@ public: /// Gets manager of notifications. AccessChangesNotifier & getChangesNotifier(); - void insertFromBackup(const std::vector> & entities_from_backup, const RestoreSettings & restore_settings, std::shared_ptr restore_coordination) override; - private: class ContextAccessCache; class CustomSettingsPrefixes; diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index 231e325196d..994abc7b53a 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -650,19 +651,24 @@ void DiskAccessStorage::deleteAccessEntityOnDisk(const UUID & id) const } -void DiskAccessStorage::insertFromBackup( - const std::vector> & entities_from_backup, - const RestoreSettings & restore_settings, - std::shared_ptr) +void DiskAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) { if (!isRestoreAllowed()) throwRestoreNotAllowed(); - bool replace_if_exists = (restore_settings.create_access == RestoreAccessCreationMode::kReplace); - bool throw_if_exists = (restore_settings.create_access == RestoreAccessCreationMode::kCreate); + auto entities = restorer.getAccessEntitiesToRestore(); + if (entities.empty()) + return; - for (const auto & [id, entity] : entities_from_backup) - insertWithID(id, entity, replace_if_exists, throw_if_exists); + auto create_access = restorer.getRestoreSettings().create_access; + bool replace_if_exists = (create_access == RestoreAccessCreationMode::kReplace); + bool throw_if_exists = (create_access == RestoreAccessCreationMode::kCreate); + + restorer.addDataRestoreTask([this, entities = std::move(entities), replace_if_exists, throw_if_exists] + { + for (const auto & [id, entity] : entities) + insertWithID(id, entity, replace_if_exists, throw_if_exists); + }); } } diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h index 1bdefbf82f9..d3bd61ff353 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -30,7 +30,7 @@ public: bool exists(const UUID & id) const override; bool isBackupAllowed() const override { return backup_allowed; } - void insertFromBackup(const std::vector> & entities_from_backup, const RestoreSettings & restore_settings, std::shared_ptr restore_coordination) override; + void restoreFromBackup(RestorerFromBackup & restorer) override; private: std::optional findImpl(AccessEntityType type, const String & name) const override; diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 476b1674ce1..fa9c78816c7 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -2,9 +2,12 @@ #include #include #include +#include +#include #include #include #include +#include #include #include #include @@ -520,26 +523,30 @@ bool IAccessStorage::isAddressAllowed(const User & user, const Poco::Net::IPAddr } -bool IAccessStorage::isRestoreAllowed() const -{ - return isBackupAllowed() && !isReadOnly(); -} - -std::vector> IAccessStorage::readAllForBackup(AccessEntityType type, const BackupSettings &) const +void IAccessStorage::backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const { if (!isBackupAllowed()) throwBackupNotAllowed(); - auto res = readAllWithIDs(type); - boost::range::remove_erase_if(res, [](const std::pair & x) { return !x.second->isBackupAllowed(); }); - return res; + auto entities = readAllWithIDs(type); + boost::range::remove_erase_if(entities, [](const std::pair & x) { return !x.second->isBackupAllowed(); }); + + auto backup_entry = makeBackupEntryForAccess( + entities, + data_path_in_backup, + backup_entries_collector.getAccessCounter(type), + backup_entries_collector.getContext()->getAccessControl()); + + backup_entries_collector.addBackupEntry(backup_entry); } -void IAccessStorage::insertFromBackup(const std::vector> &, const RestoreSettings &, std::shared_ptr) + +void IAccessStorage::restoreFromBackup(RestorerFromBackup &) { if (!isRestoreAllowed()) throwRestoreNotAllowed(); - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "insertFromBackup() is not implemented in {}", getStorageType()); + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "restoreFromBackup() is not implemented in {}", getStorageType()); } diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index 7b43309204d..394d3ed6358 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -18,9 +18,8 @@ struct User; class Credentials; class ExternalAuthenticators; enum class AuthenticationType; -struct BackupSettings; -struct RestoreSettings; -class IRestoreCoordination; +class BackupEntriesCollector; +class RestorerFromBackup; /// Contains entities, i.e. instances of classes derived from IAccessEntity. /// The implementations of this class MUST be thread-safe. @@ -158,11 +157,11 @@ public: /// Returns true if this storage can be stored to or restored from a backup. virtual bool isBackupAllowed() const { return false; } - virtual bool isRestoreAllowed() const; + virtual bool isRestoreAllowed() const { return isBackupAllowed() && !isReadOnly(); } /// Makes a backup of this access storage. - virtual std::vector> readAllForBackup(AccessEntityType type, const BackupSettings & backup_settings) const; - virtual void insertFromBackup(const std::vector> & entities_from_backup, const RestoreSettings & restore_settings, std::shared_ptr restore_coordination); + virtual void backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const; + virtual void restoreFromBackup(RestorerFromBackup & restorer); protected: virtual std::optional findImpl(AccessEntityType type, const String & name) const = 0; diff --git a/src/Access/MemoryAccessStorage.cpp b/src/Access/MemoryAccessStorage.cpp index ad877e263ad..60669532e25 100644 --- a/src/Access/MemoryAccessStorage.cpp +++ b/src/Access/MemoryAccessStorage.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -272,19 +273,24 @@ void MemoryAccessStorage::setAll(const std::vector> & entities_from_backup, - const RestoreSettings & restore_settings, - std::shared_ptr) +void MemoryAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) { if (!isRestoreAllowed()) throwRestoreNotAllowed(); - bool replace_if_exists = (restore_settings.create_access == RestoreAccessCreationMode::kReplace); - bool throw_if_exists = (restore_settings.create_access == RestoreAccessCreationMode::kCreate); + auto entities = restorer.getAccessEntitiesToRestore(); + if (entities.empty()) + return; - for (const auto & [id, entity] : entities_from_backup) - insertWithID(id, entity, replace_if_exists, throw_if_exists); + auto create_access = restorer.getRestoreSettings().create_access; + bool replace_if_exists = (create_access == RestoreAccessCreationMode::kReplace); + bool throw_if_exists = (create_access == RestoreAccessCreationMode::kCreate); + + restorer.addDataRestoreTask([this, entities = std::move(entities), replace_if_exists, throw_if_exists] + { + for (const auto & [id, entity] : entities) + insertWithID(id, entity, replace_if_exists, throw_if_exists); + }); } } diff --git a/src/Access/MemoryAccessStorage.h b/src/Access/MemoryAccessStorage.h index aa4cd08252c..5c8d33ed443 100644 --- a/src/Access/MemoryAccessStorage.h +++ b/src/Access/MemoryAccessStorage.h @@ -29,7 +29,7 @@ public: bool exists(const UUID & id) const override; bool isBackupAllowed() const override { return backup_allowed; } - void insertFromBackup(const std::vector> & entities_from_backup, const RestoreSettings & restore_settings, std::shared_ptr restore_coordination) override; + void restoreFromBackup(RestorerFromBackup & restorer) override; private: std::optional findImpl(AccessEntityType type, const String & name) const override; diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index 6f654f68e57..e7151cc7b4b 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -383,40 +383,38 @@ bool MultipleAccessStorage::isRestoreAllowed() const } -std::vector> MultipleAccessStorage::readAllForBackup(AccessEntityType type, const BackupSettings & backup_settings) const +void MultipleAccessStorage::backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const { - std::vector> res; auto storages = getStoragesInternal(); - size_t count = 0; + bool allowed = false; for (const auto & storage : *storages) { if (storage->isBackupAllowed()) { - insertAtEnd(res, storage->readAllForBackup(type, backup_settings)); - ++count; + storage->backup(backup_entries_collector, data_path_in_backup, type); + allowed = true; } } - if (!count) + if (!allowed) throwBackupNotAllowed(); - - return res; } - -void MultipleAccessStorage::insertFromBackup(const std::vector> & entities_from_backup, const RestoreSettings & restore_settings, std::shared_ptr restore_coordination) +void MultipleAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) { auto storages = getStoragesInternal(); + for (const auto & storage : *storages) { if (storage->isRestoreAllowed()) { - storage->insertFromBackup(entities_from_backup, restore_settings, restore_coordination); + storage->restoreFromBackup(restorer); return; } } - throwRestoreNotAllowed(); + + throwBackupNotAllowed(); } } diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h index 2eacdafd3f3..58cf09fd0ff 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -45,8 +45,8 @@ public: bool isBackupAllowed() const override; bool isRestoreAllowed() const override; - std::vector> readAllForBackup(AccessEntityType type, const BackupSettings & backup_settings) const override; - void insertFromBackup(const std::vector> & entities_from_backup, const RestoreSettings & restore_settings, std::shared_ptr restore_coordination) override; + void backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const override; + void restoreFromBackup(RestorerFromBackup & restorer) override; protected: std::optional findImpl(AccessEntityType type, const String & name) const override; diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index 6a9d716c2f9..f6c8d0a7153 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -2,10 +2,14 @@ #include #include #include +#include +#include +#include #include +#include #include #include -#include +#include #include #include #include @@ -13,6 +17,7 @@ #include #include #include +#include namespace DB @@ -613,19 +618,64 @@ AccessEntityPtr ReplicatedAccessStorage::readImpl(const UUID & id, bool throw_if return entry.entity; } -void ReplicatedAccessStorage::insertFromBackup(const std::vector> & entities_from_backup, const RestoreSettings & restore_settings, std::shared_ptr restore_coordination) + +void ReplicatedAccessStorage::backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const +{ + if (!isBackupAllowed()) + throwBackupNotAllowed(); + + auto entities = readAllWithIDs(type); + boost::range::remove_erase_if(entities, [](const std::pair & x) { return !x.second->isBackupAllowed(); }); + + auto backup_entry_with_path = makeBackupEntryForAccess( + entities, + data_path_in_backup, + backup_entries_collector.getAccessCounter(type), + backup_entries_collector.getContext()->getAccessControl()); + + auto backup_coordination = backup_entries_collector.getBackupCoordination(); + backup_coordination->addReplicatedAccessPath(zookeeper_path, backup_entry_with_path.first); + String current_host_id = backup_entries_collector.getBackupSettings().host_id; + backup_coordination->setReplicatedAccessHost(zookeeper_path, current_host_id); + + backup_entries_collector.addPostTask( + [backup_entry = backup_entry_with_path.second, + zookeeper_path = zookeeper_path, + current_host_id, + &backup_entries_collector, + backup_coordination] + { + if (current_host_id != backup_coordination->getReplicatedAccessHost(zookeeper_path)) + return; + + for (const String & path : backup_coordination->getReplicatedAccessPaths(zookeeper_path)) + backup_entries_collector.addBackupEntry(path, backup_entry); + }); +} + + +void ReplicatedAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) { if (!isRestoreAllowed()) throwRestoreNotAllowed(); + auto restore_coordination = restorer.getRestoreCoordination(); if (!restore_coordination->acquireReplicatedAccessStorage(zookeeper_path)) return; - bool replace_if_exists = (restore_settings.create_access == RestoreAccessCreationMode::kReplace); - bool throw_if_exists = (restore_settings.create_access == RestoreAccessCreationMode::kCreate); + auto entities = restorer.getAccessEntitiesToRestore(); + if (entities.empty()) + return; - for (const auto & [id, entity] : entities_from_backup) - insertWithID(id, entity, replace_if_exists, throw_if_exists); + auto create_access = restorer.getRestoreSettings().create_access; + bool replace_if_exists = (create_access == RestoreAccessCreationMode::kReplace); + bool throw_if_exists = (create_access == RestoreAccessCreationMode::kCreate); + + restorer.addDataRestoreTask([this, entities = std::move(entities), replace_if_exists, throw_if_exists] + { + for (const auto & [id, entity] : entities) + insertWithID(id, entity, replace_if_exists, throw_if_exists); + }); } } diff --git a/src/Access/ReplicatedAccessStorage.h b/src/Access/ReplicatedAccessStorage.h index 7cccdc1793f..6311e2ac7c0 100644 --- a/src/Access/ReplicatedAccessStorage.h +++ b/src/Access/ReplicatedAccessStorage.h @@ -38,7 +38,8 @@ public: bool exists(const UUID & id) const override; bool isBackupAllowed() const override { return backup_allowed; } - void insertFromBackup(const std::vector> & entities_from_backup, const RestoreSettings & restore_settings, std::shared_ptr restore_coordination) override; + void backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const override; + void restoreFromBackup(RestorerFromBackup & restorer) override; private: String zookeeper_path; diff --git a/src/Backups/BackupCoordinationDistributed.cpp b/src/Backups/BackupCoordinationDistributed.cpp index 945239482fc..5b932229e71 100644 --- a/src/Backups/BackupCoordinationDistributed.cpp +++ b/src/Backups/BackupCoordinationDistributed.cpp @@ -131,7 +131,7 @@ namespace BackupCoordinationDistributed::BackupCoordinationDistributed(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_) : zookeeper_path(zookeeper_path_) , get_zookeeper(get_zookeeper_) - , stage_sync(zookeeper_path_ + "/stage", get_zookeeper_, &Poco::Logger::get("BackupCoordination")) + , status_sync(zookeeper_path_ + "/status", get_zookeeper_, &Poco::Logger::get("BackupCoordination")) { createRootNodes(); } @@ -145,6 +145,8 @@ void BackupCoordinationDistributed::createRootNodes() zookeeper->createIfNotExists(zookeeper_path, ""); zookeeper->createIfNotExists(zookeeper_path + "/repl_part_names", ""); zookeeper->createIfNotExists(zookeeper_path + "/repl_data_paths", ""); + zookeeper->createIfNotExists(zookeeper_path + "/repl_access_host", ""); + zookeeper->createIfNotExists(zookeeper_path + "/repl_access_paths", ""); zookeeper->createIfNotExists(zookeeper_path + "/file_names", ""); zookeeper->createIfNotExists(zookeeper_path + "/file_infos", ""); zookeeper->createIfNotExists(zookeeper_path + "/archive_suffixes", ""); @@ -157,19 +159,24 @@ void BackupCoordinationDistributed::removeAllNodes() } -void BackupCoordinationDistributed::syncStage(const String & current_host, int new_stage, const Strings & wait_hosts, std::chrono::seconds timeout) +void BackupCoordinationDistributed::setStatus(const String & current_host, const String & new_status, const String & message) { - stage_sync.syncStage(current_host, new_stage, wait_hosts, timeout); + status_sync.set(current_host, new_status, message); } -void BackupCoordinationDistributed::syncStageError(const String & current_host, const String & error_message) +Strings BackupCoordinationDistributed::setStatusAndWait(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts) { - stage_sync.syncStageError(current_host, error_message); + return status_sync.setAndWait(current_host, new_status, message, all_hosts); +} + +Strings BackupCoordinationDistributed::setStatusAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, UInt64 timeout_ms) +{ + return status_sync.setAndWaitFor(current_host, new_status, message, all_hosts, timeout_ms); } void BackupCoordinationDistributed::addReplicatedPartNames( - const String & table_zk_path, + const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) @@ -181,39 +188,39 @@ void BackupCoordinationDistributed::addReplicatedPartNames( } auto zookeeper = get_zookeeper(); - String path = zookeeper_path + "/repl_part_names/" + escapeForFileName(table_zk_path); + String path = zookeeper_path + "/repl_part_names/" + escapeForFileName(table_shared_id); zookeeper->createIfNotExists(path, ""); path += "/" + escapeForFileName(replica_name); zookeeper->create(path, ReplicatedPartNames::serialize(part_names_and_checksums, table_name_for_logs), zkutil::CreateMode::Persistent); } -Strings BackupCoordinationDistributed::getReplicatedPartNames(const String & table_zk_path, const String & replica_name) const +Strings BackupCoordinationDistributed::getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const { std::lock_guard lock{mutex}; prepareReplicatedPartNames(); - return replicated_part_names->getPartNames(table_zk_path, replica_name); + return replicated_part_names->getPartNames(table_shared_id, replica_name); } void BackupCoordinationDistributed::addReplicatedDataPath( - const String & table_zk_path, const String & data_path) + const String & table_shared_id, const String & data_path) { auto zookeeper = get_zookeeper(); - String path = zookeeper_path + "/repl_data_paths/" + escapeForFileName(table_zk_path); + String path = zookeeper_path + "/repl_data_paths/" + escapeForFileName(table_shared_id); + zookeeper->createIfNotExists(path, ""); + path += "/" + escapeForFileName(data_path); zookeeper->createIfNotExists(path, ""); - path += "/"; - zookeeper->create(path, data_path, zkutil::CreateMode::PersistentSequential); } -Strings BackupCoordinationDistributed::getReplicatedDataPaths(const String & table_zk_path) const +Strings BackupCoordinationDistributed::getReplicatedDataPaths(const String & table_shared_id) const { auto zookeeper = get_zookeeper(); - String path = zookeeper_path + "/repl_data_paths/" + escapeForFileName(table_zk_path); + String path = zookeeper_path + "/repl_data_paths/" + escapeForFileName(table_shared_id); Strings children = zookeeper->getChildren(path); Strings data_paths; data_paths.reserve(children.size()); for (const String & child : children) - data_paths.push_back(zookeeper->get(path + "/" + child)); + data_paths.push_back(unescapeForFileName(child)); return data_paths; } @@ -240,6 +247,47 @@ void BackupCoordinationDistributed::prepareReplicatedPartNames() const } +void BackupCoordinationDistributed::addReplicatedAccessPath(const String & access_zk_path, const String & file_path) +{ + auto zookeeper = get_zookeeper(); + String path = zookeeper_path + "/repl_access_paths/" + escapeForFileName(access_zk_path); + zookeeper->createIfNotExists(path, ""); + path += "/" + escapeForFileName(file_path); + zookeeper->createIfNotExists(path, ""); +} + +Strings BackupCoordinationDistributed::getReplicatedAccessPaths(const String & access_zk_path) const +{ + auto zookeeper = get_zookeeper(); + String path = zookeeper_path + "/repl_access_paths/" + escapeForFileName(access_zk_path); + Strings children = zookeeper->getChildren(path); + Strings file_paths; + file_paths.reserve(children.size()); + for (const String & child : children) + file_paths.push_back(unescapeForFileName(child)); + return file_paths; +} + +void BackupCoordinationDistributed::setReplicatedAccessHost(const String & access_zk_path, const String & host_id) +{ + auto zookeeper = get_zookeeper(); + String path = zookeeper_path + "/repl_access_host/" + escapeForFileName(access_zk_path); + auto code = zookeeper->tryCreate(path, host_id, zkutil::CreateMode::Persistent); + if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) + throw zkutil::KeeperException(code, path); + + if (code == Coordination::Error::ZNODEEXISTS) + zookeeper->set(path, host_id); +} + +String BackupCoordinationDistributed::getReplicatedAccessHost(const String & access_zk_path) const +{ + auto zookeeper = get_zookeeper(); + String path = zookeeper_path + "/repl_access_host/" + escapeForFileName(access_zk_path); + return zookeeper->get(path); +} + + void BackupCoordinationDistributed::addFileInfo(const FileInfo & file_info, bool & is_data_file_required) { auto zookeeper = get_zookeeper(); diff --git a/src/Backups/BackupCoordinationDistributed.h b/src/Backups/BackupCoordinationDistributed.h index 2872e1f3ae4..813132bd0b8 100644 --- a/src/Backups/BackupCoordinationDistributed.h +++ b/src/Backups/BackupCoordinationDistributed.h @@ -14,19 +14,26 @@ public: BackupCoordinationDistributed(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_); ~BackupCoordinationDistributed() override; - void syncStage(const String & current_host, int new_stage, const Strings & wait_hosts, std::chrono::seconds timeout) override; - void syncStageError(const String & current_host, const String & error_message) override; + void setStatus(const String & current_host, const String & new_status, const String & message) override; + Strings setStatusAndWait(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts) override; + Strings setStatusAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, UInt64 timeout_ms) override; void addReplicatedPartNames( - const String & table_zk_path, + const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) override; - Strings getReplicatedPartNames(const String & table_zk_path, const String & replica_name) const override; + Strings getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const override; - void addReplicatedDataPath(const String & table_zk_path, const String & data_path) override; - Strings getReplicatedDataPaths(const String & table_zk_path) const override; + void addReplicatedDataPath(const String & table_shared_id, const String & data_path) override; + Strings getReplicatedDataPaths(const String & table_shared_id) const override; + + void addReplicatedAccessPath(const String & access_zk_path, const String & file_path) override; + Strings getReplicatedAccessPaths(const String & access_zk_path) const override; + + void setReplicatedAccessHost(const String & access_zk_path, const String & host_id) override; + String getReplicatedAccessHost(const String & access_zk_path) const override; void addFileInfo(const FileInfo & file_info, bool & is_data_file_required) override; void updateFileInfo(const FileInfo & file_info) override; @@ -51,7 +58,7 @@ private: const String zookeeper_path; const zkutil::GetZooKeeper get_zookeeper; - BackupCoordinationStageSync stage_sync; + BackupCoordinationStatusSync status_sync; mutable std::mutex mutex; mutable std::optional replicated_part_names; diff --git a/src/Backups/BackupCoordinationHelpers.cpp b/src/Backups/BackupCoordinationHelpers.cpp index 9528f888770..7f570ba9c85 100644 --- a/src/Backups/BackupCoordinationHelpers.cpp +++ b/src/Backups/BackupCoordinationHelpers.cpp @@ -157,7 +157,7 @@ BackupCoordinationReplicatedPartNames::BackupCoordinationReplicatedPartNames() = BackupCoordinationReplicatedPartNames::~BackupCoordinationReplicatedPartNames() = default; void BackupCoordinationReplicatedPartNames::addPartNames( - const String & table_zk_path, + const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) @@ -165,7 +165,7 @@ void BackupCoordinationReplicatedPartNames::addPartNames( if (part_names_prepared) throw Exception(ErrorCodes::LOGICAL_ERROR, "addPartNames() must not be called after getPartNames()"); - auto & table_info = table_infos[table_zk_path]; + auto & table_info = table_infos[table_shared_id]; if (!table_info.covered_parts_finder) table_info.covered_parts_finder = std::make_unique(table_name_for_logs); @@ -207,10 +207,10 @@ void BackupCoordinationReplicatedPartNames::addPartNames( } } -Strings BackupCoordinationReplicatedPartNames::getPartNames(const String & table_zk_path, const String & replica_name) const +Strings BackupCoordinationReplicatedPartNames::getPartNames(const String & table_shared_id, const String & replica_name) const { preparePartNames(); - auto it = table_infos.find(table_zk_path); + auto it = table_infos.find(table_shared_id); if (it == table_infos.end()) return {}; const auto & replicas_parts = it->second.replicas_parts; @@ -243,7 +243,7 @@ void BackupCoordinationReplicatedPartNames::preparePartNames() const /// Helps to wait until all hosts come to a specified stage. -BackupCoordinationStageSync::BackupCoordinationStageSync(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, Poco::Logger * log_) +BackupCoordinationStatusSync::BackupCoordinationStatusSync(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, Poco::Logger * log_) : zookeeper_path(zookeeper_path_) , get_zookeeper(get_zookeeper_) , log(log_) @@ -251,63 +251,78 @@ BackupCoordinationStageSync::BackupCoordinationStageSync(const String & zookeepe createRootNodes(); } -void BackupCoordinationStageSync::createRootNodes() +void BackupCoordinationStatusSync::createRootNodes() { auto zookeeper = get_zookeeper(); zookeeper->createAncestors(zookeeper_path); zookeeper->createIfNotExists(zookeeper_path, ""); } -void BackupCoordinationStageSync::syncStage(const String & current_host, int new_stage, const Strings & wait_hosts, std::chrono::seconds timeout) +void BackupCoordinationStatusSync::set(const String & current_host, const String & new_status, const String & message) { - /// Put new stage to ZooKeeper. - auto zookeeper = get_zookeeper(); - zookeeper->createIfNotExists(zookeeper_path + "/" + current_host + "|" + std::to_string(new_stage), ""); + setImpl(current_host, new_status, message, {}, {}); +} - if (wait_hosts.empty() || ((wait_hosts.size() == 1) && (wait_hosts.front() == current_host))) - return; +Strings BackupCoordinationStatusSync::setAndWait(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts) +{ + return setImpl(current_host, new_status, message, all_hosts, {}); +} + +Strings BackupCoordinationStatusSync::setAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, UInt64 timeout_ms) +{ + return setImpl(current_host, new_status, message, all_hosts, timeout_ms); +} + +Strings BackupCoordinationStatusSync::setImpl(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, const std::optional & timeout_ms) +{ + /// Put new status to ZooKeeper. + auto zookeeper = get_zookeeper(); + zookeeper->createIfNotExists(zookeeper_path + "/" + current_host + "|" + new_status, message); + + if (all_hosts.empty() || (new_status == kErrorStatus)) + return {}; + + if ((all_hosts.size() == 1) && (all_hosts.front() == current_host)) + return {message}; /// Wait for other hosts. - /// Current stages of all hosts. + Strings ready_hosts_results; + ready_hosts_results.resize(all_hosts.size()); + + std::map /* index in `ready_hosts_results` */> unready_hosts; + for (size_t i = 0; i != all_hosts.size(); ++i) + unready_hosts[all_hosts[i]].push_back(i); + std::optional host_with_error; std::optional error_message; - std::map> unready_hosts; - for (const String & host : wait_hosts) - unready_hosts.emplace(host, std::optional{}); - /// Process ZooKeeper's nodes and set `all_hosts_ready` or `unready_host` or `error_message`. auto process_zk_nodes = [&](const Strings & zk_nodes) { for (const String & zk_node : zk_nodes) { - if (zk_node == "error") + if (zk_node.starts_with("remove_watch-")) + continue; + + size_t separator_pos = zk_node.find('|'); + if (separator_pos == String::npos) + throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, "Unexpected zk node {}", zookeeper_path + "/" + zk_node); + String host = zk_node.substr(0, separator_pos); + String status = zk_node.substr(separator_pos + 1); + if (status == kErrorStatus) { - String str = zookeeper->get(zookeeper_path + "/" + zk_node); - size_t separator_pos = str.find('|'); - if (separator_pos == String::npos) - throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, "Unexpected value of zk node {}: {}", zookeeper_path + "/" + zk_node, str); - host_with_error = str.substr(0, separator_pos); - error_message = str.substr(separator_pos + 1); + host_with_error = host; + error_message = zookeeper->get(zookeeper_path + "/" + zk_node); return; } - else if (!zk_node.starts_with("remove_watch-")) + auto it = unready_hosts.find(host); + if ((it != unready_hosts.end()) && (status == new_status)) { - size_t separator_pos = zk_node.find('|'); - if (separator_pos == String::npos) - throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, "Unexpected zk node {}", zookeeper_path + "/" + zk_node); - String host = zk_node.substr(0, separator_pos); - int found_stage = parseFromString(zk_node.substr(separator_pos + 1)); - auto it = unready_hosts.find(host); - if (it != unready_hosts.end()) - { - auto & stage = it->second; - if (!stage || (stage < found_stage)) - stage = found_stage; - if (stage >= new_stage) - unready_hosts.erase(it); - } + String result = zookeeper->get(zookeeper_path + "/" + zk_node); + for (size_t i : it->second) + ready_hosts_results[i] = result; + unready_hosts.erase(it); } } }; @@ -324,7 +339,8 @@ void BackupCoordinationStageSync::syncStage(const String & current_host, int new auto watch_triggered = [&] { return !watch_set; }; - bool use_timeout = (timeout.count() >= 0); + bool use_timeout = timeout_ms.has_value(); + std::chrono::milliseconds timeout{timeout_ms.value_or(0)}; std::chrono::steady_clock::time_point start_time = std::chrono::steady_clock::now(); std::chrono::steady_clock::duration elapsed; std::mutex dummy_mutex; @@ -369,12 +385,8 @@ void BackupCoordinationStageSync::syncStage(const String & current_host, int new unready_hosts.begin()->first, to_string(elapsed)); } -} -void BackupCoordinationStageSync::syncStageError(const String & current_host, const String & error_message) -{ - auto zookeeper = get_zookeeper(); - zookeeper->createIfNotExists(zookeeper_path + "/error", current_host + "|" + error_message); + return ready_hosts_results; } } diff --git a/src/Backups/BackupCoordinationHelpers.h b/src/Backups/BackupCoordinationHelpers.h index b0cd0440b98..2e9e4b3cbde 100644 --- a/src/Backups/BackupCoordinationHelpers.h +++ b/src/Backups/BackupCoordinationHelpers.h @@ -24,7 +24,7 @@ public: /// getPartNames(). /// Checksums are used only to control that parts under the same names on different replicas are the same. void addPartNames( - const String & table_zk_path, + const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums); @@ -32,7 +32,7 @@ public: /// Returns the names of the parts which a specified replica of a replicated table should put to the backup. /// This is the same list as it was added by call of the function addPartNames() but without duplications and without /// parts covered by another parts. - Strings getPartNames(const String & table_zk_path, const String & replica_name) const; + Strings getPartNames(const String & table_shared_id, const String & replica_name) const; private: void preparePartNames() const; @@ -52,22 +52,26 @@ private: std::unique_ptr covered_parts_finder; }; - std::map table_infos; /// Should be ordered because we need this map to be in the same order on every replica. + std::map table_infos; /// Should be ordered because we need this map to be in the same order on every replica. mutable bool part_names_prepared = false; }; /// Helps to wait until all hosts come to a specified stage. -class BackupCoordinationStageSync +class BackupCoordinationStatusSync { public: - BackupCoordinationStageSync(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, Poco::Logger * log_); + BackupCoordinationStatusSync(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, Poco::Logger * log_); - void syncStage(const String & current_host, int stage, const Strings & wait_hosts, std::chrono::seconds timeout); - void syncStageError(const String & current_host, const String & error_message); + void set(const String & current_host, const String & new_status, const String & message); + Strings setAndWait(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts); + Strings setAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, UInt64 timeout_ms); + + static constexpr const char * kErrorStatus = "error"; private: void createRootNodes(); + Strings setImpl(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, const std::optional & timeout_ms); String zookeeper_path; zkutil::GetZooKeeper get_zookeeper; diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index 55a3c671a6e..a7d5602ca30 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -13,43 +13,80 @@ using FileInfo = IBackupCoordination::FileInfo; BackupCoordinationLocal::BackupCoordinationLocal() = default; BackupCoordinationLocal::~BackupCoordinationLocal() = default; -void BackupCoordinationLocal::syncStage(const String &, int, const Strings &, std::chrono::seconds) +void BackupCoordinationLocal::setStatus(const String &, const String &, const String &) { } -void BackupCoordinationLocal::syncStageError(const String &, const String &) +Strings BackupCoordinationLocal::setStatusAndWait(const String &, const String &, const String &, const Strings &) { + return {}; } -void BackupCoordinationLocal::addReplicatedPartNames(const String & table_zk_path, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) +Strings BackupCoordinationLocal::setStatusAndWaitFor(const String &, const String &, const String &, const Strings &, UInt64) +{ + return {}; +} + +void BackupCoordinationLocal::addReplicatedPartNames(const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) { std::lock_guard lock{mutex}; - replicated_part_names.addPartNames(table_zk_path, table_name_for_logs, replica_name, part_names_and_checksums); + replicated_part_names.addPartNames(table_shared_id, table_name_for_logs, replica_name, part_names_and_checksums); } -Strings BackupCoordinationLocal::getReplicatedPartNames(const String & table_zk_path, const String & replica_name) const +Strings BackupCoordinationLocal::getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const { std::lock_guard lock{mutex}; - return replicated_part_names.getPartNames(table_zk_path, replica_name); + return replicated_part_names.getPartNames(table_shared_id, replica_name); } -void BackupCoordinationLocal::addReplicatedDataPath(const String & table_zk_path, const String & data_path) +void BackupCoordinationLocal::addReplicatedDataPath(const String & table_shared_id, const String & data_path) { std::lock_guard lock{mutex}; - replicated_data_paths[table_zk_path].push_back(data_path); + replicated_data_paths[table_shared_id].push_back(data_path); } -Strings BackupCoordinationLocal::getReplicatedDataPaths(const String & table_zk_path) const +Strings BackupCoordinationLocal::getReplicatedDataPaths(const String & table_shared_id) const { std::lock_guard lock{mutex}; - auto it = replicated_data_paths.find(table_zk_path); + auto it = replicated_data_paths.find(table_shared_id); if (it == replicated_data_paths.end()) return {}; return it->second; } +void BackupCoordinationLocal::addReplicatedAccessPath(const String & access_zk_path, const String & file_path) +{ + std::lock_guard lock{mutex}; + replicated_access_paths[access_zk_path].push_back(file_path); +} + +Strings BackupCoordinationLocal::getReplicatedAccessPaths(const String & access_zk_path) const +{ + std::lock_guard lock{mutex}; + auto it = replicated_access_paths.find(access_zk_path); + if (it == replicated_access_paths.end()) + return {}; + return it->second; +} + +void BackupCoordinationLocal::setReplicatedAccessHost(const String & access_zk_path, const String & host_id) +{ + std::lock_guard lock{mutex}; + replicated_access_hosts[access_zk_path] = host_id; +} + +String BackupCoordinationLocal::getReplicatedAccessHost(const String & access_zk_path) const +{ + std::lock_guard lock{mutex}; + auto it = replicated_access_hosts.find(access_zk_path); + if (it == replicated_access_hosts.end()) + return {}; + return it->second; +} + + void BackupCoordinationLocal::addFileInfo(const FileInfo & file_info, bool & is_data_file_required) { std::lock_guard lock{mutex}; diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index 6529184c61a..dcd6505a438 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -19,15 +19,22 @@ public: BackupCoordinationLocal(); ~BackupCoordinationLocal() override; - void syncStage(const String & current_host, int stage, const Strings & wait_hosts, std::chrono::seconds timeout) override; - void syncStageError(const String & current_host, const String & error_message) override; + void setStatus(const String & current_host, const String & new_status, const String & message) override; + Strings setStatusAndWait(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts) override; + Strings setStatusAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, UInt64 timeout_ms) override; - void addReplicatedPartNames(const String & table_zk_path, const String & table_name_for_logs, const String & replica_name, + void addReplicatedPartNames(const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) override; - Strings getReplicatedPartNames(const String & table_zk_path, const String & replica_name) const override; + Strings getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const override; - void addReplicatedDataPath(const String & table_zk_path, const String & data_path) override; - Strings getReplicatedDataPaths(const String & table_zk_path) const override; + void addReplicatedDataPath(const String & table_shared_id, const String & data_path) override; + Strings getReplicatedDataPaths(const String & table_shared_id) const override; + + void addReplicatedAccessPath(const String & access_zk_path, const String & file_path) override; + Strings getReplicatedAccessPaths(const String & access_zk_path) const override; + + void setReplicatedAccessHost(const String & access_zk_path, const String & host_id) override; + String getReplicatedAccessHost(const String & access_zk_path) const override; void addFileInfo(const FileInfo & file_info, bool & is_data_file_required) override; void updateFileInfo(const FileInfo & file_info) override; @@ -47,6 +54,8 @@ private: mutable std::mutex mutex; BackupCoordinationReplicatedPartNames replicated_part_names TSA_GUARDED_BY(mutex); std::unordered_map replicated_data_paths TSA_GUARDED_BY(mutex); + std::unordered_map replicated_access_paths TSA_GUARDED_BY(mutex); + std::unordered_map replicated_access_hosts TSA_GUARDED_BY(mutex); std::map file_names TSA_GUARDED_BY(mutex); /// Should be ordered alphabetically, see listFiles(). For empty files we assume checksum = 0. std::map file_infos TSA_GUARDED_BY(mutex); /// Information about files. Without empty files. Strings archive_suffixes TSA_GUARDED_BY(mutex); diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index 322bc00ee3c..d5ed9e0da2b 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -1,14 +1,18 @@ #include #include #include +#include #include +#include #include #include #include #include #include +#include #include #include +#include #include #include #include @@ -21,35 +25,63 @@ namespace DB namespace ErrorCodes { - extern const int CANNOT_COLLECT_OBJECTS_FOR_BACKUP; + extern const int INCONSISTENT_METADATA_FOR_BACKUP; extern const int CANNOT_BACKUP_TABLE; extern const int TABLE_IS_DROPPED; + extern const int UNKNOWN_TABLE; extern const int LOGICAL_ERROR; } - -bool BackupEntriesCollector::TableKey::operator ==(const TableKey & right) const +namespace { - return (name == right.name) && (is_temporary == right.is_temporary); -} + /// Finding all tables and databases which we're going to put to the backup and collecting their metadata. + constexpr const char * kGatheringMetadataStatus = "gathering metadata"; -bool BackupEntriesCollector::TableKey::operator <(const TableKey & right) const -{ - return (name < right.name) || ((name == right.name) && (is_temporary < right.is_temporary)); -} + /// Making temporary hard links and prepare backup entries. + constexpr const char * kExtractingDataFromTablesStatus = "extracting data from tables"; -std::string_view BackupEntriesCollector::toString(Stage stage) -{ - switch (stage) + /// Running special tasks for replicated tables which can also prepare some backup entries. + constexpr const char * kRunningPostTasksStatus = "running post-tasks"; + + /// Writing backup entries to the backup and removing temporary hard links. + constexpr const char * kWritingBackupStatus = "writing backup"; + + /// Error status. + constexpr const char * kErrorStatus = BackupCoordinationStatusSync::kErrorStatus; + + /// Uppercases the first character of a passed string. + String toUpperFirst(const String & str) { - case Stage::kPreparing: return "Preparing"; - case Stage::kFindingTables: return "Finding tables"; - case Stage::kExtractingDataFromTables: return "Extracting data from tables"; - case Stage::kRunningPostTasks: return "Running post tasks"; - case Stage::kWritingBackup: return "Writing backup"; - case Stage::kError: return "Error"; + String res = str; + res[0] = std::toupper(res[0]); + return res; + } + + /// Outputs "table " or "temporary table " + String tableNameWithTypeToString(const String & database_name, const String & table_name, bool first_upper) + { + String str; + if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) + str = fmt::format("temporary table {}", backQuoteIfNeed(table_name)); + else + str = fmt::format("table {}.{}", backQuoteIfNeed(database_name), backQuoteIfNeed(table_name)); + if (first_upper) + str[0] = std::toupper(str[0]); + return str; + } + + /// How long we should sleep after finding an inconsistency error. + std::chrono::milliseconds getSleepTimeAfterInconsistencyError(size_t pass) + { + size_t ms; + if (pass == 1) /* pass is 1-based */ + ms = 0; + else if ((pass % 10) != 1) + ms = 0; + else + ms = 1000; + return std::chrono::milliseconds{ms}; } - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown backup stage: {}", static_cast(stage)); } @@ -57,36 +89,38 @@ BackupEntriesCollector::BackupEntriesCollector( const ASTBackupQuery::Elements & backup_query_elements_, const BackupSettings & backup_settings_, std::shared_ptr backup_coordination_, - const ContextPtr & context_, - std::chrono::seconds timeout_) + const ContextPtr & context_) : backup_query_elements(backup_query_elements_) , backup_settings(backup_settings_) , backup_coordination(backup_coordination_) , context(context_) - , timeout(timeout_) + , consistent_metadata_snapshot_timeout(context->getConfigRef().getUInt64("backups.consistent_metadata_snapshot_timeout", 300000)) , log(&Poco::Logger::get("BackupEntriesCollector")) { } BackupEntriesCollector::~BackupEntriesCollector() = default; -BackupEntries BackupEntriesCollector::getBackupEntries() +BackupEntries BackupEntriesCollector::run() { try { - /// getBackupEntries() must not be called multiple times. - if (current_stage != Stage::kPreparing) + /// run() can be called onle once. + if (!current_status.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Already making backup entries"); - /// Calculate the root path for collecting backup entries, it's either empty or has the format "shards//replicas//". - calculateRootPathInBackup(); + /// Find other hosts working along with us to execute this ON CLUSTER query. + all_hosts + = BackupSettings::Util::filterHostIDs(backup_settings.cluster_host_ids, backup_settings.shard_num, backup_settings.replica_num); /// Do renaming in the create queries according to the renaming config. renaming_map = makeRenamingMapFromBackupQuery(backup_query_elements); + /// Calculate the root path for collecting backup entries, it's either empty or has the format "shards//replicas//". + calculateRootPathInBackup(); + /// Find databases and tables which we're going to put to the backup. - setStage(Stage::kFindingTables); - collectDatabasesAndTablesInfo(); + gatherMetadataAndCheckConsistency(); /// Make backup entries for the definitions of the found databases. makeBackupEntriesForDatabasesDefs(); @@ -95,15 +129,15 @@ BackupEntries BackupEntriesCollector::getBackupEntries() makeBackupEntriesForTablesDefs(); /// Make backup entries for the data of the found tables. - setStage(Stage::kExtractingDataFromTables); + setStatus(kExtractingDataFromTablesStatus); makeBackupEntriesForTablesData(); /// Run all the tasks added with addPostCollectingTask(). - setStage(Stage::kRunningPostTasks); - runPostCollectingTasks(); + setStatus(kRunningPostTasksStatus); + runPostTasks(); /// No more backup entries or tasks are allowed after this point. - setStage(Stage::kWritingBackup); + setStatus(kWritingBackupStatus); return std::move(backup_entries); } @@ -111,7 +145,7 @@ BackupEntries BackupEntriesCollector::getBackupEntries() { try { - setStage(Stage::kError, getCurrentExceptionMessage(false)); + setStatus(kErrorStatus, getCurrentExceptionMessage(false)); } catch (...) { @@ -120,24 +154,34 @@ BackupEntries BackupEntriesCollector::getBackupEntries() } } -void BackupEntriesCollector::setStage(Stage new_stage, const String & error_message) +Strings BackupEntriesCollector::setStatus(const String & new_status, const String & message) { - if (new_stage == Stage::kError) - LOG_ERROR(log, "{} failed with error: {}", toString(current_stage), error_message); - else - LOG_TRACE(log, "{}", toString(new_stage)); - - current_stage = new_stage; - - if (new_stage == Stage::kError) + if (new_status == kErrorStatus) { - backup_coordination->syncStageError(backup_settings.host_id, error_message); + LOG_ERROR(log, "{} failed with error: {}", toUpperFirst(current_status), message); + backup_coordination->setStatus(backup_settings.host_id, new_status, message); + return {}; } else { - auto all_hosts - = BackupSettings::Util::filterHostIDs(backup_settings.cluster_host_ids, backup_settings.shard_num, backup_settings.replica_num); - backup_coordination->syncStage(backup_settings.host_id, static_cast(new_stage), all_hosts, timeout); + LOG_TRACE(log, "{}", toUpperFirst(new_status)); + current_status = new_status; + if (new_status.starts_with(kGatheringMetadataStatus)) + { + auto now = std::chrono::steady_clock::now(); + auto end_of_timeout = std::max(now, consistent_metadata_snapshot_start_time + consistent_metadata_snapshot_timeout); + + return backup_coordination->setStatusAndWaitFor( + backup_settings.host_id, + new_status, + message, + all_hosts, + std::chrono::duration_cast(end_of_timeout - now).count()); + } + else + { + return backup_coordination->setStatusAndWait(backup_settings.host_id, new_status, message, all_hosts); + } } } @@ -156,287 +200,444 @@ void BackupEntriesCollector::calculateRootPathInBackup() } /// Finds databases and tables which we will put to the backup. -void BackupEntriesCollector::collectDatabasesAndTablesInfo() +void BackupEntriesCollector::gatherMetadataAndCheckConsistency() { - bool use_timeout = (timeout.count() >= 0); - auto start_time = std::chrono::steady_clock::now(); + consistent_metadata_snapshot_start_time = std::chrono::steady_clock::now(); + auto end_of_timeout = consistent_metadata_snapshot_start_time + consistent_metadata_snapshot_timeout; + setStatus(fmt::format("{} ({})", kGatheringMetadataStatus, 1)); - int pass = 0; - do + for (size_t pass = 1;; ++pass) { - database_infos.clear(); - table_infos.clear(); - consistent = true; - - /// Collect information about databases and tables specified in the BACKUP query. - for (const auto & element : backup_query_elements) + String new_status = fmt::format("{} ({})", kGatheringMetadataStatus, pass + 1); + std::optional inconsistency_error; + if (tryGatherMetadataAndCompareWithPrevious(inconsistency_error)) { - switch (element.type) + /// Gathered metadata and checked consistency, cool! But we have to check that other hosts cope with that too. + auto all_hosts_results = setStatus(new_status, "consistent"); + + std::optional host_with_inconsistency; + std::optional inconsistency_error_on_other_host; + for (size_t i = 0; i != all_hosts.size(); ++i) { - case ASTBackupQuery::ElementType::TABLE: + if ((i < all_hosts_results.size()) && (all_hosts_results[i] != "consistent")) { - collectTableInfo({element.database_name, element.table_name}, false, element.partitions, true); - break; - } - - case ASTBackupQuery::ElementType::TEMPORARY_TABLE: - { - collectTableInfo({"", element.table_name}, true, element.partitions, true); - break; - } - - case ASTBackupQuery::ElementType::DATABASE: - { - collectDatabaseInfo(element.database_name, element.except_tables, true); - break; - } - - case ASTBackupQuery::ElementType::ALL: - { - collectAllDatabasesInfo(element.except_databases, element.except_tables); + host_with_inconsistency = all_hosts[i]; + inconsistency_error_on_other_host = all_hosts_results[i]; break; } } + + if (!host_with_inconsistency) + break; /// All hosts managed to gather metadata and everything is consistent, so we can go further to writing the backup. + + inconsistency_error = Exception{ + ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, + "Found inconsistency on host {}: {}", + *host_with_inconsistency, + *inconsistency_error_on_other_host}; } - - /// We have to check consistency of collected information to protect from the case when some table or database is - /// renamed during this collecting making the collected information invalid. - checkConsistency(); - - /// Two passes is absolute minimum (see `previous_table_names` & `previous_database_names`). - auto elapsed = std::chrono::steady_clock::now() - start_time; - if (!consistent && (pass >= 2) && use_timeout) + else { - if (elapsed > timeout) - throw Exception( - ErrorCodes::CANNOT_COLLECT_OBJECTS_FOR_BACKUP, - "Couldn't collect tables and databases to make a backup (pass #{}, elapsed {})", - pass, - to_string(elapsed)); + /// Failed to gather metadata or something wasn't consistent. We'll let other hosts know that and try again. + setStatus(new_status, inconsistency_error->displayText()); } + /// Two passes is minimum (we need to compare with table names with previous ones to be sure we don't miss anything). if (pass >= 2) - LOG_WARNING(log, "Couldn't collect tables and databases to make a backup (pass #{}, elapsed {})", pass, to_string(elapsed)); - ++pass; - } while (!consistent); + { + if (std::chrono::steady_clock::now() > end_of_timeout) + inconsistency_error->rethrow(); + else + LOG_WARNING(log, "{}", inconsistency_error->displayText()); + } + + auto sleep_time = getSleepTimeAfterInconsistencyError(pass); + if (sleep_time.count() > 0) + sleepForNanoseconds(std::chrono::duration_cast(sleep_time).count()); + } LOG_INFO(log, "Will backup {} databases and {} tables", database_infos.size(), table_infos.size()); } -void BackupEntriesCollector::collectTableInfo( - const QualifiedTableName & table_name, bool is_temporary_table, const std::optional & partitions, bool throw_if_not_found) +bool BackupEntriesCollector::tryGatherMetadataAndCompareWithPrevious(std::optional & inconsistency_error) { - /// Gather information about the table. - DatabasePtr database; - StoragePtr storage; - TableLockHolder table_lock; - ASTPtr create_table_query; - - TableKey table_key{table_name, is_temporary_table}; - - if (throw_if_not_found) + try { - auto resolved_id = is_temporary_table - ? context->resolveStorageID(StorageID{"", table_name.table}, Context::ResolveExternal) - : context->resolveStorageID(StorageID{table_name.database, table_name.table}, Context::ResolveGlobal); - std::tie(database, storage) = DatabaseCatalog::instance().getDatabaseAndTable(resolved_id, context); - table_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); - create_table_query = storage->getCreateQueryForBackup(*this); + /// Collect information about databases and tables specified in the BACKUP query. + database_infos.clear(); + table_infos.clear(); + gatherDatabasesMetadata(); + gatherTablesMetadata(); } - else + catch (Exception & e) { - auto resolved_id = is_temporary_table - ? context->tryResolveStorageID(StorageID{"", table_name.table}, Context::ResolveExternal) - : context->tryResolveStorageID(StorageID{table_name.database, table_name.table}, Context::ResolveGlobal); - if (!resolved_id.empty()) - std::tie(database, storage) = DatabaseCatalog::instance().tryGetDatabaseAndTable(resolved_id, context); + if (e.code() != ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP) + throw; + inconsistency_error = e; + return false; + } + + /// We have to check consistency of collected information to protect from the case when some table or database is + /// renamed during this collecting making the collected information invalid. + return compareWithPrevious(inconsistency_error); +} + +void BackupEntriesCollector::gatherDatabasesMetadata() +{ + /// Collect information about databases and tables specified in the BACKUP query. + for (const auto & element : backup_query_elements) + { + switch (element.type) + { + case ASTBackupQuery::ElementType::TABLE: + { + gatherDatabaseMetadata( + element.database_name, + /* throw_if_database_not_found= */ true, + /* backup_create_database_query= */ false, + element.table_name, + /* throw_if_table_not_found= */ true, + element.partitions, + /* all_tables= */ false, + /* except_table_names= */ {}); + break; + } + + case ASTBackupQuery::ElementType::TEMPORARY_TABLE: + { + gatherDatabaseMetadata( + DatabaseCatalog::TEMPORARY_DATABASE, + /* throw_if_database_not_found= */ true, + /* backup_create_database_query= */ false, + element.table_name, + /* throw_if_table_not_found= */ true, + element.partitions, + /* all_tables= */ false, + /* except_table_names= */ {}); + break; + } + + case ASTBackupQuery::ElementType::DATABASE: + { + gatherDatabaseMetadata( + element.database_name, + /* throw_if_database_not_found= */ true, + /* backup_create_database_query= */ true, + /* table_name= */ {}, + /* throw_if_table_not_found= */ false, + /* partitions= */ {}, + /* all_tables= */ true, + /* except_table_names= */ element.except_tables); + break; + } + + case ASTBackupQuery::ElementType::ALL: + { + for (const auto & [database_name, database] : DatabaseCatalog::instance().getDatabases()) + { + if (!element.except_databases.contains(database_name)) + { + gatherDatabaseMetadata( + database_name, + /* throw_if_database_not_found= */ false, + /* backup_create_database_query= */ true, + /* table_name= */ {}, + /* throw_if_table_not_found= */ false, + /* partitions= */ {}, + /* all_tables= */ true, + /* except_table_names= */ element.except_tables); + } + } + break; + } + } + } +} + +void BackupEntriesCollector::gatherDatabaseMetadata( + const String & database_name, + bool throw_if_database_not_found, + bool backup_create_database_query, + const std::optional & table_name, + bool throw_if_table_not_found, + const std::optional & partitions, + bool all_tables, + const std::set & except_table_names) +{ + auto it = database_infos.find(database_name); + if (it == database_infos.end()) + { + DatabasePtr database; + if (throw_if_database_not_found) + { + database = DatabaseCatalog::instance().getDatabase(database_name); + } + else + { + database = DatabaseCatalog::instance().tryGetDatabase(database_name); + if (!database) + return; + } + + DatabaseInfo new_database_info; + new_database_info.database = database; + it = database_infos.emplace(database_name, new_database_info).first; + } + + DatabaseInfo & database_info = it->second; + + if (backup_create_database_query && !database_info.create_database_query && (database_name != DatabaseCatalog::TEMPORARY_DATABASE)) + { + ASTPtr create_database_query; + try + { + create_database_query = database_info.database->getCreateDatabaseQuery(); + } + catch (...) + { + throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "Couldn't get a create query for database {}", database_name); + } + + database_info.create_database_query = create_database_query; + const auto & create = create_database_query->as(); + + if (create.getDatabase() != database_name) + throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "Got a create query with unexpected name {} for database {}", backQuoteIfNeed(create.getDatabase()), backQuoteIfNeed(database_name)); + + String new_database_name = renaming_map.getNewDatabaseName(database_name); + database_info.metadata_path_in_backup = root_path_in_backup / "metadata" / (escapeForFileName(new_database_name) + ".sql"); + } + + if (table_name) + { + auto & table_params = database_info.tables[*table_name]; + if (throw_if_table_not_found) + table_params.throw_if_table_not_found = true; + if (partitions) + { + table_params.partitions.emplace(); + insertAtEnd(*table_params.partitions, *partitions); + } + database_info.except_table_names.emplace(*table_name); + } + + if (all_tables) + { + database_info.all_tables = all_tables; + for (const auto & except_table_name : except_table_names) + if (except_table_name.first == database_name) + database_info.except_table_names.emplace(except_table_name.second); + } +} + +void BackupEntriesCollector::gatherTablesMetadata() +{ + table_infos.clear(); + for (const auto & [database_name, database_info] : database_infos) + { + const auto & database = database_info.database; + bool is_temporary_database = (database_name == DatabaseCatalog::TEMPORARY_DATABASE); + + auto filter_by_table_name = [database_info = &database_info](const String & table_name) + { + /// We skip inner tables of materialized views. + if (table_name.starts_with(".inner_id.")) + return false; + + if (database_info->tables.contains(table_name)) + return true; + + if (database_info->all_tables) + return !database_info->except_table_names.contains(table_name); + + return false; + }; + + auto db_tables = database->getTablesForBackup(filter_by_table_name, context); + + std::unordered_set found_table_names; + for (const auto & db_table : db_tables) + { + const auto & create_table_query = db_table.first; + const auto & create = create_table_query->as(); + found_table_names.emplace(create.getTable()); + + if (is_temporary_database && !create.temporary) + throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "Got a non-temporary create query for {}", tableNameWithTypeToString(database_name, create.getTable(), false)); + + if (!is_temporary_database && (create.getDatabase() != database_name)) + throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "Got a create query with unexpected database name {} for {}", backQuoteIfNeed(create.getDatabase()), tableNameWithTypeToString(database_name, create.getTable(), false)); + } + + /// Check that all tables were found. + for (const auto & [table_name, table_info] : database_info.tables) + { + if (table_info.throw_if_table_not_found && !found_table_names.contains(table_name)) + throw Exception(ErrorCodes::UNKNOWN_TABLE, "{} not found", tableNameWithTypeToString(database_name, table_name, true)); + } + + for (const auto & db_table : db_tables) + { + const auto & create_table_query = db_table.first; + const auto & storage = db_table.second; + const auto & create = create_table_query->as(); + String table_name = create.getTable(); + + fs::path metadata_path_in_backup, data_path_in_backup; + auto table_name_in_backup = renaming_map.getNewTableName({database_name, table_name}); + if (table_name_in_backup.database == DatabaseCatalog::TEMPORARY_DATABASE) + { + metadata_path_in_backup = root_path_in_backup / "temporary_tables" / "metadata" / (escapeForFileName(table_name_in_backup.table) + ".sql"); + data_path_in_backup = root_path_in_backup / "temporary_tables" / "data" / escapeForFileName(table_name_in_backup.table); + } + else + { + metadata_path_in_backup + = root_path_in_backup / "metadata" / escapeForFileName(table_name_in_backup.database) / (escapeForFileName(table_name_in_backup.table) + ".sql"); + data_path_in_backup = root_path_in_backup / "data" / escapeForFileName(table_name_in_backup.database) + / escapeForFileName(table_name_in_backup.table); + } + + /// Add information to `table_infos`. + auto & res_table_info = table_infos[QualifiedTableName{database_name, table_name}]; + res_table_info.database = database; + res_table_info.storage = storage; + res_table_info.create_table_query = create_table_query; + res_table_info.metadata_path_in_backup = metadata_path_in_backup; + res_table_info.data_path_in_backup = data_path_in_backup; + + if (!backup_settings.structure_only) + { + auto it = database_info.tables.find(table_name); + if (it != database_info.tables.end()) + { + const auto & partitions = it->second.partitions; + if (partitions && !storage->supportsBackupPartition()) + { + throw Exception( + ErrorCodes::CANNOT_BACKUP_TABLE, + "Table engine {} doesn't support partitions, cannot backup {}", + storage->getName(), + tableNameWithTypeToString(database_name, table_name, false)); + } + res_table_info.partitions = partitions; + } + } + } + } +} + +void BackupEntriesCollector::lockTablesForReading() +{ + for (auto & [table_name, table_info] : table_infos) + { + auto storage = table_info.storage; + TableLockHolder table_lock; if (storage) { try { table_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); - create_table_query = storage->getCreateQueryForBackup(*this); } catch (Exception & e) { if (e.code() != ErrorCodes::TABLE_IS_DROPPED) throw; - } - } - - if (!create_table_query) - { - consistent &= !table_infos.contains(table_key); - return; - } - } - - fs::path data_path_in_backup; - if (is_temporary_table) - { - auto table_name_in_backup = renaming_map.getNewTemporaryTableName(table_name.table); - data_path_in_backup = root_path_in_backup / "temporary_tables" / "data" / escapeForFileName(table_name_in_backup); - } - else - { - auto table_name_in_backup = renaming_map.getNewTableName(table_name); - data_path_in_backup - = root_path_in_backup / "data" / escapeForFileName(table_name_in_backup.database) / escapeForFileName(table_name_in_backup.table); - } - - /// Check that information is consistent. - const auto & create = create_table_query->as(); - if ((create.getTable() != table_name.table) || (is_temporary_table != create.temporary) || (create.getDatabase() != table_name.database)) - { - /// Table was renamed recently. - consistent = false; - return; - } - - if (auto it = table_infos.find(table_key); it != table_infos.end()) - { - const auto & table_info = it->second; - if ((table_info.database != database) || (table_info.storage != storage)) - { - /// Table was renamed recently. - consistent = false; - return; - } - } - - /// Add information to `table_infos`. - auto & res_table_info = table_infos[table_key]; - res_table_info.database = database; - res_table_info.storage = storage; - res_table_info.table_lock = table_lock; - res_table_info.create_table_query = create_table_query; - res_table_info.data_path_in_backup = data_path_in_backup; - - if (partitions) - { - if (!res_table_info.partitions) - res_table_info.partitions.emplace(); - insertAtEnd(*res_table_info.partitions, *partitions); - } -} - -void BackupEntriesCollector::collectDatabaseInfo(const String & database_name, const std::set & except_table_names, bool throw_if_not_found) -{ - /// Gather information about the database. - DatabasePtr database; - ASTPtr create_database_query; - - if (throw_if_not_found) - { - database = DatabaseCatalog::instance().getDatabase(database_name); - create_database_query = database->getCreateDatabaseQueryForBackup(); - } - else - { - database = DatabaseCatalog::instance().tryGetDatabase(database_name); - if (!database) - { - consistent &= !database_infos.contains(database_name); - return; - } - - try - { - create_database_query = database->getCreateDatabaseQueryForBackup(); - } - catch (...) - { - /// The database has been dropped recently. - consistent &= !database_infos.contains(database_name); - return; - } - } - - /// Check that information is consistent. - const auto & create = create_database_query->as(); - if (create.getDatabase() != database_name) - { - /// Database was renamed recently. - consistent = false; - return; - } - - if (auto it = database_infos.find(database_name); it != database_infos.end()) - { - const auto & database_info = it->second; - if (database_info.database != database) - { - /// Database was renamed recently. - consistent = false; - return; - } - } - - /// Add information to `database_infos`. - auto & res_database_info = database_infos[database_name]; - res_database_info.database = database; - res_database_info.create_database_query = create_database_query; - - /// Add information about tables too. - for (auto it = database->getTablesIteratorForBackup(*this); it->isValid(); it->next()) - { - if (except_table_names.contains({database_name, it->name()})) - continue; - - collectTableInfo({database_name, it->name()}, /* is_temporary_table= */ false, {}, /* throw_if_not_found= */ false); - if (!consistent) - return; - } -} - -void BackupEntriesCollector::collectAllDatabasesInfo(const std::set & except_database_names, const std::set & except_table_names) -{ - for (const auto & [database_name, database] : DatabaseCatalog::instance().getDatabases()) - { - if (except_database_names.contains(database_name)) - continue; - collectDatabaseInfo(database_name, except_table_names, false); - if (!consistent) - return; - } -} - -/// Check for consistency of collected information about databases and tables. -void BackupEntriesCollector::checkConsistency() -{ - if (!consistent) - return; /// Already inconsistent, no more checks necessary - - /// Databases found while we were scanning tables and while we were scanning databases - must be the same. - for (const auto & [key, table_info] : table_infos) - { - auto it = database_infos.find(key.name.database); - if (it != database_infos.end()) - { - const auto & database_info = it->second; - if (database_info.database != table_info.database) - { - consistent = false; - return; + throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "{} is dropped", tableNameWithTypeToString(table_name.database, table_name.table, true)); } } } +} +/// Check consistency of collected information about databases and tables. +bool BackupEntriesCollector::compareWithPrevious(std::optional & inconsistency_error) +{ /// We need to scan tables at least twice to be sure that we haven't missed any table which could be renamed /// while we were scanning. - std::set database_names; - std::set table_names; - boost::range::copy(database_infos | boost::adaptors::map_keys, std::inserter(database_names, database_names.end())); - boost::range::copy(table_infos | boost::adaptors::map_keys, std::inserter(table_names, table_names.end())); + std::vector> databases_metadata; + std::vector> tables_metadata; + databases_metadata.reserve(database_infos.size()); + tables_metadata.reserve(table_infos.size()); + for (const auto & [database_name, database_info] : database_infos) + databases_metadata.emplace_back(database_name, database_info.create_database_query ? serializeAST(*database_info.create_database_query) : ""); + for (const auto & [table_name, table_info] : table_infos) + tables_metadata.emplace_back(table_name, serializeAST(*table_info.create_table_query)); - if (!previous_database_names || !previous_table_names || (*previous_database_names != database_names) - || (*previous_table_names != table_names)) + /// We need to sort the lists to make the comparison below correct. + ::sort(databases_metadata.begin(), databases_metadata.end()); + ::sort(tables_metadata.begin(), tables_metadata.end()); + + SCOPE_EXIT({ + previous_databases_metadata = std::move(databases_metadata); + previous_tables_metadata = std::move(tables_metadata); + }); + + /// Databases must be the same as during the previous scan. + if (databases_metadata != previous_databases_metadata) { - previous_database_names = std::move(database_names); - previous_table_names = std::move(table_names); - consistent = false; + std::vector> difference; + difference.reserve(databases_metadata.size()); + std::set_difference(databases_metadata.begin(), databases_metadata.end(), previous_databases_metadata.begin(), + previous_databases_metadata.end(), std::back_inserter(difference)); + + if (!difference.empty()) + { + inconsistency_error = Exception{ + ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, + "Database {} were created or changed its definition during scanning", + backQuoteIfNeed(difference[0].first)}; + return false; + } + + difference.clear(); + difference.reserve(previous_databases_metadata.size()); + std::set_difference(previous_databases_metadata.begin(), previous_databases_metadata.end(), databases_metadata.begin(), + databases_metadata.end(), std::back_inserter(difference)); + + if (!difference.empty()) + { + inconsistency_error = Exception{ + ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, + "Database {} were removed or changed its definition during scanning", + backQuoteIfNeed(difference[0].first)}; + return false; + } } + + /// Tables must be the same as during the previous scan. + if (tables_metadata != previous_tables_metadata) + { + std::vector> difference; + difference.reserve(tables_metadata.size()); + std::set_difference(tables_metadata.begin(), tables_metadata.end(), previous_tables_metadata.begin(), + previous_tables_metadata.end(), std::back_inserter(difference)); + + if (!difference.empty()) + { + inconsistency_error = Exception{ + ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, + "{} were created or changed its definition during scanning", + tableNameWithTypeToString(difference[0].first.database, difference[0].first.table, true)}; + return false; + } + + difference.clear(); + difference.reserve(previous_tables_metadata.size()); + std::set_difference(previous_tables_metadata.begin(), previous_tables_metadata.end(), tables_metadata.begin(), + tables_metadata.end(), std::back_inserter(difference)); + + if (!difference.empty()) + { + inconsistency_error = Exception{ + ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, + "{} were removed or changed its definition during scanning", + tableNameWithTypeToString(difference[0].first.database, difference[0].first.table, true)}; + return false; + } + } + + return true; } /// Make backup entries for all the definitions of all the databases found. @@ -444,14 +645,16 @@ void BackupEntriesCollector::makeBackupEntriesForDatabasesDefs() { for (const auto & [database_name, database_info] : database_infos) { + if (!database_info.create_database_query) + continue; /// We store CREATE DATABASE queries only if there was BACKUP DATABASE specified. + LOG_TRACE(log, "Adding definition of database {}", backQuoteIfNeed(database_name)); ASTPtr new_create_query = database_info.create_database_query; - renameDatabaseAndTableNameInCreateQuery(context->getGlobalContext(), renaming_map, new_create_query); - - String new_database_name = renaming_map.getNewDatabaseName(database_name); - auto metadata_path_in_backup = root_path_in_backup / "metadata" / (escapeForFileName(new_database_name) + ".sql"); + adjustCreateQueryForBackup(new_create_query, context->getGlobalContext(), nullptr); + renameDatabaseAndTableNameInCreateQuery(new_create_query, renaming_map, context->getGlobalContext()); + const String & metadata_path_in_backup = database_info.metadata_path_in_backup; backup_entries.emplace_back(metadata_path_in_backup, std::make_shared(serializeAST(*new_create_query))); } } @@ -459,26 +662,15 @@ void BackupEntriesCollector::makeBackupEntriesForDatabasesDefs() /// Calls IDatabase::backupTable() for all the tables found to make backup entries for tables. void BackupEntriesCollector::makeBackupEntriesForTablesDefs() { - for (const auto & [key, table_info] : table_infos) + for (auto & [table_name, table_info] : table_infos) { - LOG_TRACE(log, "Adding definition of {}table {}", (key.is_temporary ? "temporary " : ""), key.name.getFullName()); + LOG_TRACE(log, "Adding definition of {}", tableNameWithTypeToString(table_name.database, table_name.table, false)); ASTPtr new_create_query = table_info.create_table_query; - renameDatabaseAndTableNameInCreateQuery(context->getGlobalContext(), renaming_map, new_create_query); - - fs::path metadata_path_in_backup; - if (key.is_temporary) - { - auto new_name = renaming_map.getNewTemporaryTableName(key.name.table); - metadata_path_in_backup = root_path_in_backup / "temporary_tables" / "metadata" / (escapeForFileName(new_name) + ".sql"); - } - else - { - auto new_name = renaming_map.getNewTableName(key.name); - metadata_path_in_backup - = root_path_in_backup / "metadata" / escapeForFileName(new_name.database) / (escapeForFileName(new_name.table) + ".sql"); - } + adjustCreateQueryForBackup(new_create_query, context->getGlobalContext(), &table_info.replicated_table_shared_id); + renameDatabaseAndTableNameInCreateQuery(new_create_query, renaming_map, context->getGlobalContext()); + const String & metadata_path_in_backup = table_info.metadata_path_in_backup; backup_entries.emplace_back(metadata_path_in_backup, std::make_shared(serializeAST(*new_create_query))); } } @@ -488,63 +680,76 @@ void BackupEntriesCollector::makeBackupEntriesForTablesData() if (backup_settings.structure_only) return; - for (const auto & [key, table_info] : table_infos) + for (const auto & [table_name, table_info] : table_infos) { - LOG_TRACE(log, "Adding data of {}table {}", (key.is_temporary ? "temporary " : ""), key.name.getFullName()); const auto & storage = table_info.storage; const auto & data_path_in_backup = table_info.data_path_in_backup; - const auto & partitions = table_info.partitions; - storage->backupData(*this, data_path_in_backup, partitions); + if (storage) + { + LOG_TRACE(log, "Adding data of {}", tableNameWithTypeToString(table_name.database, table_name.table, false)); + storage->backupData(*this, data_path_in_backup, table_info.partitions); + } + else + { + /// Storage == null means this storage exists on other replicas but it has not been created on this replica yet. + /// If this table is replicated in this case we call IBackupCoordination::addReplicatedDataPath() which will cause + /// other replicas to fill the storage's data in the backup. + /// If this table is not replicated we'll do nothing leaving the storage's data empty in the backup. + if (table_info.replicated_table_shared_id) + backup_coordination->addReplicatedDataPath(*table_info.replicated_table_shared_id, data_path_in_backup); + } } } void BackupEntriesCollector::addBackupEntry(const String & file_name, BackupEntryPtr backup_entry) { - if (current_stage == Stage::kWritingBackup) + if (current_status == kWritingBackupStatus) throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding backup entries is not allowed"); backup_entries.emplace_back(file_name, backup_entry); } +void BackupEntriesCollector::addBackupEntry(const std::pair & backup_entry) +{ + addBackupEntry(backup_entry.first, backup_entry.second); +} + void BackupEntriesCollector::addBackupEntries(const BackupEntries & backup_entries_) { - if (current_stage == Stage::kWritingBackup) + if (current_status == kWritingBackupStatus) throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding backup entries is not allowed"); insertAtEnd(backup_entries, backup_entries_); } void BackupEntriesCollector::addBackupEntries(BackupEntries && backup_entries_) { - if (current_stage == Stage::kWritingBackup) + if (current_status == kWritingBackupStatus) throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding backup entries is not allowed"); insertAtEnd(backup_entries, std::move(backup_entries_)); } -void BackupEntriesCollector::addPostCollectingTask(std::function task) +void BackupEntriesCollector::addPostTask(std::function task) { - if (current_stage == Stage::kWritingBackup) + if (current_status == kWritingBackupStatus) throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding post tasks is not allowed"); - post_collecting_tasks.push(std::move(task)); + post_tasks.push(std::move(task)); } /// Runs all the tasks added with addPostCollectingTask(). -void BackupEntriesCollector::runPostCollectingTasks() +void BackupEntriesCollector::runPostTasks() { /// Post collecting tasks can add other post collecting tasks, our code is fine with that. - while (!post_collecting_tasks.empty()) + while (!post_tasks.empty()) { - auto task = std::move(post_collecting_tasks.front()); - post_collecting_tasks.pop(); + auto task = std::move(post_tasks.front()); + post_tasks.pop(); std::move(task)(); } } -void BackupEntriesCollector::throwPartitionsNotSupported(const StorageID & storage_id, const String & table_engine) +size_t BackupEntriesCollector::getAccessCounter(AccessEntityType type) { - throw Exception( - ErrorCodes::CANNOT_BACKUP_TABLE, - "Table engine {} doesn't support partitions, cannot backup table {}", - table_engine, - storage_id.getFullTableName()); + access_counters.resize(static_cast(AccessEntityType::MAX)); + return access_counters[static_cast(type)]++; } } diff --git a/src/Backups/BackupEntriesCollector.h b/src/Backups/BackupEntriesCollector.h index 1466815f3a7..9a653ee7e4d 100644 --- a/src/Backups/BackupEntriesCollector.h +++ b/src/Backups/BackupEntriesCollector.h @@ -19,6 +19,7 @@ class IBackupCoordination; class IDatabase; using DatabasePtr = std::shared_ptr; struct StorageID; +enum class AccessEntityType; /// Collects backup entries for all databases and tables which should be put to a backup. class BackupEntriesCollector : private boost::noncopyable @@ -27,84 +28,90 @@ public: BackupEntriesCollector(const ASTBackupQuery::Elements & backup_query_elements_, const BackupSettings & backup_settings_, std::shared_ptr backup_coordination_, - const ContextPtr & context_, - std::chrono::seconds timeout_ = std::chrono::seconds(-1) /* no timeout */); + const ContextPtr & context_); ~BackupEntriesCollector(); /// Collects backup entries and returns the result. - /// This function first generates a list of databases and then call IDatabase::backup() for each database from this list. - /// At this moment IDatabase::backup() calls IStorage::backup() and they both call addBackupEntry() to build a list of backup entries. - BackupEntries getBackupEntries(); + /// This function first generates a list of databases and then call IDatabase::getTablesForBackup() for each database from this list. + /// Then it calls IStorage::backupData() to build a list of backup entries. + BackupEntries run(); const BackupSettings & getBackupSettings() const { return backup_settings; } std::shared_ptr getBackupCoordination() const { return backup_coordination; } ContextPtr getContext() const { return context; } - /// Adds a backup entry which will be later returned by getBackupEntries(). - /// These function can be called by implementations of IStorage::backup() in inherited storage classes. + /// Adds a backup entry which will be later returned by run(). + /// These function can be called by implementations of IStorage::backupData() in inherited storage classes. void addBackupEntry(const String & file_name, BackupEntryPtr backup_entry); + void addBackupEntry(const std::pair & backup_entry); void addBackupEntries(const BackupEntries & backup_entries_); void addBackupEntries(BackupEntries && backup_entries_); - /// Adds a function which must be called after all IStorage::backup() have finished their work on all hosts. + /// Adds a function which must be called after all IStorage::backupData() have finished their work on all hosts. /// This function is designed to help making a consistent in some complex cases like /// 1) we need to join (in a backup) the data of replicated tables gathered on different hosts. - void addPostCollectingTask(std::function task); + void addPostTask(std::function task); - /// Writing a backup includes a few stages: - enum class Stage - { - /// Initial stage. - kPreparing, - - /// Finding all tables and databases which we're going to put to the backup. - kFindingTables, - - /// Making temporary hard links and prepare backup entries. - kExtractingDataFromTables, - - /// Running special tasks for replicated databases or tables which can also prepare some backup entries. - kRunningPostTasks, - - /// Writing backup entries to the backup and removing temporary hard links. - kWritingBackup, - - /// An error happens during any of the stages above, the backup won't be written. - kError, - }; - static std::string_view toString(Stage stage); - - /// Throws an exception that a specified table engine doesn't support partitions. - [[noreturn]] static void throwPartitionsNotSupported(const StorageID & storage_id, const String & table_engine); + /// Returns an incremental counter used to backup access control. + size_t getAccessCounter(AccessEntityType type); private: - void setStage(Stage new_stage, const String & error_message = {}); void calculateRootPathInBackup(); - void collectDatabasesAndTablesInfo(); - void collectTableInfo(const QualifiedTableName & table_name, bool is_temporary_table, const std::optional & partitions, bool throw_if_not_found); - void collectDatabaseInfo(const String & database_name, const std::set & except_table_names, bool throw_if_not_found); - void collectAllDatabasesInfo(const std::set & except_database_names, const std::set & except_table_names); - void checkConsistency(); + + void gatherMetadataAndCheckConsistency(); + + bool tryGatherMetadataAndCompareWithPrevious(std::optional & inconsistency_error); + + void gatherDatabasesMetadata(); + + void gatherDatabaseMetadata( + const String & database_name, + bool throw_if_database_not_found, + bool backup_create_database_query, + const std::optional & table_name, + bool throw_if_table_not_found, + const std::optional & partitions, + bool all_tables, + const std::set & except_table_names); + + void gatherTablesMetadata(); + void lockTablesForReading(); + bool compareWithPrevious(std::optional & inconsistency_error); + void makeBackupEntriesForDatabasesDefs(); void makeBackupEntriesForTablesDefs(); void makeBackupEntriesForTablesData(); - void runPostCollectingTasks(); + void runPostTasks(); + + Strings setStatus(const String & new_status, const String & message = ""); const ASTBackupQuery::Elements backup_query_elements; const BackupSettings backup_settings; std::shared_ptr backup_coordination; ContextPtr context; - std::chrono::seconds timeout; + std::chrono::milliseconds consistent_metadata_snapshot_timeout; Poco::Logger * log; - Stage current_stage = Stage::kPreparing; - std::filesystem::path root_path_in_backup; + Strings all_hosts; DDLRenamingMap renaming_map; + std::filesystem::path root_path_in_backup; struct DatabaseInfo { DatabasePtr database; ASTPtr create_database_query; + String metadata_path_in_backup; + + struct TableParams + { + bool throw_if_table_not_found = false; + std::optional partitions; + }; + + std::unordered_map tables; + + bool all_tables = false; + std::unordered_set except_table_names; }; struct TableInfo @@ -113,26 +120,22 @@ private: StoragePtr storage; TableLockHolder table_lock; ASTPtr create_table_query; + String metadata_path_in_backup; std::filesystem::path data_path_in_backup; + std::optional replicated_table_shared_id; std::optional partitions; }; - struct TableKey - { - QualifiedTableName name; - bool is_temporary = false; - bool operator ==(const TableKey & right) const; - bool operator <(const TableKey & right) const; - }; - + String current_status; + std::chrono::steady_clock::time_point consistent_metadata_snapshot_start_time; std::unordered_map database_infos; - std::map table_infos; - std::optional> previous_database_names; - std::optional> previous_table_names; - bool consistent = false; + std::unordered_map table_infos; + std::vector> previous_databases_metadata; + std::vector> previous_tables_metadata; BackupEntries backup_entries; - std::queue> post_collecting_tasks; + std::queue> post_tasks; + std::vector access_counters; }; } diff --git a/src/Backups/BackupUtils.cpp b/src/Backups/BackupUtils.cpp index c5de4bd7e67..9ff91050177 100644 --- a/src/Backups/BackupUtils.cpp +++ b/src/Backups/BackupUtils.cpp @@ -39,7 +39,7 @@ DDLRenamingMap makeRenamingMapFromBackupQuery(const ASTBackupQuery::Elements & e const String & new_table_name = element.new_table_name; assert(!table_name.empty()); assert(!new_table_name.empty()); - map.setNewTemporaryTableName(table_name, new_table_name); + map.setNewTableName({DatabaseCatalog::TEMPORARY_DATABASE, table_name}, {DatabaseCatalog::TEMPORARY_DATABASE, new_table_name}); break; } diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 84dc63b4f9f..635b2810941 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -166,9 +166,8 @@ UUID BackupsWorker::startMakingBackup(const ASTPtr & query, const ContextPtr & c BackupEntries backup_entries; { - auto timeout = std::chrono::seconds{context_in_use->getConfigRef().getInt("backups.backup_prepare_timeout", -1)}; - BackupEntriesCollector backup_entries_collector{backup_query->elements, backup_settings, backup_coordination, context_in_use, timeout}; - backup_entries = backup_entries_collector.getBackupEntries(); + BackupEntriesCollector backup_entries_collector{backup_query->elements, backup_settings, backup_coordination, context_in_use}; + backup_entries = backup_entries_collector.run(); } writeBackupEntries(backup, std::move(backup_entries), backups_thread_pool); @@ -272,8 +271,8 @@ UUID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePtr conte String addr_database = address->default_database.empty() ? current_database : address->default_database; for (auto & element : restore_elements) element.setCurrentDatabase(addr_database); - RestorerFromBackup dummy_restorer{restore_elements, restore_settings, nullptr, backup, context_in_use, {}}; - dummy_restorer.checkAccessOnly(); + RestorerFromBackup dummy_restorer{restore_elements, restore_settings, nullptr, backup, context_in_use}; + dummy_restorer.run(RestorerFromBackup::CHECK_ACCESS_ONLY); } } @@ -325,11 +324,9 @@ UUID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePtr conte DataRestoreTasks data_restore_tasks; { - auto timeout = std::chrono::seconds{context_in_use->getConfigRef().getInt("backups.restore_metadata_timeout", -1)}; RestorerFromBackup restorer{restore_query->elements, restore_settings, restore_coordination, - backup, context_in_use, timeout}; - restorer.restoreMetadata(); - data_restore_tasks = restorer.getDataRestoreTasks(); + backup, context_in_use}; + data_restore_tasks = restorer.run(RestorerFromBackup::RESTORE); } restoreTablesData(std::move(data_restore_tasks), restores_thread_pool); diff --git a/src/Backups/DDLAdjustingForBackupVisitor.cpp b/src/Backups/DDLAdjustingForBackupVisitor.cpp new file mode 100644 index 00000000000..8223e08f127 --- /dev/null +++ b/src/Backups/DDLAdjustingForBackupVisitor.cpp @@ -0,0 +1,113 @@ +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +namespace +{ + void visitStorageSystemTableEngine(ASTStorage &, const DDLAdjustingForBackupVisitor::Data & data) + { + /// Precondition: storage.engine && storage.engine->name.starts_with("System")) + + /// If this is a definition of a system table we'll remove columns and comment because they're redundant for backups. + auto & create = data.create_query->as(); + create.reset(create.columns_list); + create.reset(create.comment); + } + + void visitStorageReplicatedTableEngine(ASTStorage & storage, const DDLAdjustingForBackupVisitor::Data & data) + { + /// Precondition: engine_name.starts_with("Replicated") && engine_name.ends_with("MergeTree") + + if (data.replicated_table_shared_id) + *data.replicated_table_shared_id = StorageReplicatedMergeTree::tryGetTableSharedIDFromCreateQuery(*data.create_query, data.global_context); + + /// Before storing the metadata in a backup we have to find a zookeeper path in its definition and turn the table's UUID in there + /// back into "{uuid}", and also we probably can remove the zookeeper path and replica name if they're default. + /// So we're kind of reverting what we had done to the table's definition in registerStorageMergeTree.cpp before we created this table. + auto & create = data.create_query->as(); + auto & engine = *storage.engine; + + auto * engine_args_ast = typeid_cast(engine.arguments.get()); + if (!engine_args_ast) + return; + + auto & engine_args = engine_args_ast->children; + if (engine_args.size() < 2) + return; + + auto * zookeeper_path_ast = typeid_cast(engine_args[0].get()); + auto * replica_name_ast = typeid_cast(engine_args[1].get()); + if (zookeeper_path_ast && (zookeeper_path_ast->value.getType() == Field::Types::String) && + replica_name_ast && (replica_name_ast->value.getType() == Field::Types::String)) + { + String & zookeeper_path_arg = zookeeper_path_ast->value.get(); + String & replica_name_arg = replica_name_ast->value.get(); + if (create.uuid != UUIDHelpers::Nil) + { + String table_uuid_str = toString(create.uuid); + if (size_t uuid_pos = zookeeper_path_arg.find(table_uuid_str); uuid_pos != String::npos) + zookeeper_path_arg.replace(uuid_pos, table_uuid_str.size(), "{uuid}"); + } + const auto & config = data.global_context->getConfigRef(); + if ((zookeeper_path_arg == StorageReplicatedMergeTree::getDefaultZooKeeperPath(config)) + && (replica_name_arg == StorageReplicatedMergeTree::getDefaultReplicaName(config)) + && ((engine_args.size() == 2) || !engine_args[2]->as())) + { + engine_args.erase(engine_args.begin(), engine_args.begin() + 2); + } + } + } + + void visitStorage(ASTStorage & storage, const DDLAdjustingForBackupVisitor::Data & data) + { + if (!storage.engine) + return; + + const String & engine_name = storage.engine->name; + if (engine_name.starts_with("System")) + visitStorageSystemTableEngine(storage, data); + else if (engine_name.starts_with("Replicated") && engine_name.ends_with("MergeTree")) + visitStorageReplicatedTableEngine(storage, data); + } + + void visitCreateQuery(ASTCreateQuery & create, const DDLAdjustingForBackupVisitor::Data & data) + { + create.uuid = UUIDHelpers::Nil; + create.to_inner_uuid = UUIDHelpers::Nil; + + if (create.storage) + visitStorage(*create.storage, data); + } +} + + +bool DDLAdjustingForBackupVisitor::needChildVisit(const ASTPtr &, const ASTPtr &) +{ + return false; +} + +void DDLAdjustingForBackupVisitor::visit(ASTPtr ast, const Data & data) +{ + if (auto * create = ast->as()) + visitCreateQuery(*create, data); +} + +void adjustCreateQueryForBackup(ASTPtr ast, const ContextPtr & global_context, std::optional * replicated_table_shared_id) +{ + if (replicated_table_shared_id) + *replicated_table_shared_id = {}; + + DDLAdjustingForBackupVisitor::Data data{ast, global_context, replicated_table_shared_id}; + DDLAdjustingForBackupVisitor::Visitor{data}.visit(ast); +} + +} diff --git a/src/Backups/DDLAdjustingForBackupVisitor.h b/src/Backups/DDLAdjustingForBackupVisitor.h new file mode 100644 index 00000000000..63353dcc000 --- /dev/null +++ b/src/Backups/DDLAdjustingForBackupVisitor.h @@ -0,0 +1,36 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class IAST; +using ASTPtr = std::shared_ptr; +class Context; +using ContextPtr = std::shared_ptr; + +/// Changes a create query to a form which is appropriate or suitable for saving in a backup. +/// Also extracts a replicated table's shared ID from the create query if this is a create query for a replicated table. +/// `replicated_table_shared_id` can be null if you don't need that. +void adjustCreateQueryForBackup(ASTPtr ast, const ContextPtr & global_context, std::optional * replicated_table_shared_id); + +/// Visits ASTCreateQuery and changes it to a form which is appropriate or suitable for saving in a backup. +class DDLAdjustingForBackupVisitor +{ +public: + struct Data + { + ASTPtr create_query; + ContextPtr global_context; + std::optional * replicated_table_shared_id = nullptr; + }; + + using Visitor = InDepthNodeVisitor; + + static bool needChildVisit(const ASTPtr & ast, const ASTPtr & child); + static void visit(ASTPtr ast, const Data & data); +}; + +} diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index 92b7139ed5f..b4c5c7b3d88 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -13,11 +13,10 @@ class IBackupCoordination public: virtual ~IBackupCoordination() = default; - /// Sets the current stage and waits for other hosts to come to this stage too. - virtual void syncStage(const String & current_host, int stage, const Strings & wait_hosts, std::chrono::seconds timeout) = 0; - - /// Sets that the current host encountered an error, so other hosts should know that and stop waiting in syncStage(). - virtual void syncStageError(const String & current_host, const String & error_message) = 0; + /// Sets the current status and waits for other hosts to come to this status too. If status starts with "error:" it'll stop waiting on all the hosts. + virtual void setStatus(const String & current_host, const String & new_status, const String & message) = 0; + virtual Strings setStatusAndWait(const String & current_host, const String & new_status, const String & message, const Strings & other_hosts) = 0; + virtual Strings setStatusAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & other_hosts, UInt64 timeout_ms) = 0; struct PartNameAndChecksum { @@ -29,21 +28,29 @@ public: /// Multiple replicas of the replicated table call this function and then the added part names can be returned by call of the function /// getReplicatedPartNames(). /// Checksums are used only to control that parts under the same names on different replicas are the same. - virtual void addReplicatedPartNames(const String & table_zk_path, const String & table_name_for_logs, const String & replica_name, + virtual void addReplicatedPartNames(const String & table_shared_id, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) = 0; /// Returns the names of the parts which a specified replica of a replicated table should put to the backup. /// This is the same list as it was added by call of the function addReplicatedPartNames() but without duplications and without /// parts covered by another parts. - virtual Strings getReplicatedPartNames(const String & table_zk_path, const String & replica_name) const = 0; + virtual Strings getReplicatedPartNames(const String & table_shared_id, const String & replica_name) const = 0; /// Adds a data path in backup for a replicated table. /// Multiple replicas of the replicated table call this function and then all the added paths can be returned by call of the function /// getReplicatedDataPaths(). - virtual void addReplicatedDataPath(const String & table_zk_path, const String & data_path) = 0; + virtual void addReplicatedDataPath(const String & table_shared_id, const String & data_path) = 0; /// Returns all the data paths in backup added for a replicated table (see also addReplicatedDataPath()). - virtual Strings getReplicatedDataPaths(const String & table_zk_path) const = 0; + virtual Strings getReplicatedDataPaths(const String & table_shared_id) const = 0; + + /// Adds a path to access.txt file keeping access entities of a ReplicatedAccessStorage. + virtual void addReplicatedAccessPath(const String & access_zk_path, const String & file_path) = 0; + virtual Strings getReplicatedAccessPaths(const String & access_zk_path) const = 0; + + /// Sets the host id of a host storing access entities of a ReplicatedAccessStorage to backup. + virtual void setReplicatedAccessHost(const String & access_zk_path, const String & host) = 0; + virtual String getReplicatedAccessHost(const String & access_zk_path) const = 0; struct FileInfo { diff --git a/src/Backups/IRestoreCoordination.h b/src/Backups/IRestoreCoordination.h index fd9a67e1b96..ba76a6e0c99 100644 --- a/src/Backups/IRestoreCoordination.h +++ b/src/Backups/IRestoreCoordination.h @@ -13,11 +13,10 @@ class IRestoreCoordination public: virtual ~IRestoreCoordination() = default; - /// Sets the current stage and waits for other hosts to come to this stage too. - virtual void syncStage(const String & current_host, int stage, const Strings & wait_hosts, std::chrono::seconds timeout) = 0; - - /// Sets that the current host encountered an error, so other hosts should know that and stop waiting in syncStage(). - virtual void syncStageError(const String & current_host, const String & error_message) = 0; + /// Sets the current status and waits for other hosts to come to this status too. If status starts with "error:" it'll stop waiting on all the hosts. + virtual void setStatus(const String & current_host, const String & new_status, const String & message) = 0; + virtual Strings setStatusAndWait(const String & current_host, const String & new_status, const String & message, const Strings & other_hosts) = 0; + virtual Strings setStatusAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & other_hosts, UInt64 timeout_ms) = 0; /// Starts creating a table in a replicated database. Returns false if there is another host which is already creating this table. virtual bool acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) = 0; diff --git a/src/Backups/RestoreCoordinationDistributed.cpp b/src/Backups/RestoreCoordinationDistributed.cpp index e131ce7fe24..8cbaa01810d 100644 --- a/src/Backups/RestoreCoordinationDistributed.cpp +++ b/src/Backups/RestoreCoordinationDistributed.cpp @@ -9,7 +9,7 @@ namespace DB RestoreCoordinationDistributed::RestoreCoordinationDistributed(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_) : zookeeper_path(zookeeper_path_) , get_zookeeper(get_zookeeper_) - , stage_sync(zookeeper_path_ + "/stage", get_zookeeper_, &Poco::Logger::get("RestoreCoordination")) + , status_sync(zookeeper_path_ + "/status", get_zookeeper_, &Poco::Logger::get("RestoreCoordination")) { createRootNodes(); } @@ -26,14 +26,19 @@ void RestoreCoordinationDistributed::createRootNodes() zookeeper->createIfNotExists(zookeeper_path + "/repl_access_storages_acquired", ""); } -void RestoreCoordinationDistributed::syncStage(const String & current_host, int new_stage, const Strings & wait_hosts, std::chrono::seconds timeout) +void RestoreCoordinationDistributed::setStatus(const String & current_host, const String & new_status, const String & message) { - stage_sync.syncStage(current_host, new_stage, wait_hosts, timeout); + status_sync.set(current_host, new_status, message); } -void RestoreCoordinationDistributed::syncStageError(const String & current_host, const String & error_message) +Strings RestoreCoordinationDistributed::setStatusAndWait(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts) { - stage_sync.syncStageError(current_host, error_message); + return status_sync.setAndWait(current_host, new_status, message, all_hosts); +} + +Strings RestoreCoordinationDistributed::setStatusAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, UInt64 timeout_ms) +{ + return status_sync.setAndWaitFor(current_host, new_status, message, all_hosts, timeout_ms); } bool RestoreCoordinationDistributed::acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) diff --git a/src/Backups/RestoreCoordinationDistributed.h b/src/Backups/RestoreCoordinationDistributed.h index 0ea5db3f062..52b961cf0ef 100644 --- a/src/Backups/RestoreCoordinationDistributed.h +++ b/src/Backups/RestoreCoordinationDistributed.h @@ -14,11 +14,10 @@ public: RestoreCoordinationDistributed(const String & zookeeper_path, zkutil::GetZooKeeper get_zookeeper); ~RestoreCoordinationDistributed() override; - /// Sets the current stage and waits for other hosts to come to this stage too. - void syncStage(const String & current_host, int new_stage, const Strings & wait_hosts, std::chrono::seconds timeout) override; - - /// Sets that the current host encountered an error, so other hosts should know that and stop waiting in syncStage(). - void syncStageError(const String & current_host, const String & error_message) override; + /// Sets the current status and waits for other hosts to come to this status too. If status starts with "error:" it'll stop waiting on all the hosts. + void setStatus(const String & current_host, const String & new_status, const String & message) override; + Strings setStatusAndWait(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts) override; + Strings setStatusAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, UInt64 timeout_ms) override; /// Starts creating a table in a replicated database. Returns false if there is another host which is already creating this table. bool acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) override; @@ -42,7 +41,7 @@ private: const String zookeeper_path; const zkutil::GetZooKeeper get_zookeeper; - BackupCoordinationStageSync stage_sync; + BackupCoordinationStatusSync status_sync; }; } diff --git a/src/Backups/RestoreCoordinationLocal.cpp b/src/Backups/RestoreCoordinationLocal.cpp index 9cecc3f90c9..a999cc13195 100644 --- a/src/Backups/RestoreCoordinationLocal.cpp +++ b/src/Backups/RestoreCoordinationLocal.cpp @@ -7,12 +7,18 @@ namespace DB RestoreCoordinationLocal::RestoreCoordinationLocal() = default; RestoreCoordinationLocal::~RestoreCoordinationLocal() = default; -void RestoreCoordinationLocal::syncStage(const String &, int, const Strings &, std::chrono::seconds) +void RestoreCoordinationLocal::setStatus(const String &, const String &, const String &) { } -void RestoreCoordinationLocal::syncStageError(const String &, const String &) +Strings RestoreCoordinationLocal::setStatusAndWait(const String &, const String &, const String &, const Strings &) { + return {}; +} + +Strings RestoreCoordinationLocal::setStatusAndWaitFor(const String &, const String &, const String &, const Strings &, UInt64) +{ + return {}; } bool RestoreCoordinationLocal::acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) diff --git a/src/Backups/RestoreCoordinationLocal.h b/src/Backups/RestoreCoordinationLocal.h index b73f345df47..68624481a7a 100644 --- a/src/Backups/RestoreCoordinationLocal.h +++ b/src/Backups/RestoreCoordinationLocal.h @@ -17,11 +17,10 @@ public: RestoreCoordinationLocal(); ~RestoreCoordinationLocal() override; - /// Sets the current stage and waits for other hosts to come to this stage too. - void syncStage(const String & current_host, int stage, const Strings & wait_hosts, std::chrono::seconds timeout) override; - - /// Sets that the current host encountered an error, so other hosts should know that and stop waiting in syncStage(). - void syncStageError(const String & current_host, const String & error_message) override; + /// Sets the current status and waits for other hosts to come to this status too. If status starts with "error:" it'll stop waiting on all the hosts. + void setStatus(const String & current_host, const String & new_status, const String & message) override; + Strings setStatusAndWait(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts) override; + Strings setStatusAndWaitFor(const String & current_host, const String & new_status, const String & message, const Strings & all_hosts, UInt64 timeout_ms) override; /// Starts creating a table in a replicated database. Returns false if there is another host which is already creating this table. bool acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) override; diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 590d39f24f8..efa1fe2cfb8 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -74,7 +74,7 @@ namespace { case RestoreTableCreationMode::kCreate: return Field{true}; case RestoreTableCreationMode::kMustExist: return Field{false}; - case RestoreTableCreationMode::kCreateIfNotExists: return Field{"if not exists"}; + case RestoreTableCreationMode::kCreateIfNotExists: return Field{"if-not-exists"}; } throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected value of enum RestoreTableCreationMode: {}", static_cast(value)); } @@ -131,12 +131,14 @@ namespace switch (value) { case RestoreAccessCreationMode::kCreate: return Field{true}; - case RestoreAccessCreationMode::kCreateIfNotExists: return Field{"if not exists"}; + case RestoreAccessCreationMode::kCreateIfNotExists: return Field{"if-not-exists"}; case RestoreAccessCreationMode::kReplace: return Field{"replace"}; } throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected value of enum RestoreAccessCreationMode: {}", static_cast(value)); } }; + + using SettingFieldRestoreUDFCreationMode = SettingFieldRestoreAccessCreationMode; } /// List of restore settings except base_backup_name and cluster_host_ids. @@ -155,6 +157,7 @@ namespace M(Bool, allow_non_empty_tables) \ M(RestoreAccessCreationMode, create_access) \ M(Bool, allow_unresolved_access_dependencies) \ + M(RestoreUDFCreationMode, create_function) \ M(Bool, internal) \ M(String, host_id) \ M(String, coordination_zk_path) diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 5e941b79508..1bc5d867a37 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -36,6 +36,8 @@ enum class RestoreAccessCreationMode kReplace, }; +using RestoreUDFCreationMode = RestoreAccessCreationMode; + /// Settings specified in the "SETTINGS" clause of a RESTORE query. struct RestoreSettings { @@ -99,6 +101,9 @@ struct RestoreSettings /// For example, if an user has a profile assigned and that profile is not in the backup and doesn't exist locally. bool allow_unresolved_access_dependencies = false; + /// How the RESTORE command will handle if a user-defined function which it's going to restore already exists. + RestoreUDFCreationMode create_function = RestoreUDFCreationMode::kCreateIfNotExists; + /// Internal, should not be specified by user. bool internal = false; diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index 14f5b7f48f0..5b211bc50a8 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -1,9 +1,11 @@ #include #include +#include #include #include #include #include +#include #include #include #include @@ -39,54 +41,58 @@ namespace ErrorCodes namespace { - constexpr const std::string_view sql_ext = ".sql"; + /// Finding databases and tables in the backup which we're going to restore. + constexpr const char * kFindingTablesInBackupStatus = "finding tables in backup"; - String tryGetTableEngine(const IAST & ast) + /// Creating databases or finding them and checking their definitions. + constexpr const char * kCreatingDatabasesStatus = "creating databases"; + + /// Creating tables or finding them and checking their definition. + constexpr const char * kCreatingTablesStatus = "creating tables"; + + /// Inserting restored data to tables. + constexpr const char * kInsertingDataToTablesStatus = "inserting data to tables"; + + /// Error status. + constexpr const char * kErrorStatus = BackupCoordinationStatusSync::kErrorStatus; + + /// Uppercases the first character of a passed string. + String toUpperFirst(const String & str) { - const ASTCreateQuery * create = ast.as(); - if (!create) - return {}; - if (!create->storage || !create->storage->engine) - return {}; - return create->storage->engine->name; + String res = str; + res[0] = std::toupper(res[0]); + return res; } - bool hasSystemTableEngine(const IAST & ast) + /// Outputs "table " or "temporary table " + String tableNameWithTypeToString(const String & database_name, const String & table_name, bool first_upper) { - return tryGetTableEngine(ast).starts_with("System"); + String str; + if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) + str = fmt::format("temporary table {}", backQuoteIfNeed(table_name)); + else + str = fmt::format("table {}.{}", backQuoteIfNeed(database_name), backQuoteIfNeed(table_name)); + if (first_upper) + str[0] = std::toupper(str[0]); + return str; } - bool hasSystemAccessTableEngine(const IAST & ast) + /// Whether a specified name corresponds one of the tables backuping ACL. + bool isSystemAccessTableName(const QualifiedTableName & table_name) { - String engine_name = tryGetTableEngine(ast); - return (engine_name == "SystemUsers") || (engine_name == "SystemRoles") || (engine_name == "SystemSettingsProfiles") - || (engine_name == "SystemRowPolicies") || (engine_name == "SystemQuotas"); + if (table_name.database != DatabaseCatalog::SYSTEM_DATABASE) + return false; + + return (table_name.table == "users") || (table_name.table == "roles") || (table_name.table == "settings_profiles") + || (table_name.table == "row_policies") || (table_name.table == "quotas"); } -} -bool RestorerFromBackup::TableKey::operator ==(const TableKey & right) const -{ - return (name == right.name) && (is_temporary == right.is_temporary); -} - -bool RestorerFromBackup::TableKey::operator <(const TableKey & right) const -{ - return (name < right.name) || ((name == right.name) && (is_temporary < right.is_temporary)); -} - -std::string_view RestorerFromBackup::toString(Stage stage) -{ - switch (stage) + /// Whether a specified name corresponds one of the tables backuping ACL. + bool isSystemFunctionsTableName(const QualifiedTableName & table_name) { - case Stage::kPreparing: return "Preparing"; - case Stage::kFindingTablesInBackup: return "Finding tables in backup"; - case Stage::kCreatingDatabases: return "Creating databases"; - case Stage::kCreatingTables: return "Creating tables"; - case Stage::kInsertingDataToTables: return "Inserting data to tables"; - case Stage::kError: return "Error"; + return (table_name.database == DatabaseCatalog::SYSTEM_DATABASE) && (table_name.table == "functions"); } - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown restore stage: {}", static_cast(stage)); -} + } RestorerFromBackup::RestorerFromBackup( @@ -94,71 +100,66 @@ RestorerFromBackup::RestorerFromBackup( const RestoreSettings & restore_settings_, std::shared_ptr restore_coordination_, const BackupPtr & backup_, - const ContextMutablePtr & context_, - std::chrono::seconds timeout_) + const ContextMutablePtr & context_) : restore_query_elements(restore_query_elements_) , restore_settings(restore_settings_) , restore_coordination(restore_coordination_) , backup(backup_) , context(context_) - , timeout(timeout_) + , create_table_timeout(context->getConfigRef().getUInt64("backups.create_table_timeout", 300000)) , log(&Poco::Logger::get("RestorerFromBackup")) { } RestorerFromBackup::~RestorerFromBackup() = default; -void RestorerFromBackup::restoreMetadata() -{ - run(/* only_check_access= */ false); -} - -void RestorerFromBackup::checkAccessOnly() -{ - run(/* only_check_access= */ true); -} - -void RestorerFromBackup::run(bool only_check_access) +RestorerFromBackup::DataRestoreTasks RestorerFromBackup::run(Mode mode) { try { - /// restoreMetadata() must not be called multiple times. - if (current_stage != Stage::kPreparing) + /// run() can be called onle once. + if (!current_status.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Already restoring"); - /// Calculate the root path in the backup for restoring, it's either empty or has the format "shards//replicas//". - findRootPathsInBackup(); + /// Find other hosts working along with us to execute this ON CLUSTER query. + all_hosts = BackupSettings::Util::filterHostIDs( + restore_settings.cluster_host_ids, restore_settings.shard_num, restore_settings.replica_num); /// Do renaming in the create queries according to the renaming config. renaming_map = makeRenamingMapFromBackupQuery(restore_query_elements); + /// Calculate the root path in the backup for restoring, it's either empty or has the format "shards//replicas//". + findRootPathsInBackup(); + /// Find all the databases and tables which we will read from the backup. - setStage(Stage::kFindingTablesInBackup); - collectDatabaseAndTableInfos(); + setStatus(kFindingTablesInBackupStatus); + findDatabasesAndTablesInBackup(); /// Check access rights. - checkAccessForCollectedInfos(); - if (only_check_access) - return; + checkAccessForObjectsFoundInBackup(); + + if (mode == Mode::CHECK_ACCESS_ONLY) + return {}; /// Create databases using the create queries read from the backup. - setStage(Stage::kCreatingDatabases); + setStatus(kCreatingDatabasesStatus); createDatabases(); /// Create tables using the create queries read from the backup. - setStage(Stage::kCreatingTables); + setStatus(kCreatingTablesStatus); createTables(); /// All what's left is to insert data to tables. /// No more data restoring tasks are allowed after this point. - setStage(Stage::kInsertingDataToTables); + setStatus(kInsertingDataToTablesStatus); + return getDataRestoreTasks(); } catch (...) { try { /// Other hosts should know that we've encountered an error. - setStage(Stage::kError, getCurrentExceptionMessage(false)); + setStatus(kErrorStatus, getCurrentExceptionMessage(false)); } catch (...) { @@ -167,59 +168,20 @@ void RestorerFromBackup::run(bool only_check_access) } } - -RestorerFromBackup::DataRestoreTasks RestorerFromBackup::getDataRestoreTasks() +void RestorerFromBackup::setStatus(const String & new_status, const String & message) { - if (current_stage != Stage::kInsertingDataToTables) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Metadata wasn't restored"); - - if (data_restore_tasks.empty() && !access_restore_task) - return {}; - - LOG_TRACE(log, "Will insert data to tables"); - - /// Storages and table locks must exist while we're executing data restoring tasks. - auto storages = std::make_shared>(); - auto table_locks = std::make_shared>(); - storages->reserve(table_infos.size()); - table_locks->reserve(table_infos.size()); - for (const auto & table_info : table_infos | boost::adaptors::map_values) + if (new_status == kErrorStatus) { - storages->push_back(table_info.storage); - table_locks->push_back(table_info.table_lock); - } - - DataRestoreTasks res_tasks; - for (const auto & task : data_restore_tasks) - res_tasks.push_back([task, storages, table_locks] { task(); }); - - if (access_restore_task) - res_tasks.push_back([task = access_restore_task, access_control = &context->getAccessControl()] { task->restore(*access_control); }); - - return res_tasks; -} - -void RestorerFromBackup::setStage(Stage new_stage, const String & error_message) -{ - if (new_stage == Stage::kError) - LOG_ERROR(log, "{} failed with error: {}", toString(current_stage), error_message); - else - LOG_TRACE(log, "{}", toString(new_stage)); - - current_stage = new_stage; - - if (!restore_coordination) - return; - - if (new_stage == Stage::kError) - { - restore_coordination->syncStageError(restore_settings.host_id, error_message); + LOG_ERROR(log, "{} failed with {}", toUpperFirst(current_status), message); + if (restore_coordination) + restore_coordination->setStatus(restore_settings.host_id, new_status, message); } else { - auto all_hosts - = BackupSettings::Util::filterHostIDs(restore_settings.cluster_host_ids, restore_settings.shard_num, restore_settings.replica_num); - restore_coordination->syncStage(restore_settings.host_id, static_cast(new_stage), all_hosts, timeout); + LOG_TRACE(log, "{}", toUpperFirst(new_status)); + current_status = new_status; + if (restore_coordination) + restore_coordination->setStatusAndWait(restore_settings.host_id, new_status, message, all_hosts); } } @@ -302,7 +264,7 @@ void RestorerFromBackup::findRootPathsInBackup() ", ")); } -void RestorerFromBackup::collectDatabaseAndTableInfos() +void RestorerFromBackup::findDatabasesAndTablesInBackup() { database_infos.clear(); table_infos.clear(); @@ -312,22 +274,22 @@ void RestorerFromBackup::collectDatabaseAndTableInfos() { case ASTBackupQuery::ElementType::TABLE: { - collectTableInfo({element.database_name, element.table_name}, false, element.partitions); + findTableInBackup({element.database_name, element.table_name}, element.partitions); break; } case ASTBackupQuery::ElementType::TEMPORARY_TABLE: { - collectTableInfo({element.database_name, element.table_name}, true, element.partitions); + findTableInBackup({DatabaseCatalog::TEMPORARY_DATABASE, element.table_name}, element.partitions); break; } case ASTBackupQuery::ElementType::DATABASE: { - collectDatabaseInfo(element.database_name, element.except_tables, /* throw_if_no_database_metadata_in_backup= */ true); + findDatabaseInBackup(element.database_name, element.except_tables); break; } case ASTBackupQuery::ElementType::ALL: { - collectAllDatabasesInfo(element.except_databases, element.except_tables); + findEverythingInBackup(element.except_databases, element.except_tables); break; } } @@ -336,9 +298,9 @@ void RestorerFromBackup::collectDatabaseAndTableInfos() LOG_INFO(log, "Will restore {} databases and {} tables", database_infos.size(), table_infos.size()); } -void RestorerFromBackup::collectTableInfo(const QualifiedTableName & table_name_in_backup, bool is_temporary_table, const std::optional & partitions) +void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name_in_backup, const std::optional & partitions) { - String database_name_in_backup = is_temporary_table ? DatabaseCatalog::TEMPORARY_DATABASE : table_name_in_backup.database; + bool is_temporary_table = (table_name_in_backup.database == DatabaseCatalog::TEMPORARY_DATABASE); std::optional metadata_path; std::optional root_path_in_use; @@ -365,21 +327,20 @@ void RestorerFromBackup::collectTableInfo(const QualifiedTableName & table_name_ } if (!metadata_path) - throw Exception(ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Table {} not found in backup", table_name_in_backup.getFullName()); + throw Exception( + ErrorCodes::BACKUP_ENTRY_NOT_FOUND, + "{} not found in backup", + tableNameWithTypeToString(table_name_in_backup.database, table_name_in_backup.table, true)); - TableKey table_key; fs::path data_path_in_backup; if (is_temporary_table) { data_path_in_backup = *root_path_in_use / "temporary_tables" / "data" / escapeForFileName(table_name_in_backup.table); - table_key.name.table = renaming_map.getNewTemporaryTableName(table_name_in_backup.table); - table_key.is_temporary = true; } else { data_path_in_backup = *root_path_in_use / "data" / escapeForFileName(table_name_in_backup.database) / escapeForFileName(table_name_in_backup.table); - table_key.name = renaming_map.getNewTableName(table_name_in_backup); } auto read_buffer = backup->readFile(*metadata_path)->getReadBuffer(); @@ -388,27 +349,30 @@ void RestorerFromBackup::collectTableInfo(const QualifiedTableName & table_name_ read_buffer.reset(); ParserCreateQuery create_parser; ASTPtr create_table_query = parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); - renameDatabaseAndTableNameInCreateQuery(context->getGlobalContext(), renaming_map, create_table_query); + renameDatabaseAndTableNameInCreateQuery(create_table_query, renaming_map, context->getGlobalContext()); - if (auto it = table_infos.find(table_key); it != table_infos.end()) + QualifiedTableName table_name = renaming_map.getNewTableName(table_name_in_backup); + + if (auto it = table_infos.find(table_name); it != table_infos.end()) { const TableInfo & table_info = it->second; if (table_info.create_table_query && (serializeAST(*table_info.create_table_query) != serializeAST(*create_table_query))) { throw Exception( ErrorCodes::CANNOT_RESTORE_TABLE, - "Extracted two different create queries for the same {}table {}: {} and {}", - (is_temporary_table ? "temporary " : ""), - table_key.name.getFullName(), + "Extracted two different create queries for the same {}: {} and {}", + tableNameWithTypeToString(table_name.database, table_name.table, false), serializeAST(*table_info.create_table_query), serializeAST(*create_table_query)); } } - TableInfo & res_table_info = table_infos[table_key]; + TableInfo & res_table_info = table_infos[table_name]; res_table_info.create_table_query = create_table_query; + res_table_info.is_predefined_table = DatabaseCatalog::instance().isPredefinedTable(StorageID{table_name.database, table_name.table}); + res_table_info.dependencies = getDependenciesSetFromCreateQuery(context->getGlobalContext(), table_name, create_table_query); + res_table_info.has_data = backup->hasFiles(data_path_in_backup); res_table_info.data_path_in_backup = data_path_in_backup; - res_table_info.dependencies = getDependenciesSetFromCreateQuery(context->getGlobalContext(), table_key.name, create_table_query); if (partitions) { @@ -417,35 +381,45 @@ void RestorerFromBackup::collectTableInfo(const QualifiedTableName & table_name_ insertAtEnd(*res_table_info.partitions, *partitions); } - if (hasSystemAccessTableEngine(*create_table_query)) + if (!restore_settings.structure_only && isSystemAccessTableName(table_name)) { - if (!access_restore_task) - access_restore_task = std::make_shared(backup, restore_settings, restore_coordination); - access_restore_task->addDataPath(data_path_in_backup); + if (!access_restorer) + access_restorer = std::make_unique(backup, restore_settings); + access_restorer->addDataPath(data_path_in_backup, table_name); } } -void RestorerFromBackup::collectDatabaseInfo(const String & database_name_in_backup, const std::set & except_table_names, bool throw_if_no_database_metadata_in_backup) +void RestorerFromBackup::findDatabaseInBackup(const String & database_name_in_backup, const std::set & except_table_names) { std::optional metadata_path; std::unordered_set table_names_in_backup; for (const auto & root_path_in_backup : root_paths_in_backup) { - fs::path try_metadata_path = root_path_in_backup / "metadata" / (escapeForFileName(database_name_in_backup) + ".sql"); - if (!metadata_path && backup->fileExists(try_metadata_path)) + fs::path try_metadata_path, try_tables_metadata_path; + if (database_name_in_backup == DatabaseCatalog::TEMPORARY_DATABASE) + { + try_tables_metadata_path = root_path_in_backup / "temporary_tables" / "metadata"; + } + else + { + try_metadata_path = root_path_in_backup / "metadata" / (escapeForFileName(database_name_in_backup) + ".sql"); + try_tables_metadata_path = root_path_in_backup / "metadata" / escapeForFileName(database_name_in_backup); + } + + if (!metadata_path && !try_metadata_path.empty() && backup->fileExists(try_metadata_path)) metadata_path = try_metadata_path; - Strings file_names = backup->listFiles(root_path_in_backup / "metadata" / escapeForFileName(database_name_in_backup)); + Strings file_names = backup->listFiles(try_tables_metadata_path); for (const String & file_name : file_names) { - if (!file_name.ends_with(sql_ext)) + if (!file_name.ends_with(".sql")) continue; - String file_name_without_ext = file_name.substr(0, file_name.length() - sql_ext.length()); + String file_name_without_ext = file_name.substr(0, file_name.length() - strlen(".sql")); table_names_in_backup.insert(unescapeForFileName(file_name_without_ext)); } } - if (!metadata_path && throw_if_no_database_metadata_in_backup) + if (!metadata_path && table_names_in_backup.empty()) throw Exception(ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Database {} not found in backup", backQuoteIfNeed(database_name_in_backup)); if (metadata_path) @@ -456,7 +430,7 @@ void RestorerFromBackup::collectDatabaseInfo(const String & database_name_in_bac read_buffer.reset(); ParserCreateQuery create_parser; ASTPtr create_database_query = parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); - renameDatabaseAndTableNameInCreateQuery(context->getGlobalContext(), renaming_map, create_database_query); + renameDatabaseAndTableNameInCreateQuery(create_database_query, renaming_map, context->getGlobalContext()); String database_name = renaming_map.getNewDatabaseName(database_name_in_backup); DatabaseInfo & database_info = database_infos[database_name]; @@ -472,6 +446,7 @@ void RestorerFromBackup::collectDatabaseInfo(const String & database_name_in_bac } database_info.create_database_query = create_database_query; + database_info.is_predefined_database = DatabaseCatalog::isPredefinedDatabase(database_name); } for (const String & table_name_in_backup : table_names_in_backup) @@ -479,33 +454,26 @@ void RestorerFromBackup::collectDatabaseInfo(const String & database_name_in_bac if (except_table_names.contains({database_name_in_backup, table_name_in_backup})) continue; - collectTableInfo({database_name_in_backup, table_name_in_backup}, /* is_temporary_table= */ false, /* partitions= */ {}); + findTableInBackup({database_name_in_backup, table_name_in_backup}, /* partitions= */ {}); } } -void RestorerFromBackup::collectAllDatabasesInfo(const std::set & except_database_names, const std::set & except_table_names) +void RestorerFromBackup::findEverythingInBackup(const std::set & except_database_names, const std::set & except_table_names) { std::unordered_set database_names_in_backup; - std::unordered_set temporary_table_names_in_backup; for (const auto & root_path_in_backup : root_paths_in_backup) { Strings file_names = backup->listFiles(root_path_in_backup / "metadata"); for (String & file_name : file_names) { - if (file_name.ends_with(sql_ext)) - file_name.resize(file_name.length() - sql_ext.length()); + if (file_name.ends_with(".sql")) + file_name.resize(file_name.length() - strlen(".sql")); database_names_in_backup.emplace(unescapeForFileName(file_name)); } - file_names = backup->listFiles(root_path_in_backup / "temporary_tables" / "metadata"); - for (String & file_name : file_names) - { - if (!file_name.ends_with(sql_ext)) - continue; - file_name.resize(file_name.length() - sql_ext.length()); - temporary_table_names_in_backup.emplace(unescapeForFileName(file_name)); - } + if (backup->hasFiles(root_path_in_backup / "temporary_tables" / "metadata")) + database_names_in_backup.emplace(DatabaseCatalog::TEMPORARY_DATABASE); } for (const String & database_name_in_backup : database_names_in_backup) @@ -513,19 +481,16 @@ void RestorerFromBackup::collectAllDatabasesInfo(const std::set & except if (except_database_names.contains(database_name_in_backup)) continue; - collectDatabaseInfo(database_name_in_backup, except_table_names, /* throw_if_no_database_metadata_in_backup= */ false); + findDatabaseInBackup(database_name_in_backup, except_table_names); } - - for (const String & temporary_table_name_in_backup : temporary_table_names_in_backup) - collectTableInfo({"", temporary_table_name_in_backup}, /* is_temporary_table= */ true, /* partitions= */ {}); } -void RestorerFromBackup::checkAccessForCollectedInfos() const +void RestorerFromBackup::checkAccessForObjectsFoundInBackup() const { AccessRightsElements required_access; - for (const auto & database_name : database_infos | boost::adaptors::map_keys) + for (const auto & [database_name, database_info] : database_infos) { - if (DatabaseCatalog::isPredefinedDatabaseName(database_name)) + if (database_info.is_predefined_database) continue; AccessFlags flags; @@ -541,10 +506,20 @@ void RestorerFromBackup::checkAccessForCollectedInfos() const for (const auto & [table_name, table_info] : table_infos) { - if (hasSystemTableEngine(*table_info.create_table_query)) + if (table_info.is_predefined_table) + { + if (isSystemFunctionsTableName(table_name)) + { + /// CREATE_FUNCTION privilege is required to restore the "system.functions" table. + if (!restore_settings.structure_only && table_info.has_data) + required_access.emplace_back(AccessType::CREATE_FUNCTION); + } + /// Privileges required to restore ACL system tables are checked separately + /// (see access_restore_task->getRequiredAccess() below). continue; + } - if (table_name.is_temporary) + if (table_name.database == DatabaseCatalog::TEMPORARY_DATABASE) { if (restore_settings.create_table != RestoreTableCreationMode::kMustExist) required_access.emplace_back(AccessType::CREATE_TEMPORARY_TABLE); @@ -564,8 +539,7 @@ void RestorerFromBackup::checkAccessForCollectedInfos() const flags |= AccessType::CREATE_TABLE; } - if (!restore_settings.structure_only && !create.is_dictionary && !create.is_ordinary_view - && backup->hasFiles(table_info.data_path_in_backup)) + if (!restore_settings.structure_only && table_info.has_data) { flags |= AccessType::INSERT; } @@ -578,11 +552,11 @@ void RestorerFromBackup::checkAccessForCollectedInfos() const flags = AccessType::SHOW_TABLES; } - required_access.emplace_back(flags, table_name.name.database, table_name.name.table); + required_access.emplace_back(flags, table_name.database, table_name.table); } - if (access_restore_task) - insertAtEnd(required_access, access_restore_task->getRequiredAccess()); + if (access_restorer) + insertAtEnd(required_access, access_restorer->getRequiredAccess()); /// We convert to AccessRights and back to check access rights in a predictable way /// (some elements could be duplicated or not sorted). @@ -597,7 +571,7 @@ void RestorerFromBackup::createDatabases() for (const auto & [database_name, database_info] : database_infos) { bool need_create_database = (restore_settings.create_database != RestoreDatabaseCreationMode::kMustExist); - if (need_create_database && DatabaseCatalog::isPredefinedDatabaseName(database_name)) + if (database_info.is_predefined_database) need_create_database = false; /// Predefined databases always exist. if (need_create_database) @@ -610,15 +584,18 @@ void RestorerFromBackup::createDatabases() create_database_query->as().if_not_exists = true; } LOG_TRACE(log, "Creating database {}: {}", backQuoteIfNeed(database_name), serializeAST(*create_database_query)); - executeCreateQuery(create_database_query); + InterpreterCreateQuery interpreter{create_database_query, context}; + interpreter.setInternal(true); + interpreter.execute(); } DatabasePtr database = DatabaseCatalog::instance().getDatabase(database_name); - if (!restore_settings.allow_different_database_def) + if (!restore_settings.allow_different_database_def && !database_info.is_predefined_database) { /// Check that the database's definition is the same as expected. - ASTPtr create_database_query = database->getCreateDatabaseQueryForBackup(); + ASTPtr create_database_query = database->getCreateDatabaseQuery(); + adjustCreateQueryForBackup(create_database_query, context->getGlobalContext(), nullptr); ASTPtr expected_create_query = database_info.create_database_query; if (serializeAST(*create_database_query) != serializeAST(*expected_create_query)) { @@ -643,62 +620,62 @@ void RestorerFromBackup::createTables() if (tables_to_create.empty()) break; /// We've already created all the tables. - for (const auto & table_key : tables_to_create) + for (const auto & table_name : tables_to_create) { - auto & table_info = table_infos.at(table_key); + auto & table_info = table_infos.at(table_name); - DatabasePtr database; - if (table_key.is_temporary) - database = DatabaseCatalog::instance().getDatabaseForTemporaryTables(); - else - database = DatabaseCatalog::instance().getDatabase(table_key.name.database); + DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_name.database); bool need_create_table = (restore_settings.create_table != RestoreTableCreationMode::kMustExist); - if (need_create_table && hasSystemTableEngine(*table_info.create_table_query)) - need_create_table = false; /// Tables with System* table engine already exist or can't be created by SQL anyway. + if (table_info.is_predefined_table) + need_create_table = false; /// Predefined tables always exist. if (need_create_table) { - /// Execute CREATE TABLE query (we call IDatabase::createTableRestoredFromBackup() to allow the database to do some - /// database-specific things). auto create_table_query = table_info.create_table_query; if (restore_settings.create_table == RestoreTableCreationMode::kCreateIfNotExists) { create_table_query = create_table_query->clone(); create_table_query->as().if_not_exists = true; } + LOG_TRACE( log, - "Creating {}table {}: {}", - (table_key.is_temporary ? "temporary " : ""), - table_key.name.getFullName(), + "Creating {}: {}", + tableNameWithTypeToString(table_name.database, table_name.table, false), serializeAST(*create_table_query)); - database->createTableRestoredFromBackup(create_table_query, *this); + /// Execute CREATE TABLE query (we call IDatabase::createTableRestoredFromBackup() to allow the database to do some + /// database-specific things). + database->createTableRestoredFromBackup( + create_table_query, + context, + restore_coordination, + std::chrono::duration_cast(create_table_timeout).count()); } table_info.created = true; - auto resolved_id = table_key.is_temporary - ? context->resolveStorageID(StorageID{"", table_key.name.table}, Context::ResolveExternal) - : context->resolveStorageID(StorageID{table_key.name.database, table_key.name.table}, Context::ResolveGlobal); + auto resolved_id = (table_name.database == DatabaseCatalog::TEMPORARY_DATABASE) + ? context->resolveStorageID(StorageID{"", table_name.table}, Context::ResolveExternal) + : context->resolveStorageID(StorageID{table_name.database, table_name.table}, Context::ResolveGlobal); auto storage = database->getTable(resolved_id.table_name, context); table_info.storage = storage; table_info.table_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); - if (!restore_settings.allow_different_table_def) + if (!restore_settings.allow_different_table_def && !table_info.is_predefined_table) { - ASTPtr create_table_query = storage->getCreateQueryForBackup(context, nullptr); + ASTPtr create_table_query = database->getCreateTableQuery(resolved_id.table_name, context); + adjustCreateQueryForBackup(create_table_query, context->getGlobalContext(), nullptr); ASTPtr expected_create_query = table_info.create_table_query; if (serializeAST(*create_table_query) != serializeAST(*expected_create_query)) { throw Exception( ErrorCodes::CANNOT_RESTORE_TABLE, - "The {}table {} has a different definition: {} " + "{} has a different definition: {} " "comparing to its definition in the backup: {}", - (table_key.is_temporary ? "temporary " : ""), - table_key.name.getFullName(), + tableNameWithTypeToString(table_name.database, table_name.table, true), serializeAST(*create_table_query), serializeAST(*expected_create_query)); } @@ -708,6 +685,15 @@ void RestorerFromBackup::createTables() { const auto & data_path_in_backup = table_info.data_path_in_backup; const auto & partitions = table_info.partitions; + if (partitions && !storage->supportsBackupPartition()) + { + throw Exception( + ErrorCodes::CANNOT_RESTORE_TABLE, + "Table engine {} doesn't support partitions, cannot restore {}", + storage->getName(), + tableNameWithTypeToString(table_name.database, table_name.table, false)); + } + storage->restoreDataFromBackup(*this, data_path_in_backup, partitions); } } @@ -715,9 +701,9 @@ void RestorerFromBackup::createTables() } /// Returns the list of tables without dependencies or those which dependencies have been created before. -std::vector RestorerFromBackup::findTablesWithoutDependencies() const +std::vector RestorerFromBackup::findTablesWithoutDependencies() const { - std::vector tables_without_dependencies; + std::vector tables_without_dependencies; bool all_tables_created = true; for (const auto & [key, table_info] : table_infos) @@ -732,7 +718,7 @@ std::vector RestorerFromBackup::findTablesWithoutD bool all_dependencies_met = true; for (const auto & dependency : table_info.dependencies) { - auto it = table_infos.find(TableKey{dependency, false}); + auto it = table_infos.find(dependency); if ((it != table_infos.end()) && !it->second.created) { all_dependencies_met = false; @@ -751,7 +737,7 @@ std::vector RestorerFromBackup::findTablesWithoutD return {}; /// Cyclic dependency? We'll try to create those tables anyway but probably it's going to fail. - std::vector tables_with_cyclic_dependencies; + std::vector tables_with_cyclic_dependencies; for (const auto & [key, table_info] : table_infos) { if (!table_info.created) @@ -764,7 +750,7 @@ std::vector RestorerFromBackup::findTablesWithoutD "Some tables have cyclic dependency from each other: {}", boost::algorithm::join( tables_with_cyclic_dependencies - | boost::adaptors::transformed([](const TableKey & key) -> String { return key.name.getFullName(); }), + | boost::adaptors::transformed([](const QualifiedTableName & table_name) -> String { return table_name.getFullName(); }), ", ")); return tables_with_cyclic_dependencies; @@ -772,38 +758,52 @@ std::vector RestorerFromBackup::findTablesWithoutD void RestorerFromBackup::addDataRestoreTask(DataRestoreTask && new_task) { - if (current_stage == Stage::kInsertingDataToTables) + if (current_status == kInsertingDataToTablesStatus) throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding data-restoring tasks is not allowed"); data_restore_tasks.push_back(std::move(new_task)); } void RestorerFromBackup::addDataRestoreTasks(DataRestoreTasks && new_tasks) { - if (current_stage == Stage::kInsertingDataToTables) + if (current_status == kInsertingDataToTablesStatus) throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding data-restoring tasks is not allowed"); insertAtEnd(data_restore_tasks, std::move(new_tasks)); } -void RestorerFromBackup::checkPathInBackupToRestoreAccess(const String & path) +RestorerFromBackup::DataRestoreTasks RestorerFromBackup::getDataRestoreTasks() { - if (!access_restore_task || !access_restore_task->hasDataPath(path)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Path to restore access was not added"); + if (data_restore_tasks.empty()) + return {}; + + LOG_TRACE(log, "Will insert data to tables"); + + /// Storages and table locks must exist while we're executing data restoring tasks. + auto storages = std::make_shared>(); + auto table_locks = std::make_shared>(); + storages->reserve(table_infos.size()); + table_locks->reserve(table_infos.size()); + for (const auto & table_info : table_infos | boost::adaptors::map_values) + { + storages->push_back(table_info.storage); + table_locks->push_back(table_info.table_lock); + } + + DataRestoreTasks res_tasks; + for (const auto & task : data_restore_tasks) + res_tasks.push_back([task, storages, table_locks] { task(); }); + + return res_tasks; } -void RestorerFromBackup::executeCreateQuery(const ASTPtr & create_query) const +std::vector> RestorerFromBackup::getAccessEntitiesToRestore() { - InterpreterCreateQuery interpreter{create_query, context}; - interpreter.setInternal(true); - interpreter.execute(); -} + if (!access_restorer || access_restored) + return {}; -void RestorerFromBackup::throwPartitionsNotSupported(const StorageID & storage_id, const String & table_engine) -{ - throw Exception( - ErrorCodes::CANNOT_RESTORE_TABLE, - "Table engine {} doesn't support partitions, cannot table {}", - table_engine, - storage_id.getFullTableName()); + /// getAccessEntitiesToRestore() will return entities only when called first time (we don't want to restore the same entities again). + access_restored = true; + + return access_restorer->getAccessEntities(context->getAccessControl()); } void RestorerFromBackup::throwTableIsNotEmpty(const StorageID & storage_id) diff --git a/src/Backups/RestorerFromBackup.h b/src/Backups/RestorerFromBackup.h index 65139e0b946..ae2f0c76832 100644 --- a/src/Backups/RestorerFromBackup.h +++ b/src/Backups/RestorerFromBackup.h @@ -15,7 +15,9 @@ class IBackup; using BackupPtr = std::shared_ptr; class IRestoreCoordination; struct StorageID; -class AccessRestoreTask; +class AccessRestorerFromBackup; +struct IAccessEntity; +using AccessEntityPtr = std::shared_ptr; /// Restores the definition of databases and tables and prepares tasks to restore the data of the tables. class RestorerFromBackup : private boost::noncopyable @@ -26,63 +28,38 @@ public: const RestoreSettings & restore_settings_, std::shared_ptr restore_coordination_, const BackupPtr & backup_, - const ContextMutablePtr & context_, - std::chrono::seconds timeout_); + const ContextMutablePtr & context_); ~RestorerFromBackup(); - /// Restores the definition of databases and tables and prepares tasks to restore the data of the tables. - /// restoreMetadata() checks access rights internally so checkAccessRightsOnly() shouldn't be called first. - void restoreMetadata(); + enum Mode + { + /// Restores databases and tables. + RESTORE, - /// Only checks access rights without restoring anything. - void checkAccessOnly(); + /// Only checks access rights without restoring anything. + CHECK_ACCESS_ONLY + }; using DataRestoreTask = std::function; using DataRestoreTasks = std::vector; - DataRestoreTasks getDataRestoreTasks(); + + /// Restores the metadata of databases and tables and returns tasks to restore the data of tables. + DataRestoreTasks run(Mode mode); BackupPtr getBackup() const { return backup; } const RestoreSettings & getRestoreSettings() const { return restore_settings; } bool isNonEmptyTableAllowed() const { return getRestoreSettings().allow_non_empty_tables; } std::shared_ptr getRestoreCoordination() const { return restore_coordination; } - std::chrono::seconds getTimeout() const { return timeout; } ContextMutablePtr getContext() const { return context; } - void executeCreateQuery(const ASTPtr & create_query) const; /// Adds a data restore task which will be later returned by getDataRestoreTasks(). /// This function can be called by implementations of IStorage::restoreFromBackup() in inherited storage classes. void addDataRestoreTask(DataRestoreTask && new_task); void addDataRestoreTasks(DataRestoreTasks && new_tasks); - /// Adds a new data path to restore access control. - void checkPathInBackupToRestoreAccess(const String & path); - - /// Reading a backup includes a few stages: - enum class Stage - { - /// Initial stage. - kPreparing, - - /// Finding databases and tables in the backup which we're going to restore. - kFindingTablesInBackup, - - /// Creating databases or finding them and checking their definitions. - kCreatingDatabases, - - /// Creating tables or finding them and checking their definition. - kCreatingTables, - - /// Inserting restored data to tables. - kInsertingDataToTables, - - /// An error happens during any of the stages above, the backup is not restored properly. - kError = -1, - }; - static std::string_view toString(Stage stage); - - /// Throws an exception that a specified table engine doesn't support partitions. - [[noreturn]] static void throwPartitionsNotSupported(const StorageID & storage_id, const String & table_engine); + /// Returns the list of access entities to restore. + std::vector> getAccessEntitiesToRestore(); /// Throws an exception that a specified table is already non-empty. [[noreturn]] static void throwTableIsNotEmpty(const StorageID & storage_id); @@ -93,54 +70,56 @@ private: std::shared_ptr restore_coordination; BackupPtr backup; ContextMutablePtr context; - std::chrono::seconds timeout; + std::chrono::milliseconds create_table_timeout; Poco::Logger * log; - Stage current_stage = Stage::kPreparing; - std::vector root_paths_in_backup; + Strings all_hosts; DDLRenamingMap renaming_map; + std::vector root_paths_in_backup; - void run(bool only_check_access); - void setStage(Stage new_stage, const String & error_message = {}); void findRootPathsInBackup(); - void collectDatabaseAndTableInfos(); - void collectTableInfo(const QualifiedTableName & table_name_in_backup, bool is_temporary_table, const std::optional & partitions); - void collectDatabaseInfo(const String & database_name_in_backup, const std::set & except_table_names, bool throw_if_no_database_metadata_in_backup); - void collectAllDatabasesInfo(const std::set & except_database_names, const std::set & except_table_names); - void checkAccessForCollectedInfos() const; + + void findDatabasesAndTablesInBackup(); + void findTableInBackup(const QualifiedTableName & table_name_in_backup, const std::optional & partitions); + void findDatabaseInBackup(const String & database_name_in_backup, const std::set & except_table_names); + void findEverythingInBackup(const std::set & except_database_names, const std::set & except_table_names); + + void checkAccessForObjectsFoundInBackup() const; + void createDatabases(); void createTables(); + DataRestoreTasks getDataRestoreTasks(); + + void setStatus(const String & new_status, const String & message = ""); + struct DatabaseInfo { ASTPtr create_database_query; + bool is_predefined_database = false; }; struct TableInfo { ASTPtr create_table_query; - std::optional partitions; - std::filesystem::path data_path_in_backup; + bool is_predefined_table = false; std::unordered_set dependencies; + bool has_data = false; + std::filesystem::path data_path_in_backup; + std::optional partitions; bool created = false; StoragePtr storage; TableLockHolder table_lock; }; - struct TableKey - { - QualifiedTableName name; - bool is_temporary = false; - bool operator ==(const TableKey & right) const; - bool operator <(const TableKey & right) const; - }; - - std::vector findTablesWithoutDependencies() const; + std::vector findTablesWithoutDependencies() const; + String current_status; std::unordered_map database_infos; - std::map table_infos; + std::map table_infos; std::vector data_restore_tasks; - std::shared_ptr access_restore_task; + std::unique_ptr access_restorer; + bool access_restored = false; }; } diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 6f2ac41cc08..8e7eaf4c6e6 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -631,7 +631,7 @@ M(660, HDFS_ERROR) \ M(661, CANNOT_SEND_SIGNAL) \ M(662, FS_METADATA_ERROR) \ - M(663, CANNOT_COLLECT_OBJECTS_FOR_BACKUP) \ + M(663, INCONSISTENT_METADATA_FOR_BACKUP) \ M(664, ACCESS_STORAGE_DOESNT_ALLOW_BACKUP) \ \ M(999, KEEPER_EXCEPTION) \ diff --git a/src/Databases/DDLRenamingVisitor.cpp b/src/Databases/DDLRenamingVisitor.cpp index caedfc55f3d..7ea5dbeda83 100644 --- a/src/Databases/DDLRenamingVisitor.cpp +++ b/src/Databases/DDLRenamingVisitor.cpp @@ -19,7 +19,6 @@ namespace DB namespace ErrorCodes { extern const int WRONG_DDL_RENAMING_SETTINGS; - extern const int LOGICAL_ERROR; } namespace @@ -31,24 +30,40 @@ namespace { /// CREATE TEMPORARY TABLE String table_name = create.getTable(); - const auto & new_table_name = data.renaming_map.getNewTemporaryTableName(table_name); - if (new_table_name != table_name) - create.setTable(new_table_name); + QualifiedTableName full_table_name{DatabaseCatalog::TEMPORARY_DATABASE, table_name}; + const auto & new_table_name = data.renaming_map.getNewTableName(full_table_name); + if (new_table_name != full_table_name) + { + create.setTable(new_table_name.table); + if (new_table_name.database != DatabaseCatalog::TEMPORARY_DATABASE) + { + create.temporary = false; + create.setDatabase(new_table_name.database); + } + } } else if (create.table) { /// CREATE TABLE or CREATE DICTIONARY or CREATE VIEW - QualifiedTableName qualified_name; - qualified_name.table = create.getTable(); - qualified_name.database = create.getDatabase(); + QualifiedTableName full_name; + full_name.table = create.getTable(); + full_name.database = create.getDatabase(); - if (!qualified_name.database.empty() && !qualified_name.table.empty()) + if (!full_name.database.empty() && !full_name.table.empty()) { - auto new_qualified_name = data.renaming_map.getNewTableName(qualified_name); - if (new_qualified_name != qualified_name) + auto new_table_name = data.renaming_map.getNewTableName(full_name); + if (new_table_name != full_name) { - create.setTable(new_qualified_name.table); - create.setDatabase(new_qualified_name.database); + create.setTable(new_table_name.table); + if (new_table_name.database == DatabaseCatalog::TEMPORARY_DATABASE) + { + create.temporary = true; + create.setDatabase(""); + } + else + { + create.setDatabase(new_table_name.database); + } } } } @@ -291,17 +306,10 @@ void DDLRenamingVisitor::visit(ASTPtr ast, const Data & data) bool DDLRenamingVisitor::needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } -void renameDatabaseAndTableNameInCreateQuery(const ContextPtr & global_context, const DDLRenamingMap & renaming_map, ASTPtr & ast) -{ - DDLRenamingVisitor::Data data{global_context, renaming_map, ast}; - DDLRenamingVisitor::Visitor{data}.visit(ast); -} - - void DDLRenamingMap::setNewTableName(const QualifiedTableName & old_table_name, const QualifiedTableName & new_table_name) { if (old_table_name.table.empty() || old_table_name.database.empty() || new_table_name.table.empty() || new_table_name.database.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty names are not allowed"); + throw Exception(ErrorCodes::WRONG_DDL_RENAMING_SETTINGS, "Empty names are not allowed"); auto it = old_to_new_table_names.find(old_table_name); if ((it != old_to_new_table_names.end())) @@ -321,7 +329,7 @@ void DDLRenamingMap::setNewTableName(const QualifiedTableName & old_table_name, void DDLRenamingMap::setNewDatabaseName(const String & old_database_name, const String & new_database_name) { if (old_database_name.empty() || new_database_name.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty names are not allowed"); + throw Exception(ErrorCodes::WRONG_DDL_RENAMING_SETTINGS, "Empty names are not allowed"); auto it = old_to_new_database_names.find(old_database_name); if ((it != old_to_new_database_names.end())) @@ -351,28 +359,11 @@ QualifiedTableName DDLRenamingMap::getNewTableName(const QualifiedTableName & ol return {getNewDatabaseName(old_table_name.database), old_table_name.table}; } -void DDLRenamingMap::setNewTemporaryTableName(const String & old_table_name, const String & new_table_name) + +void renameDatabaseAndTableNameInCreateQuery(ASTPtr ast, const DDLRenamingMap & renaming_map, const ContextPtr & global_context) { - if (old_table_name.empty() || new_table_name.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty names are not allowed"); - - auto it = old_to_new_temporary_table_names.find(old_table_name); - if ((it != old_to_new_temporary_table_names.end())) - { - if (it->second == new_table_name) - return; - throw Exception(ErrorCodes::WRONG_DDL_RENAMING_SETTINGS, "Wrong renaming: it's specified that temporary table {} should be renamed to {} and to {} at the same time", - backQuoteIfNeed(old_table_name), backQuoteIfNeed(it->second), backQuoteIfNeed(new_table_name)); - } - old_to_new_temporary_table_names[old_table_name] = new_table_name; -} - -const String & DDLRenamingMap::getNewTemporaryTableName(const String & old_table_name) const -{ - auto it = old_to_new_temporary_table_names.find(old_table_name); - if (it != old_to_new_temporary_table_names.end()) - return it->second; - return old_table_name; + DDLRenamingVisitor::Data data{ast, renaming_map, global_context}; + DDLRenamingVisitor::Visitor{data}.visit(ast); } } diff --git a/src/Databases/DDLRenamingVisitor.h b/src/Databases/DDLRenamingVisitor.h index 9d0f770d105..44146a8ee6b 100644 --- a/src/Databases/DDLRenamingVisitor.h +++ b/src/Databases/DDLRenamingVisitor.h @@ -17,7 +17,7 @@ class DDLRenamingMap; /// Changes names of databases or tables in a create query according to a specified renaming map. /// Does not validate AST, works a best-effort way. -void renameDatabaseAndTableNameInCreateQuery(const ContextPtr & global_context, const DDLRenamingMap & renaming_map, ASTPtr & ast); +void renameDatabaseAndTableNameInCreateQuery(ASTPtr ast, const DDLRenamingMap & renaming_map, const ContextPtr & global_context); /// Renaming map keeps information about new names of databases or tables. class DDLRenamingMap @@ -25,16 +25,13 @@ class DDLRenamingMap public: void setNewTableName(const QualifiedTableName & old_table_name, const QualifiedTableName & new_table_name); void setNewDatabaseName(const String & old_database_name, const String & new_database_name); - void setNewTemporaryTableName(const String & old_table_name, const String & new_table_name); QualifiedTableName getNewTableName(const QualifiedTableName & old_table_name) const; const String & getNewDatabaseName(const String & old_database_name) const; - const String & getNewTemporaryTableName(const String & old_table_name) const; private: std::unordered_map old_to_new_table_names; std::unordered_map old_to_new_database_names; - std::unordered_map old_to_new_temporary_table_names; }; /// Visits ASTCreateQuery and changes names of databases or tables. @@ -43,9 +40,9 @@ class DDLRenamingVisitor public: struct Data { - ContextPtr global_context; - const DDLRenamingMap & renaming_map; ASTPtr create_query; + const DDLRenamingMap & renaming_map; + ContextPtr global_context; }; using Visitor = InDepthNodeVisitor; diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 5268252731f..8540c785419 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -19,6 +19,7 @@ namespace ErrorCodes { extern const int UNKNOWN_TABLE; extern const int LOGICAL_ERROR; + extern const int INCONSISTENT_METADATA_FOR_BACKUP; } DatabaseMemory::DatabaseMemory(const String & name_, ContextPtr context_) @@ -145,4 +146,42 @@ void DatabaseMemory::alterTable(ContextPtr local_context, const StorageID & tabl DatabaseCatalog::instance().updateLoadingDependencies(table_id, std::move(new_dependencies)); } +std::vector> DatabaseMemory::getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const +{ + /// We need a special processing for the temporary database. + if (getDatabaseName() != DatabaseCatalog::TEMPORARY_DATABASE) + return DatabaseWithOwnTablesBase::getTablesForBackup(filter, local_context); + + std::vector> res; + + /// `this->tables` for the temporary database doesn't contain real names of tables. + /// That's why we need to call Context::getExternalTables() and then resolve those names using tryResolveStorageID() below. + auto external_tables = local_context->getExternalTables(); + + for (const auto & [table_name, storage] : external_tables) + { + if (!filter(table_name)) + continue; + + auto storage_id = local_context->tryResolveStorageID(StorageID{"", table_name}, Context::ResolveExternal); + if (!storage_id) + throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "Couldn't resolve the name of temporary table {}", backQuoteIfNeed(table_name)); + + /// Here `storage_id.table_name` looks like looks like "_tmp_ab9b15a3-fb43-4670-abec-14a0e9eb70f1" + /// it's not the real name of the table. + auto create_table_query = tryGetCreateTableQuery(storage_id.table_name, local_context); + if (!create_table_query) + throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "Couldn't get a create query for temporary table {}", backQuoteIfNeed(table_name)); + + const auto & create = create_table_query->as(); + if (create.getTable() != table_name) + throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "Got a create query with unexpected name {} for temporary table {}", backQuoteIfNeed(create.getTable()), backQuoteIfNeed(table_name)); + + storage->adjustCreateQueryForBackup(create_table_query); + res.emplace_back(create_table_query, storage); + } + + return res; +} + } diff --git a/src/Databases/DatabaseMemory.h b/src/Databases/DatabaseMemory.h index eef9f306343..6262543b0c1 100644 --- a/src/Databases/DatabaseMemory.h +++ b/src/Databases/DatabaseMemory.h @@ -50,6 +50,8 @@ public: void alterTable(ContextPtr local_context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) override; + std::vector> getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const override; + private: const String data_path; using NameToASTCreate = std::unordered_map; diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 5a22eeaf570..a7ad632efff 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -44,6 +44,7 @@ namespace ErrorCodes extern const int INCORRECT_QUERY; extern const int ALL_CONNECTION_TRIES_FAILED; extern const int NO_ACTIVE_REPLICAS; + extern const int INCONSISTENT_METADATA_FOR_BACKUP; extern const int CANNOT_RESTORE_TABLE; } @@ -923,7 +924,50 @@ String DatabaseReplicated::readMetadataFile(const String & table_name) const } -void DatabaseReplicated::createTableRestoredFromBackup(const ASTPtr & create_table_query, const RestorerFromBackup & restorer) +std::vector> +DatabaseReplicated::getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr &) const +{ + /// Here we read metadata from ZooKeeper. We could do that by simple call of DatabaseAtomic::getTablesForBackup() however + /// reading from ZooKeeper is better because thus we won't be dependent on how fast the replication queue of this database is. + std::vector> res; + auto zookeeper = getContext()->getZooKeeper(); + auto escaped_table_names = zookeeper->getChildren(zookeeper_path + "/metadata"); + for (const auto & escaped_table_name : escaped_table_names) + { + String table_name = unescapeForFileName(escaped_table_name); + if (!filter(table_name)) + continue; + String zk_metadata; + if (!zookeeper->tryGet(zookeeper_path + "/metadata/" + escaped_table_name, zk_metadata)) + throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "Metadata for table {} was not found in ZooKeeper", table_name); + + ParserCreateQuery parser; + auto create_table_query = parseQuery(parser, zk_metadata, 0, getContext()->getSettingsRef().max_parser_depth); + + auto & create = create_table_query->as(); + create.attach = false; + create.setTable(table_name); + create.setDatabase(getDatabaseName()); + + StoragePtr storage; + if (create.uuid != UUIDHelpers::Nil) + { + storage = DatabaseCatalog::instance().tryGetByUUID(create.uuid).second; + if (storage) + storage->adjustCreateQueryForBackup(create_table_query); + } + res.emplace_back(create_table_query, storage); + } + + return res; +} + + +void DatabaseReplicated::createTableRestoredFromBackup( + const ASTPtr & create_table_query, + ContextMutablePtr local_context, + std::shared_ptr restore_coordination, + UInt64 timeout_ms) { /// Because of the replication multiple nodes can try to restore the same tables again and failed with "Table already exists" /// because of some table could be restored already on other node and then replicated to this node. @@ -931,29 +975,25 @@ void DatabaseReplicated::createTableRestoredFromBackup(const ASTPtr & create_tab /// IRestoreCoordination::acquireCreatingTableInReplicatedDatabase() and then for other nodes this function returns false which means /// this table is already being created by some other node. String table_name = create_table_query->as().getTable(); - if (restorer.getRestoreCoordination()->acquireCreatingTableInReplicatedDatabase(getZooKeeperPath(), table_name)) + if (restore_coordination->acquireCreatingTableInReplicatedDatabase(getZooKeeperPath(), table_name)) { - restorer.executeCreateQuery(create_table_query); + DatabaseAtomic::createTableRestoredFromBackup(create_table_query, local_context, restore_coordination, timeout_ms); } /// Wait until the table is actually created no matter if it's created by the current or another node and replicated to the /// current node afterwards. We have to wait because `RestorerFromBackup` is going to restore data of the table then. /// TODO: The following code doesn't look very reliable, probably we need to rewrite it somehow. - auto timeout = restorer.getTimeout(); - bool use_timeout = (timeout.count() >= 0); + auto timeout = std::chrono::milliseconds{timeout_ms}; auto start_time = std::chrono::steady_clock::now(); - while (!isTableExist(table_name, restorer.getContext())) + while (!isTableExist(table_name, local_context)) { waitForReplicaToProcessAllEntries(50); - if (use_timeout) - { - auto elapsed = std::chrono::steady_clock::now() - start_time; - if (elapsed > timeout) - throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, - "Couldn't restore table {}.{} on other node or sync it (elapsed {})", - backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(table_name), to_string(elapsed)); - } + auto elapsed = std::chrono::steady_clock::now() - start_time; + if (elapsed > timeout) + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, + "Couldn't restore table {}.{} on other node or sync it (elapsed {})", + backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(table_name), to_string(elapsed)); } } diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 3aa2aa378b7..07014702067 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -72,7 +72,8 @@ public: void shutdown() override; - void createTableRestoredFromBackup(const ASTPtr & create_table_query, const RestorerFromBackup & restorer) override; + std::vector> getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const override; + void createTableRestoredFromBackup(const ASTPtr & create_table_query, ContextMutablePtr local_context, std::shared_ptr restore_coordination, UInt64 timeout_ms) override; friend struct DatabaseReplicatedTask; friend class DatabaseReplicatedDDLWorker; diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 5dd17789e60..93a9523d115 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; extern const int CANNOT_GET_CREATE_TABLE_QUERY; + extern const int INCONSISTENT_METADATA_FOR_BACKUP; } void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemoryMetadata & metadata) @@ -322,22 +323,34 @@ StoragePtr DatabaseWithOwnTablesBase::getTableUnlocked(const String & table_name backQuote(database_name), backQuote(table_name)); } -DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIteratorForBackup(const BackupEntriesCollector & backup_entries_collector) const +std::vector> DatabaseWithOwnTablesBase::getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const { - /// Backup all the tables in this database. - /// Here we skip inner tables of materialized views. - auto skip_internal_tables = [](const String & table_name) { return !table_name.starts_with(".inner_id."); }; - return getTablesIterator(backup_entries_collector.getContext(), skip_internal_tables); + std::vector> res; + + for (auto it = getTablesIterator(local_context, filter); it->isValid(); it->next()) + { + auto create_table_query = tryGetCreateTableQuery(it->name(), local_context); + if (!create_table_query) + throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "Couldn't get a create query for table {}.{}", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(it->name())); + + const auto & create = create_table_query->as(); + if (create.getTable() != it->name()) + throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "Got a create query with unexpected name {} for table {}.{}", backQuoteIfNeed(create.getTable()), backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(it->name())); + + auto storage = it->table(); + storage->adjustCreateQueryForBackup(create_table_query); + res.emplace_back(create_table_query, storage); + } + + return res; } -void DatabaseWithOwnTablesBase::checkCreateTableQueryForBackup(const ASTPtr &, const BackupEntriesCollector &) const -{ -} - -void DatabaseWithOwnTablesBase::createTableRestoredFromBackup(const ASTPtr & create_table_query, const RestorerFromBackup & restorer) +void DatabaseWithOwnTablesBase::createTableRestoredFromBackup(const ASTPtr & create_table_query, ContextMutablePtr local_context, std::shared_ptr, UInt64) { /// Creates a table by executing a "CREATE TABLE" query. - restorer.executeCreateQuery(create_table_query); + InterpreterCreateQuery interpreter{create_table_query, local_context}; + interpreter.setInternal(true); + interpreter.execute(); } } diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h index c960d295529..c5842d7dac3 100644 --- a/src/Databases/DatabasesCommon.h +++ b/src/Databases/DatabasesCommon.h @@ -36,9 +36,8 @@ public: DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; - DatabaseTablesIteratorPtr getTablesIteratorForBackup(const BackupEntriesCollector & backup_entries_collector) const override; - void checkCreateTableQueryForBackup(const ASTPtr & create_table_query, const BackupEntriesCollector & backup_entries_collector) const override; - void createTableRestoredFromBackup(const ASTPtr & create_table_query, const RestorerFromBackup & restorer) override; + std::vector> getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const override; + void createTableRestoredFromBackup(const ASTPtr & create_table_query, ContextMutablePtr local_context, std::shared_ptr restore_coordination, UInt64 timeout_ms) override; void shutdown() override; diff --git a/src/Databases/IDatabase.cpp b/src/Databases/IDatabase.cpp index 3adba0d85c8..9e33548b0dd 100644 --- a/src/Databases/IDatabase.cpp +++ b/src/Databases/IDatabase.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include @@ -22,33 +21,15 @@ StoragePtr IDatabase::getTable(const String & name, ContextPtr context) const throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name)); } -ASTPtr IDatabase::getCreateDatabaseQueryForBackup() const +std::vector> IDatabase::getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const { - auto query = getCreateDatabaseQuery(); - - /// We don't want to see any UUIDs in backup (after RESTORE the table will have another UUID anyway). - auto & create = query->as(); - create.uuid = UUIDHelpers::Nil; - - return query; -} - -DatabaseTablesIteratorPtr IDatabase::getTablesIteratorForBackup(const BackupEntriesCollector &) const -{ - /// IDatabase doesn't own any tables. - return std::make_unique(Tables{}, getDatabaseName()); -} - -void IDatabase::checkCreateTableQueryForBackup(const ASTPtr & create_table_query, const BackupEntriesCollector &) const -{ - /// Cannot restore any table because IDatabase doesn't own any tables. + /// Cannot backup any table because IDatabase doesn't own any tables. throw Exception(ErrorCodes::CANNOT_BACKUP_TABLE, - "Database engine {} does not support backups, cannot backup table {}.{}", - getEngineName(), backQuoteIfNeed(getDatabaseName()), - backQuoteIfNeed(create_table_query->as().getTable())); + "Database engine {} does not support backups, cannot backup tables in database {}", + getEngineName(), backQuoteIfNeed(getDatabaseName())); } -void IDatabase::createTableRestoredFromBackup(const ASTPtr & create_table_query, const RestorerFromBackup &) +void IDatabase::createTableRestoredFromBackup(const ASTPtr & create_table_query, ContextMutablePtr, std::shared_ptr, UInt64) { /// Cannot restore any table because IDatabase doesn't own any tables. throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 2223d657f7f..72155bc818c 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -30,8 +30,7 @@ class SettingsChanges; using DictionariesWithID = std::vector>; struct ParsedTablesMetadata; struct QualifiedTableName; -class BackupEntriesCollector; -class RestorerFromBackup; +class IRestoreCoordination; namespace ErrorCodes { @@ -333,17 +332,11 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "Database engine {} does not run a replication thread!", getEngineName()); } - /// Returns a slightly changed version of the CREATE DATABASE query which must be written to a backup. - virtual ASTPtr getCreateDatabaseQueryForBackup() const; - - /// Returns an iterator that passes through all the tables when an user wants to backup the whole database. - virtual DatabaseTablesIteratorPtr getTablesIteratorForBackup(const BackupEntriesCollector & restorer) const; - - /// Checks a CREATE TABLE query before it will be written to a backup. Called by IStorage::getCreateQueryForBackup(). - virtual void checkCreateTableQueryForBackup(const ASTPtr & create_table_query, const BackupEntriesCollector & backup_entries_collector) const; + /// Returns CREATE TABLE queries and corresponding tables prepared for writing to a backup. + virtual std::vector> getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & context) const; /// Creates a table restored from backup. - virtual void createTableRestoredFromBackup(const ASTPtr & create_table_query, const RestorerFromBackup & restorer); + virtual void createTableRestoredFromBackup(const ASTPtr & create_table_query, ContextMutablePtr context, std::shared_ptr restore_coordination, UInt64 timeout_ms); virtual ~IDatabase() = default; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index a0579b813db..bae2aed2cd5 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -234,12 +234,13 @@ void DatabaseCatalog::shutdownImpl() view_dependencies.clear(); } -bool DatabaseCatalog::isPredefinedDatabaseName(const std::string_view & database_name) +bool DatabaseCatalog::isPredefinedDatabase(const std::string_view & database_name) { return database_name == TEMPORARY_DATABASE || database_name == SYSTEM_DATABASE || database_name == INFORMATION_SCHEMA || database_name == INFORMATION_SCHEMA_UPPERCASE; } + DatabaseAndTable DatabaseCatalog::tryGetByUUID(const UUID & uuid) const { assert(uuid != UUIDHelpers::Nil && getFirstLevelIdx(uuid) < uuid_map.size()); @@ -328,6 +329,48 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( return {database, table}; } +bool DatabaseCatalog::isPredefinedTable(const StorageID & table_id) const +{ + static const char * information_schema_views[] = {"schemata", "tables", "views", "columns"}; + static const char * information_schema_views_uppercase[] = {"SCHEMATA", "TABLES", "VIEWS", "COLUMNS"}; + + auto check_database_and_table_name = [&](const String & database_name, const String & table_name) + { + if (database_name == SYSTEM_DATABASE) + { + auto storage = getSystemDatabase()->tryGetTable(table_name, getContext()); + return storage && storage->isSystemStorage(); + } + if (database_name == INFORMATION_SCHEMA) + { + return std::find(std::begin(information_schema_views), std::end(information_schema_views), table_name) + != std::end(information_schema_views); + } + if (database_name == INFORMATION_SCHEMA_UPPERCASE) + { + return std::find(std::begin(information_schema_views_uppercase), std::end(information_schema_views_uppercase), table_name) + != std::end(information_schema_views_uppercase); + } + return false; + }; + + if (table_id.hasUUID()) + { + if (auto storage = tryGetByUUID(table_id.uuid).second) + { + if (storage->isSystemStorage()) + return true; + auto res_id = storage->getStorageID(); + String database_name = res_id.getDatabaseName(); + if (database_name != SYSTEM_DATABASE) /// If (database_name == SYSTEM_DATABASE) then we have already checked it (see isSystemStorage() above). + return check_database_and_table_name(database_name, res_id.getTableName()); + } + return false; + } + + return check_database_and_table_name(table_id.getDatabaseName(), table_id.getTableName()); +} + void DatabaseCatalog::assertDatabaseExists(const String & database_name) const { std::lock_guard lock{databases_mutex}; diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 4468cc3a5d8..133cf0c5126 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -130,8 +130,8 @@ public: static constexpr const char * INFORMATION_SCHEMA = "information_schema"; static constexpr const char * INFORMATION_SCHEMA_UPPERCASE = "INFORMATION_SCHEMA"; - /// Returns true if a passed string is one of the predefined databases' names - static bool isPredefinedDatabaseName(const std::string_view & database_name); + /// Returns true if a passed name is one of the predefined databases' names. + static bool isPredefinedDatabase(const std::string_view & database_name); static DatabaseCatalog & init(ContextMutablePtr global_context_); static DatabaseCatalog & instance(); @@ -181,6 +181,11 @@ public: ContextPtr context, std::optional * exception = nullptr) const; + /// Returns true if a passed table_id refers to one of the predefined tables' names. + /// All tables in the "system" database with System* table engine are predefined. + /// Four views (tables, views, columns, schemata) in the "information_schema" database are predefined too. + bool isPredefinedTable(const StorageID & table_id) const; + void addDependency(const StorageID & from, const StorageID & where); void removeDependency(const StorageID & from, const StorageID & where); Dependencies getDependencies(const StorageID & from) const; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 0fcf48b9afc..fc29769790d 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -14,7 +14,8 @@ #include #include #include -#include +#include +#include namespace DB @@ -24,6 +25,7 @@ namespace ErrorCodes extern const int TABLE_IS_DROPPED; extern const int NOT_IMPLEMENTED; extern const int DEADLOCK_AVOIDED; + extern const int CANNOT_RESTORE_TABLE; } bool IStorage::isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const @@ -248,48 +250,21 @@ bool IStorage::isStaticStorage() const return false; } -ASTPtr IStorage::getCreateQueryForBackup(const ContextPtr & context, DatabasePtr * database) const +void IStorage::adjustCreateQueryForBackup(ASTPtr &) const { - auto table_id = getStorageID(); - auto db = DatabaseCatalog::instance().tryGetDatabase(table_id.getDatabaseName()); - if (!db) - throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {}.{} is dropped", table_id.database_name, table_id.table_name); - ASTPtr query = db->tryGetCreateTableQuery(table_id.getTableName(), context); - if (!query) - throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {}.{} is dropped", table_id.database_name, table_id.table_name); - - /// We don't want to see any UUIDs in backup (after RESTORE the table will have another UUID anyway). - auto & create = query->as(); - create.uuid = UUIDHelpers::Nil; - create.to_inner_uuid = UUIDHelpers::Nil; - - /// If this is a definition of a system table we'll remove columns and comment because they're excessive for backups. - if (create.storage && create.storage->engine && create.storage->engine->name.starts_with("System")) - { - create.reset(create.columns_list); - create.reset(create.comment); - } - - if (database) - *database = db; - - return query; -} - -ASTPtr IStorage::getCreateQueryForBackup(const BackupEntriesCollector & backup_entries_collector) const -{ - DatabasePtr database; - auto query = getCreateQueryForBackup(backup_entries_collector.getContext(), &database); - database->checkCreateTableQueryForBackup(query, backup_entries_collector); - return query; } void IStorage::backupData(BackupEntriesCollector &, const String &, const std::optional &) { } -void IStorage::restoreDataFromBackup(RestorerFromBackup &, const String &, const std::optional &) +void IStorage::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional &) { + /// If an inherited class doesn't override restoreDataFromBackup() that means it doesn't backup any data. + auto filenames = restorer.getBackup()->listFiles(data_path_in_backup); + if (!filenames.empty()) + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: Folder {} in backup must be empty", + getStorageID().getFullTableName(), data_path_in_backup); } std::string PrewhereInfo::dump() const diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 6dd329db02b..e265c94eb11 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -223,10 +223,8 @@ public: /// Initially reserved virtual column name may be shadowed by real column. bool isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const; - /// Returns a slightly changed version of the CREATE TABLE query which must be written to a backup. - /// The function can throw `TABLE_IS_DROPPED` if this storage is not attached to a database. - virtual ASTPtr getCreateQueryForBackup(const ContextPtr & context, DatabasePtr * database) const; - virtual ASTPtr getCreateQueryForBackup(const BackupEntriesCollector & backup_entries_collector) const; + /// Modify a CREATE TABLE query to make a variant which must be written to a backup. + virtual void adjustCreateQueryForBackup(ASTPtr & create_query) const; /// Makes backup entries to backup the data of this storage. virtual void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions); @@ -234,6 +232,9 @@ public: /// Extracts data from the backup and put it to the storage. virtual void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions); + /// Returns true if the storage supports backup/restore for specific partitions. + virtual bool supportsBackupPartition() const { return false; } + private: StorageID storage_id; diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index 083cbc90cb1..dc80b0aafb6 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -615,36 +615,50 @@ void DataPartStorageOnDisk::backup( TemporaryFilesOnDisks & temp_dirs, const MergeTreeDataPartChecksums & checksums, const NameSet & files_without_checksums, + const String & path_in_backup, BackupEntries & backup_entries) const { - auto disk = volume->getDisk(); + fs::path part_path_on_disk = fs::path{root_path} / part_dir; + fs::path part_path_in_backup = fs::path{path_in_backup} / part_dir; + auto disk = volume->getDisk(); auto temp_dir_it = temp_dirs.find(disk); if (temp_dir_it == temp_dirs.end()) - temp_dir_it = temp_dirs.emplace(disk, std::make_shared(disk, "tmp/backup_")).first; + temp_dir_it = temp_dirs.emplace(disk, std::make_shared(disk, "tmp/backup/")).first; auto temp_dir_owner = temp_dir_it->second; fs::path temp_dir = temp_dir_owner->getPath(); - - fs::path temp_part_dir = temp_dir / part_dir; + fs::path temp_part_dir = temp_dir / part_path_in_backup.relative_path(); disk->createDirectories(temp_part_dir); + /// For example, + /// part_path_in_backup = /data/test/table/0_1_1_0 + /// part_path_on_disk = store/f57/f5728353-44bb-4575-85e8-28deb893657a/0_1_1_0 + /// tmp_part_dir = tmp/backup/1aaaaaa/data/test/table/0_1_1_0 + /// Or, for projections: + /// part_path_in_backup = /data/test/table/0_1_1_0/prjmax.proj + /// part_path_on_disk = store/f57/f5728353-44bb-4575-85e8-28deb893657a/0_1_1_0/prjmax.proj + /// tmp_part_dir = tmp/backup/1aaaaaa/data/test/table/0_1_1_0/prjmax.proj + for (const auto & [filepath, checksum] : checksums.files) { - String relative_filepath = fs::path(part_dir) / filepath; - String full_filepath = fs::path(root_path) / part_dir / filepath; + if (filepath.ends_with(".proj")) + continue; /// Skip *.proj files - they're actually directories and will be handled. + String filepath_on_disk = part_path_on_disk / filepath; + String filepath_in_backup = part_path_in_backup / filepath; String hardlink_filepath = temp_part_dir / filepath; - disk->createHardLink(full_filepath, hardlink_filepath); + + disk->createHardLink(filepath_on_disk, hardlink_filepath); UInt128 file_hash{checksum.file_hash.first, checksum.file_hash.second}; backup_entries.emplace_back( - relative_filepath, + filepath_in_backup, std::make_unique(disk, hardlink_filepath, checksum.file_size, file_hash, temp_dir_owner)); } for (const auto & filepath : files_without_checksums) { - String relative_filepath = fs::path(part_dir) / filepath; - String full_filepath = fs::path(root_path) / part_dir / filepath; - backup_entries.emplace_back(relative_filepath, std::make_unique(disk, full_filepath)); + String filepath_on_disk = part_path_on_disk / filepath; + String filepath_in_backup = part_path_in_backup / filepath; + backup_entries.emplace_back(filepath_in_backup, std::make_unique(disk, filepath_on_disk)); } } diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.h b/src/Storages/MergeTree/DataPartStorageOnDisk.h index d6fcb2f1442..bb1a8879d63 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.h +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.h @@ -88,6 +88,7 @@ public: TemporaryFilesOnDisks & temp_dirs, const MergeTreeDataPartChecksums & checksums, const NameSet & files_without_checksums, + const String & path_in_backup, BackupEntries & backup_entries) const override; DataPartStoragePtr freeze( diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index 0e165e74ed0..72810680812 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -173,6 +173,7 @@ public: TemporaryFilesOnDisks & temp_dirs, const MergeTreeDataPartChecksums & checksums, const NameSet & files_without_checksums, + const String & path_in_backup, BackupEntries & backup_entries) const = 0; /// Creates hardlinks into 'to/dir_path' for every file in data part. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2c4dcfa05ee..d4b2c8d177e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -150,6 +150,7 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; extern const int TOO_MANY_SIMULTANEOUS_QUERIES; extern const int INCORRECT_QUERY; + extern const int CANNOT_RESTORE_TABLE; } static void checkSampleExpression(const StorageInMemoryMetadata & metadata, bool allow_sampling_expression_not_in_primary_key, bool check_sample_column_is_correct) @@ -3993,14 +3994,23 @@ BackupEntries MergeTreeData::backupParts(const ContextPtr & local_context, const BackupEntries backup_entries; std::map> temp_dirs; - fs::path data_path_in_backup_fs = data_path_in_backup; for (const auto & part : data_parts) - part->data_part_storage->backup(temp_dirs, part->checksums, part->getFileNamesWithoutChecksums(), backup_entries); + { + part->data_part_storage->backup( + temp_dirs, part->checksums, part->getFileNamesWithoutChecksums(), data_path_in_backup, backup_entries); - /// TODO: try to write better code later. - for (auto & entry : backup_entries) - entry.first = data_path_in_backup_fs / entry.first; + auto projection_parts = part->getProjectionParts(); + for (const auto & [projection_name, projection_part] : projection_parts) + { + projection_part->data_part_storage->backup( + temp_dirs, + projection_part->checksums, + projection_part->getFileNamesWithoutChecksums(), + fs::path{data_path_in_backup} / part->name, + backup_entries); + } + } return backup_entries; } @@ -4083,7 +4093,10 @@ void MergeTreeData::restorePartsFromBackup(RestorerFromBackup & restorer, const { const auto part_info = MergeTreePartInfo::tryParsePartName(part_name, format_version); if (!part_info) - continue; + { + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File name {} doesn't look like the name of a part", + getStorageID().getFullTableName(), String{data_path_in_backup_fs / part_name}); + } if (partition_ids && !partition_ids->contains(part_info->partition_id)) continue; @@ -4116,27 +4129,39 @@ void MergeTreeData::restorePartFromBackup(std::shared_ptr r auto disk = reservation->getDisk(); String part_name = part_info.getPartName(); - auto temp_part_dir_owner = std::make_shared(disk, relative_data_path + "restoring_" + part_name + "_"); - String temp_part_dir = temp_part_dir_owner->getPath(); + auto temp_part_dir_owner = std::make_shared(disk, fs::path{relative_data_path} / ("restoring_" + part_name + "_")); + fs::path temp_part_dir = temp_part_dir_owner->getPath(); disk->createDirectories(temp_part_dir); + std::unordered_set subdirs; - assert(temp_part_dir.starts_with(relative_data_path)); - String relative_temp_part_dir = temp_part_dir.substr(relative_data_path.size()); + /// temp_part_name = "restoring__", for example "restoring_0_1_1_0_1baaaaa" + String temp_part_name = temp_part_dir.filename(); for (const String & filename : filenames) { + /// Needs to create subdirectories before copying the files. Subdirectories are used to represent projections. + auto separator_pos = filename.rfind('/'); + if (separator_pos != String::npos) + { + String subdir = filename.substr(0, separator_pos); + if (subdirs.emplace(subdir).second) + disk->createDirectories(temp_part_dir / subdir); + } + + /// TODO Transactions: Decide what to do with version metadata (if any). Let's just skip it for now. + if (filename.ends_with(IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME)) + continue; + auto backup_entry = backup->readFile(part_path_in_backup_fs / filename); auto read_buffer = backup_entry->getReadBuffer(); - auto write_buffer = disk->writeFile(fs::path(temp_part_dir) / filename); + 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 data_part_storage = std::make_shared(single_disk_volume, relative_data_path, relative_temp_part_dir); + auto data_part_storage = std::make_shared(single_disk_volume, relative_data_path, temp_part_name); auto part = createPart(part_name, part_info, data_part_storage); - /// TODO Transactions: Decide what to do with version metadata (if any). Let's just remove it for now. - disk->removeFileIfExists(fs::path(temp_part_dir) / IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME); part->version.setCreationTID(Tx::PrehistoricTID, nullptr); part->loadColumnsChecksumsIndexes(false, true); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 4fd7dd7d3cf..9aa14367f80 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -716,6 +716,9 @@ public: /// Extract data from the backup and put it to the storage. void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; + /// Returns true if the storage supports backup/restore for specific partitions. + bool supportsBackupPartition() const override { return true; } + /// Moves partition to specified Disk void movePartitionToDisk(const ASTPtr & partition, const String & name, bool moving_part, ContextPtr context); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index 7dee7b8d0f8..ea90179caa3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -353,4 +354,123 @@ ReplicatedMergeTreeTableMetadata::checkAndFindDiff(const ReplicatedMergeTreeTabl return diff; } +StorageInMemoryMetadata ReplicatedMergeTreeTableMetadata::Diff::getNewMetadata(const ColumnsDescription & new_columns, ContextPtr context, const StorageInMemoryMetadata & old_metadata) const +{ + StorageInMemoryMetadata new_metadata = old_metadata; + new_metadata.columns = new_columns; + + if (!empty()) + { + auto parse_key_expr = [] (const String & key_expr) + { + ParserNotEmptyExpressionList parser(false); + auto new_sorting_key_expr_list = parseQuery(parser, key_expr, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); + + ASTPtr order_by_ast; + if (new_sorting_key_expr_list->children.size() == 1) + order_by_ast = new_sorting_key_expr_list->children[0]; + else + { + auto tuple = makeASTFunction("tuple"); + tuple->arguments->children = new_sorting_key_expr_list->children; + order_by_ast = tuple; + } + return order_by_ast; + }; + + if (sorting_key_changed) + { + auto order_by_ast = parse_key_expr(new_sorting_key); + + new_metadata.sorting_key.recalculateWithNewAST(order_by_ast, new_metadata.columns, context); + + if (new_metadata.primary_key.definition_ast == nullptr) + { + /// Primary and sorting key become independent after this ALTER so we have to + /// save the old ORDER BY expression as the new primary key. + auto old_sorting_key_ast = old_metadata.getSortingKey().definition_ast; + new_metadata.primary_key = KeyDescription::getKeyFromAST( + old_sorting_key_ast, new_metadata.columns, context); + } + } + + if (sampling_expression_changed) + { + if (!new_sampling_expression.empty()) + { + auto sample_by_ast = parse_key_expr(new_sampling_expression); + new_metadata.sampling_key.recalculateWithNewAST(sample_by_ast, new_metadata.columns, context); + } + else /// SAMPLE BY was removed + { + new_metadata.sampling_key = {}; + } + } + + if (skip_indices_changed) + new_metadata.secondary_indices = IndicesDescription::parse(new_skip_indices, new_columns, context); + + if (constraints_changed) + new_metadata.constraints = ConstraintsDescription::parse(new_constraints); + + if (projections_changed) + new_metadata.projections = ProjectionsDescription::parse(new_projections, new_columns, context); + + if (ttl_table_changed) + { + if (!new_ttl_table.empty()) + { + ParserTTLExpressionList parser; + auto ttl_for_table_ast = parseQuery(parser, new_ttl_table, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); + new_metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( + ttl_for_table_ast, new_metadata.columns, context, new_metadata.primary_key); + } + else /// TTL was removed + { + new_metadata.table_ttl = TTLTableDescription{}; + } + } + } + + /// Changes in columns may affect following metadata fields + new_metadata.column_ttls_by_name.clear(); + for (const auto & [name, ast] : new_metadata.columns.getColumnTTLs()) + { + auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, new_metadata.columns, context, new_metadata.primary_key); + new_metadata.column_ttls_by_name[name] = new_ttl_entry; + } + + if (new_metadata.partition_key.definition_ast != nullptr) + new_metadata.partition_key.recalculateWithNewColumns(new_metadata.columns, context); + + if (!sorting_key_changed) /// otherwise already updated + new_metadata.sorting_key.recalculateWithNewColumns(new_metadata.columns, context); + + /// Primary key is special, it exists even if not defined + if (new_metadata.primary_key.definition_ast != nullptr) + { + new_metadata.primary_key.recalculateWithNewColumns(new_metadata.columns, context); + } + else + { + new_metadata.primary_key = KeyDescription::getKeyFromAST(new_metadata.sorting_key.definition_ast, new_metadata.columns, context); + new_metadata.primary_key.definition_ast = nullptr; + } + + if (!sampling_expression_changed && new_metadata.sampling_key.definition_ast != nullptr) + new_metadata.sampling_key.recalculateWithNewColumns(new_metadata.columns, context); + + if (!skip_indices_changed) /// otherwise already updated + { + for (auto & index : new_metadata.secondary_indices) + index.recalculateWithNewColumns(new_metadata.columns, context); + } + + if (!ttl_table_changed && new_metadata.table_ttl.definition_ast != nullptr) + new_metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( + new_metadata.table_ttl.definition_ast, new_metadata.columns, context, new_metadata.primary_key); + + return new_metadata; +} + } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h index 6d510d20304..eb2d087e988 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h @@ -66,6 +66,8 @@ struct ReplicatedMergeTreeTableMetadata return !sorting_key_changed && !sampling_expression_changed && !skip_indices_changed && !projections_changed && !ttl_table_changed && !constraints_changed; } + + StorageInMemoryMetadata getNewMetadata(const ColumnsDescription & new_columns, ContextPtr context, const StorageInMemoryMetadata & old_metadata) const; }; void checkEquals(const ReplicatedMergeTreeTableMetadata & from_zk, const ColumnsDescription & columns, ContextPtr context) const; diff --git a/src/Storages/MergeTree/extractZkPathFromCreateQuery.cpp b/src/Storages/MergeTree/extractZkPathFromCreateQuery.cpp new file mode 100644 index 00000000000..45d667047af --- /dev/null +++ b/src/Storages/MergeTree/extractZkPathFromCreateQuery.cpp @@ -0,0 +1,60 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +std::optional tryExtractZkPathFromCreateQuery(const IAST & create_query, const ContextPtr & global_context) +{ + const auto * create = create_query.as(); + if (!create || !create->storage || !create->storage->engine) + return {}; + + /// Check if the table engine is one of the ReplicatedMergeTree family. + const auto & ast_engine = *create->storage->engine; + if (!ast_engine.name.starts_with("Replicated") || !ast_engine.name.ends_with("MergeTree")) + return {}; + + /// Get the first argument. + const auto * ast_arguments = typeid_cast(ast_engine.arguments.get()); + if (!ast_arguments || ast_arguments->children.empty()) + return {}; + + auto * ast_zk_path = typeid_cast(ast_arguments->children[0].get()); + if (!ast_zk_path || (ast_zk_path->value.getType() != Field::Types::String)) + return {}; + + String zk_path = ast_zk_path->value.safeGet(); + + /// Expand macros. + Macros::MacroExpansionInfo info; + info.table_id.table_name = create->getTable(); + info.table_id.database_name = create->getDatabase(); + info.table_id.uuid = create->uuid; + auto database = DatabaseCatalog::instance().tryGetDatabase(info.table_id.database_name); + if (database && database->getEngineName() == "Replicated") + { + info.shard = getReplicatedDatabaseShardName(database); + info.replica = getReplicatedDatabaseReplicaName(database); + } + + try + { + zk_path = global_context->getMacros()->expand(zk_path, info); + } + catch (...) + { + return {}; /// Couldn't expand macros. + } + + return zk_path; +} + +} diff --git a/src/Storages/MergeTree/extractZkPathFromCreateQuery.h b/src/Storages/MergeTree/extractZkPathFromCreateQuery.h new file mode 100644 index 00000000000..e22f76d2cd5 --- /dev/null +++ b/src/Storages/MergeTree/extractZkPathFromCreateQuery.h @@ -0,0 +1,19 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ +class IAST; +class Context; +using ContextPtr = std::shared_ptr; + +/// Extracts a zookeeper path from a specified CREATE TABLE query. Returns std::nullopt if fails. +/// The function takes the first argument of the ReplicatedMergeTree table engine and expands macros in it. +/// It works like a part of what the create() function in registerStorageMergeTree.cpp does but in a simpler manner. +std::optional tryExtractZkPathFromCreateQuery(const IAST & create_query, const ContextPtr & global_context); + +} diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 1324ebf5b28..ac6ead54016 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -51,6 +51,7 @@ namespace ErrorCodes extern const int SIZES_OF_MARKS_FILES_ARE_INCONSISTENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_FILE_NAME; + extern const int CANNOT_RESTORE_TABLE; } /// NOTE: The lock `StorageLog::rwlock` is NOT kept locked while reading, @@ -921,11 +922,8 @@ std::optional StorageLog::totalBytes(const Settings &) const return total_bytes; } -void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) +void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & /* partitions */) { - if (partitions) - BackupEntriesCollector::throwPartitionsNotSupported(getStorageID(), getName()); - auto lock_timeout = getLockTimeout(backup_entries_collector.getContext()); loadMarks(lock_timeout); @@ -986,16 +984,16 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c } } -void StorageLog::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) +void StorageLog::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & /* partitions */) { - if (partitions) - RestorerFromBackup::throwPartitionsNotSupported(getStorageID(), getName()); + auto backup = restorer.getBackup(); + if (!backup->hasFiles(data_path_in_backup)) + return; if (!num_data_files) return; - auto backup = restorer.getBackup(); - if (!restorer.isNonEmptyTableAllowed() && total_bytes && backup->hasFiles(data_path_in_backup)) + if (!restorer.isNonEmptyTableAllowed() && total_bytes) RestorerFromBackup::throwTableIsNotEmpty(getStorageID()); auto lock_timeout = getLockTimeout(restorer.getContext()); @@ -1024,6 +1022,11 @@ void StorageLog::restoreDataImpl(const BackupPtr & backup, const String & data_p for (const auto & data_file : data_files) { String file_path_in_backup = data_path_in_backup_fs / fileName(data_file.path); + if (!backup->fileExists(file_path_in_backup)) + { + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required", + getStorageID().getFullTableName(), file_path_in_backup); + } auto backup_entry = backup->readFile(file_path_in_backup); auto in = backup_entry->getReadBuffer(); auto out = disk->writeFile(data_file.path, max_compress_block_size, WriteMode::Append); @@ -1035,6 +1038,11 @@ void StorageLog::restoreDataImpl(const BackupPtr & backup, const String & data_p /// Append marks. size_t num_extra_marks = 0; String file_path_in_backup = data_path_in_backup_fs / fileName(marks_file_path); + if (!backup->fileExists(file_path_in_backup)) + { + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required", + getStorageID().getFullTableName(), file_path_in_backup); + } size_t file_size = backup->getFileSize(file_path_in_backup); if (file_size % (num_data_files * sizeof(Mark)) != 0) throw Exception("Size of marks file is inconsistent", ErrorCodes::SIZES_OF_MARKS_FILES_ARE_INCONSISTENT); diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 2ece0af3359..b01415f9590 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -421,6 +421,13 @@ void StorageMaterializedView::restoreDataFromBackup(RestorerFromBackup & restore return getTargetTable()->restoreDataFromBackup(restorer, data_path_in_backup, partitions); } +bool StorageMaterializedView::supportsBackupPartition() const +{ + if (hasInnerTable()) + return getTargetTable()->supportsBackupPartition(); + return false; +} + std::optional StorageMaterializedView::totalRows(const Settings & settings) const { if (hasInnerTable()) diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 0adf394876c..1d8808b302e 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -97,6 +97,7 @@ public: void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; + bool supportsBackupPartition() const override; std::optional totalRows(const Settings & settings) const override; std::optional totalBytes(const Settings & settings) const override; diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 5de8c3bda43..7baecaa594f 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -38,6 +38,7 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int CANNOT_RESTORE_TABLE; } @@ -479,24 +480,21 @@ namespace }; } -void StorageMemory::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) +void StorageMemory::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & /* partitions */) { - if (partitions) - BackupEntriesCollector::throwPartitionsNotSupported(getStorageID(), getName()); - auto max_compress_block_size = backup_entries_collector.getContext()->getSettingsRef().max_compress_block_size; backup_entries_collector.addBackupEntries( std::make_shared(getInMemoryMetadataPtr(), data.get(), data_path_in_backup, max_compress_block_size) ->getBackupEntries()); } -void StorageMemory::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) +void StorageMemory::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & /* partitions */) { - if (partitions) - RestorerFromBackup::throwPartitionsNotSupported(getStorageID(), getName()); - auto backup = restorer.getBackup(); - if (!restorer.isNonEmptyTableAllowed() && total_size_bytes && backup->hasFiles(data_path_in_backup)) + if (!backup->hasFiles(data_path_in_backup)) + return; + + if (!restorer.isNonEmptyTableAllowed() && total_size_bytes) RestorerFromBackup::throwTableIsNotEmpty(getStorageID()); restorer.addDataRestoreTask( @@ -514,6 +512,11 @@ void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & dat IndexForNativeFormat index; { String index_file_path = data_path_in_backup_fs / "index.mrk"; + if (!backup->fileExists(index_file_path)) + { + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required", + getStorageID().getFullTableName(), index_file_path); + } auto backup_entry = backup->readFile(index_file_path); auto in = backup_entry->getReadBuffer(); CompressedReadBuffer compressed_in{*in}; @@ -526,6 +529,11 @@ void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & dat size_t new_rows = 0; { String data_file_path = data_path_in_backup_fs / "data.bin"; + if (!backup->fileExists(data_file_path)) + { + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required", + getStorageID().getFullTableName(), data_file_path); + } auto backup_entry = backup->readFile(data_file_path); std::unique_ptr in = backup_entry->getReadBuffer(); std::optional temp_data_copy; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e44013f39ca..2ee1e233688 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -37,6 +37,7 @@ #include #include #include +#include #include #include @@ -1100,123 +1101,8 @@ void StorageReplicatedMergeTree::checkTableStructure(const String & zookeeper_pr void StorageReplicatedMergeTree::setTableStructure( ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata::Diff & metadata_diff) { - StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); StorageInMemoryMetadata old_metadata = getInMemoryMetadata(); - - new_metadata.columns = new_columns; - - if (!metadata_diff.empty()) - { - auto parse_key_expr = [] (const String & key_expr) - { - ParserNotEmptyExpressionList parser(false); - auto new_sorting_key_expr_list = parseQuery(parser, key_expr, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); - - ASTPtr order_by_ast; - if (new_sorting_key_expr_list->children.size() == 1) - order_by_ast = new_sorting_key_expr_list->children[0]; - else - { - auto tuple = makeASTFunction("tuple"); - tuple->arguments->children = new_sorting_key_expr_list->children; - order_by_ast = tuple; - } - return order_by_ast; - }; - - if (metadata_diff.sorting_key_changed) - { - auto order_by_ast = parse_key_expr(metadata_diff.new_sorting_key); - auto & sorting_key = new_metadata.sorting_key; - auto & primary_key = new_metadata.primary_key; - - sorting_key.recalculateWithNewAST(order_by_ast, new_metadata.columns, getContext()); - - if (primary_key.definition_ast == nullptr) - { - /// Primary and sorting key become independent after this ALTER so we have to - /// save the old ORDER BY expression as the new primary key. - auto old_sorting_key_ast = old_metadata.getSortingKey().definition_ast; - primary_key = KeyDescription::getKeyFromAST( - old_sorting_key_ast, new_metadata.columns, getContext()); - } - } - - if (metadata_diff.sampling_expression_changed) - { - if (!metadata_diff.new_sampling_expression.empty()) - { - auto sample_by_ast = parse_key_expr(metadata_diff.new_sampling_expression); - new_metadata.sampling_key.recalculateWithNewAST(sample_by_ast, new_metadata.columns, getContext()); - } - else /// SAMPLE BY was removed - { - new_metadata.sampling_key = {}; - } - } - - if (metadata_diff.skip_indices_changed) - new_metadata.secondary_indices = IndicesDescription::parse(metadata_diff.new_skip_indices, new_columns, getContext()); - - if (metadata_diff.constraints_changed) - new_metadata.constraints = ConstraintsDescription::parse(metadata_diff.new_constraints); - - if (metadata_diff.projections_changed) - new_metadata.projections = ProjectionsDescription::parse(metadata_diff.new_projections, new_columns, getContext()); - - if (metadata_diff.ttl_table_changed) - { - if (!metadata_diff.new_ttl_table.empty()) - { - ParserTTLExpressionList parser; - auto ttl_for_table_ast = parseQuery(parser, metadata_diff.new_ttl_table, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); - new_metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( - ttl_for_table_ast, new_metadata.columns, getContext(), new_metadata.primary_key); - } - else /// TTL was removed - { - new_metadata.table_ttl = TTLTableDescription{}; - } - } - } - - /// Changes in columns may affect following metadata fields - new_metadata.column_ttls_by_name.clear(); - for (const auto & [name, ast] : new_metadata.columns.getColumnTTLs()) - { - auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, new_metadata.columns, getContext(), new_metadata.primary_key); - new_metadata.column_ttls_by_name[name] = new_ttl_entry; - } - - if (new_metadata.partition_key.definition_ast != nullptr) - new_metadata.partition_key.recalculateWithNewColumns(new_metadata.columns, getContext()); - - if (!metadata_diff.sorting_key_changed) /// otherwise already updated - new_metadata.sorting_key.recalculateWithNewColumns(new_metadata.columns, getContext()); - - /// Primary key is special, it exists even if not defined - if (new_metadata.primary_key.definition_ast != nullptr) - { - new_metadata.primary_key.recalculateWithNewColumns(new_metadata.columns, getContext()); - } - else - { - new_metadata.primary_key = KeyDescription::getKeyFromAST(new_metadata.sorting_key.definition_ast, new_metadata.columns, getContext()); - new_metadata.primary_key.definition_ast = nullptr; - } - - if (!metadata_diff.sampling_expression_changed && new_metadata.sampling_key.definition_ast != nullptr) - new_metadata.sampling_key.recalculateWithNewColumns(new_metadata.columns, getContext()); - - if (!metadata_diff.skip_indices_changed) /// otherwise already updated - { - for (auto & index : new_metadata.secondary_indices) - index.recalculateWithNewColumns(new_metadata.columns, getContext()); - } - - if (!metadata_diff.ttl_table_changed && new_metadata.table_ttl.definition_ast != nullptr) - new_metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( - new_metadata.table_ttl.definition_ast, new_metadata.columns, getContext(), new_metadata.primary_key); + StorageInMemoryMetadata new_metadata = metadata_diff.getNewMetadata(new_columns, getContext(), old_metadata); /// Even if the primary/sorting/partition keys didn't change we must reinitialize it /// because primary/partition key column types might have changed. @@ -7511,6 +7397,24 @@ void StorageReplicatedMergeTree::createTableSharedID() } +std::optional StorageReplicatedMergeTree::tryGetTableSharedIDFromCreateQuery(const IAST & create_query, const ContextPtr & global_context) +{ + auto zk_path = tryExtractZkPathFromCreateQuery(create_query, global_context); + if (!zk_path) + return {}; + + String zk_name = zkutil::extractZooKeeperName(*zk_path); + zk_path = zkutil::extractZooKeeperPath(*zk_path, false, nullptr); + zkutil::ZooKeeperPtr zookeeper = (zk_name == getDefaultZooKeeperName()) ? global_context->getZooKeeper() : global_context->getAuxiliaryZooKeeper(zk_name); + + String id; + if (!zookeeper->tryGet(fs::path(*zk_path) / "table_shared_id", id)) + return {}; + + return id; +} + + void StorageReplicatedMergeTree::lockSharedDataTemporary(const String & part_name, const String & part_id, const DiskPtr & disk) const { auto settings = getSettings(); @@ -8261,44 +8165,21 @@ void StorageReplicatedMergeTree::createAndStoreFreezeMetadata(DiskPtr disk, Data } -ASTPtr StorageReplicatedMergeTree::getCreateQueryForBackup(const ContextPtr & local_context, DatabasePtr * database) const +void StorageReplicatedMergeTree::adjustCreateQueryForBackup(ASTPtr & create_query) const { - ASTPtr query = MergeTreeData::getCreateQueryForBackup(local_context, database); + /// Adjust the create query using values from ZooKeeper. + auto zookeeper = getZooKeeper(); + auto columns_from_entry = ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_path) / "columns")); + auto metadata_from_entry = ReplicatedMergeTreeTableMetadata::parse(zookeeper->get(fs::path(zookeeper_path) / "metadata")); - /// Before storing the metadata in a backup we have to find a zookeeper path in its definition and turn the table's UUID in there - /// back into "{uuid}", and also we probably can remove the zookeeper path and replica name if they're default. - /// So we're kind of reverting what we had done to the table's definition in registerStorageMergeTree.cpp before we created this table. - auto & create = query->as(); - if (create.storage && create.storage->engine && (create.uuid != UUIDHelpers::Nil)) - { - auto & engine = *(create.storage->engine); - if (auto * engine_args_ast = typeid_cast(engine.arguments.get())) - { - auto & engine_args = engine_args_ast->children; - if (engine_args.size() >= 2) - { - auto * zookeeper_path_ast = typeid_cast(engine_args[0].get()); - auto * replica_name_ast = typeid_cast(engine_args[1].get()); - if (zookeeper_path_ast && (zookeeper_path_ast->value.getType() == Field::Types::String) && - replica_name_ast && (replica_name_ast->value.getType() == Field::Types::String)) - { - String & zookeeper_path_arg = zookeeper_path_ast->value.get(); - String & replica_name_arg = replica_name_ast->value.get(); - String table_uuid_str = toString(create.uuid); - if (size_t uuid_pos = zookeeper_path_arg.find(table_uuid_str); uuid_pos != String::npos) - zookeeper_path_arg.replace(uuid_pos, table_uuid_str.size(), "{uuid}"); - const auto & config = getContext()->getConfigRef(); - if ((zookeeper_path_arg == getDefaultZooKeeperPath(config)) && (replica_name_arg == getDefaultReplicaName(config)) - && ((engine_args.size() == 2) || !engine_args[2]->as())) - { - engine_args.erase(engine_args.begin(), engine_args.begin() + 2); - } - } - } - } - } + auto current_metadata = getInMemoryMetadataPtr(); + auto metadata_diff = ReplicatedMergeTreeTableMetadata(*this, current_metadata).checkAndFindDiff(metadata_from_entry, current_metadata->getColumns(), getContext()); + auto adjusted_metadata = metadata_diff.getNewMetadata(columns_from_entry, getContext(), *current_metadata); + applyMetadataChangesToCreateQuery(create_query, adjusted_metadata); - return query; + /// Check that tryGetTableSharedIDFromCreateQuery() works for this storage. + if (tryGetTableSharedIDFromCreateQuery(*create_query, getContext()) != getTableSharedID()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} has its shared ID to be different from one from the create query"); } void StorageReplicatedMergeTree::backupData( @@ -8310,8 +8191,8 @@ void StorageReplicatedMergeTree::backupData( auto backup_entries = backupParts(backup_entries_collector.getContext(), "", partitions); auto coordination = backup_entries_collector.getBackupCoordination(); - String full_zk_path = getZooKeeperName() + getZooKeeperPath(); - coordination->addReplicatedDataPath(full_zk_path, data_path_in_backup); + String shared_id = getTableSharedID(); + coordination->addReplicatedDataPath(shared_id, data_path_in_backup); std::unordered_map part_names_with_hashes_calculating; for (auto & [relative_path, backup_entry] : backup_entries) @@ -8349,23 +8230,23 @@ void StorageReplicatedMergeTree::backupData( } /// Send our list of part names to the coordination (to compare with other replicas). - coordination->addReplicatedPartNames(full_zk_path, getStorageID().getFullTableName(), getReplicaName(), part_names_with_hashes); + coordination->addReplicatedPartNames(shared_id, getStorageID().getFullTableName(), getReplicaName(), part_names_with_hashes); /// This task will be executed after all replicas have collected their parts and the coordination is ready to /// give us the final list of parts to add to the BackupEntriesCollector. - auto post_collecting_task = [full_zk_path, + auto post_collecting_task = [shared_id, replica_name = getReplicaName(), coordination, backup_entries = std::move(backup_entries), &backup_entries_collector]() { - Strings data_paths = coordination->getReplicatedDataPaths(full_zk_path); + Strings data_paths = coordination->getReplicatedDataPaths(shared_id); std::vector data_paths_fs; data_paths_fs.reserve(data_paths.size()); for (const auto & data_path : data_paths) data_paths_fs.push_back(data_path); - Strings part_names = coordination->getReplicatedPartNames(full_zk_path, replica_name); + Strings part_names = coordination->getReplicatedPartNames(shared_id, replica_name); std::unordered_set part_names_set{part_names.begin(), part_names.end()}; for (const auto & [relative_path, backup_entry] : backup_entries) @@ -8378,7 +8259,7 @@ void StorageReplicatedMergeTree::backupData( backup_entries_collector.addBackupEntry(data_path / relative_path, backup_entry); } }; - backup_entries_collector.addPostCollectingTask(post_collecting_task); + backup_entries_collector.addPostTask(post_collecting_task); } void StorageReplicatedMergeTree::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 73a08a2b921..18b9ef54777 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -232,8 +232,8 @@ public: int getMetadataVersion() const { return metadata_version; } - /// Returns a slightly changed version of the CREATE TABLE query which must be written to a backup. - ASTPtr getCreateQueryForBackup(const ContextPtr & context, DatabasePtr * database) const override; + /// Modify a CREATE TABLE query to make a variant which must be written to a backup. + void adjustCreateQueryForBackup(ASTPtr & create_query) const override; /// Makes backup entries to backup the data of the storage. void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; @@ -312,6 +312,9 @@ public: // Return table id, common for different replicas String getTableSharedID() const override; + /// Returns the same as getTableSharedID(), but extracts it from a create query. + static std::optional tryGetTableSharedIDFromCreateQuery(const IAST & create_query, const ContextPtr & global_context); + static String getDefaultZooKeeperName() { return default_zookeeper_name; } /// Check if there are new broken disks and enqueue part recovery tasks. diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index d569a81c4a7..2033d33a33d 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -55,6 +55,7 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_FILE_NAME; extern const int TIMEOUT_EXCEEDED; + extern const int CANNOT_RESTORE_TABLE; } @@ -527,11 +528,8 @@ std::optional StorageStripeLog::totalBytes(const Settings &) const } -void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) +void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & /* partitions */) { - if (partitions) - BackupEntriesCollector::throwPartitionsNotSupported(getStorageID(), getName()); - auto lock_timeout = getLockTimeout(backup_entries_collector.getContext()); loadIndices(lock_timeout); @@ -589,13 +587,13 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec data_path_in_backup_fs / "count.txt", std::make_unique(toString(num_rows))); } -void StorageStripeLog::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) +void StorageStripeLog::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & /* partitions */) { - if (partitions) - RestorerFromBackup::throwPartitionsNotSupported(getStorageID(), getName()); - auto backup = restorer.getBackup(); - if (!restorer.isNonEmptyTableAllowed() && total_bytes && backup->hasFiles(data_path_in_backup)) + if (!backup->hasFiles(data_path_in_backup)) + return; + + if (!restorer.isNonEmptyTableAllowed() && total_bytes) RestorerFromBackup::throwTableIsNotEmpty(getStorageID()); auto lock_timeout = getLockTimeout(restorer.getContext()); @@ -624,6 +622,11 @@ void StorageStripeLog::restoreDataImpl(const BackupPtr & backup, const String & auto old_data_size = file_checker.getFileSize(data_file_path); { String file_path_in_backup = data_path_in_backup_fs / fileName(data_file_path); + if (!backup->fileExists(file_path_in_backup)) + { + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required", + getStorageID().getFullTableName(), file_path_in_backup); + } auto backup_entry = backup->readFile(file_path_in_backup); auto in = backup_entry->getReadBuffer(); auto out = disk->writeFile(data_file_path, max_compress_block_size, WriteMode::Append); @@ -634,6 +637,11 @@ void StorageStripeLog::restoreDataImpl(const BackupPtr & backup, const String & { String index_path_in_backup = data_path_in_backup_fs / fileName(index_file_path); IndexForNativeFormat extra_indices; + if (!backup->fileExists(index_path_in_backup)) + { + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required", + getStorageID().getFullTableName(), index_path_in_backup); + } auto backup_entry = backup->readFile(index_path_in_backup); auto index_in = backup_entry->getReadBuffer(); CompressedReadBuffer index_compressed_in{*index_in}; diff --git a/src/Storages/System/StorageSystemFunctions.cpp b/src/Storages/System/StorageSystemFunctions.cpp index b3f1231bd1a..e2bc699d3f1 100644 --- a/src/Storages/System/StorageSystemFunctions.cpp +++ b/src/Storages/System/StorageSystemFunctions.cpp @@ -9,6 +9,16 @@ #include #include #include +#include +#include +#include +#include +#include +#include +#include + +namespace fs = std::filesystem; + namespace DB { @@ -20,6 +30,11 @@ enum class FunctionOrigin : Int8 EXECUTABLE_USER_DEFINED = 2 }; +namespace ErrorCodes +{ + extern const int CANNOT_RESTORE_TABLE; +} + namespace { template @@ -99,4 +114,66 @@ void StorageSystemFunctions::fillData(MutableColumns & res_columns, ContextPtr c fillRow(res_columns, function_name, UInt64(0), "", FunctionOrigin::EXECUTABLE_USER_DEFINED, user_defined_executable_functions_factory); } } + +void StorageSystemFunctions::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & /* partitions */) +{ + const auto & user_defined_sql_functions_factory = UserDefinedSQLFunctionFactory::instance(); + const auto & user_defined_sql_functions_names = user_defined_sql_functions_factory.getAllRegisteredNames(); + fs::path data_path_in_backup_fs{data_path_in_backup}; + for (const auto & function_name : user_defined_sql_functions_names) + { + auto ast = user_defined_sql_functions_factory.tryGet(function_name); + if (!ast) + continue; + backup_entries_collector.addBackupEntry( + data_path_in_backup_fs / (escapeForFileName(function_name) + ".sql"), + std::make_shared(queryToString(ast))); + } +} + +void StorageSystemFunctions::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & /* partitions */) +{ + auto backup = restorer.getBackup(); + fs::path data_path_in_backup_fs{data_path_in_backup}; + + Strings filenames = backup->listFiles(data_path_in_backup); + for (const auto & filename : filenames) + { + if (!filename.ends_with(".sql")) + { + throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File name {} doesn't have the extension .sql", + getStorageID().getFullTableName(), String{data_path_in_backup_fs / filename}); + } + } + + auto & user_defined_sql_functions_factory = UserDefinedSQLFunctionFactory::instance(); + const auto & restore_settings = restorer.getRestoreSettings(); + auto context = restorer.getContext(); + + for (const auto & filename : filenames) + { + String escaped_function_name = filename.substr(0, filename.length() - strlen(".sql")); + String function_name = unescapeForFileName(escaped_function_name); + + String filepath = data_path_in_backup_fs / filename; + auto function_def_entry = backup->readFile(filepath); + auto function_def_in = function_def_entry->getReadBuffer(); + String function_def; + readStringUntilEOF(function_def, *function_def_in); + + ParserCreateFunctionQuery parser; + ASTPtr ast = parseQuery( + parser, + function_def.data(), + function_def.data() + function_def.size(), + "in file " + filepath + " from backup " + backup->getName(), + 0, + context->getSettingsRef().max_parser_depth); + + bool replace = (restore_settings.create_function == RestoreUDFCreationMode::kReplace); + bool if_not_exists = (restore_settings.create_function == RestoreUDFCreationMode::kCreateIfNotExists); + user_defined_sql_functions_factory.registerFunction(context, function_name, ast, replace, if_not_exists, true); + } +} + } diff --git a/src/Storages/System/StorageSystemFunctions.h b/src/Storages/System/StorageSystemFunctions.h index fdbe79e29a2..606694a4c0b 100644 --- a/src/Storages/System/StorageSystemFunctions.h +++ b/src/Storages/System/StorageSystemFunctions.h @@ -19,6 +19,9 @@ public: static NamesAndTypesList getNamesAndTypes(); + void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; + void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; + protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemQuotas.cpp b/src/Storages/System/StorageSystemQuotas.cpp index efe6b93fe57..046db151684 100644 --- a/src/Storages/System/StorageSystemQuotas.cpp +++ b/src/Storages/System/StorageSystemQuotas.cpp @@ -122,23 +122,17 @@ void StorageSystemQuotas::fillData(MutableColumns & res_columns, ContextPtr cont } void StorageSystemQuotas::backupData( - BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) + BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & /* partitions */) { - if (partitions) - BackupEntriesCollector::throwPartitionsNotSupported(getStorageID(), getName()); - const auto & access_control = backup_entries_collector.getContext()->getAccessControl(); - access_control.backup(backup_entries_collector, AccessEntityType::QUOTA, data_path_in_backup); + access_control.backup(backup_entries_collector, data_path_in_backup, AccessEntityType::QUOTA); } void StorageSystemQuotas::restoreDataFromBackup( - RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) + RestorerFromBackup & restorer, const String & /* data_path_in_backup */, const std::optional & /* partitions */) { - if (partitions) - RestorerFromBackup::throwPartitionsNotSupported(getStorageID(), getName()); - auto & access_control = restorer.getContext()->getAccessControl(); - access_control.restore(restorer, data_path_in_backup); + access_control.restoreFromBackup(restorer); } } diff --git a/src/Storages/System/StorageSystemRoles.cpp b/src/Storages/System/StorageSystemRoles.cpp index ff3490ce8ba..e5b8d53ce7e 100644 --- a/src/Storages/System/StorageSystemRoles.cpp +++ b/src/Storages/System/StorageSystemRoles.cpp @@ -60,23 +60,17 @@ void StorageSystemRoles::fillData(MutableColumns & res_columns, ContextPtr conte } void StorageSystemRoles::backupData( - BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) + BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & /* partitions */) { - if (partitions) - BackupEntriesCollector::throwPartitionsNotSupported(getStorageID(), getName()); - const auto & access_control = backup_entries_collector.getContext()->getAccessControl(); - access_control.backup(backup_entries_collector, AccessEntityType::ROLE, data_path_in_backup); + access_control.backup(backup_entries_collector, data_path_in_backup, AccessEntityType::ROLE); } void StorageSystemRoles::restoreDataFromBackup( - RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) + RestorerFromBackup & restorer, const String & /* data_path_in_backup */, const std::optional & /* partitions */) { - if (partitions) - RestorerFromBackup::throwPartitionsNotSupported(getStorageID(), getName()); - auto & access_control = restorer.getContext()->getAccessControl(); - access_control.restore(restorer, data_path_in_backup); + access_control.restoreFromBackup(restorer); } } diff --git a/src/Storages/System/StorageSystemRowPolicies.cpp b/src/Storages/System/StorageSystemRowPolicies.cpp index 680f90adff7..064f610730d 100644 --- a/src/Storages/System/StorageSystemRowPolicies.cpp +++ b/src/Storages/System/StorageSystemRowPolicies.cpp @@ -140,23 +140,17 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr } void StorageSystemRowPolicies::backupData( - BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) + BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & /* partitions */) { - if (partitions) - BackupEntriesCollector::throwPartitionsNotSupported(getStorageID(), getName()); - const auto & access_control = backup_entries_collector.getContext()->getAccessControl(); - access_control.backup(backup_entries_collector, AccessEntityType::ROW_POLICY, data_path_in_backup); + access_control.backup(backup_entries_collector, data_path_in_backup, AccessEntityType::ROW_POLICY); } void StorageSystemRowPolicies::restoreDataFromBackup( - RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) + RestorerFromBackup & restorer, const String & /* data_path_in_backup */, const std::optional & /* partitions */) { - if (partitions) - RestorerFromBackup::throwPartitionsNotSupported(getStorageID(), getName()); - auto & access_control = restorer.getContext()->getAccessControl(); - access_control.restore(restorer, data_path_in_backup); + access_control.restoreFromBackup(restorer); } } diff --git a/src/Storages/System/StorageSystemSettingsProfiles.cpp b/src/Storages/System/StorageSystemSettingsProfiles.cpp index 7c3ccfe863a..d03848ba68b 100644 --- a/src/Storages/System/StorageSystemSettingsProfiles.cpp +++ b/src/Storages/System/StorageSystemSettingsProfiles.cpp @@ -87,23 +87,17 @@ void StorageSystemSettingsProfiles::fillData(MutableColumns & res_columns, Conte } void StorageSystemSettingsProfiles::backupData( - BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) + BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & /* partitions */) { - if (partitions) - BackupEntriesCollector::throwPartitionsNotSupported(getStorageID(), getName()); - const auto & access_control = backup_entries_collector.getContext()->getAccessControl(); - access_control.backup(backup_entries_collector, AccessEntityType::SETTINGS_PROFILE, data_path_in_backup); + access_control.backup(backup_entries_collector, data_path_in_backup, AccessEntityType::SETTINGS_PROFILE); } void StorageSystemSettingsProfiles::restoreDataFromBackup( - RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) + RestorerFromBackup & restorer, const String & /* data_path_in_backup */, const std::optional & /* partitions */) { - if (partitions) - RestorerFromBackup::throwPartitionsNotSupported(getStorageID(), getName()); - auto & access_control = restorer.getContext()->getAccessControl(); - access_control.restore(restorer, data_path_in_backup); + access_control.restoreFromBackup(restorer); } } diff --git a/src/Storages/System/StorageSystemUsers.cpp b/src/Storages/System/StorageSystemUsers.cpp index f2cae638d45..be56abfa3e8 100644 --- a/src/Storages/System/StorageSystemUsers.cpp +++ b/src/Storages/System/StorageSystemUsers.cpp @@ -215,23 +215,17 @@ void StorageSystemUsers::fillData(MutableColumns & res_columns, ContextPtr conte } void StorageSystemUsers::backupData( - BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) + BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & /* partitions */) { - if (partitions) - BackupEntriesCollector::throwPartitionsNotSupported(getStorageID(), getName()); - const auto & access_control = backup_entries_collector.getContext()->getAccessControl(); - access_control.backup(backup_entries_collector, AccessEntityType::USER, data_path_in_backup); + access_control.backup(backup_entries_collector, data_path_in_backup, AccessEntityType::USER); } void StorageSystemUsers::restoreDataFromBackup( - RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) + RestorerFromBackup & restorer, const String & /* data_path_in_backup */, const std::optional & /* partitions */) { - if (partitions) - RestorerFromBackup::throwPartitionsNotSupported(getStorageID(), getName()); - auto & access_control = restorer.getContext()->getAccessControl(); - access_control.restore(restorer, data_path_in_backup); + access_control.restoreFromBackup(restorer); } } diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 3996a31e7c9..a930ddac7df 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -138,7 +138,7 @@ def test_backup_table_under_another_name(): assert instance.query("SELECT count(), sum(x) FROM test.table2") == "100\t4950\n" -def test_materialized_view(): +def test_materialized_view_select_1(): backup_name = new_backup_name() instance.query( "CREATE MATERIALIZED VIEW mv_1(x UInt8) ENGINE=MergeTree ORDER BY tuple() POPULATE AS SELECT 1 AS x" @@ -456,18 +456,32 @@ def test_temporary_table(): ) == TSV([["e"], ["q"], ["w"]]) -# "BACKUP DATABASE _temporary_and_external_tables" is allowed but the backup must not contain these tables. -def test_temporary_tables_database(): +# The backup created by "BACKUP DATABASE _temporary_and_external_tables" must not contain tables from other sessions. +def test_temporary_database(): session_id = new_session_id() instance.http_query( "CREATE TEMPORARY TABLE temp_tbl(s String)", params={"session_id": session_id} ) - backup_name = new_backup_name() - instance.query(f"BACKUP DATABASE _temporary_and_external_tables TO {backup_name}") + other_session_id = new_session_id() + instance.http_query( + "CREATE TEMPORARY TABLE other_temp_tbl(s String)", + params={"session_id": other_session_id}, + ) - assert os.listdir(os.path.join(get_path_to_backup(backup_name), "metadata/")) == [ - "_temporary_and_external_tables.sql" # database metadata only + backup_name = new_backup_name() + instance.http_query( + f"BACKUP DATABASE _temporary_and_external_tables TO {backup_name}", + params={"session_id": session_id}, + ) + + assert os.listdir( + os.path.join(get_path_to_backup(backup_name), "temporary_tables/metadata") + ) == ["temp_tbl.sql"] + + assert sorted(os.listdir(get_path_to_backup(backup_name))) == [ + ".backup", + "temporary_tables", ] @@ -711,3 +725,107 @@ def test_system_users_async(): instance.query("SHOW CREATE USER u1") == "CREATE USER u1 IDENTIFIED WITH sha256_password SETTINGS custom_c = 3\n" ) + + +def test_projection(): + create_and_fill_table(n=3) + + instance.query("ALTER TABLE test.table ADD PROJECTION prjmax (SELECT MAX(x))") + instance.query(f"INSERT INTO test.table VALUES (100, 'a'), (101, 'b')") + + assert ( + instance.query( + "SELECT count() FROM system.projection_parts WHERE database='test' AND table='table' AND name='prjmax'" + ) + == "2\n" + ) + + backup_name = new_backup_name() + instance.query(f"BACKUP TABLE test.table TO {backup_name}") + + assert os.path.exists( + os.path.join( + get_path_to_backup(backup_name), "data/test/table/1_5_5_0/data.bin" + ) + ) + + assert os.path.exists( + os.path.join( + get_path_to_backup(backup_name), + "data/test/table/1_5_5_0/prjmax.proj/data.bin", + ) + ) + + instance.query("DROP TABLE test.table") + + assert ( + instance.query( + "SELECT count() FROM system.projection_parts WHERE database='test' AND table='table' AND name='prjmax'" + ) + == "0\n" + ) + + instance.query(f"RESTORE TABLE test.table FROM {backup_name}") + + assert instance.query("SELECT * FROM test.table ORDER BY x") == TSV( + [[0, "0"], [1, "1"], [2, "2"], [100, "a"], [101, "b"]] + ) + + assert ( + instance.query( + "SELECT count() FROM system.projection_parts WHERE database='test' AND table='table' AND name='prjmax'" + ) + == "2\n" + ) + + +def test_system_functions(): + instance.query("CREATE FUNCTION linear_equation AS (x, k, b) -> k*x + b;") + + instance.query("CREATE FUNCTION parity_str AS (n) -> if(n % 2, 'odd', 'even');") + + backup_name = new_backup_name() + instance.query(f"BACKUP TABLE system.functions TO {backup_name}") + + instance.query("DROP FUNCTION linear_equation") + instance.query("DROP FUNCTION parity_str") + + instance.query(f"RESTORE TABLE system.functions FROM {backup_name}") + + assert instance.query( + "SELECT number, linear_equation(number, 2, 1) FROM numbers(3)" + ) == TSV([[0, 1], [1, 3], [2, 5]]) + + assert instance.query("SELECT number, parity_str(number) FROM numbers(3)") == TSV( + [[0, "even"], [1, "odd"], [2, "even"]] + ) + + +def test_backup_partition(): + create_and_fill_table(n=30) + + backup_name = new_backup_name() + instance.query(f"BACKUP TABLE test.table PARTITIONS '1', '4' TO {backup_name}") + + instance.query("DROP TABLE test.table") + + instance.query(f"RESTORE TABLE test.table FROM {backup_name}") + + assert instance.query("SELECT * FROM test.table ORDER BY x") == TSV( + [[1, "1"], [4, "4"], [11, "11"], [14, "14"], [21, "21"], [24, "24"]] + ) + + +def test_restore_partition(): + create_and_fill_table(n=30) + + backup_name = new_backup_name() + instance.query(f"BACKUP TABLE test.table TO {backup_name}") + + instance.query("DROP TABLE test.table") + + instance.query(f"RESTORE TABLE test.table PARTITIONS '2', '3' FROM {backup_name}") + + assert instance.query("SELECT * FROM test.table ORDER BY x") == TSV( + [[2, "2"], [3, "3"], [12, "12"], [13, "13"], [22, "22"], [23, "23"]] + ) diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index 6264959fbce..02f855cf766 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -514,3 +514,141 @@ def test_system_users(): node1.query("SHOW CREATE USER u1") == "CREATE USER u1 SETTINGS custom_a = 123\n" ) assert node1.query("SHOW GRANTS FOR u1") == "GRANT SELECT ON default.tbl TO u1\n" + + +def test_projection(): + node1.query( + "CREATE TABLE tbl ON CLUSTER 'cluster' (x UInt32, y String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/tbl/', '{replica}') " + "ORDER BY y PARTITION BY x%10" + ) + node1.query(f"INSERT INTO tbl SELECT number, toString(number) FROM numbers(3)") + + node1.query("ALTER TABLE tbl ADD PROJECTION prjmax (SELECT MAX(x))") + node1.query(f"INSERT INTO tbl VALUES (100, 'a'), (101, 'b')") + + assert ( + node1.query( + "SELECT count() FROM system.projection_parts WHERE database='default' AND table='tbl' AND name='prjmax'" + ) + == "2\n" + ) + + backup_name = new_backup_name() + node1.query(f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}") + + node1.query(f"DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY") + + assert ( + node1.query( + "SELECT count() FROM system.projection_parts WHERE database='default' AND table='tbl' AND name='prjmax'" + ) + == "0\n" + ) + + node1.query(f"RESTORE TABLE tbl FROM {backup_name}") + + assert node1.query("SELECT * FROM tbl ORDER BY x") == TSV( + [[0, "0"], [1, "1"], [2, "2"], [100, "a"], [101, "b"]] + ) + + assert ( + node1.query( + "SELECT count() FROM system.projection_parts WHERE database='default' AND table='tbl' AND name='prjmax'" + ) + == "2\n" + ) + + +def test_replicated_table_with_not_synced_def(): + node1.query( + "CREATE TABLE tbl (" + "x UInt8, y String" + ") ENGINE=ReplicatedMergeTree('/clickhouse/tables/tbl/', '{replica}')" + "ORDER BY tuple()" + ) + + node2.query( + "CREATE TABLE tbl (" + "x UInt8, y String" + ") ENGINE=ReplicatedMergeTree('/clickhouse/tables/tbl/', '{replica}')" + "ORDER BY tuple()" + ) + + node2.query("SYSTEM STOP REPLICATION QUEUES tbl") + node1.query("ALTER TABLE tbl MODIFY COLUMN x String") + + # Not synced because the replication queue is stopped + assert node1.query( + "SELECT name, type FROM system.columns WHERE database='default' AND table='tbl'" + ) == TSV([["x", "String"], ["y", "String"]]) + assert node2.query( + "SELECT name, type FROM system.columns WHERE database='default' AND table='tbl'" + ) == TSV([["x", "UInt8"], ["y", "String"]]) + + backup_name = new_backup_name() + node2.query(f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}") + + node1.query("DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY") + + # But synced after RESTORE anyway + node1.query( + f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name} SETTINGS replica_num_in_backup=1" + ) + assert node1.query( + "SELECT name, type FROM system.columns WHERE database='default' AND table='tbl'" + ) == TSV([["x", "String"], ["y", "String"]]) + assert node2.query( + "SELECT name, type FROM system.columns WHERE database='default' AND table='tbl'" + ) == TSV([["x", "String"], ["y", "String"]]) + + node1.query("DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY") + + node2.query( + f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name} SETTINGS replica_num_in_backup=2" + ) + assert node1.query( + "SELECT name, type FROM system.columns WHERE database='default' AND table='tbl'" + ) == TSV([["x", "String"], ["y", "String"]]) + assert node2.query( + "SELECT name, type FROM system.columns WHERE database='default' AND table='tbl'" + ) == TSV([["x", "String"], ["y", "String"]]) + + +def test_table_in_replicated_database_with_not_synced_def(): + node1.query( + "CREATE DATABASE mydb ON CLUSTER 'cluster' ENGINE=Replicated('/clickhouse/path/','{shard}','{replica}')" + ) + + node1.query( + "CREATE TABLE mydb.tbl (x UInt8, y String) ENGINE=ReplicatedMergeTree ORDER BY tuple()" + ) + + node1.query("ALTER TABLE mydb.tbl MODIFY COLUMN x String") + + backup_name = new_backup_name() + node2.query(f"BACKUP DATABASE mydb ON CLUSTER 'cluster' TO {backup_name}") + + node1.query("DROP DATABASE mydb ON CLUSTER 'cluster' NO DELAY") + + # But synced after RESTORE anyway + node1.query( + f"RESTORE DATABASE mydb ON CLUSTER 'cluster' FROM {backup_name} SETTINGS replica_num_in_backup=1" + ) + assert node1.query( + "SELECT name, type FROM system.columns WHERE database='mydb' AND table='tbl'" + ) == TSV([["x", "String"], ["y", "String"]]) + assert node2.query( + "SELECT name, type FROM system.columns WHERE database='mydb' AND table='tbl'" + ) == TSV([["x", "String"], ["y", "String"]]) + + node1.query("DROP DATABASE mydb ON CLUSTER 'cluster' NO DELAY") + + node2.query( + f"RESTORE DATABASE mydb ON CLUSTER 'cluster' FROM {backup_name} SETTINGS replica_num_in_backup=2" + ) + assert node1.query( + "SELECT name, type FROM system.columns WHERE database='mydb' AND table='tbl'" + ) == TSV([["x", "String"], ["y", "String"]]) + assert node2.query( + "SELECT name, type FROM system.columns WHERE database='mydb' AND table='tbl'" + ) == TSV([["x", "String"], ["y", "String"]])