This commit is contained in:
Vitaly Baranov 2024-09-19 01:34:33 +02:00 committed by GitHub
commit ac422c99ad
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
60 changed files with 1244 additions and 500 deletions

View File

@ -30,6 +30,7 @@ namespace ErrorCodes
{ {
extern const int CANNOT_RESTORE_TABLE; extern const int CANNOT_RESTORE_TABLE;
extern const int ACCESS_ENTITY_ALREADY_EXISTS; extern const int ACCESS_ENTITY_ALREADY_EXISTS;
extern const int ACCESS_ENTITY_NOT_FOUND;
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
@ -41,6 +42,7 @@ namespace
{ {
std::unordered_map<UUID, AccessEntityPtr> entities; std::unordered_map<UUID, AccessEntityPtr> entities;
std::unordered_map<UUID, std::pair<String, AccessEntityType>> dependencies; std::unordered_map<UUID, std::pair<String, AccessEntityType>> dependencies;
std::unordered_map<UUID, AccessEntityPtr> dependents;
BackupEntryPtr toBackupEntry() const BackupEntryPtr toBackupEntry() const
{ {
@ -72,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()); return std::make_shared<BackupEntryFromMemory>(buf.str());
} }
@ -81,59 +101,71 @@ namespace
{ {
AccessEntitiesInBackup res; AccessEntitiesInBackup res;
bool dependencies_found = false; bool reading_dependencies = false;
bool reading_dependents = false;
while (!buf->eof()) while (!buf->eof())
{ {
String line; String line;
readStringUntilNewlineInto(line, *buf); readStringUntilNewlineInto(line, *buf);
buf->ignore(); buf->ignore();
if (line == "DEPENDENCIES") if (line == "DEPENDENCIES")
{ {
dependencies_found = true; reading_dependencies = true;
break; 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'))); size_t separator1 = line.find('\t');
line.clear(); 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; UUID id = parse<UUID>(line.substr(0, separator1));
while (!buf->eof()) AccessEntityType type = AccessEntityTypeInfo::parseType(line.substr(separator1 + 1, separator2 - separator1 - 1));
String name = line.substr(separator2 + 1);
if (reading_dependencies)
{ {
String query; res.dependencies.emplace(id, std::pair{name, type});
readStringUntilNewlineInto(query, *buf);
buf->ignore();
if (query.empty())
break;
if (!queries.empty())
queries.append("\n");
queries.append(query);
} }
else
AccessEntityPtr entity = deserializeAccessEntity(queries);
res.entities.emplace(id, entity);
}
if (dependencies_found)
{
while (!buf->eof())
{ {
String id_as_string; String queries;
readStringInto(id_as_string, *buf); while (!buf->eof())
buf->ignore(); {
UUID id = parse<UUID>(id_as_string); String query;
readStringUntilNewlineInto(query, *buf);
buf->ignore();
if (query.empty())
break;
if (!queries.empty())
queries.append("\n");
queries.append(query);
}
String type_as_string; AccessEntityPtr entity = deserializeAccessEntity(queries);
readStringInto(type_as_string, *buf);
buf->ignore();
AccessEntityType type = AccessEntityTypeInfo::parseType(type_as_string);
String name; if (name != entity->getName())
readStringInto(name, *buf); throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Unexpected name {} is specified for {}", name, entity->formatTypeWithName());
buf->ignore(); 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)) if (reading_dependents)
res.dependencies.emplace(id, std::pair{name, type}); res.dependents.emplace(id, entity);
else
res.entities.emplace(id, entity);
} }
} }
@ -146,190 +178,59 @@ 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;
}
/// Checks if new entities (which we're going to restore) already exist,
/// and either skips them or throws an exception depending on the restore settings.
void checkExistingEntities(std::vector<std::pair<UUID, AccessEntityPtr>> & entities,
std::unordered_map<UUID, UUID> & old_to_new_id,
const AccessControl & access_control,
RestoreAccessCreationMode creation_mode)
{
if (creation_mode == RestoreAccessCreationMode::kReplace)
return;
auto should_skip = [&](const std::pair<UUID, AccessEntityPtr> & id_and_entity)
{
const auto & id = id_and_entity.first;
const auto & entity = *id_and_entity.second;
auto existing_id = access_control.find(entity.getType(), entity.getName());
if (!existing_id)
{
return false;
}
else if (creation_mode == RestoreAccessCreationMode::kCreateIfNotExists)
{
old_to_new_id[id] = *existing_id;
return true;
}
else
{
throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Cannot restore {} because it already exists", entity.formatTypeWithName());
}
};
std::erase_if(entities, should_skip);
}
/// If new entities (which we're going to restore) depend on other entities which are not going to be restored or not present in the backup
/// then we should try to replace those dependencies with already existing entities.
void resolveDependencies(const std::unordered_map<UUID, std::pair<String, AccessEntityType>> & dependencies,
std::unordered_map<UUID, UUID> & old_to_new_ids,
const AccessControl & access_control,
bool allow_unresolved_dependencies)
{
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);
}
}
/// Generates random IDs for the new entities.
void generateRandomIDs(std::vector<std::pair<UUID, AccessEntityPtr>> & entities, std::unordered_map<UUID, UUID> & old_to_new_ids)
{
Poco::UUIDGenerator generator;
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;
}
}
/// Updates dependencies of the new entities using a specified map.
void replaceDependencies(std::vector<std::pair<UUID, AccessEntityPtr>> & entities,
const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
for (auto & entity : entities | boost::adaptors::map_values)
IAccessEntity::replaceDependencies(entity, old_to_new_ids);
}
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;
}
} }
std::pair<String, BackupEntryPtr> makeBackupEntryForAccess( std::pair<String, BackupEntryPtr> makeBackupEntryForAccessEntities(
const std::vector<std::pair<UUID, AccessEntityPtr>> & access_entities, const std::vector<UUID> & entities_ids,
const String & data_path_in_backup, const std::unordered_map<UUID, AccessEntityPtr> & all_entities,
size_t counter, bool write_dependents,
const AccessControl & access_control) const String & data_path_in_backup)
{ {
auto dependencies = readDependenciesNamesAndTypes(findDependencies(access_entities), access_control);
AccessEntitiesInBackup ab; 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;
String filename = fmt::format("access{:02}.txt", counter + 1); /// access01.txt, access02.txt, ... 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; String file_path_in_backup = fs::path{data_path_in_backup} / filename;
return {file_path_in_backup, ab.toBackupEntry()}; return {file_path_in_backup, ab.toBackupEntry()};
} }
@ -339,61 +240,427 @@ AccessRestorerFromBackup::AccessRestorerFromBackup(
const BackupPtr & backup_, const RestoreSettings & restore_settings_) const BackupPtr & backup_, const RestoreSettings & restore_settings_)
: backup(backup_) : backup(backup_)
, creation_mode(restore_settings_.create_access) , creation_mode(restore_settings_.create_access)
, allow_unresolved_dependencies(restore_settings_.allow_unresolved_access_dependencies) , skip_unresolved_dependencies(restore_settings_.skip_unresolved_access_entities_dependencies)
, update_dependents(restore_settings_.update_access_entities_dependents)
, log(getLogger("AccessRestorerFromBackup"))
{ {
} }
AccessRestorerFromBackup::~AccessRestorerFromBackup() = default; AccessRestorerFromBackup::~AccessRestorerFromBackup() = default;
void AccessRestorerFromBackup::addDataPath(const String & data_path)
void AccessRestorerFromBackup::addDataPath(const String & data_path_in_backup)
{ {
if (!data_paths.emplace(data_path).second) if (loaded)
return; throw Exception(ErrorCodes::LOGICAL_ERROR, "Access entities already loaded");
fs::path data_path_in_backup_fs = data_path; if (std::find(data_paths_in_backup.begin(), data_paths_in_backup.end(), data_path_in_backup) == data_paths_in_backup.end())
Strings filenames = backup->listFiles(data_path, /*recursive*/ false); data_paths_in_backup.emplace_back(data_path_in_backup);
if (filenames.empty())
return;
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});
}
::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);
} }
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];
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.restore = true;
entity_info.data_path_index = data_path_index;
}
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;
}
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;
}
}
}
loaded = true;
}
AccessRightsElements AccessRestorerFromBackup::getRequiredAccess() const 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;
if (!loaded)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Access entities not loaded");
/// 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_set<UUID> ids_to_restore;
std::unordered_map<UUID, UUID> old_to_new_ids; std::unordered_map<UUID, UUID> old_to_new_ids;
checkExistingEntities(new_entities, old_to_new_ids, access_control, creation_mode); std::unordered_set<UUID> unresolved_ids;
resolveDependencies(dependencies, old_to_new_ids, access_control, allow_unresolved_dependencies);
generateRandomIDs(new_entities, old_to_new_ids);
replaceDependencies(new_entities, old_to_new_ids);
return new_entities; for (const auto & [id, entity_info] : entity_infos)
{
if (entity_info.restore)
ids_to_restore.insert(id);
if (entity_info.new_id)
old_to_new_ids[id] = *entity_info.new_id;
else
unresolved_ids.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(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.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];
}
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_set<UUID> restored_ids;
std::unordered_map<UUID, UUID> new_to_existing_ids;
std::vector<std::pair<UUID, AccessEntityPtr>> more_dependents;
more_dependents.reserve(entities.size());
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));
restored_ids.emplace(id);
}
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);
}
}
if (!new_to_existing_ids.empty())
{
std::vector<UUID> ids_to_update;
ids_to_update.reserve(restored_ids.size());
boost::copy(restored_ids, std::inserter(ids_to_update, ids_to_update.end()));
std::unordered_set<UUID> new_ids;
boost::copy(new_to_existing_ids | boost::adaptors::map_keys, std::inserter(new_ids, new_ids.end()));
/// 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);
}
auto do_update_dependents = [&](const std::vector<std::pair<UUID, AccessEntityPtr>> & dependents_to_update)
{
if (dependents_to_update.empty())
return;
std::vector<UUID> ids_to_update;
ids_to_update.reserve(dependents_to_update.size());
std::unordered_map<UUID, AccessEntityPtr> id_to_source;
for (const auto & [id, source] : dependents_to_update)
{
if (!destination_access_storage.isReadOnly(id))
{
ids_to_update.emplace_back(id);
auto new_source = source->clone();
new_source->replaceDependencies(new_to_existing_ids);
id_to_source[id] = new_source;
}
}
/// 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 & source = *id_to_source.at(id);
if (!source.hasDependencies(restored_ids))
return entity;
LOG_TRACE(log, "{}: Updating dependent", entity->formatTypeWithName());
auto res = entity->clone();
res->copyDependenciesFrom(source, 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 #pragma once
#include <Common/Logger.h>
#include <Core/UUID.h> #include <Core/UUID.h>
#include <unordered_map> #include <unordered_map>
#include <unordered_set>
namespace DB namespace DB
@ -12,6 +12,7 @@ enum class AccessEntityType : uint8_t;
struct IAccessEntity; struct IAccessEntity;
using AccessEntityPtr = std::shared_ptr<const IAccessEntity>; using AccessEntityPtr = std::shared_ptr<const IAccessEntity>;
class AccessRightsElements; class AccessRightsElements;
class IAccessStorage;
class IBackup; class IBackup;
using BackupPtr = std::shared_ptr<const IBackup>; using BackupPtr = std::shared_ptr<const IBackup>;
class IBackupEntry; class IBackupEntry;
@ -20,15 +21,22 @@ struct RestoreSettings;
enum class RestoreAccessCreationMode : uint8_t; enum class RestoreAccessCreationMode : uint8_t;
/// Makes a backup of access entities of a specified type. /// Makes a backup entry for of a set of access entities.
std::pair<String, BackupEntryPtr> makeBackupEntryForAccess( std::pair<String, BackupEntryPtr> makeBackupEntryForAccessEntities(
const std::vector<std::pair<UUID, AccessEntityPtr>> & access_entities, const std::vector<UUID> & entities_ids,
const String & data_path_in_backup, const std::unordered_map<UUID, AccessEntityPtr> & all_entities,
size_t counter, bool write_dependents,
const AccessControl & access_control); const String & data_path_in_backup);
/// Restores access entities from a backup. /// 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 class AccessRestorerFromBackup
{ {
public: public:
@ -36,21 +44,82 @@ public:
~AccessRestorerFromBackup(); ~AccessRestorerFromBackup();
/// Adds a data path to loads access entities from. /// Adds a data path to loads access entities from.
void addDataPath(const String & data_path); void addDataPath(const String & data_path_in_backup);
/// Loads access entities from the backup.
void loadFromBackup();
/// Checks that the current user can do restoring. /// Checks that the current user can do restoring.
/// Function loadFromBackup() must be called before that.
AccessRightsElements getRequiredAccess() const; AccessRightsElements getRequiredAccess() const;
/// Inserts all access entities loaded from all the paths added by addDataPath(). /// Generates random IDs for access entities we're restoring to insert them into an access storage;
std::vector<std::pair<UUID, AccessEntityPtr>> getAccessEntities(const AccessControl & access_control) const; /// 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: private:
BackupPtr backup; const BackupPtr backup;
RestoreAccessCreationMode creation_mode; const RestoreAccessCreationMode creation_mode;
bool allow_unresolved_dependencies = false; const bool skip_unresolved_dependencies;
std::vector<std::pair<UUID, AccessEntityPtr>> entities; const bool update_dependents;
std::unordered_map<UUID, std::pair<String, AccessEntityType>> dependencies; const LoggerPtr log;
std::unordered_set<String> data_paths;
/// Whether loadFromBackup() finished.
bool loaded = false;
/// Whether generateRandomIDsAndResolveDependencies() finished.
bool ids_assigned = false;
Strings 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

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

View File

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

View File

@ -176,6 +176,16 @@ std::vector<UUID> GrantedRoles::findDependencies() const
return res; 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) void GrantedRoles::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{ {
std::vector<UUID> 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); } friend bool operator !=(const GrantedRoles & left, const GrantedRoles & right) { return !(left == right); }
std::vector<UUID> findDependencies() 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 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: private:
boost::container::flat_set<UUID> roles; boost::container::flat_set<UUID> roles;

View File

@ -9,28 +9,4 @@ bool IAccessEntity::equal(const IAccessEntity & other) const
return (name == other.name) && (getType() == other.getType()); return (name == other.name) && (getType() == other.getType());
} }
void IAccessEntity::replaceDependencies(std::shared_ptr<const IAccessEntity> & entity, const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
if (old_to_new_ids.empty())
return;
bool need_replace_dependencies = false;
auto dependencies = entity->findDependencies();
for (const auto & dependency : dependencies)
{
if (old_to_new_ids.contains(dependency))
{
need_replace_dependencies = true;
break;
}
}
if (!need_replace_dependencies)
return;
auto new_entity = entity->clone();
new_entity->replaceDependencies(old_to_new_ids);
entity = new_entity;
}
} }

