Merge pull request #69346 from vitlibar/restore-access-dependencies

Improve restoring of access entities' dependencies
This commit is contained in:
Vitaly Baranov 2024-09-17 18:21:55 +00:00 committed by GitHub
commit fcda762a27
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
28 changed files with 322 additions and 155 deletions

View File

@ -29,6 +29,7 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int CANNOT_RESTORE_TABLE; extern const int CANNOT_RESTORE_TABLE;
extern const int ACCESS_ENTITY_ALREADY_EXISTS;
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
@ -175,9 +176,46 @@ namespace
return res; 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) for (const auto & [id, name_and_type] : dependencies)
{ {
std::optional<UUID> new_id; std::optional<UUID> new_id;
@ -188,9 +226,9 @@ namespace
if (new_id) if (new_id)
old_to_new_ids.emplace(id, *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) void generateRandomIDs(std::vector<std::pair<UUID, AccessEntityPtr>> & entities, std::unordered_map<UUID, UUID> & old_to_new_ids)
{ {
Poco::UUIDGenerator generator; 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) for (auto & entity : entities | boost::adaptors::map_values)
{ IAccessEntity::replaceDependencies(entity, old_to_new_ids);
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;
}
} }
AccessRightsElements getRequiredAccessToRestore(const std::vector<std::pair<UUID, AccessEntityPtr>> & entities) AccessRightsElements getRequiredAccessToRestore(const std::vector<std::pair<UUID, AccessEntityPtr>> & entities)
@ -314,7 +337,9 @@ std::pair<String, BackupEntryPtr> makeBackupEntryForAccess(
AccessRestorerFromBackup::AccessRestorerFromBackup( AccessRestorerFromBackup::AccessRestorerFromBackup(
const BackupPtr & backup_, const RestoreSettings & restore_settings_) 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 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); generateRandomIDs(new_entities, old_to_new_ids);
replaceDependencies(new_entities, old_to_new_ids); replaceDependencies(new_entities, old_to_new_ids);

View File

@ -17,6 +17,7 @@ using BackupPtr = std::shared_ptr<const IBackup>;
class IBackupEntry; class IBackupEntry;
using BackupEntryPtr = std::shared_ptr<const IBackupEntry>; using BackupEntryPtr = std::shared_ptr<const IBackupEntry>;
struct RestoreSettings; struct RestoreSettings;
enum class RestoreAccessCreationMode : uint8_t;
/// Makes a backup of access entities of a specified type. /// Makes a backup of access entities of a specified type.
@ -45,7 +46,8 @@ public:
private: private:
BackupPtr backup; BackupPtr backup;
bool allow_unresolved_access_dependencies = false; RestoreAccessCreationMode creation_mode;
bool allow_unresolved_dependencies = false;
std::vector<std::pair<UUID, AccessEntityPtr>> entities; std::vector<std::pair<UUID, AccessEntityPtr>> entities;
std::unordered_map<UUID, std::pair<String, AccessEntityType>> dependencies; std::unordered_map<UUID, std::pair<String, AccessEntityType>> dependencies;
std::unordered_set<String> data_paths; std::unordered_set<String> data_paths;

View File

@ -544,9 +544,9 @@ scope_guard AccessControl::subscribeForChanges(const std::vector<UUID> & ids, co
return changes_notifier->subscribeForChanges(ids, handler); 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(); changes_notifier->sendNotifications();
return true; return true;

View File

@ -243,7 +243,7 @@ private:
class CustomSettingsPrefixes; class CustomSettingsPrefixes;
class PasswordComplexityRules; 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 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 updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;

View File

@ -1,8 +1,6 @@
#include <Access/DiskAccessStorage.h> #include <Access/DiskAccessStorage.h>
#include <Access/AccessEntityIO.h> #include <Access/AccessEntityIO.h>
#include <Access/AccessChangesNotifier.h> #include <Access/AccessChangesNotifier.h>
#include <Backups/RestorerFromBackup.h>
#include <Backups/RestoreSettings.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
@ -418,7 +416,7 @@ void DiskAccessStorage::setAllInMemory(const std::vector<std::pair<UUID, AccessE
/// Insert or update entities. /// Insert or update entities.
for (const auto & [id, entity] : entities_without_conflicts) 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) 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}; 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(); const String & name = new_entity->getName();
AccessEntityType type = new_entity->getType(); AccessEntityType type = new_entity->getType();
@ -533,10 +531,16 @@ bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & ne
if (name_collision && !replace_if_exists) if (name_collision && !replace_if_exists)
{ {
if (throw_if_exists) if (throw_if_exists)
{
throwNameCollisionCannotInsert(type, name); throwNameCollisionCannotInsert(type, name);
}
else else
{
if (conflicting_id)
*conflicting_id = id_by_name;
return false; return false;
} }
}
auto it_by_id = entries_by_id.find(id); auto it_by_id = entries_by_id.find(id);
bool id_collision = (it_by_id != entries_by_id.end()); bool id_collision = (it_by_id != entries_by_id.end());
@ -548,8 +552,12 @@ bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & ne
throwIDCollisionCannotInsert(id, type, name, existing_entry.type, existing_entry.name); throwIDCollisionCannotInsert(id, type, name, existing_entry.type, existing_entry.name);
} }
else else
{
if (conflicting_id)
*conflicting_id = id;
return false; return false;
} }
}
if (write_on_disk) if (write_on_disk)
scheduleWriteLists(type); scheduleWriteLists(type);
@ -727,25 +735,4 @@ void DiskAccessStorage::deleteAccessEntityOnDisk(const UUID & id) const
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Couldn't delete {}", file_path); 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);
});
}
} }

