Merge pull request #38537 from vitlibar/backup-improvements-8

Backup Improvements 8
This commit is contained in:
Vitaly Baranov 2022-07-01 11:20:46 +02:00 committed by GitHub
commit cadb496894
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
76 changed files with 2434 additions and 1349 deletions

View File

@ -10,6 +10,7 @@
#include <Backups/BackupEntriesCollector.h>
#include <Backups/BackupEntryFromMemory.h>
#include <Backups/IBackup.h>
#include <Backups/RestoreSettings.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromString.h>
@ -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<UUID, AccessEntityPtr> & entities)
AccessRightsElements getRequiredAccessToRestore(const std::vector<std::pair<UUID, AccessEntityPtr>> & 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<String, BackupEntryPtr> makeBackupEntryForAccess(
const std::vector<std::pair<UUID, AccessEntityPtr>> 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<IRestoreCoordination> 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<std::pair<UUID, AccessEntityPtr>> 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<std::pair<UUID, AccessEntityPtr>> 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;
}
}

View File

@ -1,6 +1,6 @@
#pragma once
#include <Backups/RestoreSettings.h>
#include <Core/UUID.h>
#include <unordered_map>
#include <unordered_set>
@ -9,46 +9,45 @@ namespace DB
{
class AccessControl;
enum class AccessEntityType;
class BackupEntriesCollector;
class RestorerFromBackup;
class IBackup;
using BackupPtr = std::shared_ptr<const IBackup>;
class IRestoreCoordination;
struct IAccessEntity;
using AccessEntityPtr = std::shared_ptr<const IAccessEntity>;
class AccessRightsElements;
class IBackup;
using BackupPtr = std::shared_ptr<const IBackup>;
class IBackupEntry;
using BackupEntryPtr = std::shared_ptr<const IBackupEntry>;
struct RestoreSettings;
struct QualifiedTableName;
/// Makes a backup of access entities of a specified type.
void backupAccessEntities(
BackupEntriesCollector & backup_entries_collector,
std::pair<String, BackupEntryPtr> makeBackupEntryForAccess(
const std::vector<std::pair<UUID, AccessEntityPtr>> 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<IRestoreCoordination> 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<std::pair<UUID, AccessEntityPtr>> getAccessEntities(const AccessControl & access_control) const;
private:
BackupPtr backup;
RestoreSettings restore_settings;
std::shared_ptr<IRestoreCoordination> restore_coordination;
std::unordered_map<UUID, AccessEntityPtr> entities;
bool allow_unresolved_access_dependencies = false;
std::vector<std::pair<UUID, AccessEntityPtr>> entities;
std::unordered_map<UUID, std::pair<String, AccessEntityType>> dependencies;
std::unordered_set<String> data_paths;
};

View File

@ -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<std::pair<UUID, AccessEntityPtr>> & entities_from_backup, const RestoreSettings & restore_settings, std::shared_ptr<IRestoreCoordination> restore_coordination)
{
MultipleAccessStorage::insertFromBackup(entities_from_backup, restore_settings, restore_coordination);
MultipleAccessStorage::restoreFromBackup(restorer);
changes_notifier->sendNotifications();
}

View File

@ -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<std::pair<UUID, AccessEntityPtr>> & entities_from_backup, const RestoreSettings & restore_settings, std::shared_ptr<IRestoreCoordination> restore_coordination) override;
private:
class ContextAccessCache;
class CustomSettingsPrefixes;

View File

@ -1,6 +1,7 @@
#include <Access/DiskAccessStorage.h>
#include <Access/AccessEntityIO.h>
#include <Access/AccessChangesNotifier.h>
#include <Backups/RestorerFromBackup.h>
#include <Backups/RestoreSettings.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
@ -650,19 +651,24 @@ void DiskAccessStorage::deleteAccessEntityOnDisk(const UUID & id) const
}
void DiskAccessStorage::insertFromBackup(
const std::vector<std::pair<UUID, AccessEntityPtr>> & entities_from_backup,
const RestoreSettings & restore_settings,
std::shared_ptr<IRestoreCoordination>)
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);
});
}
}