View File

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

View File

@ -4,8 +4,10 @@
#include <Access/User.h> #include <Access/User.h>
#include <Access/AccessBackup.h> #include <Access/AccessBackup.h>
#include <Backups/BackupEntriesCollector.h> #include <Backups/BackupEntriesCollector.h>
#include <Backups/RestorerFromBackup.h> #include <Backups/IBackupCoordination.h>
#include <Backups/IRestoreCoordination.h>
#include <Backups/RestoreSettings.h> #include <Backups/RestoreSettings.h>
#include <Backups/RestorerFromBackup.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <Common/callOnce.h> #include <Common/callOnce.h>
@ -14,6 +16,7 @@
#include <Poco/UUIDGenerator.h> #include <Poco/UUIDGenerator.h>
#include <Poco/Logger.h> #include <Poco/Logger.h>
#include <base/FnTraits.h> #include <base/FnTraits.h>
#include <base/range.h>
#include <boost/algorithm/string/join.hpp> #include <boost/algorithm/string/join.hpp>
#include <boost/algorithm/string/replace.hpp> #include <boost/algorithm/string/replace.hpp>
#include <boost/range/adaptor/map.hpp> #include <boost/range/adaptor/map.hpp>
@ -71,6 +74,18 @@ std::vector<UUID> IAccessStorage::find(AccessEntityType type, const Strings & na
} }
std::vector<UUID> IAccessStorage::findAllImpl() const
{
std::vector<UUID> res;
for (auto type : collections::range(AccessEntityType::MAX))
{
auto ids = findAllImpl(type);
res.insert(res.end(), ids.begin(), ids.end());
}
return res;
}
UUID IAccessStorage::getID(AccessEntityType type, const String & name) const UUID IAccessStorage::getID(AccessEntityType type, const String & name) const
{ {
auto id = findImpl(type, name); auto id = findImpl(type, name);
@ -598,67 +613,60 @@ void IAccessStorage::backup(BackupEntriesCollector & backup_entries_collector, c
if (!isBackupAllowed()) if (!isBackupAllowed())
throwBackupNotAllowed(); throwBackupNotAllowed();
auto entities = readAllWithIDs(type); auto entities_ids = findAll(type);
std::erase_if(entities, [](const std::pair<UUID, AccessEntityPtr> & x) { return !x.second->isBackupAllowed(); }); if (entities_ids.empty())
if (entities.empty())
return; return;
auto backup_entry = makeBackupEntryForAccess( auto backup_entry_with_path = makeBackupEntryForAccessEntities(
entities, entities_ids,
data_path_in_backup, backup_entries_collector.getAllAccessEntities(),
backup_entries_collector.getAccessCounter(type), backup_entries_collector.getBackupSettings().write_access_entities_dependents,
backup_entries_collector.getContext()->getAccessControl()); data_path_in_backup);
backup_entries_collector.addBackupEntry(backup_entry); if (isReplicated())
{
auto backup_coordination = backup_entries_collector.getBackupCoordination();
auto replication_id = getReplicationID();
backup_coordination->addReplicatedAccessFilePath(replication_id, type, backup_entry_with_path.first);
backup_entries_collector.addPostTask(
[backup_entry = backup_entry_with_path.second,
replication_id,
type,
&backup_entries_collector,
backup_coordination]
{
for (const String & path : backup_coordination->getReplicatedAccessFilePaths(replication_id, type))
backup_entries_collector.addBackupEntry(path, backup_entry);
});
}
else
{
backup_entries_collector.addBackupEntry(backup_entry_with_path);
}
} }
void IAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) void IAccessStorage::restoreFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup)
{ {
if (!isRestoreAllowed()) if (!isRestoreAllowed())
throwRestoreNotAllowed(); throwRestoreNotAllowed();
if (isReplicated() && !acquireReplicatedRestore(restorer)) if (isReplicated())
return;
auto entities = restorer.getAccessEntitiesToRestore();
if (entities.empty())
return;
auto create_access = restorer.getRestoreSettings().create_access;
bool replace_if_exists = (create_access == RestoreAccessCreationMode::kReplace);
bool throw_if_exists = (create_access == RestoreAccessCreationMode::kCreate);
restorer.addDataRestoreTask([this, entities_to_restore = std::move(entities), replace_if_exists, throw_if_exists] mutable
{ {
std::unordered_map<UUID, UUID> new_to_existing_ids; auto restore_coordination = restorer.getRestoreCoordination();
for (auto & [id, entity] : entities_to_restore) if (!restore_coordination->acquireReplicatedAccessStorage(getReplicationID()))
{ return;
UUID existing_entity_id; }
if (!insert(id, entity, replace_if_exists, throw_if_exists, &existing_entity_id))
{
/// Couldn't insert `entity` because there is an existing entity with the same name.
new_to_existing_ids[id] = existing_entity_id;
}
}
if (!new_to_existing_ids.empty()) restorer.addDataRestoreTask(
[this, &restorer, data_path_in_backup]
{ {
/// If new entities restored from backup have dependencies on other entities from backup which were not restored because they existed, auto entities = restorer.getAccessEntitiesToRestore(data_path_in_backup);
/// then we should correct those dependencies. auto dependents = restorer.getAccessEntitiesToRestoreDependents(data_path_in_backup);
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr const auto & restore_settings = restorer.getRestoreSettings();
{ restoreAccessEntitiesFromBackup(*this, entities, dependents, restore_settings);
auto res = entity; });
IAccessEntity::replaceDependencies(res, new_to_existing_ids);
return res;
};
std::vector<UUID> ids;
ids.reserve(entities_to_restore.size());
boost::copy(entities_to_restore | boost::adaptors::map_keys, std::back_inserter(ids));
tryUpdate(ids, update_func);
}
});
} }