View File

@ -34,14 +34,13 @@ public:
bool exists(const UUID & id) const override; bool exists(const UUID & id) const override;
bool isBackupAllowed() const override { return backup_allowed; } bool isBackupAllowed() const override { return backup_allowed; }
void restoreFromBackup(RestorerFromBackup & restorer) override;
private: private:
std::optional<UUID> findImpl(AccessEntityType type, const String & name) const override; std::optional<UUID> findImpl(AccessEntityType type, const String & name) const override;
std::vector<UUID> findAllImpl(AccessEntityType type) const override; std::vector<UUID> findAllImpl(AccessEntityType type) const override;
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) 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; 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 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 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 listsWritingThreadFunc() TSA_NO_THREAD_SAFETY_ANALYSIS;
void stopListsWritingThread(); 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 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); bool removeNoLock(const UUID & id, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex);

View File

@ -9,4 +9,28 @@ bool IAccessEntity::equal(const IAccessEntity & other) const
return (name == other.name) && (getType() == other.getType()); 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;
}
} }

View File

@ -50,7 +50,8 @@ struct IAccessEntity
virtual std::vector<UUID> findDependencies() const { return {}; } virtual std::vector<UUID> findDependencies() const { return {}; }
/// Replaces dependencies according to a specified map. /// 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. /// Whether this access entity should be written to a backup.
virtual bool isBackupAllowed() const { return false; } virtual bool isBackupAllowed() const { return false; }
@ -66,6 +67,8 @@ protected:
{ {
return std::make_shared<EntityClassT>(typeid_cast<const EntityClassT &>(*this)); 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>; using AccessEntityPtr = std::shared_ptr<const IAccessEntity>;

View File

@ -4,6 +4,8 @@
#include <Access/User.h> #include <Access/User.h>
#include <Access/AccessBackup.h> #include <Access/AccessBackup.h>
#include <Backups/BackupEntriesCollector.h> #include <Backups/BackupEntriesCollector.h>
#include <Backups/RestorerFromBackup.h>
#include <Backups/RestoreSettings.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <Common/callOnce.h> #include <Common/callOnce.h>
@ -14,10 +16,11 @@
#include <base/FnTraits.h> #include <base/FnTraits.h>
#include <boost/algorithm/string/join.hpp> #include <boost/algorithm/string/join.hpp>
#include <boost/algorithm/string/replace.hpp> #include <boost/algorithm/string/replace.hpp>
#include <boost/range/adaptor/map.hpp>
#include <boost/range/adaptor/reversed.hpp> #include <boost/range/adaptor/reversed.hpp>
#include <boost/range/algorithm/copy.hpp>
#include <boost/range/algorithm_ext/erase.hpp> #include <boost/range/algorithm_ext/erase.hpp>
namespace DB namespace DB
{ {
namespace ErrorCodes namespace ErrorCodes
@ -178,20 +181,20 @@ UUID IAccessStorage::insert(const AccessEntityPtr & entity)
return *insert(entity, /* replace_if_exists = */ false, /* throw_if_exists = */ true); 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(); 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 id;
return std::nullopt; 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()) if (isReadOnly())
throwReadonlyCannotInsert(entity->getType(), entity->getName()); throwReadonlyCannotInsert(entity->getType(), entity->getName());
@ -611,12 +614,51 @@ void IAccessStorage::backup(BackupEntriesCollector & backup_entries_collector, c
} }
void IAccessStorage::restoreFromBackup(RestorerFromBackup &) void IAccessStorage::restoreFromBackup(RestorerFromBackup & restorer)
{ {
if (!isRestoreAllowed()) if (!isRestoreAllowed())
throwRestoreNotAllowed(); 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, entities_to_restore = std::move(entities), replace_if_exists, throw_if_exists] mutable
{
std::unordered_map<UUID, UUID> new_to_existing_ids;
for (auto & [id, entity] : entities_to_restore)
{
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;
ids.reserve(entities_to_restore.size());
boost::copy(entities_to_restore | boost::adaptors::map_keys, std::back_inserter(ids));
tryUpdate(ids, update_func);
}
});
} }