View File

@ -30,7 +30,7 @@ public:
bool exists(const UUID & id) const override;
bool isBackupAllowed() const override { return backup_allowed; }
void insertFromBackup(const std::vector<std::pair<UUID, AccessEntityPtr>> & entities_from_backup, const RestoreSettings & restore_settings, std::shared_ptr<IRestoreCoordination> restore_coordination) override;
void restoreFromBackup(RestorerFromBackup & restorer) override;
private:
std::optional<UUID> findImpl(AccessEntityType type, const String & name) const override;

View File

@ -2,9 +2,12 @@
#include <Access/Authentication.h>
#include <Access/Credentials.h>
#include <Access/User.h>
#include <Access/AccessBackup.h>
#include <Backups/BackupEntriesCollector.h>
#include <Common/Exception.h>
#include <Common/quoteString.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <Poco/UUIDGenerator.h>
#include <Poco/Logger.h>
#include <base/FnTraits.h>
@ -520,26 +523,30 @@ bool IAccessStorage::isAddressAllowed(const User & user, const Poco::Net::IPAddr
}
bool IAccessStorage::isRestoreAllowed() const
{
return isBackupAllowed() && !isReadOnly();
}
std::vector<std::pair<UUID, AccessEntityPtr>> 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<UUID, AccessEntityPtr> & x) { return !x.second->isBackupAllowed(); });
return res;
auto entities = readAllWithIDs(type);
boost::range::remove_erase_if(entities, [](const std::pair<UUID, AccessEntityPtr> & 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<std::pair<UUID, AccessEntityPtr>> &, const RestoreSettings &, std::shared_ptr<IRestoreCoordination>)
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());
}

View File

@ -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<std::pair<UUID, AccessEntityPtr>> readAllForBackup(AccessEntityType type, const BackupSettings & backup_settings) const;
virtual void insertFromBackup(const std::vector<std::pair<UUID, AccessEntityPtr>> & entities_from_backup, const RestoreSettings & restore_settings, std::shared_ptr<IRestoreCoordination> 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<UUID> findImpl(AccessEntityType type, const String & name) const = 0;

View File

@ -1,5 +1,6 @@
#include <Access/MemoryAccessStorage.h>
#include <Access/AccessChangesNotifier.h>
#include <Backups/RestorerFromBackup.h>
#include <Backups/RestoreSettings.h>
#include <base/scope_guard.h>
#include <boost/container/flat_set.hpp>
@ -272,19 +273,24 @@ void MemoryAccessStorage::setAll(const std::vector<std::pair<UUID, AccessEntityP
}
void MemoryAccessStorage::insertFromBackup(
const std::vector<std::pair<UUID, AccessEntityPtr>> & entities_from_backup,
const RestoreSettings & restore_settings,
std::shared_ptr<IRestoreCoordination>)
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);
});
}
}

View File

@ -29,7 +29,7 @@ public:
bool exists(const UUID & id) const override;
bool isBackupAllowed() const override { return backup_allowed; }
void insertFromBackup(const std::vector<std::pair<UUID, AccessEntityPtr>> & entities_from_backup, const RestoreSettings & restore_settings, std::shared_ptr<IRestoreCoordination> restore_coordination) override;
void restoreFromBackup(RestorerFromBackup & restorer) override;
private:
std::optional<UUID> findImpl(AccessEntityType type, const String & name) const override;

View File

@ -383,40 +383,38 @@ bool MultipleAccessStorage::isRestoreAllowed() const
}
std::vector<std::pair<UUID, AccessEntityPtr>> 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<std::pair<UUID, AccessEntityPtr>> 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<std::pair<UUID, AccessEntityPtr>> & entities_from_backup, const RestoreSettings & restore_settings, std::shared_ptr<IRestoreCoordination> 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();
}
}

View File

