mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Improve gathering metadata for storing ACL in backups.
This commit is contained in:
parent
031ca28fdc
commit
5456bde4a2
@ -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>
|
||||
@ -226,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)
|
||||
@ -295,74 +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, const QualifiedTableName & table_name_for_logs)
|
||||
void AccessRestorerFromBackup::addDataPath(const String & data_path, const QualifiedTableName & table_name_for_logs)
|
||||
{
|
||||
if (!data_paths.emplace(data_path).second)
|
||||
return;
|
||||
|
||||
if (!backup->hasFiles(data_path))
|
||||
fs::path data_path_in_backup_fs = data_path;
|
||||
Strings filenames = backup->listFiles(data_path);
|
||||
if (filenames.empty())
|
||||
return;
|
||||
|
||||
String file_path = fs::path{data_path} / "access.txt";
|
||||
if (!backup->fileExists(file_path))
|
||||
for (const String & filename : filenames)
|
||||
{
|
||||
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required",
|
||||
table_name_for_logs.getFullName(), file_path);
|
||||
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});
|
||||
}
|
||||
|
||||
auto backup_entry = backup->readFile(file_path);
|
||||
auto ab = AccessEntitiesInBackup::fromBackupEntry(*backup_entry, file_path);
|
||||
::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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Backups/RestoreSettings.h>
|
||||
#include <Core/UUID.h>
|
||||
#include <unordered_map>
|
||||
#include <unordered_set>
|
||||
|
||||
@ -9,47 +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, const QualifiedTableName & table_name_for_logs);
|
||||
bool hasDataPath(const String & data_path) const;
|
||||
|
||||
/// 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;
|
||||
};
|
||||
|
@ -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.checkPathInBackupIsRegisteredToRestoreAccess(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();
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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,29 +523,34 @@ 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());
|
||||
}
|
||||
|
||||
|
||||
|
||||
UUID IAccessStorage::generateRandomID()
|
||||
{
|
||||
static Poco::UUIDGenerator generator;
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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", "");
|
||||
@ -245,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();
|
||||
|
@ -29,6 +29,12 @@ public:
|
||||
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;
|
||||
|
||||
|
@ -56,6 +56,37 @@ Strings BackupCoordinationLocal::getReplicatedDataPaths(const String & table_sha
|
||||
}
|
||||
|
||||
|
||||
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};
|
||||
|
@ -30,6 +30,12 @@ public:
|
||||
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;
|
||||
|
||||
@ -48,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);
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Access/Common/AccessEntityType.h>
|
||||
#include <base/chrono_io.h>
|
||||
#include <base/insertAtEnd.h>
|
||||
#include <base/sleep.h>
|
||||
@ -706,6 +707,11 @@ void BackupEntriesCollector::addBackupEntry(const String & file_name, BackupEntr
|
||||
backup_entries.emplace_back(file_name, backup_entry);
|
||||
}
|
||||
|
||||
void BackupEntriesCollector::addBackupEntry(const std::pair<String, BackupEntryPtr> & backup_entry)
|
||||
{
|
||||
addBackupEntry(backup_entry.first, backup_entry.second);
|
||||
}
|
||||
|
||||
void BackupEntriesCollector::addBackupEntries(const BackupEntries & backup_entries_)
|
||||
{
|
||||
if (current_status == kWritingBackupStatus)
|
||||
@ -739,4 +745,10 @@ void BackupEntriesCollector::runPostTasks()
|
||||
}
|
||||
}
|
||||
|
||||
size_t BackupEntriesCollector::getAccessCounter(AccessEntityType type)
|
||||
{
|
||||
access_counters.resize(static_cast<size_t>(AccessEntityType::MAX));
|
||||
return access_counters[static_cast<size_t>(type)]++;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
@ -42,6 +43,7 @@ public:
|
||||
/// 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_);
|
||||
|
||||
@ -50,6 +52,9 @@ public:
|
||||
/// 1) we need to join (in a backup) the data of replicated tables gathered on different hosts.
|
||||
void addPostTask(std::function<void()> task);
|
||||
|
||||
/// Returns an incremental counter used to backup access control.
|
||||
size_t getAccessCounter(AccessEntityType type);
|
||||
|
||||
private:
|
||||
void calculateRootPathInBackup();
|
||||
|
||||
@ -130,6 +135,7 @@ private:
|
||||
|
||||
BackupEntries backup_entries;
|
||||
std::queue<std::function<void()>> post_tasks;
|
||||
std::vector<size_t> access_counters;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -44,6 +44,14 @@ public:
|
||||
/// Returns all the data paths in backup added for a replicated table (see also addReplicatedDataPath()).
|
||||
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
|
||||
{
|
||||
String file_name;
|
||||
|
@ -383,9 +383,9 @@ void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name
|
||||
|
||||
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, table_name);
|
||||
if (!access_restorer)
|
||||
access_restorer = std::make_unique<AccessRestorerFromBackup>(backup, restore_settings);
|
||||
access_restorer->addDataPath(data_path_in_backup, table_name);
|
||||
}
|
||||
}
|
||||
|
||||
@ -555,8 +555,8 @@ void RestorerFromBackup::checkAccessForObjectsFoundInBackup() const
|
||||
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).
|
||||
@ -770,15 +770,9 @@ void RestorerFromBackup::addDataRestoreTasks(DataRestoreTasks && new_tasks)
|
||||
insertAtEnd(data_restore_tasks, std::move(new_tasks));
|
||||
}
|
||||
|
||||
void RestorerFromBackup::checkPathInBackupIsRegisteredToRestoreAccess(const String & path)
|
||||
{
|
||||
if (!access_restore_task || !access_restore_task->hasDataPath(path))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Path to restore access was not added");
|
||||
}
|
||||
|
||||
RestorerFromBackup::DataRestoreTasks RestorerFromBackup::getDataRestoreTasks()
|
||||
{
|
||||
if (data_restore_tasks.empty() && !access_restore_task)
|
||||
if (data_restore_tasks.empty())
|
||||
return {};
|
||||
|
||||
LOG_TRACE(log, "Will insert data to tables");
|
||||
@ -798,12 +792,20 @@ RestorerFromBackup::DataRestoreTasks RestorerFromBackup::getDataRestoreTasks()
|
||||
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;
|
||||
}
|
||||
|
||||
std::vector<std::pair<UUID, AccessEntityPtr>> RestorerFromBackup::getAccessEntitiesToRestore()
|
||||
{
|
||||
if (!access_restorer || access_restored)
|
||||
return {};
|
||||
|
||||
/// 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)
|
||||
{
|
||||
throw Exception(
|
||||
|
@ -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
|
||||
@ -56,8 +58,8 @@ public:
|
||||
void addDataRestoreTask(DataRestoreTask && new_task);
|
||||
void addDataRestoreTasks(DataRestoreTasks && new_tasks);
|
||||
|
||||
/// Checks that a specified path is already registered to be used for restoring access control.
|
||||
void checkPathInBackupIsRegisteredToRestoreAccess(const String & path);
|
||||
/// 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);
|
||||
@ -116,7 +118,8 @@ private:
|
||||
std::unordered_map<String, DatabaseInfo> database_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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -125,14 +125,14 @@ void StorageSystemQuotas::backupData(
|
||||
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
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 */)
|
||||
{
|
||||
auto & access_control = restorer.getContext()->getAccessControl();
|
||||
access_control.restore(restorer, data_path_in_backup);
|
||||
access_control.restoreFromBackup(restorer);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -63,14 +63,14 @@ void StorageSystemRoles::backupData(
|
||||
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
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 */)
|
||||
{
|
||||
auto & access_control = restorer.getContext()->getAccessControl();
|
||||
access_control.restore(restorer, data_path_in_backup);
|
||||
access_control.restoreFromBackup(restorer);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -143,14 +143,14 @@ void StorageSystemRowPolicies::backupData(
|
||||
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
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 */)
|
||||
{
|
||||
auto & access_control = restorer.getContext()->getAccessControl();
|
||||
access_control.restore(restorer, data_path_in_backup);
|
||||
access_control.restoreFromBackup(restorer);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -90,14 +90,14 @@ void StorageSystemSettingsProfiles::backupData(
|
||||
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
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 */)
|
||||
{
|
||||
auto & access_control = restorer.getContext()->getAccessControl();
|
||||
access_control.restore(restorer, data_path_in_backup);
|
||||
access_control.restoreFromBackup(restorer);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -218,14 +218,14 @@ void StorageSystemUsers::backupData(
|
||||
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
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 */)
|
||||
{
|
||||
auto & access_control = restorer.getContext()->getAccessControl();
|
||||
access_control.restore(restorer, data_path_in_backup);
|
||||
access_control.restoreFromBackup(restorer);
|
||||
}
|
||||
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user