View File

@ -64,6 +64,9 @@ public:
/// Returns true if this entity is readonly. /// Returns true if this entity is readonly.
virtual bool isReadOnly(const UUID &) const { return isReadOnly(); } 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. /// Starts periodic reloading and updating of entities in this storage.
virtual void startPeriodicReloading() {} virtual void startPeriodicReloading() {}
@ -153,8 +156,8 @@ public:
/// Inserts an entity to the storage. Returns ID of a new entry in the storage. /// Inserts an entity to the storage. Returns ID of a new entry in the storage.
/// Throws an exception if the specified name already exists. /// Throws an exception if the specified name already exists.
UUID insert(const AccessEntityPtr & entity); UUID insert(const AccessEntityPtr & entity);
std::optional<UUID> insert(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); 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, 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); 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 std::vector<UUID> findAllImpl(AccessEntityType type) const = 0;
virtual AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) 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 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 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 bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists);
virtual std::optional<AuthResult> authenticateImpl( virtual std::optional<AuthResult> authenticateImpl(
@ -240,6 +243,7 @@ protected:
LoggerPtr getLogger() const; LoggerPtr getLogger() const;
static String formatEntityTypeWithName(AccessEntityType type, const String & name) { return AccessEntityTypeInfo::get(type).formatEntityNameWithType(name); } 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_); 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(const UUID & id) const;
[[noreturn]] void throwNotFound(AccessEntityType type, const String & name) 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); [[noreturn]] static void throwBadCast(const UUID & id, AccessEntityType type, const String & name, AccessEntityType required_type);

View File

@ -1,7 +1,5 @@
#include <Access/MemoryAccessStorage.h> #include <Access/MemoryAccessStorage.h>
#include <Access/AccessChangesNotifier.h> #include <Access/AccessChangesNotifier.h>
#include <Backups/RestorerFromBackup.h>
#include <Backups/RestoreSettings.h>
#include <base/scope_guard.h> #include <base/scope_guard.h>
#include <boost/container/flat_set.hpp> #include <boost/container/flat_set.hpp>
#include <boost/range/adaptor/map.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}; 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(); const String & name = new_entity->getName();
AccessEntityType type = new_entity->getType(); AccessEntityType type = new_entity->getType();
@ -86,10 +84,16 @@ bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr &
if (name_collision && !replace_if_exists) if (name_collision && !replace_if_exists)
{ {
if (throw_if_exists) if (throw_if_exists)
{
throwNameCollisionCannotInsert(type, name); throwNameCollisionCannotInsert(type, name);
}
else else
{
if (conflicting_id)
*conflicting_id = id_by_name;
return false; return false;
} }
}
auto it_by_id = entries_by_id.find(id); auto it_by_id = entries_by_id.find(id);
bool id_collision = (it_by_id != entries_by_id.end()); bool id_collision = (it_by_id != entries_by_id.end());
@ -97,10 +101,16 @@ bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr &
{ {
const auto & existing_entry = it_by_id->second; const auto & existing_entry = it_by_id->second;
if (throw_if_exists) if (throw_if_exists)
{
throwIDCollisionCannotInsert(id, type, name, existing_entry.entity->getType(), existing_entry.entity->getName()); throwIDCollisionCannotInsert(id, type, name, existing_entry.entity->getType(), existing_entry.entity->getName());
}
else else
{
if (conflicting_id)
*conflicting_id = id;
return false; return false;
} }
}
/// Remove collisions if necessary. /// Remove collisions if necessary.
if (name_collision && (id_by_name != id)) if (name_collision && (id_by_name != id))
@ -270,28 +280,7 @@ void MemoryAccessStorage::setAll(const std::vector<std::pair<UUID, AccessEntityP
/// Insert or update entities. /// Insert or update entities.
for (const auto & [id, entity] : entities_without_conflicts) for (const auto & [id, entity] : entities_without_conflicts)
insertNoLock(id, entity, /* replace_if_exists = */ true, /* throw_if_exists = */ false); insertNoLock(id, entity, /* replace_if_exists = */ true, /* throw_if_exists = */ false, /* conflicting_id = */ nullptr);
}
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);
});
} }
} }