View File

@ -66,6 +66,7 @@ public:
/// Returns true if this storage is replicated. /// Returns true if this storage is replicated.
virtual bool isReplicated() const { return false; } virtual bool isReplicated() const { return false; }
virtual String getReplicationID() const { return ""; }
/// Starts periodic reloading and updating of entities in this storage. /// Starts periodic reloading and updating of entities in this storage.
virtual void startPeriodicReloading() {} virtual void startPeriodicReloading() {}
@ -90,8 +91,9 @@ public:
/// Returns the identifiers of all the entities of a specified type contained in the storage. /// Returns the identifiers of all the entities of a specified type contained in the storage.
std::vector<UUID> findAll(AccessEntityType type) const; std::vector<UUID> findAll(AccessEntityType type) const;
template <typename EntityClassT> /// Returns the identifiers of all the entities in the storage.
std::vector<UUID> findAll() const { return findAll(EntityClassT::TYPE); } template <typename EntityClassT = IAccessEntity>
std::vector<UUID> findAll() const;
/// Searches for an entity with specified type and name. Returns std::nullopt if not found. /// Searches for an entity with specified type and name. Returns std::nullopt if not found.
std::optional<UUID> find(AccessEntityType type, const String & name) const; std::optional<UUID> find(AccessEntityType type, const String & name) const;
@ -148,7 +150,7 @@ public:
std::optional<std::pair<String, AccessEntityType>> tryReadNameWithType(const UUID & id) const; std::optional<std::pair<String, AccessEntityType>> tryReadNameWithType(const UUID & id) const;
/// Reads all entities and returns them with their IDs. /// Reads all entities and returns them with their IDs.
template <typename EntityClassT> template <typename EntityClassT = IAccessEntity>
std::vector<std::pair<UUID, std::shared_ptr<const EntityClassT>>> readAllWithIDs() const; std::vector<std::pair<UUID, std::shared_ptr<const EntityClassT>>> readAllWithIDs() const;
std::vector<std::pair<UUID, AccessEntityPtr>> readAllWithIDs(AccessEntityType type) const; std::vector<std::pair<UUID, AccessEntityPtr>> readAllWithIDs(AccessEntityType type) const;
@ -180,7 +182,7 @@ public:
/// Removes multiple entities from the storage. Returns the list of successfully dropped. /// Removes multiple entities from the storage. Returns the list of successfully dropped.
std::vector<UUID> tryRemove(const std::vector<UUID> & ids); 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. /// 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); bool update(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists = true);
@ -214,11 +216,12 @@ public:
/// Makes a backup of this access storage. /// 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 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: protected:
virtual std::optional<UUID> findImpl(AccessEntityType type, const String & name) const = 0; virtual std::optional<UUID> findImpl(AccessEntityType type, const String & name) const = 0;
virtual std::vector<UUID> findAllImpl(AccessEntityType type) const = 0; virtual std::vector<UUID> findAllImpl(AccessEntityType type) const = 0;
virtual std::vector<UUID> findAllImpl() const;
virtual AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const = 0; virtual AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const = 0;
virtual std::optional<std::pair<String, AccessEntityType>> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const; virtual std::optional<std::pair<String, AccessEntityType>> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const;
virtual bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id); virtual bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id);
@ -267,6 +270,16 @@ private:
}; };
template <typename EntityClassT>
std::vector<UUID> IAccessStorage::findAll() const
{
if constexpr (std::is_same_v<EntityClassT, IAccessEntity>)
return findAllImpl();
else
return findAllImpl(EntityClassT::TYPE);
}
template <typename EntityClassT> template <typename EntityClassT>
std::shared_ptr<const EntityClassT> IAccessStorage::read(const UUID & id, bool throw_if_not_exists) const std::shared_ptr<const EntityClassT> IAccessStorage::read(const UUID & id, bool throw_if_not_exists) const
{ {

View File

@ -163,7 +163,7 @@ void LDAPAccessStorage::applyRoleChangeNoLock(bool grant, const UUID & role_id,
// Update the granted roles of the relevant users. // Update the granted roles of the relevant users.
if (!user_ids.empty()) 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_)) 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) if (it != external_role_hashes.end() && it->second == external_roles_hash)
return; 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_)) if (auto user = typeid_cast<std::shared_ptr<const User>>(entity_))
{ {

View File

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

View File

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

View File

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

View File

@ -24,9 +24,33 @@ std::vector<UUID> Quota::findDependencies() const
return to_roles.findDependencies(); return to_roles.findDependencies();
} }
void Quota::doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) 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); 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);
}
void Quota::clearAllExceptDependencies()
{
all_limits.clear();
key_type = QuotaKeyType::NONE;
}
} }

