Compare commits

...

8 Commits

51 changed files with 1156 additions and 351 deletions

View File

@ -29,6 +29,8 @@ namespace DB
namespace ErrorCodes
{
extern const int CANNOT_RESTORE_TABLE;
extern const int ACCESS_ENTITY_ALREADY_EXISTS;
extern const int ACCESS_ENTITY_NOT_FOUND;
extern const int LOGICAL_ERROR;
}
@ -174,125 +176,6 @@ 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)
{
std::unordered_map<UUID, UUID> old_to_new_ids;
for (const auto & [id, name_and_type] : dependencies)
{
std::optional<UUID> new_id;
if (allow_unresolved_dependencies)
new_id = access_control.find(name_and_type.second, name_and_type.first);
else
new_id = access_control.getID(name_and_type.second, name_and_type.first);
if (new_id)
old_to_new_ids.emplace(id, *new_id);
}
return 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;
for (auto & [id, entity] : entities)
{
UUID new_id;
generator.createRandom().copyTo(reinterpret_cast<char *>(&new_id));
old_to_new_ids.emplace(id, new_id);
id = new_id;
}
}
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;
}
}
AccessRightsElements getRequiredAccessToRestore(const std::vector<std::pair<UUID, AccessEntityPtr>> & entities)
{
AccessRightsElements res;
for (const auto & entity : entities | boost::adaptors::map_values)
{
auto entity_type = entity->getType();
switch (entity_type)
{
case User::TYPE:
{
const auto & user = typeid_cast<const User &>(*entity);
res.emplace_back(AccessType::CREATE_USER);
auto elements = user.access.getElements();
for (auto & element : elements)
{
if (element.is_partial_revoke)
continue;
element.grant_option = true;
res.emplace_back(element);
}
if (!user.granted_roles.isEmpty())
res.emplace_back(AccessType::ROLE_ADMIN);
break;
}
case Role::TYPE:
{
const auto & role = typeid_cast<const Role &>(*entity);
res.emplace_back(AccessType::CREATE_ROLE);
auto elements = role.access.getElements();
for (auto & element : elements)
{
if (element.is_partial_revoke)
continue;
element.grant_option = true;
res.emplace_back(element);
}
if (!role.granted_roles.isEmpty())
res.emplace_back(AccessType::ROLE_ADMIN);
break;
}
case SettingsProfile::TYPE:
{
res.emplace_back(AccessType::CREATE_SETTINGS_PROFILE);
break;
}
case RowPolicy::TYPE:
{
const auto & policy = typeid_cast<const RowPolicy &>(*entity);
res.emplace_back(AccessType::CREATE_ROW_POLICY, policy.getDatabase(), policy.getTableName());
break;
}
case Quota::TYPE:
{
res.emplace_back(AccessType::CREATE_QUOTA);
break;
}
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown type: {}", toString(entity_type));
}
}
return res;
}
}
@ -314,59 +197,447 @@ 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)
, skip_unresolved_dependencies(restore_settings_.skip_unresolved_access_entities_dependencies)
, update_dependents(restore_settings_.update_access_entities_dependents)
, log(getLogger("AccessRestorerFromBackup"))
{
}
AccessRestorerFromBackup::~AccessRestorerFromBackup() = default;
void AccessRestorerFromBackup::addDataPath(const String & data_path)
void AccessRestorerFromBackup::addDataPath(const String & data_path_in_backup, bool dependents_only)
{
if (!data_paths.emplace(data_path).second)
if (loaded)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Access entities already loaded");
if (dependents_only && !update_dependents)
return;
fs::path data_path_in_backup_fs = data_path;
Strings filenames = backup->listFiles(data_path, /*recursive*/ false);
if (filenames.empty())
return;
for (const String & filename : filenames)
for (auto & stored_data_path : data_paths_in_backup)
{
if (!filename.starts_with("access") || !filename.ends_with(".txt"))
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File name {} doesn't match the wildcard \"access*.txt\"",
String{data_path_in_backup_fs / filename});
if (stored_data_path.first == data_path_in_backup)
{
stored_data_path.second &= dependents_only;
return;
}
}
::sort(filenames.begin(), filenames.end());
for (const String & filename : filenames)
{
String filepath_in_backup = data_path_in_backup_fs / filename;
auto read_buffer_from_backup = backup->readFile(filepath_in_backup);
auto ab = AccessEntitiesInBackup::fromBackupEntry(std::move(read_buffer_from_backup), filepath_in_backup);
boost::range::copy(ab.entities, std::back_inserter(entities));
boost::range::copy(ab.dependencies, std::inserter(dependencies, dependencies.end()));
}
for (const auto & id : entities | boost::adaptors::map_keys)
dependencies.erase(id);
data_paths_in_backup.emplace_back(data_path_in_backup, dependents_only);
}
void AccessRestorerFromBackup::loadFromBackup()
{
if (loaded)
return;
/// Parse files "access*.txt" found in the added data paths in the backup.
for (size_t data_path_index = 0; data_path_index != data_paths_in_backup.size(); ++data_path_index)
{
const String & data_path_in_backup = data_paths_in_backup[data_path_index].first;
bool dependents_only = data_paths_in_backup[data_path_index].second;
fs::path data_path_in_backup_fs = data_path_in_backup;
Strings filenames = backup->listFiles(data_path_in_backup_fs, /*recursive*/ false);
if (filenames.empty())
continue;
for (const String & filename : filenames)
{
if (!filename.starts_with("access") || !filename.ends_with(".txt"))
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File name {} doesn't match the wildcard \"access*.txt\"",
String{data_path_in_backup_fs / filename});
}
for (const String & filename : filenames)
{
String filepath_in_backup = data_path_in_backup_fs / filename;
AccessEntitiesInBackup ab;
try
{
auto read_buffer_from_backup = backup->readFile(filepath_in_backup);
ab = AccessEntitiesInBackup::fromBackupEntry(std::move(read_buffer_from_backup), filepath_in_backup);
}
catch (Exception & e)
{
e.addMessage("While reading access entities from {} in backup", filepath_in_backup);
throw;
}
for (const auto & [id, entity] : ab.entities)
{
auto it = entity_infos.find(id);
if (it == entity_infos.end())
{
it = entity_infos.emplace(id, EntityInfo{.id = id, .name = entity->getName(), .type = entity->getType()}).first;
}
EntityInfo & entity_info = it->second;
entity_info.entity = entity;
entity_info.data_path_index = data_path_index;
if (!dependents_only)
entity_info.restore = true;
}
for (const auto & [id, name_and_type] : ab.dependencies)
{
auto it = entity_infos.find(id);
if (it == entity_infos.end())
{
it = entity_infos.emplace(id, EntityInfo{.id = id, .name = name_and_type.first, .type = name_and_type.second}).first;
}
EntityInfo & entity_info = it->second;
entity_info.is_dependency = true;
}
}
}
loaded = true;
}
AccessRightsElements AccessRestorerFromBackup::getRequiredAccess() const
{
return getRequiredAccessToRestore(entities);
if (!loaded)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Access entities not loaded");
AccessRightsElements res;
for (const auto & [id, entity_info] : entity_infos)
{
if (!entity_info.restore)
continue;
const auto & entity = entity_info.entity;
auto entity_type = entity->getType();
switch (entity_type)
{
case User::TYPE:
{
const auto & user = typeid_cast<const User &>(*entity);
res.emplace_back(AccessType::CREATE_USER);
auto elements = user.access.getElements();
for (auto & element : elements)
{
if (element.is_partial_revoke)
continue;
element.grant_option = true;
res.emplace_back(element);
}
if (!user.granted_roles.isEmpty())
res.emplace_back(AccessType::ROLE_ADMIN);
break;
}
case Role::TYPE:
{
const auto & role = typeid_cast<const Role &>(*entity);
res.emplace_back(AccessType::CREATE_ROLE);
auto elements = role.access.getElements();
for (auto & element : elements)
{
if (element.is_partial_revoke)
continue;
element.grant_option = true;
res.emplace_back(element);
}
if (!role.granted_roles.isEmpty())
res.emplace_back(AccessType::ROLE_ADMIN);
break;
}
case SettingsProfile::TYPE:
{
res.emplace_back(AccessType::CREATE_SETTINGS_PROFILE);
break;
}
case RowPolicy::TYPE:
{
const auto & policy = typeid_cast<const RowPolicy &>(*entity);
res.emplace_back(AccessType::CREATE_ROW_POLICY, policy.getDatabase(), policy.getTableName());
break;
}
case Quota::TYPE:
{
res.emplace_back(AccessType::CREATE_QUOTA);
break;
}
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown type: {}", toString(entity_type));
}
}
return res;
}
std::vector<std::pair<UUID, AccessEntityPtr>> AccessRestorerFromBackup::getAccessEntities(const AccessControl & access_control) const
void AccessRestorerFromBackup::generateRandomIDsAndResolveDependencies(const AccessControl & access_control)
{
auto new_entities = entities;
if (ids_assigned)
return;
auto old_to_new_ids = resolveDependencies(dependencies, access_control, allow_unresolved_access_dependencies);
generateRandomIDs(new_entities, old_to_new_ids);
replaceDependencies(new_entities, old_to_new_ids);
if (!loaded)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Access entities not loaded");
return new_entities;
/// Calculate `new_id` for each entity info.
/// Check which ones of the loaded access entities already exist.
/// Generate random UUIDs for access entities which we're going to restore if they don't exist.
for (auto & [id, entity_info] : entity_infos)
{
const String & name = entity_info.name;
auto type = entity_info.type;
if (entity_info.restore && (creation_mode == RestoreAccessCreationMode::kReplace))
{
entity_info.new_id = UUIDHelpers::generateV4();
LOG_TRACE(log, "{}: Generated new UUID {}", AccessEntityTypeInfo::get(type).formatEntityNameWithType(name), *entity_info.new_id);
continue;
}
if (auto existing_id = access_control.find(type, name))
{
if (entity_info.restore && (creation_mode == RestoreAccessCreationMode::kCreate))
{
throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Cannot restore {} because it already exists",
AccessEntityTypeInfo::get(type).formatEntityNameWithType(name));
}
bool was_going_to_restore = entity_info.restore;
entity_info.new_id = *existing_id;
entity_info.restore = false;
LOG_TRACE(log, "{}: Found with UUID {}{}", AccessEntityTypeInfo::get(type).formatEntityNameWithType(name), *existing_id,
(was_going_to_restore ? ", will not restore" : ""));
}
else
{
if (entity_info.is_dependency && !entity_info.restore && !skip_unresolved_dependencies)
{
throw Exception(ErrorCodes::ACCESS_ENTITY_NOT_FOUND, "Cannot resolve {} while restoring from backup",
AccessEntityTypeInfo::get(type).formatEntityNameWithType(name));
}
if (entity_info.restore)
{
entity_info.new_id = UUIDHelpers::generateV4();
LOG_TRACE(log, "{}: Generated new UUID {}", AccessEntityTypeInfo::get(type).formatEntityNameWithType(name), *entity_info.new_id);
}
else
{
LOG_TRACE(log, "{}: Not found, ignoring", AccessEntityTypeInfo::get(type).formatEntityNameWithType(name));
}
}
}
/// Prepare map from old UUIDs to new UUIDs.
std::unordered_map<UUID, UUID> old_to_new_ids;
std::unordered_set<UUID> unresolved_ids;
std::unordered_set<UUID> old_ids_to_restore;
for (const auto & [id, entity_info] : entity_infos)
{
if (entity_info.new_id)
old_to_new_ids[id] = *entity_info.new_id;
else
unresolved_ids.insert(id);
if (entity_info.restore)
old_ids_to_restore.insert(id);
}
/// Calculate `is_dependent` for each entity info.
if (update_dependents)
{
for (auto & [id, entity_info] : entity_infos)
{
if (!entity_info.restore && entity_info.new_id && entity_info.entity && entity_info.entity->hasDependencies(old_ids_to_restore))
entity_info.is_dependent = true;
}
}
/// Remap the UUIDs of dependencies in the access entities we're going to restore.
for (auto & [id, entity_info] : entity_infos)
{
if (entity_info.entity && (entity_info.restore || entity_info.is_dependent))
{
auto new_entity = entity_info.entity->clone();
new_entity->replaceDependencies(old_to_new_ids);
new_entity->removeDependencies(unresolved_ids);
entity_info.entity = new_entity;
}
if (entity_info.restore && data_path_with_entities_to_restore.empty())
data_path_with_entities_to_restore = data_paths_in_backup[entity_info.data_path_index].first;
}
ids_assigned = true;
}
std::vector<std::pair<UUID, AccessEntityPtr>> AccessRestorerFromBackup::getEntities(const String & data_path_in_backup) const
{
if (!ids_assigned)
throw Exception(ErrorCodes::LOGICAL_ERROR, "IDs not assigned");
if (data_path_in_backup != data_path_with_entities_to_restore)
return {};
std::vector<std::pair<UUID, AccessEntityPtr>> res;
res.reserve(entity_infos.size());
for (const auto & [id, entity_info] : entity_infos)
{
if (entity_info.restore)
res.emplace_back(*entity_info.new_id, entity_info.entity);
}
return res;
}
std::vector<std::pair<UUID, AccessEntityPtr>> AccessRestorerFromBackup::getDependents(const String & data_path_in_backup) const
{
if (!ids_assigned)
throw Exception(ErrorCodes::LOGICAL_ERROR, "IDs not assigned");
if (data_path_in_backup != data_path_with_entities_to_restore)
return {};
std::vector<std::pair<UUID, AccessEntityPtr>> res;
res.reserve(entity_infos.size());
for (const auto & [id, entity_info] : entity_infos)
{
if (entity_info.is_dependent)
res.emplace_back(*entity_info.new_id, entity_info.entity);
}
return res;
}
void restoreAccessEntitiesFromBackup(
IAccessStorage & destination_access_storage,
const std::vector<std::pair<UUID, AccessEntityPtr>> & entities,
const std::vector<std::pair<UUID, AccessEntityPtr>> & dependents,
const RestoreSettings & restore_settings)
{
if (entities.empty())
return; /// Nothing to restore.
auto log = getLogger("AccessRestorerFromBackup");
bool replace_if_exists = (restore_settings.create_access == RestoreAccessCreationMode::kReplace);
bool throw_if_exists = (restore_settings.create_access == RestoreAccessCreationMode::kCreate);
bool update_dependents = restore_settings.update_access_entities_dependents;
std::unordered_map<UUID, UUID> new_to_existing_ids;
std::vector<std::pair<UUID, AccessEntityPtr>> more_dependents;
for (const auto & [id, entity] : entities)
{
const String & name = entity->getName();
auto type = entity->getType();
LOG_TRACE(log, "{}: Adding with UUID {}", AccessEntityTypeInfo::get(type).formatEntityNameWithType(name), id);
UUID existing_id;
if (destination_access_storage.insert(id, entity, replace_if_exists, throw_if_exists, &existing_id))
{
LOG_TRACE(log, "{}: Added successfully", AccessEntityTypeInfo::get(type).formatEntityNameWithType(name));
}
else
{
/// Couldn't insert `entity` because there is an existing entity with the same name.
LOG_TRACE(log, "{}: Not added because already exists with UUID {}", AccessEntityTypeInfo::get(type).formatEntityNameWithType(name), existing_id);
new_to_existing_ids[id] = existing_id;
if (update_dependents)
more_dependents.emplace_back(existing_id, entity);
}
}
std::vector<UUID> ids_to_update;
if (!new_to_existing_ids.empty())
{
std::unordered_set<UUID> new_ids;
boost::copy(new_to_existing_ids | boost::adaptors::map_keys, std::inserter(new_ids, new_ids.end()));
ids_to_update.clear();
ids_to_update.reserve(entities.size());
for (const auto & [id, entity] : entities)
{
bool is_restored = !new_to_existing_ids.contains(id);
if (is_restored)
ids_to_update.emplace_back(id);
}
/// 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, const UUID &) -> AccessEntityPtr
{
if (!entity->hasDependencies(new_ids))
return entity;
LOG_TRACE(log, "{}: Updating dependencies", entity->formatTypeWithName());
auto res = entity->clone();
res->replaceDependencies(new_to_existing_ids);
return res;
};
/// It's totally ok if some UUIDs from `ids_to_update` don't exist anymore, that's why we use tryUpdate() here.
destination_access_storage.tryUpdate(ids_to_update, update_func);
}
std::unordered_set<UUID> restored_ids;
auto do_update_dependents = [&](const std::vector<std::pair<UUID, AccessEntityPtr>> & dependents_to_update)
{
if (dependents_to_update.empty())
return;
ids_to_update.clear();
ids_to_update.reserve(dependents_to_update.size());
std::unordered_map<UUID, AccessEntityPtr> id_to_dependent;
for (const auto & [id, entity] : dependents_to_update)
{
if (!destination_access_storage.isReadOnly(id))
{
auto new_entity = entity->clone();
new_entity->replaceDependencies(new_to_existing_ids);
id_to_dependent[id] = new_entity;
ids_to_update.emplace_back(id);
}
}
if (restored_ids.empty())
{
for (const auto & [id, entity] : entities)
{
bool is_restored = !new_to_existing_ids.contains(id);
if (is_restored)
restored_ids.insert(id);
}
}
/// 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, const UUID & id) -> AccessEntityPtr
{
const auto & src_entity = *id_to_dependent.at(id);
if (!src_entity.hasDependencies(restored_ids))
return entity;
LOG_TRACE(log, "{}: Updating dependent", entity->formatTypeWithName());
auto res = entity->clone();
res->copyDependenciesFrom(src_entity, restored_ids);
return res;
};
/// It's totally ok if some UUIDs from `ids_to_update` don't exist anymore, that's why we use tryUpdate() here.
destination_access_storage.tryUpdate(ids_to_update, update_func);
};
do_update_dependents(dependents);
do_update_dependents(more_dependents);
}
}

