Support writing dependents of access entities to backup, added backup setting "write_access_entities_dependents".

This commit is contained in:
Vitaly Baranov 2024-09-18 13:13:24 +02:00
parent 995b5f7d4b
commit 7ff8bf2958
22 changed files with 227 additions and 95 deletions

View File

@ -42,6 +42,7 @@ namespace
{
std::unordered_map<UUID, AccessEntityPtr> entities;
std::unordered_map<UUID, std::pair<String, AccessEntityType>> dependencies;
std::unordered_map<UUID, AccessEntityPtr> dependents;
BackupEntryPtr toBackupEntry() const
{
@ -73,6 +74,24 @@ namespace
}
}
if (!dependents.empty())
{
if (!dependencies.empty())
writeText("\n", buf);
writeText("DEPENDENTS\n", buf);
for (const auto & [id, entity] : dependents)
{
writeText(id, buf);
writeChar('\t', buf);
writeText(entity->getTypeInfo().name, buf);
writeChar('\t', buf);
writeText(entity->getName(), buf);
writeChar('\n', buf);
writeText(serializeAccessEntity(*entity), buf);
writeChar('\n', buf);
}
}
return std::make_shared<BackupEntryFromMemory>(buf.str());
}
@ -82,59 +101,71 @@ namespace
{
AccessEntitiesInBackup res;
bool dependencies_found = false;
bool reading_dependencies = false;
bool reading_dependents = false;
while (!buf->eof())
{
String line;
readStringUntilNewlineInto(line, *buf);
buf->ignore();
if (line == "DEPENDENCIES")
{
dependencies_found = true;
break;
reading_dependencies = true;
reading_dependents = false;
continue;
}
else if (line == "DEPENDENTS")
{
reading_dependents = true;
reading_dependencies = false;
continue;
}
else if (line.empty())
{
continue;
}
UUID id = parse<UUID>(line.substr(0, line.find('\t')));
line.clear();
size_t separator1 = line.find('\t');
size_t separator2 = line.find('\t', separator1 + 1);
if ((separator1 == String::npos) || (separator2 == String::npos))
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Separators not found in line {}", line);
String queries;
while (!buf->eof())
UUID id = parse<UUID>(line.substr(0, separator1));
AccessEntityType type = AccessEntityTypeInfo::parseType(line.substr(separator1 + 1, separator2 - separator1 - 1));
String name = line.substr(separator2 + 1);
if (reading_dependencies)
{
String query;
readStringUntilNewlineInto(query, *buf);
buf->ignore();
if (query.empty())
break;
if (!queries.empty())
queries.append("\n");
queries.append(query);
res.dependencies.emplace(id, std::pair{name, type});
}
AccessEntityPtr entity = deserializeAccessEntity(queries);
res.entities.emplace(id, entity);
}
if (dependencies_found)
{
while (!buf->eof())
else
{
String id_as_string;
readStringInto(id_as_string, *buf);
buf->ignore();
UUID id = parse<UUID>(id_as_string);
String queries;
while (!buf->eof())
{
String query;
readStringUntilNewlineInto(query, *buf);
buf->ignore();
if (query.empty())
break;
if (!queries.empty())
queries.append("\n");
queries.append(query);
}
String type_as_string;
readStringInto(type_as_string, *buf);
buf->ignore();
AccessEntityType type = AccessEntityTypeInfo::parseType(type_as_string);
AccessEntityPtr entity = deserializeAccessEntity(queries);
String name;
readStringInto(name, *buf);
buf->ignore();
if (name != entity->getName())
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Unexpected name {} is specified for {}", name, entity->formatTypeWithName());
if (type != entity->getType())
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Unexpected type {} is specified for {}", AccessEntityTypeInfo::get(type).name, entity->formatTypeWithName());
if (!res.entities.contains(id))
res.dependencies.emplace(id, std::pair{name, type});
if (reading_dependents)
res.dependents.emplace(id, entity);
else
res.entities.emplace(id, entity);
}
}
@ -147,47 +178,58 @@ namespace
}
}
};
std::vector<UUID> findDependencies(const std::vector<std::pair<UUID, AccessEntityPtr>> & entities)
{
std::vector<UUID> res;
for (const auto & entity : entities | boost::adaptors::map_values)
insertAtEnd(res, entity->findDependencies());
/// Remove duplicates in the list of dependencies (some entities can refer to other entities).
::sort(res.begin(), res.end());
res.erase(std::unique(res.begin(), res.end()), res.end());
for (const auto & id : entities | boost::adaptors::map_keys)
{
auto it = std::lower_bound(res.begin(), res.end(), id);
if ((it != res.end()) && (*it == id))
res.erase(it);
}
return res;
}
std::unordered_map<UUID, std::pair<String, AccessEntityType>> readDependenciesNamesAndTypes(const std::vector<UUID> & dependencies, const AccessControl & access_control)
{
std::unordered_map<UUID, std::pair<String, AccessEntityType>> res;
for (const auto & id : dependencies)
{
if (auto name_and_type = access_control.tryReadNameWithType(id))
res.emplace(id, name_and_type.value());
}
return res;
}
}
std::pair<String, BackupEntryPtr> makeBackupEntryForAccessEntities(
const std::vector<std::pair<UUID, AccessEntityPtr>> & access_entities,
const String & data_path_in_backup,
const AccessControl & access_control)
const std::vector<UUID> & entities_ids,
const std::unordered_map<UUID, AccessEntityPtr> & all_entities,
bool write_dependents,
const String & data_path_in_backup)
{
auto dependencies = readDependenciesNamesAndTypes(findDependencies(access_entities), access_control);
AccessEntitiesInBackup ab;
boost::range::copy(access_entities, std::inserter(ab.entities, ab.entities.end()));
ab.dependencies = std::move(dependencies);
std::unordered_set<UUID> entities_ids_set;
for (const auto & id : entities_ids)
entities_ids_set.emplace(id);
for (const auto & id : entities_ids)
{
auto it = all_entities.find(id);
if (it != all_entities.end())
{
AccessEntityPtr entity = it->second;
ab.entities.emplace(id, entity);
auto dependencies = entity->findDependencies();
for (const auto & dependency_id : dependencies)
{
if (!entities_ids_set.contains(dependency_id))
{
auto it_dependency = all_entities.find(dependency_id);
if (it_dependency != all_entities.end())
{
auto dependency_entity = it_dependency->second;
ab.dependencies.emplace(dependency_id, std::make_pair(dependency_entity->getName(), dependency_entity->getType()));
}
}
}
}
}
if (write_dependents)
{
for (const auto & [id, possible_dependent] : all_entities)
{
if (!entities_ids_set.contains(id) && possible_dependent->hasDependencies(entities_ids_set))
{
auto dependent = possible_dependent->clone();
dependent->clearAllExceptDependencies();
ab.dependents.emplace(id, dependent);
}
}
}
String filename = fmt::format("access-{}.txt", UUIDHelpers::generateV4());
String file_path_in_backup = fs::path{data_path_in_backup} / filename;
return {file_path_in_backup, ab.toBackupEntry()};
@ -278,6 +320,18 @@ void AccessRestorerFromBackup::loadFromBackup()
EntityInfo & entity_info = it->second;
entity_info.is_dependency = true;
}
for (const auto & [id, entity] : ab.dependents)
{
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;
if (!entity_info.restore)
entity_info.entity = entity;
}
}
}