@ -45,8 +45,8 @@ public:
bool isBackupAllowed() const override;
bool isRestoreAllowed() const override;
std::vector<std::pair<UUID, AccessEntityPtr>> readAllForBackup(AccessEntityType type, const BackupSettings & backup_settings) const override;
void insertFromBackup(const std::vector<std::pair<UUID, AccessEntityPtr>> & entities_from_backup, const RestoreSettings & restore_settings, std::shared_ptr<IRestoreCoordination> 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<UUID> findImpl(AccessEntityType type, const String & name) const override;

View File

@ -2,10 +2,14 @@
#include <Access/MemoryAccessStorage.h>
#include <Access/ReplicatedAccessStorage.h>
#include <Access/AccessChangesNotifier.h>
#include <Access/AccessBackup.h>
#include <Backups/BackupEntriesCollector.h>
#include <Backups/RestorerFromBackup.h>
#include <Backups/RestoreSettings.h>
#include <Backups/IBackupCoordination.h>
#include <Backups/IRestoreCoordination.h>
#include <IO/ReadHelpers.h>
#include <boost/container/flat_set.hpp>
#include <Interpreters/Context.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/ZooKeeper/Types.h>
#include <Common/ZooKeeper/ZooKeeper.h>
@ -13,6 +17,7 @@
#include <Common/setThreadName.h>
#include <base/range.h>
#include <base/sleep.h>
#include <boost/range/algorithm_ext/erase.hpp>
namespace DB
@ -613,19 +618,64 @@ AccessEntityPtr ReplicatedAccessStorage::readImpl(const UUID & id, bool throw_if
return entry.entity;
}
void ReplicatedAccessStorage::insertFromBackup(const std::vector<std::pair<UUID, AccessEntityPtr>> & entities_from_backup, const RestoreSettings & restore_settings, std::shared_ptr<IRestoreCoordination> 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<UUID, AccessEntityPtr> & 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);
});
}
}

View File

@ -38,7 +38,8 @@ public:
bool exists(const UUID & id) const override;
bool isBackupAllowed() const override { return backup_allowed; }
void insertFromBackup(const std::vector<std::pair<UUID, AccessEntityPtr>> & entities_from_backup, const RestoreSettings & restore_settings, std::shared_ptr<IRestoreCoordination> 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;

View File

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

View File

@ -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<PartNameAndChecksum> & 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<BackupCoordinationReplicatedPartNames> replicated_part_names;

View File

@ -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<PartNameAndChecksum> & 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<CoveredPartsFinder>(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<UInt64> & 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<String, std::vector<size_t> /* 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<String> host_with_error;
std::optional<String> error_message;
std::map<String, std::optional<int>> unready_hosts;
for (const String & host : wait_hosts)
unready_hosts.emplace(host, std::optional<int>{});
/// 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<int>(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;
}
}

View File

@ -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<PartNameAndChecksum> & 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<CoveredPartsFinder> covered_parts_finder;
};
std::map<String /* table_zk_path */, TableInfo> table_infos; /// Should be ordered because we need this map to be in the same order on every replica.
std::map<String /* table_shared_id */, TableInfo> 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<UInt64> & timeout_ms);
String zookeeper_path;
zkutil::GetZooKeeper get_zookeeper;

View File

@ -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<PartNameAndChecksum> & 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<PartNameAndChecksum> & 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};

View File

@ -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<PartNameAndChecksum> & 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<String, Strings> replicated_data_paths TSA_GUARDED_BY(mutex);
std::unordered_map<String, Strings> replicated_access_paths TSA_GUARDED_BY(mutex);
std::unordered_map<String, String> replicated_access_hosts TSA_GUARDED_BY(mutex);
std::map<String /* file_name */, SizeAndChecksum> file_names TSA_GUARDED_BY(mutex); /// Should be ordered alphabetically, see listFiles(). For empty files we assume checksum = 0.
std::map<SizeAndChecksum, FileInfo> file_infos TSA_GUARDED_BY(mutex); /// Information about files. Without empty files.
Strings archive_suffixes TSA_GUARDED_BY(mutex);

File diff suppressed because it is too large Load Diff

View File