View File

@ -1,8 +1,8 @@
#pragma once
#include <Common/Logger.h>
#include <Core/UUID.h>
#include <unordered_map>
#include <unordered_set>
namespace DB
@ -12,11 +12,13 @@ enum class AccessEntityType : uint8_t;
struct IAccessEntity;
using AccessEntityPtr = std::shared_ptr<const IAccessEntity>;
class AccessRightsElements;
class IAccessStorage;
class IBackup;
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.
@ -26,8 +28,15 @@ std::pair<String, BackupEntryPtr> makeBackupEntryForAccess(
size_t counter,
const AccessControl & access_control);
/// Restores access entities from a backup.
void restoreAccessEntitiesFromBackup(
IAccessStorage & access_storage,
const std::vector<std::pair<UUID, AccessEntityPtr>> & entities,
const std::vector<std::pair<UUID, AccessEntityPtr>> & dependents,
const RestoreSettings & restore_settings);
/// Loads access entities from a backup and prepares them for insertion into an access storage.
class AccessRestorerFromBackup
{
public:
@ -35,20 +44,82 @@ public:
~AccessRestorerFromBackup();
/// Adds a data path to loads access entities from.
void addDataPath(const String & data_path);
void addDataPath(const String & data_path_in_backup, bool dependents_only = false);
/// Loads access entities from the backup.
void loadFromBackup();
/// Checks that the current user can do restoring.
/// Function loadFromBackup() must be called before that.
AccessRightsElements getRequiredAccess() const;
/// Inserts all access entities loaded from all the paths added by addDataPath().
std::vector<std::pair<UUID, AccessEntityPtr>> getAccessEntities(const AccessControl & access_control) const;
/// Generates random IDs for access entities we're restoring to insert them into an access storage;
/// and finds IDs of existing access entities which are used as dependencies.
void generateRandomIDsAndResolveDependencies(const AccessControl & access_control);
/// Returns access entities prepared for insertion into an access storage and new random UUIDs generated for those access entities.
/// Both functions loadFromBackup() and generateRandomIDsAndResolveDependencies() must be called before that.
std::vector<std::pair<UUID, AccessEntityPtr>> getEntities(const String & data_path_in_backup) const;
/// Returns dependents of the access entities we're going to restore.
/// Dependents are access entities which exist already and they should be updated after restoring.
/// For example, if there were a role granted to a user: `CREATE USER user1; CREATE ROLE role1; GRANT role1 TO user1`,
/// and we're restoring only role `role1` because user `user1` already exists,
/// then user `user1` should be modified after restoring role `role1` to add this grant `GRANT role1 TO user1`.
std::vector<std::pair<UUID, AccessEntityPtr>> getDependents(const String & data_path_in_backup) const;
private:
BackupPtr backup;
bool allow_unresolved_access_dependencies = false;
std::vector<std::pair<UUID, AccessEntityPtr>> entities;
std::unordered_map<UUID, std::pair<String, AccessEntityType>> dependencies;
std::unordered_set<String> data_paths;
const BackupPtr backup;
const RestoreAccessCreationMode creation_mode;
const bool skip_unresolved_dependencies;
const bool update_dependents;
const LoggerPtr log;
/// Whether loadFromBackup() finished.
bool loaded = false;
/// Whether generateRandomIDsAndResolveDependencies() finished.
bool ids_assigned = false;
std::vector<std::pair<String, bool /* dependents_only */>> data_paths_in_backup;
String data_path_with_entities_to_restore;
/// Information about an access entity loaded from the backup.
struct EntityInfo
{
UUID id;
String name;
AccessEntityType type;
AccessEntityPtr entity = nullptr; /// Can be nullptr if `restore=false`.
/// Index in `data_paths_in_backup`.
size_t data_path_index = 0;
/// Whether we're going to restore this entity.
/// For example,
/// in case of `RESTORE TABLE system.roles` this flag is true for all the roles loaded from the backup, and
/// in case of `RESTORE ALL` this flag is always true.
bool restore = false;
/// Whether this entity info was added as a dependency of another entity which we're going to restore.
/// For example, if we're going to restore the following user: `CREATE USER user1 DEFAULT ROLE role1, role2 SETTINGS PROFILE profile1, profile2`
/// then `restore=true` for `user1` and `is_dependency=true` for `role1`, `role2`, `profile1`, `profile2`.
/// Flags `restore` and `is_dependency` both can be set at the same time.
bool is_dependency = false;
/// Whether this entity info is a dependent of another entity which we're going to restore.
/// For example, if we're going to restore role `role1` and there is also the following user stored in the backup:
/// `CREATE USER user1 DEFAULT ROLE role1`, then `is_dependent=true` for `user1`.
/// This flags is set by generateRandomIDsAndResolveDependencies().
bool is_dependent = false;
/// New UUID for this entity - either randomly generated or copied from an existing entity.
/// This UUID is assigned by generateRandomIDsAndResolveDependencies().
std::optional<UUID> new_id = std::nullopt;
};
std::unordered_map<UUID, EntityInfo> entity_infos;
};
}