View File

@ -23,9 +23,10 @@ enum class RestoreAccessCreationMode : uint8_t;
/// Makes a backup entry for of a set of access entities.
std::pair<String, BackupEntryPtr> makeBackupEntryForAccessEntities(
const std::vector<std::pair<UUID, AccessEntityPtr>> & access_entities,
const String & data_path_in_backup,
const AccessControl & access_control);
const std::vector<UUID> & entities_ids,
const std::unordered_map<UUID, AccessEntityPtr> & all_entities,
bool write_dependents,
const String & data_path_in_backup);
/// Restores access entities from a backup.
void restoreAccessEntitiesFromBackup(

View File

@ -54,6 +54,7 @@ struct IAccessEntity
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 */) {}
virtual void clearAllExceptDependencies() {}
/// Whether this access entity should be written to a backup.
virtual bool isBackupAllowed() const { return false; }

View File

@ -613,16 +613,15 @@ void IAccessStorage::backup(BackupEntriesCollector & backup_entries_collector, c
if (!isBackupAllowed())
throwBackupNotAllowed();
auto entities = readAllWithIDs(type);
std::erase_if(entities, [](const std::pair<UUID, AccessEntityPtr> & x) { return !x.second->isBackupAllowed(); });
if (entities.empty())
auto entities_ids = findAll(type);
if (entities_ids.empty())
return;
auto backup_entry_with_path = makeBackupEntryForAccessEntities(
entities,
data_path_in_backup,
backup_entries_collector.getContext()->getAccessControl());
entities_ids,
backup_entries_collector.getAllAccessEntities(),
backup_entries_collector.getBackupSettings().write_access_entities_dependents,
data_path_in_backup);
if (isReplicated())
{

View File

@ -47,4 +47,10 @@ void Quota::removeDependencies(const std::unordered_set<UUID> & ids)
to_roles.removeDependencies(ids);
}
void Quota::clearAllExceptDependencies()
{
all_limits.clear();
key_type = QuotaKeyType::NONE;
}
}