@ -19,6 +19,7 @@ class IBackupCoordination;
class IDatabase;
using DatabasePtr = std::shared_ptr<IDatabase>;
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<IBackupCoordination> 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<IBackupCoordination> 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<String, BackupEntryPtr> & 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<void()> task);
void addPostTask(std::function<void()> 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<ASTs> & partitions, bool throw_if_not_found);
void collectDatabaseInfo(const String & database_name, const std::set<DatabaseAndTableName> & except_table_names, bool throw_if_not_found);
void collectAllDatabasesInfo(const std::set<String> & except_database_names, const std::set<DatabaseAndTableName> & except_table_names);
void checkConsistency();
void gatherMetadataAndCheckConsistency();
bool tryGatherMetadataAndCompareWithPrevious(std::optional<Exception> & inconsistency_error);
void gatherDatabasesMetadata();
void gatherDatabaseMetadata(
const String & database_name,
bool throw_if_database_not_found,
bool backup_create_database_query,
const std::optional<String> & table_name,
bool throw_if_table_not_found,
const std::optional<ASTs> & partitions,
bool all_tables,
const std::set<DatabaseAndTableName> & except_table_names);
void gatherTablesMetadata();
void lockTablesForReading();
bool compareWithPrevious(std::optional<Exception> & 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<IBackupCoordination> 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<ASTs> partitions;
};
std::unordered_map<String, TableParams> tables;
bool all_tables = false;
std::unordered_set<String> 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<String> replicated_table_shared_id;
std::optional<ASTs> 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<String, DatabaseInfo> database_infos;
std::map<TableKey, TableInfo> table_infos;
std::optional<std::set<String>> previous_database_names;
std::optional<std::set<TableKey>> previous_table_names;
bool consistent = false;
std::unordered_map<QualifiedTableName, TableInfo> table_infos;
std::vector<std::pair<String, String>> previous_databases_metadata;
std::vector<std::pair<QualifiedTableName, String>> previous_tables_metadata;
BackupEntries backup_entries;
std::queue<std::function<void()>> post_collecting_tasks;
std::queue<std::function<void()>> post_tasks;
std::vector<size_t> access_counters;
};
}

View File

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

View File

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

View File

@ -0,0 +1,113 @@
#include <Backups/DDLAdjustingForBackupVisitor.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Interpreters/Context.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Parsers/formatAST.h>
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<ASTCreateQuery &>();
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<ASTCreateQuery &>();
auto & engine = *storage.engine;
auto * engine_args_ast = typeid_cast<ASTExpressionList *>(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<ASTLiteral *>(engine_args[0].get());
auto * replica_name_ast = typeid_cast<ASTLiteral *>(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>();
String & replica_name_arg = replica_name_ast->value.get<String>();
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<ASTLiteral>()))
{
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<ASTCreateQuery>())
visitCreateQuery(*create, data);
}
void adjustCreateQueryForBackup(ASTPtr ast, const ContextPtr & global_context, std::optional<String> * 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);
}
}

View File

@ -0,0 +1,36 @@
#pragma once
#include <Interpreters/InDepthNodeVisitor.h>
#include <memory>
namespace DB
{
class IAST;
using ASTPtr = std::shared_ptr<IAST>;
class Context;
using ContextPtr = std::shared_ptr<const Context>;
/// 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<String> * 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<String> * replicated_table_shared_id = nullptr;
};
using Visitor = InDepthNodeVisitor<DDLAdjustingForBackupVisitor, false>;
static bool needChildVisit(const ASTPtr & ast, const ASTPtr & child);
static void visit(ASTPtr ast, const Data & data);
};
}

View File