View File

@ -34,17 +34,16 @@ public:
bool exists(const UUID & id) const override; bool exists(const UUID & id) const override;
bool isBackupAllowed() const override { return backup_allowed; } bool isBackupAllowed() const override { return backup_allowed; }
void restoreFromBackup(RestorerFromBackup & restorer) override;
private: private:
std::optional<UUID> findImpl(AccessEntityType type, const String & name) const override; std::optional<UUID> findImpl(AccessEntityType type, const String & name) const override;
std::vector<UUID> findAllImpl(AccessEntityType type) const override; std::vector<UUID> findAllImpl(AccessEntityType type) const override;
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) 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 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 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 removeNoLock(const UUID & id, bool throw_if_not_exists);
bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists);

View File

@ -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; std::shared_ptr<IAccessStorage> storage_for_insertion;
@ -376,7 +376,7 @@ bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr &
getStorageName()); 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}; std::lock_guard lock{mutex};
ids_cache.set(id, storage_for_insertion); ids_cache.set(id, storage_for_insertion);

View File

@ -67,7 +67,7 @@ protected:
std::vector<UUID> findAllImpl(AccessEntityType type) const override; std::vector<UUID> findAllImpl(AccessEntityType type) const override;
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) 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; 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 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 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; 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;

View File

@ -24,7 +24,7 @@ std::vector<UUID> Quota::findDependencies() const
return to_roles.findDependencies(); 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); to_roles.replaceDependencies(old_to_new_ids);
} }

View File

@ -47,7 +47,7 @@ struct Quota : public IAccessEntity
AccessEntityType getType() const override { return TYPE; } AccessEntityType getType() const override { return TYPE; }
std::vector<UUID> findDependencies() const 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 true; } bool isBackupAllowed() const override { return true; }
}; };

View File

@ -5,10 +5,9 @@
#include <Access/AccessChangesNotifier.h> #include <Access/AccessChangesNotifier.h>
#include <Access/AccessBackup.h> #include <Access/AccessBackup.h>
#include <Backups/BackupEntriesCollector.h> #include <Backups/BackupEntriesCollector.h>
#include <Backups/RestorerFromBackup.h>
#include <Backups/RestoreSettings.h>
#include <Backups/IBackupCoordination.h> #include <Backups/IBackupCoordination.h>
#include <Backups/IRestoreCoordination.h> #include <Backups/IRestoreCoordination.h>
#include <Backups/RestorerFromBackup.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Common/ZooKeeper/KeeperException.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 AccessEntityTypeInfo type_info = AccessEntityTypeInfo::get(new_entity->getType());
const String & name = new_entity->getName(); const String & name = new_entity->getName();
@ -128,7 +127,7 @@ bool ReplicatedAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr
auto zookeeper = getZooKeeper(); auto zookeeper = getZooKeeper();
bool ok = false; 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) if (!ok)
return false; return false;
@ -143,7 +142,8 @@ bool ReplicatedAccessStorage::insertZooKeeper(
const UUID & id, const UUID & id,
const AccessEntityPtr & new_entity, const AccessEntityPtr & new_entity,
bool replace_if_exists, bool replace_if_exists,
bool throw_if_exists) bool throw_if_exists,
UUID * conflicting_id)
{ {
const String & name = new_entity->getName(); const String & name = new_entity->getName();
const AccessEntityType type = new_entity->getType(); const AccessEntityType type = new_entity->getType();
@ -167,16 +167,16 @@ bool ReplicatedAccessStorage::insertZooKeeper(
if (res == Coordination::Error::ZNODEEXISTS) if (res == Coordination::Error::ZNODEEXISTS)
{ {
if (!throw_if_exists && !replace_if_exists) if (!replace_if_exists)
return false; /// Couldn't insert a new entity.
if (throw_if_exists)
{ {
if (responses[0]->error == Coordination::Error::ZNODEEXISTS) if (responses[0]->error == Coordination::Error::ZNODEEXISTS)
{
/// 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. /// 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. /// This itself can fail if the conflicting uuid disappears in the meantime.
/// If that happens, then we'll just retry from the start. /// If that happens, then retryOnZooKeeperUserError() will just retry the operation from the start.
String existing_entity_definition = zookeeper->get(entity_path); String existing_entity_definition = zookeeper->get(entity_path);
AccessEntityPtr existing_entity = deserializeAccessEntity(existing_entity_definition, entity_path); AccessEntityPtr existing_entity = deserializeAccessEntity(existing_entity_definition, entity_path);
@ -186,9 +186,34 @@ bool ReplicatedAccessStorage::insertZooKeeper(
} }
else else
{ {
/// Couldn't insert the new entity because there is an existing entity with such name. 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); 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
{
zkutil::KeeperMultiException::check(res, ops, responses);
}
} }
assert(replace_if_exists); assert(replace_if_exists);
@ -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(); auto restore_coordination = restorer.getRestoreCoordination();
if (!restore_coordination->acquireReplicatedAccessStorage(zookeeper_path)) return 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);
});
} }
} }