View File

@ -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;
@ -629,9 +629,9 @@ AuthResult AccessControl::authenticate(const Credentials & credentials, const Po
}
}
void AccessControl::restoreFromBackup(RestorerFromBackup & restorer)
void AccessControl::restoreFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup)
{
MultipleAccessStorage::restoreFromBackup(restorer);
MultipleAccessStorage::restoreFromBackup(restorer, data_path_in_backup);
changes_notifier->sendNotifications();
}

View File

@ -124,7 +124,7 @@ public:
AuthResult authenticate(const Credentials & credentials, const Poco::Net::IPAddress & address, const String & forwarded_address) const;
/// Makes a backup of access entities.
void restoreFromBackup(RestorerFromBackup & restorer) override;
void restoreFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup) override;
void setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config);
@ -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;

View File

@ -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)
@ -668,7 +676,7 @@ bool DiskAccessStorage::updateNoLock(const UUID & id, const UpdateFunc & update_
if (!entry.entity)
entry.entity = readAccessEntityFromDisk(id);
auto old_entity = entry.entity;
auto new_entity = update_func(old_entity);
auto new_entity = update_func(old_entity, id);
if (!new_entity->isTypeOf(old_entity->getType()))
throwBadCast(id, new_entity->getType(), new_entity->getName(), old_entity->getType());
@ -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);
});
}
}