View File

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

View File

@ -4,10 +4,6 @@
#include <Access/ReplicatedAccessStorage.h> #include <Access/ReplicatedAccessStorage.h>
#include <Access/AccessChangesNotifier.h> #include <Access/AccessChangesNotifier.h>
#include <Access/AccessBackup.h> #include <Access/AccessBackup.h>
#include <Backups/BackupEntriesCollector.h>
#include <Backups/IBackupCoordination.h>
#include <Backups/IRestoreCoordination.h>
#include <Backups/RestorerFromBackup.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Common/ZooKeeper/KeeperException.h> #include <Common/ZooKeeper/KeeperException.h>
@ -359,7 +355,7 @@ bool ReplicatedAccessStorage::updateZooKeeper(const zkutil::ZooKeeperPtr & zooke
} }
const AccessEntityPtr old_entity = deserializeAccessEntity(old_entity_definition, entity_path); 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())) if (!new_entity->isTypeOf(old_entity->getType()))
throwBadCast(id, new_entity->getType(), new_entity->getName(), old_entity->getType()); throwBadCast(id, new_entity->getType(), new_entity->getName(), old_entity->getType());
@ -684,44 +680,4 @@ AccessEntityPtr ReplicatedAccessStorage::readImpl(const UUID & id, bool throw_if
return memory_storage.read(id, throw_if_not_exists); return memory_storage.read(id, throw_if_not_exists);
} }
void ReplicatedAccessStorage::backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const
{
if (!isBackupAllowed())
throwBackupNotAllowed();
auto entities = readAllWithIDs(type);
std::erase_if(entities, [](const std::pair<UUID, AccessEntityPtr> & x) { return !x.second->isBackupAllowed(); });
if (entities.empty())
return;
auto backup_entry_with_path = makeBackupEntryForAccess(
entities,
data_path_in_backup,
backup_entries_collector.getAccessCounter(type),
backup_entries_collector.getContext()->getAccessControl());
auto backup_coordination = backup_entries_collector.getBackupCoordination();
backup_coordination->addReplicatedAccessFilePath(zookeeper_path, type, backup_entry_with_path.first);
backup_entries_collector.addPostTask(
[backup_entry = backup_entry_with_path.second,
my_zookeeper_path = zookeeper_path,
type,
&backup_entries_collector,
backup_coordination]
{
for (const String & path : backup_coordination->getReplicatedAccessFilePaths(my_zookeeper_path, type))
backup_entries_collector.addBackupEntry(path, backup_entry);
});
}
bool ReplicatedAccessStorage::acquireReplicatedRestore(RestorerFromBackup & restorer) const
{
auto restore_coordination = restorer.getRestoreCoordination();
return restore_coordination->acquireReplicatedAccessStorage(zookeeper_path);
}
} }