View File

@ -51,6 +51,7 @@ struct Quota : public IAccessEntity
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;
void clearAllExceptDependencies() override;
bool isBackupAllowed() const override { return true; }
};

View File

@ -47,4 +47,10 @@ void Role::removeDependencies(const std::unordered_set<UUID> & ids)
settings.removeDependencies(ids);
}
void Role::clearAllExceptDependencies()
{
access = {};
settings.removeSettingsKeepProfiles();
}
}

View File

@ -25,6 +25,7 @@ struct Role : public IAccessEntity
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;
void clearAllExceptDependencies() override;
bool isBackupAllowed() const override { return settings.isBackupAllowed(); }
};

View File

@ -86,4 +86,10 @@ void RowPolicy::removeDependencies(const std::unordered_set<UUID> & ids)
to_roles.removeDependencies(ids);
}
void RowPolicy::clearAllExceptDependencies()
{
for (auto & filter : filters)
filter = {};
}
}

View File

@ -54,6 +54,7 @@ struct RowPolicy : public IAccessEntity
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;
void clearAllExceptDependencies() override;
bool isBackupAllowed() const override { return true; }

View File

@ -47,4 +47,9 @@ void SettingsProfile::removeDependencies(const std::unordered_set<UUID> & ids)
to_roles.removeDependencies(ids);
}
void SettingsProfile::clearAllExceptDependencies()
{
elements.removeSettingsKeepProfiles();
}
}

View File

@ -26,6 +26,7 @@ struct SettingsProfile : public IAccessEntity
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;
void clearAllExceptDependencies() override;
bool isBackupAllowed() const override { return elements.isBackupAllowed(); }
};

View File

@ -211,6 +211,15 @@ void SettingsProfileElements::removeDependencies(const std::unordered_set<UUID>
}
void SettingsProfileElements::removeSettingsKeepProfiles()
{
for (auto & element : *this)
element.setting_name.clear();
std::erase_if(*this, [&](const SettingsProfileElement & element) { return element.setting_name.empty() && !element.parent_profile; });
}
void SettingsProfileElements::merge(const SettingsProfileElements & other)
{
insert(end(), other.begin(), other.end());

View File

@ -68,6 +68,8 @@ public:
void copyDependenciesFrom(const SettingsProfileElements & src, const std::unordered_set<UUID> & ids);
void removeDependencies(const std::unordered_set<UUID> & ids);
void removeSettingsKeepProfiles();
void merge(const SettingsProfileElements & other);
Settings toSettings() const;

View File

@ -81,4 +81,14 @@ void User::removeDependencies(const std::unordered_set<UUID> & ids)
settings.removeDependencies(ids);
}
void User::clearAllExceptDependencies()
{
authentication_methods.clear();
allowed_client_hosts = AllowedClientHosts::AnyHostTag{};
access = {};
settings.removeSettingsKeepProfiles();
default_database = {};
valid_until = 0;
}
}

View File

@ -36,6 +36,7 @@ struct User : public IAccessEntity
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;
void clearAllExceptDependencies() override;
bool isBackupAllowed() const override { return settings.isBackupAllowed(); }
};

View File