@ -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<PartNameAndChecksum> & 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
{

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -1,9 +1,11 @@
#include <Backups/RestorerFromBackup.h>
#include <Backups/IRestoreCoordination.h>
#include <Backups/BackupCoordinationHelpers.h>
#include <Backups/BackupSettings.h>
#include <Backups/IBackup.h>
#include <Backups/IBackupEntry.h>
#include <Backups/BackupUtils.h>
#include <Backups/DDLAdjustingForBackupVisitor.h>
#include <Access/AccessBackup.h>
#include <Access/AccessRights.h>
#include <Parsers/ParserCreateQuery.h>
@ -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<ASTCreateQuery>();
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 <name>" or "temporary table <name>"
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<int>(stage));
}
}
RestorerFromBackup::RestorerFromBackup(
@ -94,71 +100,66 @@ RestorerFromBackup::RestorerFromBackup(
const RestoreSettings & restore_settings_,
std::shared_ptr<IRestoreCoordination> 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/<shard_num>/replicas/<replica_num>/".
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/<shard_num>/replicas/<replica_num>/".
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<std::vector<StoragePtr>>();
auto table_locks = std::make_shared<std::vector<TableLockHolder>>();
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<int>(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<ASTs> & partitions)
void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name_in_backup, const std::optional<ASTs> & 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<fs::path> metadata_path;
std::optional<fs::path> 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<AccessRestoreTask>(backup, restore_settings, restore_coordination);
access_restore_task->addDataPath(data_path_in_backup);
if (!access_restorer)
access_restorer = std::make_unique<AccessRestorerFromBackup>(backup, restore_settings);
access_restorer->addDataPath(data_path_in_backup, table_name);
}
}
void RestorerFromBackup::collectDatabaseInfo(const String & database_name_in_backup, const std::set<DatabaseAndTableName> & except_table_names, bool throw_if_no_database_metadata_in_backup)
void RestorerFromBackup::findDatabaseInBackup(const String & database_name_in_backup, const std::set<DatabaseAndTableName> & except_table_names)
{
std::optional<fs::path> metadata_path;
std::unordered_set<String> 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<String> & except_database_names, const std::set<DatabaseAndTableName> & except_table_names)
void RestorerFromBackup::findEverythingInBackup(const std::set<String> & except_database_names, const std::set<DatabaseAndTableName> & except_table_names)
{
std::unordered_set<String> database_names_in_backup;
std::unordered_set<String> 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<String> & 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<ASTCreateQuery &>().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<ASTCreateQuery &>().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<std::chrono::milliseconds>(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::TableKey> RestorerFromBackup::findTablesWithoutDependencies() const
std::vector<QualifiedTableName> RestorerFromBackup::findTablesWithoutDependencies() const
{
std::vector<TableKey> tables_without_dependencies;
std::vector<QualifiedTableName> tables_without_dependencies;
bool all_tables_created = true;
for (const auto & [key, table_info] : table_infos)
@ -732,7 +718,7 @@ std::vector<RestorerFromBackup::TableKey> 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::TableKey> RestorerFromBackup::findTablesWithoutD
return {};
/// Cyclic dependency? We'll try to create those tables anyway but probably it's going to fail.
std::vector<TableKey> tables_with_cyclic_dependencies;
std::vector<QualifiedTableName> tables_with_cyclic_dependencies;
for (const auto & [key, table_info] : table_infos)
{
if (!table_info.created)
@ -764,7 +750,7 @@ std::vector<RestorerFromBackup::TableKey> 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::TableKey> 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<std::vector<StoragePtr>>();
auto table_locks = std::make_shared<std::vector<TableLockHolder>>();
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<std::pair<UUID, AccessEntityPtr>> 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)

View File

@ -15,7 +15,9 @@ class IBackup;
using BackupPtr = std::shared_ptr<const IBackup>;
class IRestoreCoordination;
struct StorageID;
class AccessRestoreTask;
class AccessRestorerFromBackup;
struct IAccessEntity;
using AccessEntityPtr = std::shared_ptr<const IAccessEntity>;
/// 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<IRestoreCoordination> 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<void()>;
using DataRestoreTasks = std::vector<DataRestoreTask>;
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<IRestoreCoordination> 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<std::pair<UUID, AccessEntityPtr>> 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<IRestoreCoordination> 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<std::filesystem::path> root_paths_in_backup;
Strings all_hosts;
DDLRenamingMap renaming_map;
std::vector<std::filesystem::path> 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<ASTs> & partitions);
void collectDatabaseInfo(const String & database_name_in_backup, const std::set<DatabaseAndTableName> & except_table_names, bool throw_if_no_database_metadata_in_backup);
void collectAllDatabasesInfo(const std::set<String> & except_database_names, const std::set<DatabaseAndTableName> & except_table_names);
void checkAccessForCollectedInfos() const;
void findDatabasesAndTablesInBackup();
void findTableInBackup(const QualifiedTableName & table_name_in_backup, const std::optional<ASTs> & partitions);
void findDatabaseInBackup(const String & database_name_in_backup, const std::set<DatabaseAndTableName> & except_table_names);
void findEverythingInBackup(const std::set<String> & except_database_names, const std::set<DatabaseAndTableName> & 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<ASTs> partitions;
std::filesystem::path data_path_in_backup;
bool is_predefined_table = false;
std::unordered_set<QualifiedTableName> dependencies;
bool has_data = false;
std::filesystem::path data_path_in_backup;
std::optional<ASTs> 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<TableKey> findTablesWithoutDependencies() const;
std::vector<QualifiedTableName> findTablesWithoutDependencies() const;
String current_status;
std::unordered_map<String, DatabaseInfo> database_infos;
std::map<TableKey, TableInfo> table_infos;
std::map<QualifiedTableName, TableInfo> table_infos;
std::vector<DataRestoreTask> data_restore_tasks;
std::shared_ptr<AccessRestoreTask> access_restore_task;
std::unique_ptr<AccessRestorerFromBackup> access_restorer;
bool access_restored = false;
};
}

View File

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

View File

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

View File

@ -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<QualifiedTableName, QualifiedTableName> old_to_new_table_names;
std::unordered_map<String, String> old_to_new_database_names;
std::unordered_map<String, String> 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<DDLRenamingVisitor, false>;

View File

@ -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<std::pair<ASTPtr, StoragePtr>> 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<std::pair<ASTPtr, StoragePtr>> 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<const ASTCreateQuery &>();
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;
}
}