View File

@ -26,7 +26,9 @@ public:
void shutdown() override; void shutdown() override;
const char * getStorageType() const override { return STORAGE_TYPE; } const char * getStorageType() const override { return STORAGE_TYPE; }
bool isReplicated() const override { return true; } bool isReplicated() const override { return true; }
String getReplicationID() const override { return zookeeper_path; }
void startPeriodicReloading() override { startWatchingThread(); } void startPeriodicReloading() override { startWatchingThread(); }
void stopPeriodicReloading() override { stopWatchingThread(); } void stopPeriodicReloading() override { stopWatchingThread(); }
@ -35,7 +37,6 @@ public:
bool exists(const UUID & id) const override; bool exists(const UUID & id) const override;
bool isBackupAllowed() const override { return backup_allowed; } bool isBackupAllowed() const override { return backup_allowed; }
void backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const override;
private: private:
String zookeeper_path; String zookeeper_path;
@ -80,7 +81,6 @@ private:
std::optional<UUID> findImpl(AccessEntityType type, const String & name) const override; std::optional<UUID> findImpl(AccessEntityType type, const String & name) const override;
std::vector<UUID> findAllImpl(AccessEntityType type) const override; std::vector<UUID> findAllImpl(AccessEntityType type) const override;
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override;
bool acquireReplicatedRestore(RestorerFromBackup & restorer) const override;
mutable std::mutex mutex; mutable std::mutex mutex;
MemoryAccessStorage memory_storage TSA_GUARDED_BY(mutex); MemoryAccessStorage memory_storage TSA_GUARDED_BY(mutex);

View File

@ -21,10 +21,36 @@ std::vector<UUID> Role::findDependencies() const
return res; return res;
} }
void Role::doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) 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); granted_roles.replaceDependencies(old_to_new_ids);
settings.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);
}
void Role::clearAllExceptDependencies()
{
access = {};
settings.removeSettingsKeepProfiles();
}
} }

View File

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

View File

@ -295,6 +295,23 @@ std::vector<UUID> RolesOrUsersSet::findDependencies() const
return res; 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) void RolesOrUsersSet::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{ {
std::vector<UUID> 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())); 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); } friend bool operator !=(const RolesOrUsersSet & lhs, const RolesOrUsersSet & rhs) { return !(lhs == rhs); }
std::vector<UUID> findDependencies() const; 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 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; bool all = false;
boost::container::flat_set<UUID> ids; boost::container::flat_set<UUID> ids;

View File

@ -63,9 +63,33 @@ std::vector<UUID> RowPolicy::findDependencies() const
return to_roles.findDependencies(); return to_roles.findDependencies();
} }
void RowPolicy::doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) 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); 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);
}
void RowPolicy::clearAllExceptDependencies()
{
for (auto & filter : filters)
filter = {};
}
} }

View File

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

View File

@ -21,10 +21,35 @@ std::vector<UUID> SettingsProfile::findDependencies() const
return res; return res;
} }
void SettingsProfile::doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) 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); elements.replaceDependencies(old_to_new_ids);
to_roles.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);
}
void SettingsProfile::clearAllExceptDependencies()
{
elements.removeSettingsKeepProfiles();
}
} }

View File

@ -22,7 +22,12 @@ struct SettingsProfile : public IAccessEntity
AccessEntityType getType() const override { return TYPE; } AccessEntityType getType() const override { return TYPE; }
std::vector<UUID> findDependencies() const override; std::vector<UUID> findDependencies() const override;
void doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) 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;
void clearAllExceptDependencies() override;
bool isBackupAllowed() const override { return elements.isBackupAllowed(); } 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) void SettingsProfileElements::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{ {
for (auto & element : *this) for (auto & element : *this)
@ -176,6 +188,38 @@ 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::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) void SettingsProfileElements::merge(const SettingsProfileElements & other)
{ {
insert(end(), other.begin(), other.end()); insert(end(), other.begin(), other.end());

View File

@ -63,7 +63,12 @@ public:
std::shared_ptr<ASTSettingsProfileElements> toASTWithNames(const AccessControl & access_control) const; std::shared_ptr<ASTSettingsProfileElements> toASTWithNames(const AccessControl & access_control) const;
std::vector<UUID> findDependencies() 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 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 removeSettingsKeepProfiles();
void merge(const SettingsProfileElements & other); void merge(const SettingsProfileElements & other);

View File

@ -49,7 +49,12 @@ std::vector<UUID> User::findDependencies() const
return res; return res;
} }
void User::doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) 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); default_roles.replaceDependencies(old_to_new_ids);
granted_roles.replaceDependencies(old_to_new_ids); granted_roles.replaceDependencies(old_to_new_ids);
@ -57,4 +62,33 @@ void User::doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_n
settings.replaceDependencies(old_to_new_ids); 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);
}
void User::clearAllExceptDependencies()
{
authentication_methods.clear();
allowed_client_hosts = AllowedClientHosts::AnyHostTag{};
access = {};
settings.removeSettingsKeepProfiles();
default_database = {};
valid_until = 0;
}
} }

