mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Fix restoring access entities dependant on existing ones.
This commit is contained in:
parent
386d54cedf
commit
1ccd461c97
@ -29,6 +29,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_RESTORE_TABLE;
|
||||
extern const int ACCESS_ENTITY_ALREADY_EXISTS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
@ -175,9 +176,46 @@ namespace
|
||||
return res;
|
||||
}
|
||||
|
||||
std::unordered_map<UUID, UUID> resolveDependencies(const std::unordered_map<UUID, std::pair<String, AccessEntityType>> & dependencies, const AccessControl & access_control, bool allow_unresolved_dependencies)
|
||||
/// Checks if new entities (which we're going to restore) already exist,
|
||||
/// and either skips them or throws an exception depending on the restore settings.
|
||||
void checkExistingEntities(std::vector<std::pair<UUID, AccessEntityPtr>> & entities,
|
||||
std::unordered_map<UUID, UUID> & old_to_new_id,
|
||||
const AccessControl & access_control,
|
||||
RestoreAccessCreationMode creation_mode)
|
||||
{
|
||||
if (creation_mode == RestoreAccessCreationMode::kReplace)
|
||||
return;
|
||||
|
||||
auto should_skip = [&](const std::pair<UUID, AccessEntityPtr> & id_and_entity)
|
||||
{
|
||||
const auto & id = id_and_entity.first;
|
||||
const auto & entity = *id_and_entity.second;
|
||||
auto existing_id = access_control.find(entity.getType(), entity.getName());
|
||||
if (!existing_id)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
else if (creation_mode == RestoreAccessCreationMode::kCreateIfNotExists)
|
||||
{
|
||||
old_to_new_id[id] = *existing_id;
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Cannot restore {} because it already exists", entity.formatTypeWithName());
|
||||
}
|
||||
};
|
||||
|
||||
std::erase_if(entities, should_skip);
|
||||
}
|
||||
|
||||
/// If new entities (which we're going to restore) depend on other entities which are not going to be restored or not present in the backup
|
||||
/// then we should try to replace those dependencies with already existing entities.
|
||||
void resolveDependencies(const std::unordered_map<UUID, std::pair<String, AccessEntityType>> & dependencies,
|
||||
std::unordered_map<UUID, UUID> & old_to_new_ids,
|
||||
const AccessControl & access_control,
|
||||
bool allow_unresolved_dependencies)
|
||||
{
|
||||
std::unordered_map<UUID, UUID> old_to_new_ids;
|
||||
for (const auto & [id, name_and_type] : dependencies)
|
||||
{
|
||||
std::optional<UUID> new_id;
|
||||
@ -188,9 +226,9 @@ namespace
|
||||
if (new_id)
|
||||
old_to_new_ids.emplace(id, *new_id);
|
||||
}
|
||||
return old_to_new_ids;
|
||||
}
|
||||
|
||||
/// Generates random IDs for the new entities.
|
||||
void generateRandomIDs(std::vector<std::pair<UUID, AccessEntityPtr>> & entities, std::unordered_map<UUID, UUID> & old_to_new_ids)
|
||||
{
|
||||
Poco::UUIDGenerator generator;
|
||||
@ -203,27 +241,12 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
void replaceDependencies(std::vector<std::pair<UUID, AccessEntityPtr>> & entities, const std::unordered_map<UUID, UUID> & old_to_new_ids)
|
||||
/// Updates dependencies of the new entities using a specified map.
|
||||
void replaceDependencies(std::vector<std::pair<UUID, AccessEntityPtr>> & entities,
|
||||
const std::unordered_map<UUID, UUID> & old_to_new_ids)
|
||||
{
|
||||
for (auto & entity : entities | boost::adaptors::map_values)
|
||||
{
|
||||
bool need_replace = false;
|
||||
for (const auto & dependency : entity->findDependencies())
|
||||
{
|
||||
if (old_to_new_ids.contains(dependency))
|
||||
{
|
||||
need_replace = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (!need_replace)
|
||||
continue;
|
||||
|
||||
auto new_entity = entity->clone();
|
||||
new_entity->replaceDependencies(old_to_new_ids);
|
||||
entity = new_entity;
|
||||
}
|
||||
IAccessEntity::replaceDependencies(entity, old_to_new_ids);
|
||||
}
|
||||
|
||||
AccessRightsElements getRequiredAccessToRestore(const std::vector<std::pair<UUID, AccessEntityPtr>> & entities)
|
||||
@ -314,7 +337,9 @@ std::pair<String, BackupEntryPtr> makeBackupEntryForAccess(
|
||||
|
||||
AccessRestorerFromBackup::AccessRestorerFromBackup(
|
||||
const BackupPtr & backup_, const RestoreSettings & restore_settings_)
|
||||
: backup(backup_), allow_unresolved_access_dependencies(restore_settings_.allow_unresolved_access_dependencies)
|
||||
: backup(backup_)
|
||||
, creation_mode(restore_settings_.create_access)
|
||||
, allow_unresolved_dependencies(restore_settings_.allow_unresolved_access_dependencies)
|
||||
{
|
||||
}
|
||||
|
||||
@ -362,7 +387,9 @@ std::vector<std::pair<UUID, AccessEntityPtr>> AccessRestorerFromBackup::getAcces
|
||||
{
|
||||
auto new_entities = entities;
|
||||
|
||||
auto old_to_new_ids = resolveDependencies(dependencies, access_control, allow_unresolved_access_dependencies);
|
||||
std::unordered_map<UUID, UUID> old_to_new_ids;
|
||||
checkExistingEntities(new_entities, old_to_new_ids, access_control, creation_mode);
|
||||
resolveDependencies(dependencies, old_to_new_ids, access_control, allow_unresolved_dependencies);
|
||||
generateRandomIDs(new_entities, old_to_new_ids);
|
||||
replaceDependencies(new_entities, old_to_new_ids);
|
||||
|
||||
|
@ -17,6 +17,7 @@ using BackupPtr = std::shared_ptr<const IBackup>;
|
||||
class IBackupEntry;
|
||||
using BackupEntryPtr = std::shared_ptr<const IBackupEntry>;
|
||||
struct RestoreSettings;
|
||||
enum class RestoreAccessCreationMode : uint8_t;
|
||||
|
||||
|
||||
/// Makes a backup of access entities of a specified type.
|
||||
@ -45,7 +46,8 @@ public:
|
||||
|
||||
private:
|
||||
BackupPtr backup;
|
||||
bool allow_unresolved_access_dependencies = false;
|
||||
RestoreAccessCreationMode creation_mode;
|
||||
bool allow_unresolved_dependencies = false;
|
||||
std::vector<std::pair<UUID, AccessEntityPtr>> entities;
|
||||
std::unordered_map<UUID, std::pair<String, AccessEntityType>> dependencies;
|
||||
std::unordered_set<String> data_paths;
|
||||
|
@ -544,9 +544,9 @@ scope_guard AccessControl::subscribeForChanges(const std::vector<UUID> & ids, co
|
||||
return changes_notifier->subscribeForChanges(ids, handler);
|
||||
}
|
||||
|
||||
bool AccessControl::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists)
|
||||
bool AccessControl::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id)
|
||||
{
|
||||
if (MultipleAccessStorage::insertImpl(id, entity, replace_if_exists, throw_if_exists))
|
||||
if (MultipleAccessStorage::insertImpl(id, entity, replace_if_exists, throw_if_exists, conflicting_id))
|
||||
{
|
||||
changes_notifier->sendNotifications();
|
||||
return true;
|
||||
|
@ -243,7 +243,7 @@ private:
|
||||
class CustomSettingsPrefixes;
|
||||
class PasswordComplexityRules;
|
||||
|
||||
bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override;
|
||||
bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override;
|
||||
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
|
||||
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;
|
||||
|
||||
|
@ -1,8 +1,6 @@
|
||||
#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>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
@ -418,7 +416,7 @@ void DiskAccessStorage::setAllInMemory(const std::vector<std::pair<UUID, AccessE
|
||||
|
||||
/// Insert or update entities.
|
||||
for (const auto & [id, entity] : entities_without_conflicts)
|
||||
insertNoLock(id, entity, /* replace_if_exists = */ true, /* throw_if_exists = */ false, /* write_on_disk= */ false);
|
||||
insertNoLock(id, entity, /* replace_if_exists = */ true, /* throw_if_exists = */ false, /* conflicting_id = */ nullptr, /* write_on_disk= */ false);
|
||||
}
|
||||
|
||||
void DiskAccessStorage::removeAllExceptInMemory(const boost::container::flat_set<UUID> & ids_to_keep)
|
||||
@ -507,14 +505,14 @@ std::optional<std::pair<String, AccessEntityType>> DiskAccessStorage::readNameWi
|
||||
}
|
||||
|
||||
|
||||
bool DiskAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists)
|
||||
bool DiskAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, /* write_on_disk = */ true);
|
||||
return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, conflicting_id, /* write_on_disk = */ true);
|
||||
}
|
||||
|
||||
|
||||
bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk)
|
||||
bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id, bool write_on_disk)
|
||||
{
|
||||
const String & name = new_entity->getName();
|
||||
AccessEntityType type = new_entity->getType();
|
||||
@ -533,9 +531,15 @@ bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & ne
|
||||
if (name_collision && !replace_if_exists)
|
||||
{
|
||||
if (throw_if_exists)
|
||||
{
|
||||
throwNameCollisionCannotInsert(type, name);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (conflicting_id)
|
||||
*conflicting_id = id_by_name;
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
auto it_by_id = entries_by_id.find(id);
|
||||
@ -548,7 +552,11 @@ bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & ne
|
||||
throwIDCollisionCannotInsert(id, type, name, existing_entry.type, existing_entry.name);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (conflicting_id)
|
||||
*conflicting_id = id;
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
if (write_on_disk)
|
||||
@ -727,25 +735,4 @@ void DiskAccessStorage::deleteAccessEntityOnDisk(const UUID & id) const
|
||||
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Couldn't delete {}", file_path);
|
||||
}
|
||||
|
||||
|
||||
void DiskAccessStorage::restoreFromBackup(RestorerFromBackup & restorer)
|
||||
{
|
||||
if (!isRestoreAllowed())
|
||||
throwRestoreNotAllowed();
|
||||
|
||||
auto entities = restorer.getAccessEntitiesToRestore();
|
||||
if (entities.empty())
|
||||
return;
|
||||
|
||||
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, my_entities = std::move(entities), replace_if_exists, throw_if_exists]
|
||||
{
|
||||
for (const auto & [id, entity] : my_entities)
|
||||
insert(id, entity, replace_if_exists, throw_if_exists);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -34,14 +34,13 @@ public:
|
||||
bool exists(const UUID & id) const override;
|
||||
|
||||
bool isBackupAllowed() const override { return backup_allowed; }
|
||||
void restoreFromBackup(RestorerFromBackup & restorer) override;
|
||||
|
||||
private:
|
||||
std::optional<UUID> findImpl(AccessEntityType type, const String & name) const override;
|
||||
std::vector<UUID> findAllImpl(AccessEntityType type) const override;
|
||||
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override;
|
||||
std::optional<std::pair<String, AccessEntityType>> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override;
|
||||
bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override;
|
||||
bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override;
|
||||
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
|
||||
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;
|
||||
|
||||
@ -55,7 +54,7 @@ private:
|
||||
void listsWritingThreadFunc() TSA_NO_THREAD_SAFETY_ANALYSIS;
|
||||
void stopListsWritingThread();
|
||||
|
||||
bool insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk) TSA_REQUIRES(mutex);
|
||||
bool insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id, bool write_on_disk) TSA_REQUIRES(mutex);
|
||||
bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex);
|
||||
bool removeNoLock(const UUID & id, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex);
|
||||
|
||||
|
@ -9,4 +9,28 @@ bool IAccessEntity::equal(const IAccessEntity & other) const
|
||||
return (name == other.name) && (getType() == other.getType());
|
||||
}
|
||||
|
||||
void IAccessEntity::replaceDependencies(std::shared_ptr<const IAccessEntity> & entity, const std::unordered_map<UUID, UUID> & old_to_new_ids)
|
||||
{
|
||||
if (old_to_new_ids.empty())
|
||||
return;
|
||||
|
||||
bool need_replace_dependencies = false;
|
||||
auto dependencies = entity->findDependencies();
|
||||
for (const auto & dependency : dependencies)
|
||||
{
|
||||
if (old_to_new_ids.contains(dependency))
|
||||
{
|
||||
need_replace_dependencies = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (!need_replace_dependencies)
|
||||
return;
|
||||
|
||||
auto new_entity = entity->clone();
|
||||
new_entity->replaceDependencies(old_to_new_ids);
|
||||
entity = new_entity;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -50,7 +50,8 @@ struct IAccessEntity
|
||||
virtual std::vector<UUID> findDependencies() const { return {}; }
|
||||
|
||||
/// Replaces dependencies according to a specified map.
|
||||
virtual void replaceDependencies(const std::unordered_map<UUID, UUID> & /* old_to_new_ids */) {}
|
||||
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) { doReplaceDependencies(old_to_new_ids); }
|
||||
static void replaceDependencies(std::shared_ptr<const IAccessEntity> & entity, const std::unordered_map<UUID, UUID> & old_to_new_ids);
|
||||
|
||||
/// Whether this access entity should be written to a backup.
|
||||
virtual bool isBackupAllowed() const { return false; }
|
||||
@ -66,6 +67,8 @@ protected:
|
||||
{
|
||||
return std::make_shared<EntityClassT>(typeid_cast<const EntityClassT &>(*this));
|
||||
}
|
||||
|
||||
virtual void doReplaceDependencies(const std::unordered_map<UUID, UUID> & /* old_to_new_ids */) {}
|
||||
};
|
||||
|
||||
using AccessEntityPtr = std::shared_ptr<const IAccessEntity>;
|
||||
|
@ -4,6 +4,8 @@
|
||||
#include <Access/User.h>
|
||||
#include <Access/AccessBackup.h>
|
||||
#include <Backups/BackupEntriesCollector.h>
|
||||
#include <Backups/RestorerFromBackup.h>
|
||||
#include <Backups/RestoreSettings.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/callOnce.h>
|
||||
@ -14,10 +16,11 @@
|
||||
#include <base/FnTraits.h>
|
||||
#include <boost/algorithm/string/join.hpp>
|
||||
#include <boost/algorithm/string/replace.hpp>
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
#include <boost/range/adaptor/reversed.hpp>
|
||||
#include <boost/range/algorithm/copy.hpp>
|
||||
#include <boost/range/algorithm_ext/erase.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
@ -178,20 +181,20 @@ UUID IAccessStorage::insert(const AccessEntityPtr & entity)
|
||||
return *insert(entity, /* replace_if_exists = */ false, /* throw_if_exists = */ true);
|
||||
}
|
||||
|
||||
std::optional<UUID> IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists)
|
||||
std::optional<UUID> IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id)
|
||||
{
|
||||
auto id = generateRandomID();
|
||||
|
||||
if (insert(id, entity, replace_if_exists, throw_if_exists))
|
||||
if (insert(id, entity, replace_if_exists, throw_if_exists, conflicting_id))
|
||||
return id;
|
||||
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
|
||||
bool IAccessStorage::insert(const DB::UUID & id, const DB::AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists)
|
||||
bool IAccessStorage::insert(const DB::UUID & id, const DB::AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id)
|
||||
{
|
||||
return insertImpl(id, entity, replace_if_exists, throw_if_exists);
|
||||
return insertImpl(id, entity, replace_if_exists, throw_if_exists, conflicting_id);
|
||||
}
|
||||
|
||||
|
||||
@ -285,7 +288,7 @@ std::vector<UUID> IAccessStorage::insertOrReplace(const std::vector<AccessEntity
|
||||
}
|
||||
|
||||
|
||||
bool IAccessStorage::insertImpl(const UUID &, const AccessEntityPtr & entity, bool, bool)
|
||||
bool IAccessStorage::insertImpl(const UUID &, const AccessEntityPtr & entity, bool, bool, UUID *)
|
||||
{
|
||||
if (isReadOnly())
|
||||
throwReadonlyCannotInsert(entity->getType(), entity->getName());
|
||||
@ -611,12 +614,50 @@ void IAccessStorage::backup(BackupEntriesCollector & backup_entries_collector, c
|
||||
}
|
||||
|
||||
|
||||
void IAccessStorage::restoreFromBackup(RestorerFromBackup &)
|
||||
void IAccessStorage::restoreFromBackup(RestorerFromBackup & restorer)
|
||||
{
|
||||
if (!isRestoreAllowed())
|
||||
throwRestoreNotAllowed();
|
||||
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "restoreFromBackup() is not implemented in {}", getStorageType());
|
||||
if (isReplicated() && !acquireReplicatedRestore(restorer))
|
||||
return;
|
||||
|
||||
auto entities = restorer.getAccessEntitiesToRestore();
|
||||
if (entities.empty())
|
||||
return;
|
||||
|
||||
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, my_entities = std::move(entities), replace_if_exists, throw_if_exists] mutable
|
||||
{
|
||||
std::unordered_map<UUID, UUID> new_to_existing_ids;
|
||||
for (auto & [id, entity] : my_entities)
|
||||
{
|
||||
UUID existing_entity_id;
|
||||
if (!insert(id, entity, replace_if_exists, throw_if_exists, &existing_entity_id))
|
||||
{
|
||||
/// Couldn't insert `entity` because there is an existing entity with the same name.
|
||||
new_to_existing_ids[id] = existing_entity_id;
|
||||
}
|
||||
}
|
||||
|
||||
if (!new_to_existing_ids.empty())
|
||||
{
|
||||
/// If new entities restored from backup have dependencies on other entities from backup which were not restored because they existed,
|
||||
/// then we should correct those dependencies.
|
||||
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
|
||||
{
|
||||
auto res = entity;
|
||||
IAccessEntity::replaceDependencies(res, new_to_existing_ids);
|
||||
return res;
|
||||
};
|
||||
std::vector<UUID> ids;
|
||||
boost::copy(my_entities | boost::adaptors::map_keys, std::back_inserter(ids));
|
||||
tryUpdate(ids, update_func);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
|
@ -64,6 +64,9 @@ public:
|
||||
/// Returns true if this entity is readonly.
|
||||
virtual bool isReadOnly(const UUID &) const { return isReadOnly(); }
|
||||
|
||||
/// Returns true if this storage is replicated.
|
||||
virtual bool isReplicated() const { return false; }
|
||||
|
||||
/// Starts periodic reloading and updating of entities in this storage.
|
||||
virtual void startPeriodicReloading() {}
|
||||
|
||||
@ -153,8 +156,8 @@ public:
|
||||
/// Inserts an entity to the storage. Returns ID of a new entry in the storage.
|
||||
/// Throws an exception if the specified name already exists.
|
||||
UUID insert(const AccessEntityPtr & entity);
|
||||
std::optional<UUID> insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists);
|
||||
bool insert(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists);
|
||||
std::optional<UUID> insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id = nullptr);
|
||||
bool insert(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id = nullptr);
|
||||
std::vector<UUID> insert(const std::vector<AccessEntityPtr> & multiple_entities, bool replace_if_exists = false, bool throw_if_exists = true);
|
||||
std::vector<UUID> insert(const std::vector<AccessEntityPtr> & multiple_entities, const std::vector<UUID> & ids, bool replace_if_exists = false, bool throw_if_exists = true);
|
||||
|
||||
@ -218,7 +221,7 @@ protected:
|
||||
virtual std::vector<UUID> findAllImpl(AccessEntityType type) const = 0;
|
||||
virtual AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const = 0;
|
||||
virtual std::optional<std::pair<String, AccessEntityType>> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const;
|
||||
virtual bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists);
|
||||
virtual bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id);
|
||||
virtual bool removeImpl(const UUID & id, bool throw_if_not_exists);
|
||||
virtual bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists);
|
||||
virtual std::optional<AuthResult> authenticateImpl(
|
||||
@ -240,6 +243,7 @@ protected:
|
||||
LoggerPtr getLogger() const;
|
||||
static String formatEntityTypeWithName(AccessEntityType type, const String & name) { return AccessEntityTypeInfo::get(type).formatEntityNameWithType(name); }
|
||||
static void clearConflictsInEntitiesList(std::vector<std::pair<UUID, AccessEntityPtr>> & entities, LoggerPtr log_);
|
||||
virtual bool acquireReplicatedRestore(RestorerFromBackup &) const { return false; }
|
||||
[[noreturn]] void throwNotFound(const UUID & id) const;
|
||||
[[noreturn]] void throwNotFound(AccessEntityType type, const String & name) const;
|
||||
[[noreturn]] static void throwBadCast(const UUID & id, AccessEntityType type, const String & name, AccessEntityType required_type);
|
||||
|
@ -1,7 +1,5 @@
|
||||
#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>
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
@ -63,14 +61,14 @@ AccessEntityPtr MemoryAccessStorage::readImpl(const UUID & id, bool throw_if_not
|
||||
}
|
||||
|
||||
|
||||
bool MemoryAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists)
|
||||
bool MemoryAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists);
|
||||
return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, conflicting_id);
|
||||
}
|
||||
|
||||
|
||||
bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists)
|
||||
bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id)
|
||||
{
|
||||
const String & name = new_entity->getName();
|
||||
AccessEntityType type = new_entity->getType();
|
||||
@ -86,9 +84,15 @@ bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr &
|
||||
if (name_collision && !replace_if_exists)
|
||||
{
|
||||
if (throw_if_exists)
|
||||
{
|
||||
throwNameCollisionCannotInsert(type, name);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (conflicting_id)
|
||||
*conflicting_id = id_by_name;
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
auto it_by_id = entries_by_id.find(id);
|
||||
@ -97,9 +101,15 @@ bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr &
|
||||
{
|
||||
const auto & existing_entry = it_by_id->second;
|
||||
if (throw_if_exists)
|
||||
{
|
||||
throwIDCollisionCannotInsert(id, type, name, existing_entry.entity->getType(), existing_entry.entity->getName());
|
||||
}
|
||||
else
|
||||
{
|
||||
if (conflicting_id)
|
||||
*conflicting_id = id;
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/// Remove collisions if necessary.
|
||||
@ -270,28 +280,7 @@ void MemoryAccessStorage::setAll(const std::vector<std::pair<UUID, AccessEntityP
|
||||
|
||||
/// Insert or update entities.
|
||||
for (const auto & [id, entity] : entities_without_conflicts)
|
||||
insertNoLock(id, entity, /* replace_if_exists = */ true, /* throw_if_exists = */ false);
|
||||
}
|
||||
|
||||
|
||||
void MemoryAccessStorage::restoreFromBackup(RestorerFromBackup & restorer)
|
||||
{
|
||||
if (!isRestoreAllowed())
|
||||
throwRestoreNotAllowed();
|
||||
|
||||
auto entities = restorer.getAccessEntitiesToRestore();
|
||||
if (entities.empty())
|
||||
return;
|
||||
|
||||
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, my_entities = std::move(entities), replace_if_exists, throw_if_exists]
|
||||
{
|
||||
for (const auto & [id, entity] : my_entities)
|
||||
insert(id, entity, replace_if_exists, throw_if_exists);
|
||||
});
|
||||
insertNoLock(id, entity, /* replace_if_exists = */ true, /* throw_if_exists = */ false, /* conflicting_id = */ nullptr);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -34,17 +34,16 @@ public:
|
||||
bool exists(const UUID & id) const override;
|
||||
|
||||
bool isBackupAllowed() const override { return backup_allowed; }
|
||||
void restoreFromBackup(RestorerFromBackup & restorer) override;
|
||||
|
||||
private:
|
||||
std::optional<UUID> findImpl(AccessEntityType type, const String & name) const override;
|
||||
std::vector<UUID> findAllImpl(AccessEntityType type) const override;
|
||||
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override;
|
||||
bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override;
|
||||
bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override;
|
||||
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
|
||||
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;
|
||||
|
||||
bool insertNoLock(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists);
|
||||
bool insertNoLock(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id);
|
||||
bool removeNoLock(const UUID & id, bool throw_if_not_exists);
|
||||
bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists);
|
||||
|
||||
|
@ -353,7 +353,7 @@ void MultipleAccessStorage::reload(ReloadMode reload_mode)
|
||||
}
|
||||
|
||||
|
||||
bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists)
|
||||
bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id)
|
||||
{
|
||||
std::shared_ptr<IAccessStorage> storage_for_insertion;
|
||||
|
||||
@ -376,7 +376,7 @@ bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr &
|
||||
getStorageName());
|
||||
}
|
||||
|
||||
if (storage_for_insertion->insert(id, entity, replace_if_exists, throw_if_exists))
|
||||
if (storage_for_insertion->insert(id, entity, replace_if_exists, throw_if_exists, conflicting_id))
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
ids_cache.set(id, storage_for_insertion);
|
||||
|
@ -67,7 +67,7 @@ protected:
|
||||
std::vector<UUID> findAllImpl(AccessEntityType type) const override;
|
||||
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override;
|
||||
std::optional<std::pair<String, AccessEntityType>> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override;
|
||||
bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override;
|
||||
bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override;
|
||||
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
|
||||
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;
|
||||
std::optional<AuthResult> authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override;
|
||||
|
@ -24,7 +24,7 @@ std::vector<UUID> Quota::findDependencies() const
|
||||
return to_roles.findDependencies();
|
||||
}
|
||||
|
||||
void Quota::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
|
||||
void Quota::doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
|
||||
{
|
||||
to_roles.replaceDependencies(old_to_new_ids);
|
||||
}
|
||||
|
@ -47,7 +47,7 @@ struct Quota : public IAccessEntity
|
||||
AccessEntityType getType() const override { return TYPE; }
|
||||
|
||||
std::vector<UUID> findDependencies() const override;
|
||||
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
|
||||
void doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
|
||||
bool isBackupAllowed() const override { return true; }
|
||||
};
|
||||
|
||||
|
@ -5,10 +5,9 @@
|
||||
#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 <Backups/RestorerFromBackup.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/ZooKeeper/KeeperException.h>
|
||||
@ -120,7 +119,7 @@ static void retryOnZooKeeperUserError(size_t attempts, Func && function)
|
||||
}
|
||||
}
|
||||
|
||||
bool ReplicatedAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists)
|
||||
bool ReplicatedAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id)
|
||||
{
|
||||
const AccessEntityTypeInfo type_info = AccessEntityTypeInfo::get(new_entity->getType());
|
||||
const String & name = new_entity->getName();
|
||||
@ -128,7 +127,7 @@ bool ReplicatedAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr
|
||||
|
||||
auto zookeeper = getZooKeeper();
|
||||
bool ok = false;
|
||||
retryOnZooKeeperUserError(10, [&]{ ok = insertZooKeeper(zookeeper, id, new_entity, replace_if_exists, throw_if_exists); });
|
||||
retryOnZooKeeperUserError(10, [&]{ ok = insertZooKeeper(zookeeper, id, new_entity, replace_if_exists, throw_if_exists, conflicting_id); });
|
||||
|
||||
if (!ok)
|
||||
return false;
|
||||
@ -143,7 +142,8 @@ bool ReplicatedAccessStorage::insertZooKeeper(
|
||||
const UUID & id,
|
||||
const AccessEntityPtr & new_entity,
|
||||
bool replace_if_exists,
|
||||
bool throw_if_exists)
|
||||
bool throw_if_exists,
|
||||
UUID * conflicting_id)
|
||||
{
|
||||
const String & name = new_entity->getName();
|
||||
const AccessEntityType type = new_entity->getType();
|
||||
@ -167,27 +167,52 @@ bool ReplicatedAccessStorage::insertZooKeeper(
|
||||
|
||||
if (res == Coordination::Error::ZNODEEXISTS)
|
||||
{
|
||||
if (!throw_if_exists && !replace_if_exists)
|
||||
return false; /// Couldn't insert a new entity.
|
||||
|
||||
if (throw_if_exists)
|
||||
if (!replace_if_exists)
|
||||
{
|
||||
if (responses[0]->error == Coordination::Error::ZNODEEXISTS)
|
||||
{
|
||||
/// To fail with a nice error message, we need info about what already exists.
|
||||
/// This itself could fail if the conflicting uuid disappears in the meantime.
|
||||
/// If that happens, then we'll just retry from the start.
|
||||
String existing_entity_definition = zookeeper->get(entity_path);
|
||||
/// Couldn't insert the new entity because there is an existing entity with such UUID.
|
||||
if (throw_if_exists)
|
||||
{
|
||||
/// To fail with a nice error message, we need info about what already exists.
|
||||
/// This itself can fail if the conflicting uuid disappears in the meantime.
|
||||
/// If that happens, then retryOnZooKeeperUserError() will just retry the operation from the start.
|
||||
String existing_entity_definition = zookeeper->get(entity_path);
|
||||
|
||||
AccessEntityPtr existing_entity = deserializeAccessEntity(existing_entity_definition, entity_path);
|
||||
AccessEntityType existing_type = existing_entity->getType();
|
||||
String existing_name = existing_entity->getName();
|
||||
throwIDCollisionCannotInsert(id, type, name, existing_type, existing_name);
|
||||
AccessEntityPtr existing_entity = deserializeAccessEntity(existing_entity_definition, entity_path);
|
||||
AccessEntityType existing_type = existing_entity->getType();
|
||||
String existing_name = existing_entity->getName();
|
||||
throwIDCollisionCannotInsert(id, type, name, existing_type, existing_name);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (conflicting_id)
|
||||
*conflicting_id = id;
|
||||
return false;
|
||||
}
|
||||
}
|
||||
else if (responses[1]->error == Coordination::Error::ZNODEEXISTS)
|
||||
{
|
||||
/// Couldn't insert the new entity because there is an existing entity with the same name.
|
||||
if (throw_if_exists)
|
||||
{
|
||||
throwNameCollisionCannotInsert(type, name);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (conflicting_id)
|
||||
{
|
||||
/// Get UUID of the existing entry with the same name.
|
||||
/// This itself can fail if the conflicting name disappears in the meantime.
|
||||
/// If that happens, then retryOnZooKeeperUserError() will just retry the operation from the start.
|
||||
*conflicting_id = parseUUID(zookeeper->get(name_path));
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Couldn't insert the new entity because there is an existing entity with such name.
|
||||
throwNameCollisionCannotInsert(type, name);
|
||||
zkutil::KeeperMultiException::check(res, ops, responses);
|
||||
}
|
||||
}
|
||||
|
||||
@ -693,28 +718,10 @@ void ReplicatedAccessStorage::backup(BackupEntriesCollector & backup_entries_col
|
||||
}
|
||||
|
||||
|
||||
void ReplicatedAccessStorage::restoreFromBackup(RestorerFromBackup & restorer)
|
||||
bool ReplicatedAccessStorage::acquireReplicatedRestore(RestorerFromBackup & restorer) const
|
||||
{
|
||||
if (!isRestoreAllowed())
|
||||
throwRestoreNotAllowed();
|
||||
|
||||
auto restore_coordination = restorer.getRestoreCoordination();
|
||||
if (!restore_coordination->acquireReplicatedAccessStorage(zookeeper_path))
|
||||
return;
|
||||
|
||||
auto entities = restorer.getAccessEntitiesToRestore();
|
||||
if (entities.empty())
|
||||
return;
|
||||
|
||||
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, my_entities = std::move(entities), replace_if_exists, throw_if_exists]
|
||||
{
|
||||
for (const auto & [id, entity] : my_entities)
|
||||
insert(id, entity, replace_if_exists, throw_if_exists);
|
||||
});
|
||||
return restore_coordination->acquireReplicatedAccessStorage(zookeeper_path);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -26,6 +26,7 @@ public:
|
||||
void shutdown() override;
|
||||
|
||||
const char * getStorageType() const override { return STORAGE_TYPE; }
|
||||
bool isReplicated() const override { return true; }
|
||||
|
||||
void startPeriodicReloading() override { startWatchingThread(); }
|
||||
void stopPeriodicReloading() override { stopWatchingThread(); }
|
||||
@ -35,7 +36,6 @@ public:
|
||||
|
||||
bool isBackupAllowed() const override { return backup_allowed; }
|
||||
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;
|
||||
@ -48,11 +48,11 @@ private:
|
||||
std::unique_ptr<ThreadFromGlobalPool> watching_thread;
|
||||
std::shared_ptr<ConcurrentBoundedQueue<UUID>> watched_queue;
|
||||
|
||||
bool insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) override;
|
||||
bool insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override;
|
||||
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
|
||||
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;
|
||||
|
||||
bool insertZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists);
|
||||
bool insertZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id);
|
||||
bool removeZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, bool throw_if_not_exists);
|
||||
bool updateZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists);
|
||||
|
||||
@ -80,6 +80,7 @@ private:
|
||||
std::optional<UUID> findImpl(AccessEntityType type, const String & name) const override;
|
||||
std::vector<UUID> findAllImpl(AccessEntityType type) const override;
|
||||
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override;
|
||||
bool acquireReplicatedRestore(RestorerFromBackup & restorer) const override;
|
||||
|
||||
mutable std::mutex mutex;
|
||||
MemoryAccessStorage memory_storage TSA_GUARDED_BY(mutex);
|
||||
|
@ -21,7 +21,7 @@ std::vector<UUID> Role::findDependencies() const
|
||||
return res;
|
||||
}
|
||||
|
||||
void Role::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
|
||||
void Role::doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
|
||||
{
|
||||
granted_roles.replaceDependencies(old_to_new_ids);
|
||||
settings.replaceDependencies(old_to_new_ids);
|
||||
|
@ -21,7 +21,7 @@ struct Role : public IAccessEntity
|
||||
AccessEntityType getType() const override { return TYPE; }
|
||||
|
||||
std::vector<UUID> findDependencies() const override;
|
||||
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
|
||||
void doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
|
||||
bool isBackupAllowed() const override { return settings.isBackupAllowed(); }
|
||||
};
|
||||
|
||||
|
@ -63,7 +63,7 @@ std::vector<UUID> RowPolicy::findDependencies() const
|
||||
return to_roles.findDependencies();
|
||||
}
|
||||
|
||||
void RowPolicy::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
|
||||
void RowPolicy::doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
|
||||
{
|
||||
to_roles.replaceDependencies(old_to_new_ids);
|
||||
}
|
||||
|
@ -50,7 +50,7 @@ struct RowPolicy : public IAccessEntity
|
||||
AccessEntityType getType() const override { return TYPE; }
|
||||
|
||||
std::vector<UUID> findDependencies() const override;
|
||||
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
|
||||
void doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
|
||||
bool isBackupAllowed() const override { return true; }
|
||||
|
||||
/// Which roles or users should use this row policy.
|
||||
|
@ -21,7 +21,7 @@ std::vector<UUID> SettingsProfile::findDependencies() const
|
||||
return res;
|
||||
}
|
||||
|
||||
void SettingsProfile::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
|
||||
void SettingsProfile::doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
|
||||
{
|
||||
elements.replaceDependencies(old_to_new_ids);
|
||||
to_roles.replaceDependencies(old_to_new_ids);
|
||||
|
@ -22,7 +22,7 @@ struct SettingsProfile : public IAccessEntity
|
||||
AccessEntityType getType() const override { return TYPE; }
|
||||
|
||||
std::vector<UUID> findDependencies() const override;
|
||||
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
|
||||
void doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
|
||||
bool isBackupAllowed() const override { return elements.isBackupAllowed(); }
|
||||
};
|
||||
|
||||
|
@ -49,7 +49,7 @@ std::vector<UUID> User::findDependencies() const
|
||||
return res;
|
||||
}
|
||||
|
||||
void User::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
|
||||
void User::doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
|
||||
{
|
||||
default_roles.replaceDependencies(old_to_new_ids);
|
||||
granted_roles.replaceDependencies(old_to_new_ids);
|
||||
|
@ -32,7 +32,7 @@ struct User : public IAccessEntity
|
||||
void setName(const String & name_) override;
|
||||
|
||||
std::vector<UUID> findDependencies() const override;
|
||||
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
|
||||
void doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
|
||||
bool isBackupAllowed() const override { return settings.isBackupAllowed(); }
|
||||
};
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user