View File

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

View File

@ -176,6 +176,16 @@ std::vector<UUID> GrantedRoles::findDependencies() const
return res;
}
bool GrantedRoles::hasDependencies(const std::unordered_set<UUID> & ids) const
{
for (const auto & role_id : roles)
{
if (ids.contains(role_id))
return true;
}
return false;
}
void GrantedRoles::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
std::vector<UUID> new_ids;
@ -221,4 +231,56 @@ void GrantedRoles::replaceDependencies(const std::unordered_map<UUID, UUID> & ol
}
}
void GrantedRoles::copyDependenciesFrom(const GrantedRoles & src, const std::unordered_set<UUID> & ids)
{
bool found = false;
for (const auto & role_id : src.roles)
{
if (ids.contains(role_id))
{
roles.emplace(role_id);
found = true;
}
}
if (found)
{
for (const auto & role_id : src.roles_with_admin_option)
{
if (ids.contains(role_id))
roles_with_admin_option.emplace(role_id);
}
}
}
void GrantedRoles::removeDependencies(const std::unordered_set<UUID> & ids)
{
bool found = false;
for (auto it = roles.begin(); it != roles.end();)
{
if (ids.contains(*it))
{
it = roles.erase(it);
found = true;
}
else
{
++it;
}
}
if (found)
{
for (auto it = roles_with_admin_option.begin(); it != roles_with_admin_option.end();)
{
if (ids.contains(*it))
it = roles_with_admin_option.erase(it);
else
++it;
}
}
}
}

View File

@ -58,7 +58,10 @@ public:
friend bool operator !=(const GrantedRoles & left, const GrantedRoles & right) { return !(left == right); }
std::vector<UUID> findDependencies() const;
bool hasDependencies(const std::unordered_set<UUID> & ids) const;
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids);
void copyDependenciesFrom(const GrantedRoles & src, const std::unordered_set<UUID> & ids);
void removeDependencies(const std::unordered_set<UUID> & ids);
private:
boost::container::flat_set<UUID> roles;

View File

@ -48,9 +48,12 @@ struct IAccessEntity
/// Finds all dependencies.
virtual std::vector<UUID> findDependencies() const { return {}; }
virtual bool hasDependencies(const std::unordered_set<UUID> & /* ids */) const { return false; }
/// Replaces dependencies according to a specified map.
virtual void replaceDependencies(const std::unordered_map<UUID, UUID> & /* old_to_new_ids */) {}
virtual void copyDependenciesFrom(const IAccessEntity & /* src */, const std::unordered_set<UUID> & /* ids */) {}
virtual void removeDependencies(const std::unordered_set<UUID> & /* ids */) {}
/// Whether this access entity should be written to a backup.
virtual bool isBackupAllowed() const { return false; }
@ -66,6 +69,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>;

View File

@ -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,22 @@ void IAccessStorage::backup(BackupEntriesCollector & backup_entries_collector, c
}
void IAccessStorage::restoreFromBackup(RestorerFromBackup &)
void IAccessStorage::restoreFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup)
{
if (!isRestoreAllowed())
throwRestoreNotAllowed();
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "restoreFromBackup() is not implemented in {}", getStorageType());
if (isReplicated() && !acquireReplicatedRestore(restorer))
return;
restorer.addDataRestoreTask(
[this, &restorer, data_path_in_backup]
{
auto entities = restorer.getAccessEntitiesToRestore(data_path_in_backup);
auto dependents = restorer.getAccessEntitiesToRestoreDependents(data_path_in_backup);
const auto & restore_settings = restorer.getRestoreSettings();
restoreAccessEntitiesFromBackup(*this, entities, dependents, restore_settings);
});
}

View File

@ -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);
@ -177,7 +180,7 @@ public:
/// Removes multiple entities from the storage. Returns the list of successfully dropped.
std::vector<UUID> tryRemove(const std::vector<UUID> & ids);
using UpdateFunc = std::function<AccessEntityPtr(const AccessEntityPtr &)>;
using UpdateFunc = std::function<AccessEntityPtr(const AccessEntityPtr &, const UUID &)>;
/// Updates an entity stored in the storage. Throws an exception if couldn't update.
bool update(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists = true);
@ -211,14 +214,14 @@ public:
/// Makes a backup of this access storage.
virtual void backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const;
virtual void restoreFromBackup(RestorerFromBackup & restorer);
virtual void restoreFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup);
protected:
virtual std::optional<UUID> findImpl(AccessEntityType type, const String & name) 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 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);

View File