View File

@ -50,6 +50,8 @@ public:
void alterTable(ContextPtr local_context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) override;
std::vector<std::pair<ASTPtr, StoragePtr>> getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const override;
private:
const String data_path;
using NameToASTCreate = std::unordered_map<String, ASTPtr>;

View File

@ -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<std::pair<ASTPtr, StoragePtr>>
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<std::pair<ASTPtr, StoragePtr>> 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<ASTCreateQuery &>();
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<IRestoreCoordination> 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<const ASTCreateQuery &>().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));
}
}

View File

@ -72,7 +72,8 @@ public:
void shutdown() override;
void createTableRestoredFromBackup(const ASTPtr & create_table_query, const RestorerFromBackup & restorer) override;
std::vector<std::pair<ASTPtr, StoragePtr>> getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const override;
void createTableRestoredFromBackup(const ASTPtr & create_table_query, ContextMutablePtr local_context, std::shared_ptr<IRestoreCoordination> restore_coordination, UInt64 timeout_ms) override;
friend struct DatabaseReplicatedTask;
friend class DatabaseReplicatedDDLWorker;

View File

@ -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<std::pair<ASTPtr, StoragePtr>> 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<std::pair<ASTPtr, StoragePtr>> 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<const ASTCreateQuery &>();
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<IRestoreCoordination>, 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();
}
}

View File

@ -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<std::pair<ASTPtr, StoragePtr>> getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const override;
void createTableRestoredFromBackup(const ASTPtr & create_table_query, ContextMutablePtr local_context, std::shared_ptr<IRestoreCoordination> restore_coordination, UInt64 timeout_ms) override;
void shutdown() override;

View File