View File

@ -26,6 +26,7 @@ public:
void shutdown() override; void shutdown() override;
const char * getStorageType() const override { return STORAGE_TYPE; } const char * getStorageType() const override { return STORAGE_TYPE; }
bool isReplicated() const override { return true; }
void startPeriodicReloading() override { startWatchingThread(); } void startPeriodicReloading() override { startWatchingThread(); }
void stopPeriodicReloading() override { stopWatchingThread(); } void stopPeriodicReloading() override { stopWatchingThread(); }
@ -35,7 +36,6 @@ public:
bool isBackupAllowed() const override { return backup_allowed; } bool isBackupAllowed() const override { return backup_allowed; }
void backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const override; void backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const override;
void restoreFromBackup(RestorerFromBackup & restorer) override;
private: private:
String zookeeper_path; String zookeeper_path;
@ -48,11 +48,11 @@ private:
std::unique_ptr<ThreadFromGlobalPool> watching_thread; std::unique_ptr<ThreadFromGlobalPool> watching_thread;
std::shared_ptr<ConcurrentBoundedQueue<UUID>> watched_queue; 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 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 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 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); 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::optional<UUID> findImpl(AccessEntityType type, const String & name) const override;
std::vector<UUID> findAllImpl(AccessEntityType type) const override; std::vector<UUID> findAllImpl(AccessEntityType type) const override;
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override;
bool acquireReplicatedRestore(RestorerFromBackup & restorer) const override;
mutable std::mutex mutex; mutable std::mutex mutex;
MemoryAccessStorage memory_storage TSA_GUARDED_BY(mutex); MemoryAccessStorage memory_storage TSA_GUARDED_BY(mutex);

View File

@ -21,7 +21,7 @@ std::vector<UUID> Role::findDependencies() const
return res; 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); granted_roles.replaceDependencies(old_to_new_ids);
settings.replaceDependencies(old_to_new_ids); settings.replaceDependencies(old_to_new_ids);

View File

@ -21,7 +21,7 @@ struct Role : public IAccessEntity
AccessEntityType getType() const override { return TYPE; } AccessEntityType getType() const override { return TYPE; }
std::vector<UUID> findDependencies() const 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(); } bool isBackupAllowed() const override { return settings.isBackupAllowed(); }
}; };

View File

@ -63,7 +63,7 @@ std::vector<UUID> RowPolicy::findDependencies() const
return to_roles.findDependencies(); 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); to_roles.replaceDependencies(old_to_new_ids);
} }

View File

@ -50,7 +50,7 @@ struct RowPolicy : public IAccessEntity
AccessEntityType getType() const override { return TYPE; } AccessEntityType getType() const override { return TYPE; }
std::vector<UUID> findDependencies() const 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 true; } bool isBackupAllowed() const override { return true; }
/// Which roles or users should use this row policy. /// Which roles or users should use this row policy.

View File