@ -163,7 +163,7 @@ void LDAPAccessStorage::applyRoleChangeNoLock(bool grant, const UUID & role_id,
// Update the granted roles of the relevant users.
if (!user_ids.empty())
{
auto update_func = [&role_id, &grant] (const AccessEntityPtr & entity_) -> AccessEntityPtr
auto update_func = [&role_id, &grant] (const AccessEntityPtr & entity_, const UUID &) -> AccessEntityPtr
{
if (auto user = typeid_cast<std::shared_ptr<const User>>(entity_))
{
@ -301,7 +301,7 @@ void LDAPAccessStorage::updateAssignedRolesNoLock(const UUID & id, const String
if (it != external_role_hashes.end() && it->second == external_roles_hash)
return;
auto update_func = [this, &external_roles, external_roles_hash] (const AccessEntityPtr & entity_) -> AccessEntityPtr
auto update_func = [this, &external_roles, external_roles_hash] (const AccessEntityPtr & entity_, const UUID &) -> AccessEntityPtr
{
if (auto user = typeid_cast<std::shared_ptr<const User>>(entity_))
{

View File

@ -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.
@ -194,7 +204,7 @@ bool MemoryAccessStorage::updateNoLock(const UUID & id, const UpdateFunc & updat
Entry & entry = it->second;
auto old_entity = entry.entity;
auto new_entity = update_func(old_entity);
auto new_entity = update_func(old_entity, id);
if (!new_entity->isTypeOf(old_entity->getType()))
throwBadCast(id, new_entity->getType(), new_entity->getName(), old_entity->getType());
@ -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);
}
}

View File

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

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;
@ -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);
@ -416,7 +416,7 @@ bool MultipleAccessStorage::updateImpl(const UUID & id, const UpdateFunc & updat
{
if (auto old_entity = storage_for_updating->tryRead(id))
{
auto new_entity = update_func(old_entity);
auto new_entity = update_func(old_entity, id);
if (new_entity->getName() != old_entity->getName())
{
for (const auto & storage : *storages)
@ -508,7 +508,7 @@ void MultipleAccessStorage::backup(BackupEntriesCollector & backup_entries_colle
throwBackupNotAllowed();
}
void MultipleAccessStorage::restoreFromBackup(RestorerFromBackup & restorer)
void MultipleAccessStorage::restoreFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup)
{
auto storages = getStoragesInternal();
@ -516,7 +516,7 @@ void MultipleAccessStorage::restoreFromBackup(RestorerFromBackup & restorer)
{
if (storage->isRestoreAllowed())
{
storage->restoreFromBackup(restorer);
storage->restoreFromBackup(restorer, data_path_in_backup);
return;
}
}

View File

@ -59,7 +59,7 @@ public:
bool isBackupAllowed() const override;
bool isRestoreAllowed() const override;
void backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const override;
void restoreFromBackup(RestorerFromBackup & restorer) override;
void restoreFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup) override;
bool containsStorage(std::string_view storage_type) const;
protected:
@ -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;

View File

@ -24,9 +24,27 @@ std::vector<UUID> Quota::findDependencies() const
return to_roles.findDependencies();
}
bool Quota::hasDependencies(const std::unordered_set<UUID> & ids) const
{
return to_roles.hasDependencies(ids);
}
void Quota::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
to_roles.replaceDependencies(old_to_new_ids);
}
void Quota::copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids)
{
if (getType() != src.getType())
return;
const auto & src_quota = typeid_cast<const Quota &>(src);
to_roles.copyDependenciesFrom(src_quota.to_roles, ids);
}
void Quota::removeDependencies(const std::unordered_set<UUID> & ids)
{
to_roles.removeDependencies(ids);
}
}

View File

@ -47,7 +47,11 @@ struct Quota : public IAccessEntity
AccessEntityType getType() const override { return TYPE; }
std::vector<UUID> findDependencies() const override;
bool hasDependencies(const std::unordered_set<UUID> & ids) const override;
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
void copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids) override;
void removeDependencies(const std::unordered_set<UUID> & ids) override;
bool isBackupAllowed() const override { return true; }
};

View File

@ -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);
}
}
@ -334,7 +359,7 @@ bool ReplicatedAccessStorage::updateZooKeeper(const zkutil::ZooKeeperPtr & zooke
}
const AccessEntityPtr old_entity = deserializeAccessEntity(old_entity_definition, entity_path);
const AccessEntityPtr new_entity = update_func(old_entity);
const AccessEntityPtr new_entity = update_func(old_entity, id);
if (!new_entity->isTypeOf(old_entity->getType()))
throwBadCast(id, new_entity->getType(), new_entity->getName(), old_entity->getType());
@ -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);
}
}

View File

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

View File

@ -21,10 +21,30 @@ std::vector<UUID> Role::findDependencies() const
return res;
}
bool Role::hasDependencies(const std::unordered_set<UUID> & ids) const
{
return granted_roles.hasDependencies(ids) || settings.hasDependencies(ids);
}
void Role::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
granted_roles.replaceDependencies(old_to_new_ids);
settings.replaceDependencies(old_to_new_ids);
}
void Role::copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids)
{
if (getType() != src.getType())
return;
const auto & src_role = typeid_cast<const Role &>(src);
granted_roles.copyDependenciesFrom(src_role.granted_roles, ids);
settings.copyDependenciesFrom(src_role.settings, ids);
}
void Role::removeDependencies(const std::unordered_set<UUID> & ids)
{
granted_roles.removeDependencies(ids);
settings.removeDependencies(ids);
}
}

View File

@ -21,7 +21,11 @@ struct Role : public IAccessEntity
AccessEntityType getType() const override { return TYPE; }
std::vector<UUID> findDependencies() const override;
bool hasDependencies(const std::unordered_set<UUID> & ids) const override;
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
void copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids) override;
void removeDependencies(const std::unordered_set<UUID> & ids) override;
bool isBackupAllowed() const override { return settings.isBackupAllowed(); }
};

View File

@ -295,6 +295,23 @@ std::vector<UUID> RolesOrUsersSet::findDependencies() const
return res;
}
bool RolesOrUsersSet::hasDependencies(const std::unordered_set<UUID> & dependencies_ids) const
{
for (const auto & id : ids)
{
if (dependencies_ids.contains(id))
return true;
}
for (const auto & id : except_ids)
{
if (dependencies_ids.contains(id))
return true;
}
return false;
}
void RolesOrUsersSet::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
std::vector<UUID> new_ids;
@ -337,4 +354,41 @@ void RolesOrUsersSet::replaceDependencies(const std::unordered_map<UUID, UUID> &
boost::range::copy(new_ids, std::inserter(except_ids, except_ids.end()));
}
void RolesOrUsersSet::copyDependenciesFrom(const RolesOrUsersSet & src, const std::unordered_set<UUID> & dependencies_ids)
{
if (all != src.all)
return;
for (const auto & id : src.ids)
{
if (dependencies_ids.contains(id))
ids.emplace(id);
}
for (const auto & id : src.except_ids)
{
if (dependencies_ids.contains(id))
except_ids.emplace(id);
}
}
void RolesOrUsersSet::removeDependencies(const std::unordered_set<UUID> & dependencies_ids)
{
for (auto it = ids.begin(); it != ids.end();)
{
if (dependencies_ids.contains(*it))
it = ids.erase(it);
else
++it;
}
for (auto it = except_ids.begin(); it != except_ids.end();)
{
if (dependencies_ids.contains(*it))
except_ids.erase(it);
else
++it;
}
}
}

View File

@ -64,7 +64,10 @@ struct RolesOrUsersSet
friend bool operator !=(const RolesOrUsersSet & lhs, const RolesOrUsersSet & rhs) { return !(lhs == rhs); }
std::vector<UUID> findDependencies() const;
bool hasDependencies(const std::unordered_set<UUID> & dependencies_ids) const;
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids);
void copyDependenciesFrom(const RolesOrUsersSet & src, const std::unordered_set<UUID> & dependencies_ids);
void removeDependencies(const std::unordered_set<UUID> & dependencies_ids);
bool all = false;
boost::container::flat_set<UUID> ids;

View File

@ -63,9 +63,27 @@ std::vector<UUID> RowPolicy::findDependencies() const
return to_roles.findDependencies();
}
bool RowPolicy::hasDependencies(const std::unordered_set<UUID> & ids) const
{
return to_roles.hasDependencies(ids);
}
void RowPolicy::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
to_roles.replaceDependencies(old_to_new_ids);
}
void RowPolicy::copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids)
{
if (getType() != src.getType())
return;
const auto & src_policy = typeid_cast<const RowPolicy &>(src);
to_roles.copyDependenciesFrom(src_policy.to_roles, ids);
}
void RowPolicy::removeDependencies(const std::unordered_set<UUID> & ids)
{
to_roles.removeDependencies(ids);
}
}

View File

@ -50,7 +50,11 @@ struct RowPolicy : public IAccessEntity
AccessEntityType getType() const override { return TYPE; }
std::vector<UUID> findDependencies() const override;
bool hasDependencies(const std::unordered_set<UUID> & ids) const override;
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
void copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids) override;
void removeDependencies(const std::unordered_set<UUID> & ids) override;
bool isBackupAllowed() const override { return true; }
/// Which roles or users should use this row policy.