@ -1,4 +1,5 @@
#include <Access/Common/AccessEntityType.h>
#include <Access/AccessControl.h>
#include <Backups/BackupCoordinationStage.h>
#include <Backups/BackupEntriesCollector.h>
#include <Backups/BackupEntryFromMemory.h>
@ -903,4 +904,20 @@ void BackupEntriesCollector::runPostTasks()
LOG_TRACE(log, "All post tasks successfully executed");
}
std::unordered_map<UUID, AccessEntityPtr> BackupEntriesCollector::getAllAccessEntities()
{
std::lock_guard lock(mutex);
if (!all_access_entities)
{
all_access_entities.emplace();
auto entities_with_ids = context->getAccessControl().readAllWithIDs();
for (const auto & [id, entity] : entities_with_ids)
{
if (entity->isBackupAllowed())
all_access_entities->emplace(id, entity);
}
}
return *all_access_entities;
}
}

View File

@ -21,6 +21,8 @@ class IBackupCoordination;
class IDatabase;
using DatabasePtr = std::shared_ptr<IDatabase>;
struct StorageID;
struct IAccessEntity;
using AccessEntityPtr = std::shared_ptr<const IAccessEntity>;
class QueryStatus;
using QueryStatusPtr = std::shared_ptr<QueryStatus>;
@ -48,6 +50,9 @@ public:
ContextPtr getContext() const { return context; }
const ZooKeeperRetriesInfo & getZooKeeperRetriesInfo() const { return global_zookeeper_retries_info; }
/// Returns all access entities which can be put into a backup.
std::unordered_map<UUID, AccessEntityPtr> getAllAccessEntities();
/// Adds a backup entry which will be later returned by run().
/// These function can be called by implementations of IStorage::backupData() in inherited storage classes.
void addBackupEntry(const String & file_name, BackupEntryPtr backup_entry);
@ -173,6 +178,8 @@ private:
std::vector<std::pair<String, String>> previous_databases_metadata;
std::vector<std::pair<QualifiedTableName, String>> previous_tables_metadata;
std::optional<std::unordered_map<UUID, AccessEntityPtr>> all_access_entities;
BackupEntries backup_entries;
std::queue<std::function<void()>> post_tasks;

View File

@ -37,6 +37,7 @@ namespace ErrorCodes
M(Bool, check_parts) \
M(Bool, check_projection_parts) \
M(Bool, allow_backup_broken_projections) \
M(Bool, write_access_entities_dependents) \
M(Bool, internal) \
M(String, host_id) \
M(OptionalUUID, backup_uuid)

View File

@ -77,6 +77,11 @@ struct BackupSettings
/// Allow to create backup with broken projections.
bool allow_backup_broken_projections = false;
/// Whether dependents of access entities should be written along with the access entities.
/// For example, if a role is granted to a user and we're making a backup of system.roles (but not system.users)
/// this is whether the backup will contain information to grant the role to the corresponding user again.
bool write_access_entities_dependents = false;
/// Internal, should not be specified by user.
/// Whether this backup is a part of a distributed backup created by BACKUP ON CLUSTER.
bool internal = false;

View File

@ -8,3 +8,6 @@ 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

@ -19,7 +19,7 @@ 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 "BACKUP TABLE system.users, TABLE system.roles TO ${backup_name} SETTINGS write_access_entities_dependents = true FORMAT Null"
${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null"
do_check()
@ -81,16 +81,11 @@ ${CLICKHOUSE_CLIENT} --query "SELECT 'role_b', count() FROM system.roles WHERE n
${CLICKHOUSE_CLIENT} --query "RESTORE TABLE system.users FROM ${backup_name} FORMAT Null"
do_check
# TODO: Cannot restore system.users, then system.roles correctly. The result after the second RESTORE ALL is the following:
# CREATE USER user_a IDENTIFIED WITH no_password DEFAULT ROLE NONE SETTINGS custom_x = 2; CREATE ROLE role_b SETTINGS custom_x = 1
# because there is no `role_b` at the time when `user_a` is restored,
# and no information about the default role and the grant at the time when `role_b` is restored.
#
# 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} SETTINGS allow_unresolved_access_dependencies=true 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} SETTINGS update_access_entities_dependents=true FORMAT Null"
# do_check
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} SETTINGS allow_unresolved_access_dependencies=true 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} SETTINGS update_access_entities_dependents=true FORMAT Null"
do_check