View File

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

View File

@ -570,7 +570,7 @@ void BackupCoordinationRemote::prepareReplicatedAccess() const
if (replicated_access) if (replicated_access)
return; return;
std::vector<BackupCoordinationReplicatedAccess::FilePathForAccessEntitry> file_path_for_access_entities; std::vector<BackupCoordinationReplicatedAccess::FilePathForAccessEntity> file_path_for_access_entities;
auto holder = with_retries.createRetriesControlHolder("prepareReplicatedAccess"); auto holder = with_retries.createRetriesControlHolder("prepareReplicatedAccess");
holder.retries_ctl.retryLoop( holder.retries_ctl.retryLoop(
[&, &zk = holder.faulty_zookeeper]() [&, &zk = holder.faulty_zookeeper]()

View File

@ -1,5 +1,9 @@
#include <Backups/BackupCoordinationReplicatedAccess.h> #include <Backups/BackupCoordinationReplicatedAccess.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB namespace DB
{ {
@ -7,7 +11,7 @@ namespace DB
BackupCoordinationReplicatedAccess::BackupCoordinationReplicatedAccess() = default; BackupCoordinationReplicatedAccess::BackupCoordinationReplicatedAccess() = default;
BackupCoordinationReplicatedAccess::~BackupCoordinationReplicatedAccess() = default; BackupCoordinationReplicatedAccess::~BackupCoordinationReplicatedAccess() = default;
void BackupCoordinationReplicatedAccess::addFilePath(FilePathForAccessEntitry && file_path_for_access_entity) void BackupCoordinationReplicatedAccess::addFilePath(FilePathForAccessEntity && file_path_for_access_entity)
{ {
const auto & access_zk_path = file_path_for_access_entity.access_zk_path; const auto & access_zk_path = file_path_for_access_entity.access_zk_path;
const auto & access_entity_type = file_path_for_access_entity.access_entity_type; const auto & access_entity_type = file_path_for_access_entity.access_entity_type;
@ -28,10 +32,19 @@ Strings BackupCoordinationReplicatedAccess::getFilePaths(const String & access_z
return {}; return {};
const auto & file_paths = it->second; const auto & file_paths = it->second;
if (file_paths.host_to_store_access != host_id) if ((file_paths.host_to_store_access != host_id) || file_paths.file_paths.empty())
return {}; return {};
Strings res{file_paths.file_paths.begin(), file_paths.file_paths.end()}; /// Use the same filename for all the paths in backup.
/// Those filenames have format "access-<UUID>.txt", where UUID is random.
/// It's not really necessary, however it looks better if those files have the same filename
/// for a backup of ReplicatedAccessStorage on different hosts.
Strings res;
res.reserve(file_paths.file_paths.size());
String filename = fs::path{*file_paths.file_paths.begin()}.filename();
for (const auto & file_path : file_paths.file_paths)
res.emplace_back(fs::path{file_path}.replace_filename(filename));
return res; return res;
} }

View File

@ -2,7 +2,7 @@
#include <Core/Types.h> #include <Core/Types.h>
#include <map> #include <map>
#include <unordered_set> #include <set>
namespace DB namespace DB
@ -28,7 +28,7 @@ public:
BackupCoordinationReplicatedAccess(); BackupCoordinationReplicatedAccess();
~BackupCoordinationReplicatedAccess(); ~BackupCoordinationReplicatedAccess();
struct FilePathForAccessEntitry struct FilePathForAccessEntity
{ {
String access_zk_path; String access_zk_path;
AccessEntityType access_entity_type; AccessEntityType access_entity_type;
@ -37,7 +37,7 @@ public:
}; };
/// Adds a path to access*.txt file keeping access entities of a ReplicatedAccessStorage. /// Adds a path to access*.txt file keeping access entities of a ReplicatedAccessStorage.
void addFilePath(FilePathForAccessEntitry && file_path_for_access_entity); void addFilePath(FilePathForAccessEntity && file_path_for_access_entity);
/// Returns all paths added by addFilePath() if `host_id` is a host chosen to store access. /// Returns all paths added by addFilePath() if `host_id` is a host chosen to store access.
Strings getFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const; Strings getFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const;
@ -47,7 +47,7 @@ private:
struct FilePathsAndHost struct FilePathsAndHost
{ {
std::unordered_set<String> file_paths; std::set<String> file_paths;
String host_to_store_access; String host_to_store_access;
}; };

View File

@ -32,6 +32,9 @@ namespace BackupCoordinationStage
/// Finding databases and tables in the backup which we're going to restore. /// Finding databases and tables in the backup which we're going to restore.
constexpr const char * FINDING_TABLES_IN_BACKUP = "finding tables in backup"; 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. /// Creating databases or finding them and checking their definitions.
constexpr const char * CREATING_DATABASES = "creating databases"; constexpr const char * CREATING_DATABASES = "creating databases";

View File

@ -1,4 +1,5 @@
#include <Access/Common/AccessEntityType.h> #include <Access/Common/AccessEntityType.h>
#include <Access/AccessControl.h>
#include <Backups/BackupCoordinationStage.h> #include <Backups/BackupCoordinationStage.h>
#include <Backups/BackupEntriesCollector.h> #include <Backups/BackupEntriesCollector.h>
#include <Backups/BackupEntryFromMemory.h> #include <Backups/BackupEntryFromMemory.h>
@ -903,11 +904,20 @@ void BackupEntriesCollector::runPostTasks()
LOG_TRACE(log, "All post tasks successfully executed"); LOG_TRACE(log, "All post tasks successfully executed");
} }
size_t BackupEntriesCollector::getAccessCounter(AccessEntityType type) std::unordered_map<UUID, AccessEntityPtr> BackupEntriesCollector::getAllAccessEntities()
{ {
std::lock_guard lock(mutex); std::lock_guard lock(mutex);
access_counters.resize(static_cast<size_t>(AccessEntityType::MAX)); if (!all_access_entities)
return access_counters[static_cast<size_t>(type)]++; {
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,7 +21,8 @@ class IBackupCoordination;
class IDatabase; class IDatabase;
using DatabasePtr = std::shared_ptr<IDatabase>; using DatabasePtr = std::shared_ptr<IDatabase>;
struct StorageID; struct StorageID;
enum class AccessEntityType : uint8_t; struct IAccessEntity;
using AccessEntityPtr = std::shared_ptr<const IAccessEntity>;
class QueryStatus; class QueryStatus;
using QueryStatusPtr = std::shared_ptr<QueryStatus>; using QueryStatusPtr = std::shared_ptr<QueryStatus>;
@ -49,6 +50,9 @@ public:
ContextPtr getContext() const { return context; } ContextPtr getContext() const { return context; }
const ZooKeeperRetriesInfo & getZooKeeperRetriesInfo() const { return global_zookeeper_retries_info; } 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(). /// 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. /// These function can be called by implementations of IStorage::backupData() in inherited storage classes.
void addBackupEntry(const String & file_name, BackupEntryPtr backup_entry); void addBackupEntry(const String & file_name, BackupEntryPtr backup_entry);
@ -61,9 +65,6 @@ public:
/// 1) we need to join (in a backup) the data of replicated tables gathered on different hosts. /// 1) we need to join (in a backup) the data of replicated tables gathered on different hosts.
void addPostTask(std::function<void()> task); void addPostTask(std::function<void()> task);
/// Returns an incremental counter used to backup access control.
size_t getAccessCounter(AccessEntityType type);
private: private:
void calculateRootPathInBackup(); void calculateRootPathInBackup();
@ -177,9 +178,10 @@ private:
std::vector<std::pair<String, String>> previous_databases_metadata; std::vector<std::pair<String, String>> previous_databases_metadata;
std::vector<std::pair<QualifiedTableName, String>> previous_tables_metadata; std::vector<std::pair<QualifiedTableName, String>> previous_tables_metadata;
std::optional<std::unordered_map<UUID, AccessEntityPtr>> all_access_entities;
BackupEntries backup_entries; BackupEntries backup_entries;
std::queue<std::function<void()>> post_tasks; std::queue<std::function<void()>> post_tasks;
std::vector<size_t> access_counters;
ThreadPool & threadpool; ThreadPool & threadpool;
std::mutex mutex; std::mutex mutex;

View File

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

View File

@ -77,6 +77,11 @@ struct BackupSettings
/// Allow to create backup with broken projections. /// Allow to create backup with broken projections.
bool allow_backup_broken_projections = false; 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 = true;
/// Internal, should not be specified by user. /// Internal, should not be specified by user.
/// Whether this backup is a part of a distributed backup created by BACKUP ON CLUSTER. /// Whether this backup is a part of a distributed backup created by BACKUP ON CLUSTER.
bool internal = false; bool internal = false;

View File

@ -160,7 +160,8 @@ namespace
M(UInt64, replica_num_in_backup) \ M(UInt64, replica_num_in_backup) \
M(Bool, allow_non_empty_tables) \ M(Bool, allow_non_empty_tables) \
M(RestoreAccessCreationMode, create_access) \ 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(RestoreUDFCreationMode, create_function) \
M(Bool, allow_s3_native_copy) \ M(Bool, allow_s3_native_copy) \
M(Bool, use_same_s3_credentials_for_base_backup) \ M(Bool, use_same_s3_credentials_for_base_backup) \
@ -187,7 +188,12 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query)
else else
LIST_OF_RESTORE_SETTINGS(GET_SETTINGS_FROM_RESTORE_QUERY_HELPER) 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. /// 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; RestoreAccessCreationMode create_access = RestoreAccessCreationMode::kCreateIfNotExists;
/// Skip dependencies of access entities which can't be resolved. /// Ignore 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. /// For example: if a backup contains a profile assigned to a user: `CREATE PROFILE p1; CREATE USER u1 SETTINGS PROFILE p1`
bool allow_unresolved_access_dependencies = false; /// 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. /// How the RESTORE command will handle if a user-defined function which it's going to restore already exists.
RestoreUDFCreationMode create_function = RestoreUDFCreationMode::kCreateIfNotExists; RestoreUDFCreationMode create_function = RestoreUDFCreationMode::kCreateIfNotExists;

View File

@ -136,6 +136,8 @@ void RestorerFromBackup::run(Mode mode)
waitFutures(); waitFutures();
/// Check access rights. /// Check access rights.
setStage(Stage::CHECKING_ACCESS_RIGHTS);
loadSystemAccessTables();
checkAccessForObjectsFoundInBackup(); checkAccessForObjectsFoundInBackup();
if (mode == Mode::CHECK_ACCESS_ONLY) if (mode == Mode::CHECK_ACCESS_ONLY)
@ -482,25 +484,6 @@ void RestorerFromBackup::findTableInBackupImpl(const QualifiedTableName & table_
res_table_info.partitions.emplace(); res_table_info.partitions.emplace();
insertAtEnd(*res_table_info.partitions, *partitions); 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) void RestorerFromBackup::findDatabaseInBackup(const String & database_name_in_backup, const std::set<DatabaseAndTableName> & except_table_names)
@ -624,6 +607,27 @@ size_t RestorerFromBackup::getNumTables() const
return table_infos.size(); return table_infos.size();
} }
void RestorerFromBackup::loadSystemAccessTables()
{
if (restore_settings.structure_only)
return;
/// 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);
}
}
if (access_restorer)
access_restorer->loadFromBackup();
}
void RestorerFromBackup::checkAccessForObjectsFoundInBackup() const void RestorerFromBackup::checkAccessForObjectsFoundInBackup() const
{ {
AccessRightsElements required_access; AccessRightsElements required_access;
@ -708,6 +712,24 @@ void RestorerFromBackup::checkAccessForObjectsFoundInBackup() const
context->checkAccess(required_access); 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() void RestorerFromBackup::createDatabases()
{ {
Strings database_names; Strings database_names;
@ -1066,19 +1088,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) void RestorerFromBackup::throwTableIsNotEmpty(const StorageID & storage_id)
{ {
throw Exception( throw Exception(

View File

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

View File

@ -111,7 +111,7 @@ BlockIO InterpreterCreateQuotaQuery::execute()
if (query.alter) 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()); auto updated_quota = typeid_cast<std::shared_ptr<Quota>>(entity->clone());
updateQuotaFromQueryImpl(*updated_quota, query, {}, roles_from_query); updateQuotaFromQueryImpl(*updated_quota, query, {}, roles_from_query);

View File

@ -74,7 +74,7 @@ BlockIO InterpreterCreateRoleQuery::execute()
if (query.alter) 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()); auto updated_role = typeid_cast<std::shared_ptr<Role>>(entity->clone());
updateRoleFromQueryImpl(*updated_role, query, {}, settings_from_query); updateRoleFromQueryImpl(*updated_role, query, {}, settings_from_query);

View File

@ -88,7 +88,7 @@ BlockIO InterpreterCreateRowPolicyQuery::execute()
Strings names = query.names->toStrings(); Strings names = query.names->toStrings();
if (query.alter) 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()); auto updated_policy = typeid_cast<std::shared_ptr<RowPolicy>>(entity->clone());
updateRowPolicyFromQueryImpl(*updated_policy, query, {}, roles_from_query); updateRowPolicyFromQueryImpl(*updated_policy, query, {}, roles_from_query);

View File

@ -90,7 +90,7 @@ BlockIO InterpreterCreateSettingsProfileQuery::execute()
if (query.alter) 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()); auto updated_profile = typeid_cast<std::shared_ptr<SettingsProfile>>(entity->clone());
updateSettingsProfileFromQueryImpl(*updated_profile, query, {}, settings_from_query, roles_from_query); updateSettingsProfileFromQueryImpl(*updated_profile, query, {}, settings_from_query, roles_from_query);

View File

@ -264,7 +264,7 @@ BlockIO InterpreterCreateUserQuery::execute()
if (query.grantees) if (query.grantees)
grantees_from_query = RolesOrUsersSet{*query.grantees, access_control}; 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()); auto updated_user = typeid_cast<std::shared_ptr<User>>(entity->clone());
updateUserFromQueryImpl( updateUserFromQueryImpl(
@ -317,7 +317,7 @@ BlockIO InterpreterCreateUserQuery::execute()
if (query.grantees) if (query.grantees)
{ {
RolesOrUsersSet grantees_from_query = RolesOrUsersSet{*query.grantees, access_control}; 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()); auto updated_user = typeid_cast<std::shared_ptr<User>>(entity->clone());
updated_user->grantees = grantees_from_query; updated_user->grantees = grantees_from_query;

View File

@ -474,7 +474,7 @@ BlockIO InterpreterGrantQuery::execute()
calculateCurrentGrantRightsWithIntersection(new_rights, current_user_access, elements_to_grant); calculateCurrentGrantRightsWithIntersection(new_rights, current_user_access, elements_to_grant);
/// Update roles and users listed in `grantees`. /// 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(); auto clone = entity->clone();
if (query.current_grants) 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); std::vector<UUID> to_users = RolesOrUsersSet{*query.to_users, access_control, getContext()->getUserID()}.getMatchingIDs(access_control);
RolesOrUsersSet roles_from_query{*query.roles, 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()); auto updated_user = typeid_cast<std::shared_ptr<User>>(entity->clone());
updateUserSetDefaultRoles(*updated_user, roles_from_query); updateUserSetDefaultRoles(*updated_user, roles_from_query);

View File

@ -150,10 +150,10 @@ void StorageSystemQuotas::backupData(
} }
void StorageSystemQuotas::restoreDataFromBackup( 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(); 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( 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(); 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( 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(); 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( 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(); 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( 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(); auto & access_control = restorer.getContext()->getAccessControl();
access_control.restoreFromBackup(restorer); access_control.restoreFromBackup(restorer, data_path_in_backup);
} }
} }

View File

@ -0,0 +1,7 @@
CREATE USER user_03231 IDENTIFIED WITH no_password DEFAULT ROLE role_a_03231 SETTINGS custom_x = \'x\'
GRANT role_a_03231 TO user_03231
CREATE ROLE role_a_03231
GRANT INSERT ON *.* TO role_a_03231
GRANT role_b_03231 TO role_a_03231
CREATE ROLE role_b_03231
GRANT SELECT ON *.* TO role_b_03231

View File

@ -0,0 +1,30 @@
#!/usr/bin/env bash
# Tags: no-parallel
# Disabled parallel since we drop and restore fixed users and roles.
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# In this test we restore from "/tests/queries/0_stateless/backups/old_backup_without_access_entities_dependents.zip"
backup_name="$($CURDIR/helpers/install_predefined_backup.sh old_backup_without_access_entities_dependents.zip)"
${CLICKHOUSE_CLIENT} -m --query "
DROP USER IF EXISTS user_03231;
DROP ROLE IF EXISTS role_a_03231, role_b_03231;
"
${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM Disk('backups', '${backup_name}') FORMAT Null"
${CLICKHOUSE_CLIENT} --query "SHOW CREATE USER user_03231"
${CLICKHOUSE_CLIENT} --query "SHOW GRANTS FOR user_03231"
${CLICKHOUSE_CLIENT} --query "SHOW CREATE ROLE role_a_03231"
${CLICKHOUSE_CLIENT} --query "SHOW GRANTS FOR role_a_03231"
${CLICKHOUSE_CLIENT} --query "SHOW CREATE ROLE role_b_03231"
${CLICKHOUSE_CLIENT} --query "SHOW GRANTS FOR role_b_03231"
${CLICKHOUSE_CLIENT} -m --query "
DROP USER user_03231;
DROP ROLE role_a_03231, role_b_03231;
"

View File

@ -1,6 +1,13 @@
Everything dropped Everything dropped
User dropped User dropped
Role dropped
Nothing dropped Nothing dropped
Nothing dropped, mode=replace Nothing dropped, mode=replace
Nothing dropped, mode=create Nothing dropped, mode=create
ACCESS_ENTITY_ALREADY_EXISTS 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

@ -55,14 +55,10 @@ ${CLICKHOUSE_CLIENT} --query "DROP USER ${user_a}"
${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null" ${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null"
do_check do_check
# TODO: Cannot restore a dropped role granted to an existing user. The result after RESTORE ALL below is the following: echo "Role dropped"
# CREATE USER user_a DEFAULT ROLE NONE SETTINGS custom_x = 2; GRANT NONE TO user_a; CREATE ROLE role_b SETTINGS custom_x = 1 ${CLICKHOUSE_CLIENT} --query "DROP ROLE ${role_b}"
# because `role_b` is restored but not granted to existing 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" echo "Nothing dropped"
${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null" ${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null"
@ -75,3 +71,21 @@ do_check
echo "Nothing dropped, mode=create" 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" ${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} SETTINGS create_access='create' FORMAT Null" 2>&1 | grep -om1 "ACCESS_ENTITY_ALREADY_EXISTS"
do_check 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
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