View File

@ -21,10 +21,30 @@ std::vector<UUID> SettingsProfile::findDependencies() const
return res;
}
bool SettingsProfile::hasDependencies(const std::unordered_set<UUID> & ids) const
{
return elements.hasDependencies(ids) || to_roles.hasDependencies(ids);
}
void SettingsProfile::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
elements.replaceDependencies(old_to_new_ids);
to_roles.replaceDependencies(old_to_new_ids);
}
void SettingsProfile::copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids)
{
if (getType() != src.getType())
return;
const auto & src_profile = typeid_cast<const SettingsProfile &>(src);
elements.copyDependenciesFrom(src_profile.elements, ids);
to_roles.copyDependenciesFrom(src_profile.to_roles, ids);
}
void SettingsProfile::removeDependencies(const std::unordered_set<UUID> & ids)
{
elements.removeDependencies(ids);
to_roles.removeDependencies(ids);
}
}

View File

@ -22,7 +22,11 @@ struct SettingsProfile : public IAccessEntity
AccessEntityType getType() const override { return TYPE; }
std::vector<UUID> findDependencies() const override;
bool hasDependencies(const std::unordered_set<UUID> & ids) const override;
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
void copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids) override;
void removeDependencies(const std::unordered_set<UUID> & ids) override;
bool isBackupAllowed() const override { return elements.isBackupAllowed(); }
};

View File

@ -158,6 +158,18 @@ std::vector<UUID> SettingsProfileElements::findDependencies() const
}
bool SettingsProfileElements::hasDependencies(const std::unordered_set<UUID> & ids) const
{
std::vector<UUID> res;
for (const auto & element : *this)
{
if (element.parent_profile && ids.contains(*element.parent_profile))
return true;
}
return false;
}
void SettingsProfileElements::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
for (auto & element : *this)
@ -176,6 +188,29 @@ void SettingsProfileElements::replaceDependencies(const std::unordered_map<UUID,
}
void SettingsProfileElements::copyDependenciesFrom(const SettingsProfileElements & src, const std::unordered_set<UUID> & ids)
{
SettingsProfileElements new_elements;
for (const auto & element : src)
{
if (element.parent_profile && ids.contains(*element.parent_profile))
{
SettingsProfileElement new_element;
new_element.parent_profile = *element.parent_profile;
new_elements.emplace_back(new_element);
}
}
insert(begin(), new_elements.begin(), new_elements.end());
}
void SettingsProfileElements::removeDependencies(const std::unordered_set<UUID> & ids)
{
std::erase_if(
*this, [&](const SettingsProfileElement & element) { return element.parent_profile && ids.contains(*element.parent_profile); });
}
void SettingsProfileElements::merge(const SettingsProfileElements & other)
{
insert(end(), other.begin(), other.end());

View File

@ -63,7 +63,10 @@ public:
std::shared_ptr<ASTSettingsProfileElements> toASTWithNames(const AccessControl & access_control) const;
std::vector<UUID> findDependencies() const;
bool hasDependencies(const std::unordered_set<UUID> & ids) const;
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids);
void copyDependenciesFrom(const SettingsProfileElements & src, const std::unordered_set<UUID> & ids);
void removeDependencies(const std::unordered_set<UUID> & ids);
void merge(const SettingsProfileElements & other);

View File

@ -49,6 +49,11 @@ std::vector<UUID> User::findDependencies() const
return res;
}
bool User::hasDependencies(const std::unordered_set<UUID> & ids) const
{
return default_roles.hasDependencies(ids) || granted_roles.hasDependencies(ids) || grantees.hasDependencies(ids) || settings.hasDependencies(ids);
}
void User::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
default_roles.replaceDependencies(old_to_new_ids);
@ -57,4 +62,23 @@ void User::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new
settings.replaceDependencies(old_to_new_ids);
}
void User::copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids)
{
if (getType() != src.getType())
return;
const auto & src_user = typeid_cast<const User &>(src);
default_roles.copyDependenciesFrom(src_user.default_roles, ids);
granted_roles.copyDependenciesFrom(src_user.granted_roles, ids);
grantees.copyDependenciesFrom(src_user.grantees, ids);
settings.copyDependenciesFrom(src_user.settings, ids);
}
void User::removeDependencies(const std::unordered_set<UUID> & ids)
{
default_roles.removeDependencies(ids);
granted_roles.removeDependencies(ids);
grantees.removeDependencies(ids);
settings.removeDependencies(ids);
}
}

View File

@ -32,7 +32,11 @@ struct User : public IAccessEntity
void setName(const String & name_) override;
std::vector<UUID> findDependencies() const override;
bool hasDependencies(const std::unordered_set<UUID> & ids) const override;
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
void copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids) override;
void removeDependencies(const std::unordered_set<UUID> & ids) override;
bool isBackupAllowed() const override { return settings.isBackupAllowed(); }
};

View File

@ -32,6 +32,9 @@ namespace BackupCoordinationStage
/// Finding databases and tables in the backup which we're going to restore.
constexpr const char * FINDING_TABLES_IN_BACKUP = "finding tables in backup";
/// Loading system access tables and then checking if the current user has enough access to restore.
constexpr const char * CHECKING_ACCESS_RIGHTS = "checking access rights";
/// Creating databases or finding them and checking their definitions.
constexpr const char * CREATING_DATABASES = "creating databases";

View File

@ -160,7 +160,8 @@ namespace
M(UInt64, replica_num_in_backup) \
M(Bool, allow_non_empty_tables) \
M(RestoreAccessCreationMode, create_access) \
M(Bool, allow_unresolved_access_dependencies) \
M(Bool, skip_unresolved_access_entities_dependencies) \
M(Bool, update_access_entities_dependents) \
M(RestoreUDFCreationMode, create_function) \
M(Bool, allow_s3_native_copy) \
M(Bool, use_same_s3_credentials_for_base_backup) \
@ -187,7 +188,12 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query)
else
LIST_OF_RESTORE_SETTINGS(GET_SETTINGS_FROM_RESTORE_QUERY_HELPER)
throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.name);
/// `allow_unresolved_access_dependencies` is an obsolete name.
if (setting.name == "allow_unresolved_access_dependencies")
res.skip_unresolved_access_entities_dependencies = SettingFieldBool{setting.value}.value;
else
throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.name);
}
}

View File

@ -100,9 +100,25 @@ struct RestoreSettings
/// How the RESTORE command will handle if an user (or role or profile) which it's going to restore already exists.
RestoreAccessCreationMode create_access = RestoreAccessCreationMode::kCreateIfNotExists;
/// Skip dependencies of access entities which can't be resolved.
/// For example, if an user has a profile assigned and that profile is not in the backup and doesn't exist locally.
bool allow_unresolved_access_dependencies = false;
/// Ignore dependencies of access entities which can't be resolved.
/// For example: if a backup contains a profile assigned to a user: `CREATE PROFILE p1; CREATE USER u1 SETTINGS PROFILE p1`
/// and now we're restoring only user `u1` and profile `p1` doesn't exists, then
/// this flag is whether RESTORE should continue with restoring user `u1` without assigning profile `p1`.
/// Another example: if a backup contains a role granted to a user: `CREATE USER u2; CREATE ROLE r2; GRANT r2 TO u2`
/// and now we're restoring only user `u2` and role `r2` doesn't exist, then
/// this flag is whether RESTORE should continue with restoring user `u2` without that grant.
/// If this flag is false then RESTORE will throw an exception in that case.
bool skip_unresolved_access_entities_dependencies = true;
/// Try to update dependents of restored access entities.
/// For example: if a backup contains a profile assigned to a user: `CREATE PROFILE p1; CREATE USER u1 SETTINGS PROFILE p1`
/// and now we're restoring only profile `p1` and user `u1` already exists, then
/// this flag is whether restored profile `p1` should be assigned to user `u1` again.
/// Another example, if a backup contains a role granted to a user: `CREATE USER u2; CREATE ROLE r2; GRANT r2 TO u2`
/// and now we're restoring only role `r2` and user `u2` already exists, then
/// this flag is whether restored role `r2` should be granted to user `u2` again.
/// If this flag is false then RESTORE won't update existing access entities.
bool update_access_entities_dependents = true;
/// How the RESTORE command will handle if a user-defined function which it's going to restore already exists.
RestoreUDFCreationMode create_function = RestoreUDFCreationMode::kCreateIfNotExists;