@ -21,7 +21,7 @@ std::vector<UUID> SettingsProfile::findDependencies() const
return res; 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); elements.replaceDependencies(old_to_new_ids);
to_roles.replaceDependencies(old_to_new_ids); to_roles.replaceDependencies(old_to_new_ids);

View File

@ -22,7 +22,7 @@ struct SettingsProfile : public IAccessEntity
AccessEntityType getType() const override { return TYPE; } AccessEntityType getType() const override { return TYPE; }
std::vector<UUID> findDependencies() const 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 elements.isBackupAllowed(); } bool isBackupAllowed() const override { return elements.isBackupAllowed(); }
}; };

View File

@ -49,7 +49,7 @@ std::vector<UUID> User::findDependencies() const
return res; 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); default_roles.replaceDependencies(old_to_new_ids);
granted_roles.replaceDependencies(old_to_new_ids); granted_roles.replaceDependencies(old_to_new_ids);

View File

@ -32,7 +32,7 @@ struct User : public IAccessEntity
void setName(const String & name_) override; void setName(const String & name_) override;
std::vector<UUID> findDependencies() const 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(); } bool isBackupAllowed() const override { return settings.isBackupAllowed(); }
}; };

View File

@ -0,0 +1,6 @@
Everything dropped
User dropped
Nothing dropped
Nothing dropped, mode=replace
Nothing dropped, mode=create
ACCESS_ENTITY_ALREADY_EXISTS

View File

@ -0,0 +1,77 @@
#!/usr/bin/env bash
# Tags: no-parallel
# Disabled parallel since RESTORE can only restore either all users or no users
# (it can't restore only users added by the current test run),
# so a RESTORE from a parallel test run could recreate our users before we expect that.
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
user_a="user_a_${CLICKHOUSE_TEST_UNIQUE_NAME}"
role_b="role_b_${CLICKHOUSE_TEST_UNIQUE_NAME}"
${CLICKHOUSE_CLIENT} -m --query "
CREATE ROLE ${role_b} SETTINGS custom_x=1;
CREATE USER ${user_a} DEFAULT ROLE ${role_b} SETTINGS custom_x=2;
"
backup_name="Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}')"
${CLICKHOUSE_CLIENT} --query "BACKUP TABLE system.users, TABLE system.roles TO ${backup_name} FORMAT Null"
${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null"
do_check()
{
local replacements
replacements="s/${user_a}/user_a/g; s/${role_b}/role_b/g"
local check_info
check_info=$(${CLICKHOUSE_CLIENT} -mq "
SHOW CREATE USER ${user_a};
SHOW GRANTS FOR ${user_a};
SHOW CREATE ROLE ${role_b};
SHOW GRANTS FOR ${role_b};
" | sed "${replacements}")
local expected
expected=$'CREATE USER user_a IDENTIFIED WITH no_password DEFAULT ROLE role_b SETTINGS custom_x = 2\nGRANT role_b TO user_a\nCREATE ROLE role_b SETTINGS custom_x = 1'
if [[ "${check_info}" != "${expected}" ]]; then
echo "Assertion failed:"
echo "\"${check_info}\""
echo "!="
echo "\"${expected}\""
echo "Test database: ${CLICKHOUSE_DATABASE}" >&2
fi
}
echo "Everything dropped"
${CLICKHOUSE_CLIENT} --query "DROP USER ${user_a}"
${CLICKHOUSE_CLIENT} --query "DROP ROLE ${role_b}"
${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null"
do_check
echo "User dropped"
${CLICKHOUSE_CLIENT} --query "DROP USER ${user_a}"
${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null"
do_check
# TODO: Cannot restore a dropped role granted to an existing user. The result after RESTORE ALL below is the following:
# CREATE USER user_a DEFAULT ROLE NONE SETTINGS custom_x = 2; GRANT NONE TO user_a; CREATE ROLE role_b SETTINGS custom_x = 1
# because `role_b` is restored but not granted to existing user `user_a`.
#
# echo "Role dropped"
# ${CLICKHOUSE_CLIENT} --query "DROP ROLE ${role_b}"
# ${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null"
# do_check
echo "Nothing dropped"
${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null"
do_check
echo "Nothing dropped, mode=replace"
${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} SETTINGS create_access='replace' FORMAT Null"
do_check
echo "Nothing dropped, mode=create"
${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} SETTINGS create_access='create' FORMAT Null" 2>&1 | grep -om1 "ACCESS_ENTITY_ALREADY_EXISTS"
do_check