@ -1,6 +1,5 @@
#include <Databases/IDatabase.h>
#include <Storages/IStorage.h>
#include <Backups/BackupEntriesCollector.h>
#include <Parsers/ASTCreateQuery.h>
#include <Common/quoteString.h>
@ -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<std::pair<ASTPtr, StoragePtr>> 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<ASTCreateQuery &>();
create.uuid = UUIDHelpers::Nil;
return query;
}
DatabaseTablesIteratorPtr IDatabase::getTablesIteratorForBackup(const BackupEntriesCollector &) const
{
/// IDatabase doesn't own any tables.
return std::make_unique<DatabaseTablesSnapshotIterator>(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<const ASTCreateQuery &>().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<IRestoreCoordination>, UInt64)
{
/// Cannot restore any table because IDatabase doesn't own any tables.
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE,

View File

@ -30,8 +30,7 @@ class SettingsChanges;
using DictionariesWithID = std::vector<std::pair<String, UUID>>;
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<std::pair<ASTPtr, StoragePtr>> 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<IRestoreCoordination> restore_coordination, UInt64 timeout_ms);
virtual ~IDatabase() = default;

View File

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

View File

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

View File

@ -14,7 +14,8 @@
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Storages/AlterCommands.h>
#include <Backups/BackupEntriesCollector.h>
#include <Backups/RestorerFromBackup.h>
#include <Backups/IBackup.h>
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<ASTCreateQuery &>();
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<ASTs> &)
{
}
void IStorage::restoreDataFromBackup(RestorerFromBackup &, const String &, const std::optional<ASTs> &)
void IStorage::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> &)
{
/// 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

View File

@ -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<ASTs> & 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<ASTs> & partitions);
/// Returns true if the storage supports backup/restore for specific partitions.
virtual bool supportsBackupPartition() const { return false; }
private:
StorageID storage_id;

View File

@ -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<TemporaryFileOnDisk>(disk, "tmp/backup_")).first;
temp_dir_it = temp_dirs.emplace(disk, std::make_shared<TemporaryFileOnDisk>(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<BackupEntryFromImmutableFile>(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<BackupEntryFromSmallFile>(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<BackupEntryFromSmallFile>(disk, filepath_on_disk));
}
}

View File

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

View File

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

View File

@ -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<DiskPtr, std::shared_ptr<TemporaryFileOnDisk>> 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<RestoredPartsHolder> r
auto disk = reservation->getDisk();
String part_name = part_info.getPartName();
auto temp_part_dir_owner = std::make_shared<TemporaryFileOnDisk>(disk, relative_data_path + "restoring_" + part_name + "_");
String temp_part_dir = temp_part_dir_owner->getPath();
auto temp_part_dir_owner = std::make_shared<TemporaryFileOnDisk>(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<String> 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_<part_name>_<random_chars>", 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<SingleDiskVolume>(disk->getName(), disk, 0);
auto data_part_storage = std::make_shared<DataPartStorageOnDisk>(single_disk_volume, relative_data_path, relative_temp_part_dir);
auto data_part_storage = std::make_shared<DataPartStorageOnDisk>(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);

View File

@ -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<ASTs> & 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);

View File

@ -2,6 +2,7 @@
#include <Storages/MergeTree/MergeTreeData.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ExpressionListParsers.h>
#include <IO/Operators.h>
@ -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;
}
}

View File

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

View File

@ -0,0 +1,60 @@
#include <Storages/MergeTree/extractZkPathFromCreateQuery.h>
#include <Common/Macros.h>
#include <Databases/DatabaseReplicatedHelpers.h>
#include <Databases/IDatabase.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Interpreters/Context.h>
namespace DB
{
std::optional<String> tryExtractZkPathFromCreateQuery(const IAST & create_query, const ContextPtr & global_context)
{
const auto * create = create_query.as<const ASTCreateQuery>();
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<ASTExpressionList *>(ast_engine.arguments.get());
if (!ast_arguments || ast_arguments->children.empty())
return {};
auto * ast_zk_path = typeid_cast<ASTLiteral *>(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<String>();
/// 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;
}
}

View File

@ -0,0 +1,19 @@
#pragma once
#include <base/types.h>
#include <memory>
#include <optional>
namespace DB
{
class IAST;
class Context;
using ContextPtr = std::shared_ptr<const Context>;
/// 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<String> tryExtractZkPathFromCreateQuery(const IAST & create_query, const ContextPtr & global_context);
}

View File

@ -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<UInt64> StorageLog::totalBytes(const Settings &) const
return total_bytes;
}
void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* 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<ASTs> & partitions)
void StorageLog::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* 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);

View File

@ -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<UInt64> StorageMaterializedView::totalRows(const Settings & settings) const
{
if (hasInnerTable())

View File

@ -97,6 +97,7 @@ public:
void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & partitions) override;
void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions) override;
bool supportsBackupPartition() const override;
std::optional<UInt64> totalRows(const Settings & settings) const override;
std::optional<UInt64> totalBytes(const Settings & settings) const override;