View File

@ -72,6 +72,12 @@ namespace
|| (table_name.table == "row_policies") || (table_name.table == "quotas");
}
const Strings & getAllSystemAccessTableNames()
{
static const Strings all_system_access_table_names{"users", "roles", "quotas", "settings_profiles", "row_policies"};
return all_system_access_table_names;
}
/// Whether a specified name corresponds one of the tables backuping ACL.
bool isSystemFunctionsTableName(const QualifiedTableName & table_name)
{
@ -136,6 +142,8 @@ void RestorerFromBackup::run(Mode mode)
waitFutures();
/// Check access rights.
setStage(Stage::CHECKING_ACCESS_RIGHTS);
loadSystemAccessTables();
checkAccessForObjectsFoundInBackup();
if (mode == Mode::CHECK_ACCESS_ONLY)
@ -482,25 +490,6 @@ void RestorerFromBackup::findTableInBackupImpl(const QualifiedTableName & table_
res_table_info.partitions.emplace();
insertAtEnd(*res_table_info.partitions, *partitions);
}
/// Special handling for ACL-related system tables.
if (!restore_settings.structure_only && isSystemAccessTableName(table_name))
{
if (!access_restorer)
access_restorer = std::make_unique<AccessRestorerFromBackup>(backup, restore_settings);
try
{
/// addDataPath() will parse access*.txt files and extract access entities from them.
/// We need to do that early because we need those access entities to check access.
access_restorer->addDataPath(data_path_in_backup);
}
catch (Exception & e)
{
e.addMessage("While parsing data of {} from backup", tableNameWithTypeToString(table_name.database, table_name.table, false));
throw;
}
}
}
void RestorerFromBackup::findDatabaseInBackup(const String & database_name_in_backup, const std::set<DatabaseAndTableName> & except_table_names)
@ -624,6 +613,58 @@ size_t RestorerFromBackup::getNumTables() const
return table_infos.size();
}
void RestorerFromBackup::loadSystemAccessTables()
{
if (restore_settings.structure_only)
return;
std::unordered_set<String> system_access_tables;
/// Special handling for ACL-related system tables.
std::lock_guard lock{mutex};
for (const auto & [table_name, table_info] : table_infos)
{
if (isSystemAccessTableName(table_name))
{
if (!access_restorer)
access_restorer = std::make_unique<AccessRestorerFromBackup>(backup, restore_settings);
access_restorer->addDataPath(table_info.data_path_in_backup, /* dependents_only = */ false);
system_access_tables.emplace(table_name.table);
}
}
if (!access_restorer)
return;
if (restore_settings.update_access_entities_dependents)
{
/// Collect dependents from all the system access tables, even from those ones which we aren't restoring at the moment.
for (const auto & table_name : getAllSystemAccessTableNames())
{
if (!system_access_tables.contains(table_name))
{
std::optional<fs::path> root_path_in_use;
for (const auto & root_path_in_backup : root_paths_in_backup)
{
fs::path try_metadata_path = root_path_in_backup / "metadata" / DatabaseCatalog::SYSTEM_DATABASE / (escapeForFileName(table_name) + ".sql");
if (backup->fileExists(try_metadata_path))
{
root_path_in_use = root_path_in_backup;
break;
}
}
if (root_path_in_use)
{
String data_path_in_backup = *root_path_in_use / "data" / DatabaseCatalog::SYSTEM_DATABASE / escapeForFileName(table_name);
access_restorer->addDataPath(data_path_in_backup, /* dependents_only = */ true);
}
}
}
}
access_restorer->loadFromBackup();
}
void RestorerFromBackup::checkAccessForObjectsFoundInBackup() const
{
AccessRightsElements required_access;
@ -708,6 +749,24 @@ void RestorerFromBackup::checkAccessForObjectsFoundInBackup() const
context->checkAccess(required_access);
}
std::vector<std::pair<UUID, AccessEntityPtr>> RestorerFromBackup::getAccessEntitiesToRestore(const String & data_path_in_backup) const
{
std::lock_guard lock{mutex};
if (!access_restorer)
return {};
access_restorer->generateRandomIDsAndResolveDependencies(context->getAccessControl());
return access_restorer->getEntities(data_path_in_backup);
}
std::vector<std::pair<UUID, AccessEntityPtr>> RestorerFromBackup::getAccessEntitiesToRestoreDependents(const String & data_path_in_backup) const
{
std::lock_guard lock{mutex};
if (!access_restorer)
return {};
access_restorer->generateRandomIDsAndResolveDependencies(context->getAccessControl());
return access_restorer->getDependents(data_path_in_backup);
}
void RestorerFromBackup::createDatabases()
{
Strings database_names;
@ -1066,19 +1125,6 @@ void RestorerFromBackup::runDataRestoreTasks()
}
}
std::vector<std::pair<UUID, AccessEntityPtr>> RestorerFromBackup::getAccessEntitiesToRestore()
{
std::lock_guard lock{mutex};
if (!access_restorer || access_restored)
return {};
/// getAccessEntitiesToRestore() will return entities only when called first time (we don't want to restore the same entities again).
access_restored = true;
return access_restorer->getAccessEntities(context->getAccessControl());
}
void RestorerFromBackup::throwTableIsNotEmpty(const StorageID & storage_id)
{
throw Exception(

View File

@ -68,7 +68,8 @@ public:
void addDataRestoreTasks(DataRestoreTasks && new_tasks);
/// Returns the list of access entities to restore.
std::vector<std::pair<UUID, AccessEntityPtr>> getAccessEntitiesToRestore();
std::vector<std::pair<UUID, AccessEntityPtr>> getAccessEntitiesToRestore(const String & data_path_in_backup) const;
std::vector<std::pair<UUID, AccessEntityPtr>> getAccessEntitiesToRestoreDependents(const String & data_path_in_backup) const;
/// Throws an exception that a specified table is already non-empty.
[[noreturn]] static void throwTableIsNotEmpty(const StorageID & storage_id);
@ -101,6 +102,7 @@ private:
size_t getNumDatabases() const;
size_t getNumTables() const;
void loadSystemAccessTables();
void checkAccessForObjectsFoundInBackup() const;
void createDatabases();

View File

@ -111,7 +111,7 @@ BlockIO InterpreterCreateQuotaQuery::execute()
if (query.alter)
{
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
auto update_func = [&](const AccessEntityPtr & entity, const UUID &) -> AccessEntityPtr
{
auto updated_quota = typeid_cast<std::shared_ptr<Quota>>(entity->clone());
updateQuotaFromQueryImpl(*updated_quota, query, {}, roles_from_query);

View File

@ -74,7 +74,7 @@ BlockIO InterpreterCreateRoleQuery::execute()
if (query.alter)
{
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
auto update_func = [&](const AccessEntityPtr & entity, const UUID &) -> AccessEntityPtr
{
auto updated_role = typeid_cast<std::shared_ptr<Role>>(entity->clone());
updateRoleFromQueryImpl(*updated_role, query, {}, settings_from_query);

View File

@ -88,7 +88,7 @@ BlockIO InterpreterCreateRowPolicyQuery::execute()
Strings names = query.names->toStrings();
if (query.alter)
{
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
auto update_func = [&](const AccessEntityPtr & entity, const UUID &) -> AccessEntityPtr
{
auto updated_policy = typeid_cast<std::shared_ptr<RowPolicy>>(entity->clone());
updateRowPolicyFromQueryImpl(*updated_policy, query, {}, roles_from_query);

View File

@ -90,7 +90,7 @@ BlockIO InterpreterCreateSettingsProfileQuery::execute()
if (query.alter)
{
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
auto update_func = [&](const AccessEntityPtr & entity, const UUID &) -> AccessEntityPtr
{
auto updated_profile = typeid_cast<std::shared_ptr<SettingsProfile>>(entity->clone());
updateSettingsProfileFromQueryImpl(*updated_profile, query, {}, settings_from_query, roles_from_query);

View File

@ -264,7 +264,7 @@ BlockIO InterpreterCreateUserQuery::execute()
if (query.grantees)
grantees_from_query = RolesOrUsersSet{*query.grantees, access_control};
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
auto update_func = [&](const AccessEntityPtr & entity, const UUID &) -> AccessEntityPtr
{
auto updated_user = typeid_cast<std::shared_ptr<User>>(entity->clone());
updateUserFromQueryImpl(
@ -317,7 +317,7 @@ BlockIO InterpreterCreateUserQuery::execute()
if (query.grantees)
{
RolesOrUsersSet grantees_from_query = RolesOrUsersSet{*query.grantees, access_control};
access_control.update(ids, [&](const AccessEntityPtr & entity) -> AccessEntityPtr
access_control.update(ids, [&](const AccessEntityPtr & entity, const UUID &) -> AccessEntityPtr
{
auto updated_user = typeid_cast<std::shared_ptr<User>>(entity->clone());
updated_user->grantees = grantees_from_query;

View File

@ -474,7 +474,7 @@ BlockIO InterpreterGrantQuery::execute()
calculateCurrentGrantRightsWithIntersection(new_rights, current_user_access, elements_to_grant);
/// Update roles and users listed in `grantees`.
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
auto update_func = [&](const AccessEntityPtr & entity, const UUID &) -> AccessEntityPtr
{
auto clone = entity->clone();
if (query.current_grants)

View File

@ -46,7 +46,7 @@ void InterpreterSetRoleQuery::setDefaultRole(const ASTSetRoleQuery & query)
std::vector<UUID> to_users = RolesOrUsersSet{*query.to_users, access_control, getContext()->getUserID()}.getMatchingIDs(access_control);
RolesOrUsersSet roles_from_query{*query.roles, access_control};
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
auto update_func = [&](const AccessEntityPtr & entity, const UUID &) -> AccessEntityPtr
{
auto updated_user = typeid_cast<std::shared_ptr<User>>(entity->clone());
updateUserSetDefaultRoles(*updated_user, roles_from_query);

View File

@ -150,10 +150,10 @@ void StorageSystemQuotas::backupData(
}
void StorageSystemQuotas::restoreDataFromBackup(
RestorerFromBackup & restorer, const String & /* data_path_in_backup */, const std::optional<ASTs> & /* partitions */)
RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
{
auto & access_control = restorer.getContext()->getAccessControl();
access_control.restoreFromBackup(restorer);
access_control.restoreFromBackup(restorer, data_path_in_backup);
}
}

View File

@ -70,10 +70,10 @@ void StorageSystemRoles::backupData(
}
void StorageSystemRoles::restoreDataFromBackup(
RestorerFromBackup & restorer, const String & /* data_path_in_backup */, const std::optional<ASTs> & /* partitions */)
RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
{
auto & access_control = restorer.getContext()->getAccessControl();
access_control.restoreFromBackup(restorer);
access_control.restoreFromBackup(restorer, data_path_in_backup);
}
}

View File

@ -160,10 +160,10 @@ void StorageSystemRowPolicies::backupData(
}
void StorageSystemRowPolicies::restoreDataFromBackup(
RestorerFromBackup & restorer, const String & /* data_path_in_backup */, const std::optional<ASTs> & /* partitions */)
RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
{
auto & access_control = restorer.getContext()->getAccessControl();
access_control.restoreFromBackup(restorer);
access_control.restoreFromBackup(restorer, data_path_in_backup);
}
}

View File

@ -101,10 +101,10 @@ void StorageSystemSettingsProfiles::backupData(
}
void StorageSystemSettingsProfiles::restoreDataFromBackup(
RestorerFromBackup & restorer, const String & /* data_path_in_backup */, const std::optional<ASTs> & /* partitions */)
RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
{
auto & access_control = restorer.getContext()->getAccessControl();
access_control.restoreFromBackup(restorer);
access_control.restoreFromBackup(restorer, data_path_in_backup);
}
}

View File

@ -261,10 +261,10 @@ void StorageSystemUsers::backupData(
}
void StorageSystemUsers::restoreDataFromBackup(
RestorerFromBackup & restorer, const String & /* data_path_in_backup */, const std::optional<ASTs> & /* partitions */)
RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
{
auto & access_control = restorer.getContext()->getAccessControl();
access_control.restoreFromBackup(restorer);
access_control.restoreFromBackup(restorer, data_path_in_backup);
}
}

View File

@ -0,0 +1,13 @@
Everything dropped
User dropped
Role dropped
Nothing dropped
Nothing dropped, mode=replace
Nothing dropped, mode=create
ACCESS_ENTITY_ALREADY_EXISTS
Everything dropped, restore system.roles, then system.users
user_a 0
role_b 1
Everything dropped, restore system.users, then system.roles
user_a 1
role_b 0

View File

@ -0,0 +1,93 @@
#!/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 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
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
echo "Everything dropped, restore system.roles, then system.users"
${CLICKHOUSE_CLIENT} --query "DROP USER ${user_a}"
${CLICKHOUSE_CLIENT} --query "DROP ROLE ${role_b}"
${CLICKHOUSE_CLIENT} --query "RESTORE TABLE system.roles FROM ${backup_name} FORMAT Null"
${CLICKHOUSE_CLIENT} --query "SELECT 'user_a', count() FROM system.users WHERE name = '${user_a}'"
${CLICKHOUSE_CLIENT} --query "SELECT 'role_b', count() FROM system.roles WHERE name = '${role_b}'"
${CLICKHOUSE_CLIENT} --query "RESTORE TABLE system.users FROM ${backup_name} FORMAT Null"
do_check
# RESTORE with "update_access_entities_dependents=true" must collect dependents from all the system access tables,
# even from those ones which we aren't restoring at the moment.
echo "Everything dropped, restore system.users, then system.roles"
${CLICKHOUSE_CLIENT} --query "DROP USER ${user_a}"
${CLICKHOUSE_CLIENT} --query "DROP ROLE ${role_b}"
${CLICKHOUSE_CLIENT} --query "RESTORE TABLE system.users FROM ${backup_name} FORMAT Null"
${CLICKHOUSE_CLIENT} --query "SELECT 'user_a', count() FROM system.users WHERE name = '${user_a}'"
${CLICKHOUSE_CLIENT} --query "SELECT 'role_b', count() FROM system.roles WHERE name = '${role_b}'"
${CLICKHOUSE_CLIENT} --query "RESTORE TABLE system.roles FROM ${backup_name} FORMAT Null"
do_check