View File

@ -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<ASTs> & partitions)
void StorageMemory::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* 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<MemoryBackupEntriesBatch>(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<ASTs> & partitions)
void StorageMemory::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* 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<ReadBuffer> in = backup_entry->getReadBuffer();
std::optional<Poco::TemporaryFile> temp_data_copy;

View File

@ -37,6 +37,7 @@
#include <Storages/MergeTree/MergeTreeReaderCompact.h>
#include <Storages/MergeTree/LeaderElection.h>
#include <Storages/MergeTree/ZeroCopyLock.h>
#include <Storages/MergeTree/extractZkPathFromCreateQuery.h>
#include <Storages/Freeze.h>
#include <Databases/DatabaseOnDisk.h>
@ -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<String> 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<ASTCreateQuery &>();
if (create.storage && create.storage->engine && (create.uuid != UUIDHelpers::Nil))
{
auto & engine = *(create.storage->engine);
if (auto * engine_args_ast = typeid_cast<ASTExpressionList *>(engine.arguments.get()))
{
auto & engine_args = engine_args_ast->children;
if (engine_args.size() >= 2)
{
auto * zookeeper_path_ast = typeid_cast<ASTLiteral *>(engine_args[0].get());
auto * replica_name_ast = typeid_cast<ASTLiteral *>(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>();
String & replica_name_arg = replica_name_ast->value.get<String>();
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<ASTLiteral>()))
{
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<String, SipHash> 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<fs::path> 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<std::string_view> 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<ASTs> & partitions)

View File

@ -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<ASTs> & 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<String> 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.

View File

@ -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<UInt64> StorageStripeLog::totalBytes(const Settings &) const
}
void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* 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<BackupEntryFromMemory>(toString(num_rows)));
}
void StorageStripeLog::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
void StorageStripeLog::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* 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};

View File

@ -9,6 +9,16 @@
#include <Interpreters/UserDefinedSQLFunctionFactory.h>
#include <Interpreters/UserDefinedExecutableFunctionFactory.h>
#include <Storages/System/StorageSystemFunctions.h>
#include <Common/escapeForFileName.h>
#include <Backups/BackupEntriesCollector.h>
#include <Backups/BackupEntryFromMemory.h>
#include <Backups/RestorerFromBackup.h>
#include <Backups/IBackup.h>
#include <Parsers/ParserCreateFunctionQuery.h>
#include <Parsers/parseQuery.h>
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 <typename Factory>
@ -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<ASTs> & /* 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<BackupEntryFromMemory>(queryToString(ast)));
}
}
void StorageSystemFunctions::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* 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);
}
}
}

View File

@ -19,6 +19,9 @@ public:
static NamesAndTypesList getNamesAndTypes();
void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & partitions) override;
void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions) override;
protected:
using IStorageSystemOneBlock::IStorageSystemOneBlock;

View File

@ -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<ASTs> & partitions)
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* 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<ASTs> & partitions)
RestorerFromBackup & restorer, const String & /* data_path_in_backup */, const std::optional<ASTs> & /* 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);
}
}

View File

@ -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<ASTs> & partitions)
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* 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<ASTs> & partitions)
RestorerFromBackup & restorer, const String & /* data_path_in_backup */, const std::optional<ASTs> & /* 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);
}
}

View File

@ -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<ASTs> & partitions)
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* 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<ASTs> & partitions)
RestorerFromBackup & restorer, const String & /* data_path_in_backup */, const std::optional<ASTs> & /* 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);
}
}

View File

@ -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<ASTs> & partitions)
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* 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<ASTs> & partitions)
RestorerFromBackup & restorer, const String & /* data_path_in_backup */, const std::optional<ASTs> & /* 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);
}
}

View File

@ -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<ASTs> & partitions)
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* 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<ASTs> & partitions)
RestorerFromBackup & restorer, const String & /* data_path_in_backup */, const std::optional<ASTs> & /* 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);
}
}

View File

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

View File

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