mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
commit
0f04ff0749
@ -44,6 +44,7 @@
|
||||
#include <Interpreters/DNSCacheUpdater.h>
|
||||
#include <Interpreters/SystemLog.cpp>
|
||||
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
|
||||
#include <Access/AccessControlManager.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/System/attachSystemTables.h>
|
||||
#include <AggregateFunctions/registerAggregateFunctions.h>
|
||||
@ -493,6 +494,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
users_config_reloader->reload();
|
||||
});
|
||||
|
||||
/// Sets a local directory storing information about access control.
|
||||
std::string access_control_local_path = config().getString("access_control_path", "");
|
||||
if (!access_control_local_path.empty())
|
||||
global_context->getAccessControlManager().setLocalDirectory(access_control_local_path);
|
||||
|
||||
/// Limit on total number of concurrently executed queries.
|
||||
global_context->getProcessList().setMaxSize(config().getInt("max_concurrent_queries", 0));
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Access/MultipleAccessStorage.h>
|
||||
#include <Access/MemoryAccessStorage.h>
|
||||
#include <Access/UsersConfigAccessStorage.h>
|
||||
#include <Access/DiskAccessStorage.h>
|
||||
#include <Access/AccessRightsContextFactory.h>
|
||||
#include <Access/RoleContextFactory.h>
|
||||
#include <Access/RowPolicyContextFactory.h>
|
||||
@ -15,10 +16,14 @@ namespace
|
||||
std::vector<std::unique_ptr<IAccessStorage>> createStorages()
|
||||
{
|
||||
std::vector<std::unique_ptr<IAccessStorage>> list;
|
||||
list.emplace_back(std::make_unique<MemoryAccessStorage>());
|
||||
list.emplace_back(std::make_unique<DiskAccessStorage>());
|
||||
list.emplace_back(std::make_unique<UsersConfigAccessStorage>());
|
||||
list.emplace_back(std::make_unique<MemoryAccessStorage>());
|
||||
return list;
|
||||
}
|
||||
|
||||
constexpr size_t DISK_ACCESS_STORAGE_INDEX = 0;
|
||||
constexpr size_t USERS_CONFIG_ACCESS_STORAGE_INDEX = 1;
|
||||
}
|
||||
|
||||
|
||||
@ -37,10 +42,17 @@ AccessControlManager::~AccessControlManager()
|
||||
}
|
||||
|
||||
|
||||
void AccessControlManager::loadFromConfig(const Poco::Util::AbstractConfiguration & users_config)
|
||||
void AccessControlManager::setLocalDirectory(const String & directory_path)
|
||||
{
|
||||
auto & users_config_access_storage = dynamic_cast<UsersConfigAccessStorage &>(getStorageByIndex(1));
|
||||
users_config_access_storage.loadFromConfig(users_config);
|
||||
auto & disk_access_storage = dynamic_cast<DiskAccessStorage &>(getStorageByIndex(DISK_ACCESS_STORAGE_INDEX));
|
||||
disk_access_storage.setDirectory(directory_path);
|
||||
}
|
||||
|
||||
|
||||
void AccessControlManager::setUsersConfig(const Poco::Util::AbstractConfiguration & users_config)
|
||||
{
|
||||
auto & users_config_access_storage = dynamic_cast<UsersConfigAccessStorage &>(getStorageByIndex(USERS_CONFIG_ACCESS_STORAGE_INDEX));
|
||||
users_config_access_storage.setConfiguration(users_config);
|
||||
}
|
||||
|
||||
|
||||
|
@ -45,7 +45,8 @@ public:
|
||||
AccessControlManager();
|
||||
~AccessControlManager();
|
||||
|
||||
void loadFromConfig(const Poco::Util::AbstractConfiguration & users_config);
|
||||
void setLocalDirectory(const String & directory);
|
||||
void setUsersConfig(const Poco::Util::AbstractConfiguration & users_config);
|
||||
|
||||
AccessRightsContextPtr getAccessRightsContext(
|
||||
const UUID & user_id,
|
||||
|
@ -9,6 +9,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INVALID_GRANT;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
@ -45,6 +46,13 @@ namespace
|
||||
const AccessFlags create_table_flag = AccessType::CREATE_TABLE;
|
||||
const AccessFlags create_temporary_table_flag = AccessType::CREATE_TEMPORARY_TABLE;
|
||||
};
|
||||
|
||||
std::string_view checkCurrentDatabase(const std::string_view & current_database)
|
||||
{
|
||||
if (current_database.empty())
|
||||
throw Exception("No current database", ErrorCodes::LOGICAL_ERROR);
|
||||
return current_database;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -520,21 +528,21 @@ void AccessRights::grantImpl(const AccessRightsElement & element, std::string_vi
|
||||
else if (element.any_table)
|
||||
{
|
||||
if (element.database.empty())
|
||||
grantImpl(element.access_flags, current_database);
|
||||
grantImpl(element.access_flags, checkCurrentDatabase(current_database));
|
||||
else
|
||||
grantImpl(element.access_flags, element.database);
|
||||
}
|
||||
else if (element.any_column)
|
||||
{
|
||||
if (element.database.empty())
|
||||
grantImpl(element.access_flags, current_database, element.table);
|
||||
grantImpl(element.access_flags, checkCurrentDatabase(current_database), element.table);
|
||||
else
|
||||
grantImpl(element.access_flags, element.database, element.table);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (element.database.empty())
|
||||
grantImpl(element.access_flags, current_database, element.table, element.columns);
|
||||
grantImpl(element.access_flags, checkCurrentDatabase(current_database), element.table, element.columns);
|
||||
else
|
||||
grantImpl(element.access_flags, element.database, element.table, element.columns);
|
||||
}
|
||||
@ -575,21 +583,21 @@ void AccessRights::revokeImpl(const AccessRightsElement & element, std::string_v
|
||||
else if (element.any_table)
|
||||
{
|
||||
if (element.database.empty())
|
||||
revokeImpl<mode>(element.access_flags, current_database);
|
||||
revokeImpl<mode>(element.access_flags, checkCurrentDatabase(current_database));
|
||||
else
|
||||
revokeImpl<mode>(element.access_flags, element.database);
|
||||
}
|
||||
else if (element.any_column)
|
||||
{
|
||||
if (element.database.empty())
|
||||
revokeImpl<mode>(element.access_flags, current_database, element.table);
|
||||
revokeImpl<mode>(element.access_flags, checkCurrentDatabase(current_database), element.table);
|
||||
else
|
||||
revokeImpl<mode>(element.access_flags, element.database, element.table);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (element.database.empty())
|
||||
revokeImpl<mode>(element.access_flags, current_database, element.table, element.columns);
|
||||
revokeImpl<mode>(element.access_flags, checkCurrentDatabase(current_database), element.table, element.columns);
|
||||
else
|
||||
revokeImpl<mode>(element.access_flags, element.database, element.table, element.columns);
|
||||
}
|
||||
@ -710,21 +718,21 @@ bool AccessRights::isGrantedImpl(const AccessRightsElement & element, std::strin
|
||||
else if (element.any_table)
|
||||
{
|
||||
if (element.database.empty())
|
||||
return isGrantedImpl(element.access_flags, current_database);
|
||||
return isGrantedImpl(element.access_flags, checkCurrentDatabase(current_database));
|
||||
else
|
||||
return isGrantedImpl(element.access_flags, element.database);
|
||||
}
|
||||
else if (element.any_column)
|
||||
{
|
||||
if (element.database.empty())
|
||||
return isGrantedImpl(element.access_flags, current_database, element.table);
|
||||
return isGrantedImpl(element.access_flags, checkCurrentDatabase(current_database), element.table);
|
||||
else
|
||||
return isGrantedImpl(element.access_flags, element.database, element.table);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (element.database.empty())
|
||||
return isGrantedImpl(element.access_flags, current_database, element.table, element.columns);
|
||||
return isGrantedImpl(element.access_flags, checkCurrentDatabase(current_database), element.table, element.columns);
|
||||
else
|
||||
return isGrantedImpl(element.access_flags, element.database, element.table, element.columns);
|
||||
}
|
||||
|
775
dbms/src/Access/DiskAccessStorage.cpp
Normal file
775
dbms/src/Access/DiskAccessStorage.cpp
Normal file
@ -0,0 +1,775 @@
|
||||
#include <Access/DiskAccessStorage.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <Access/User.h>
|
||||
#include <Access/Role.h>
|
||||
#include <Access/RowPolicy.h>
|
||||
#include <Access/Quota.h>
|
||||
#include <Parsers/ASTCreateUserQuery.h>
|
||||
#include <Parsers/ASTCreateRoleQuery.h>
|
||||
#include <Parsers/ASTCreateRowPolicyQuery.h>
|
||||
#include <Parsers/ASTCreateQuotaQuery.h>
|
||||
#include <Parsers/ASTGrantQuery.h>
|
||||
#include <Parsers/ParserCreateUserQuery.h>
|
||||
#include <Parsers/ParserCreateRoleQuery.h>
|
||||
#include <Parsers/ParserCreateRowPolicyQuery.h>
|
||||
#include <Parsers/ParserCreateQuotaQuery.h>
|
||||
#include <Parsers/ParserGrantQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Interpreters/InterpreterCreateUserQuery.h>
|
||||
#include <Interpreters/InterpreterCreateRoleQuery.h>
|
||||
#include <Interpreters/InterpreterCreateRowPolicyQuery.h>
|
||||
#include <Interpreters/InterpreterCreateQuotaQuery.h>
|
||||
#include <Interpreters/InterpreterGrantQuery.h>
|
||||
#include <Interpreters/InterpreterShowCreateAccessEntityQuery.h>
|
||||
#include <Interpreters/InterpreterShowGrantsQuery.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <boost/range/adaptor/map.hpp>
|
||||
#include <boost/range/algorithm/copy.hpp>
|
||||
#include <boost/range/algorithm_ext/push_back.hpp>
|
||||
#include <filesystem>
|
||||
#include <fstream>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int DIRECTORY_DOESNT_EXIST;
|
||||
extern const int FILE_DOESNT_EXIST;
|
||||
extern const int INCORRECT_ACCESS_ENTITY_DEFINITION;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
/// Special parser for the 'ATTACH access entity' queries.
|
||||
class ParserAttachAccessEntity : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "ATTACH access entity query"; }
|
||||
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
|
||||
{
|
||||
if (ParserCreateUserQuery{}.enableAttachMode(true).parse(pos, node, expected))
|
||||
return true;
|
||||
if (ParserCreateRoleQuery{}.enableAttachMode(true).parse(pos, node, expected))
|
||||
return true;
|
||||
if (ParserCreateRowPolicyQuery{}.enableAttachMode(true).parse(pos, node, expected))
|
||||
return true;
|
||||
if (ParserCreateQuotaQuery{}.enableAttachMode(true).parse(pos, node, expected))
|
||||
return true;
|
||||
if (ParserGrantQuery{}.enableAttachMode(true).parse(pos, node, expected))
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// Reads a file containing ATTACH queries and then parses it to build an access entity.
|
||||
AccessEntityPtr readAccessEntityFile(const std::filesystem::path & file_path)
|
||||
{
|
||||
/// Read the file.
|
||||
ReadBufferFromFile in{file_path};
|
||||
String file_contents;
|
||||
readStringUntilEOF(file_contents, in);
|
||||
|
||||
/// Parse the file contents.
|
||||
ASTs queries;
|
||||
ParserAttachAccessEntity parser;
|
||||
const char * begin = file_contents.data(); /// begin of current query
|
||||
const char * pos = begin; /// parser moves pos from begin to the end of current query
|
||||
const char * end = begin + file_contents.size();
|
||||
while (pos < end)
|
||||
{
|
||||
queries.emplace_back(parseQueryAndMovePosition(parser, pos, end, "", true, 0));
|
||||
while (isWhitespaceASCII(*pos) || *pos == ';')
|
||||
++pos;
|
||||
}
|
||||
|
||||
/// Interpret the AST to build an access entity.
|
||||
std::shared_ptr<User> user;
|
||||
std::shared_ptr<Role> role;
|
||||
std::shared_ptr<RowPolicy> policy;
|
||||
std::shared_ptr<Quota> quota;
|
||||
AccessEntityPtr res;
|
||||
|
||||
for (const auto & query : queries)
|
||||
{
|
||||
if (auto create_user_query = query->as<ASTCreateUserQuery>())
|
||||
{
|
||||
if (res)
|
||||
throw Exception("Two access entities in one file " + file_path.string(), ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
|
||||
res = user = std::make_unique<User>();
|
||||
InterpreterCreateUserQuery::updateUserFromQuery(*user, *create_user_query);
|
||||
}
|
||||
else if (auto create_role_query = query->as<ASTCreateRoleQuery>())
|
||||
{
|
||||
if (res)
|
||||
throw Exception("Two access entities in one file " + file_path.string(), ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
|
||||
res = role = std::make_unique<Role>();
|
||||
InterpreterCreateRoleQuery::updateRoleFromQuery(*role, *create_role_query);
|
||||
}
|
||||
else if (auto create_policy_query = query->as<ASTCreateRowPolicyQuery>())
|
||||
{
|
||||
if (res)
|
||||
throw Exception("Two access entities in one file " + file_path.string(), ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
|
||||
res = policy = std::make_unique<RowPolicy>();
|
||||
InterpreterCreateRowPolicyQuery::updateRowPolicyFromQuery(*policy, *create_policy_query);
|
||||
}
|
||||
else if (auto create_quota_query = query->as<ASTCreateQuotaQuery>())
|
||||
{
|
||||
if (res)
|
||||
throw Exception("Two access entities are attached in the same file " + file_path.string(), ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
|
||||
res = quota = std::make_unique<Quota>();
|
||||
InterpreterCreateQuotaQuery::updateQuotaFromQuery(*quota, *create_quota_query);
|
||||
}
|
||||
else if (auto grant_query = query->as<ASTGrantQuery>())
|
||||
{
|
||||
if (!user && !role)
|
||||
throw Exception("A user or role should be attached before grant in file " + file_path.string(), ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
|
||||
if (user)
|
||||
InterpreterGrantQuery::updateUserFromQuery(*user, *grant_query);
|
||||
else
|
||||
InterpreterGrantQuery::updateRoleFromQuery(*role, *grant_query);
|
||||
}
|
||||
else
|
||||
throw Exception("Two access entities are attached in the same file " + file_path.string(), ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
|
||||
}
|
||||
|
||||
if (!res)
|
||||
throw Exception("No access entities attached in file " + file_path.string(), ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
/// Writes ATTACH queries for building a specified access entity to a file.
|
||||
void writeAccessEntityFile(const std::filesystem::path & file_path, const IAccessEntity & entity)
|
||||
{
|
||||
/// Build list of ATTACH queries.
|
||||
ASTs queries;
|
||||
queries.push_back(InterpreterShowCreateAccessEntityQuery::getAttachQuery(entity));
|
||||
if (entity.getType() == typeid(User) || entity.getType() == typeid(Role))
|
||||
boost::range::push_back(queries, InterpreterShowGrantsQuery::getAttachGrantQueries(entity));
|
||||
|
||||
/// Serialize the list of ATTACH queries to a string.
|
||||
std::stringstream ss;
|
||||
for (const ASTPtr & query : queries)
|
||||
ss << *query << ";\n";
|
||||
String file_contents = std::move(ss).str();
|
||||
|
||||
/// First we save *.tmp file and then we rename if everything's ok.
|
||||
auto tmp_file_path = std::filesystem::path{file_path}.replace_extension(".tmp");
|
||||
bool succeeded = false;
|
||||
SCOPE_EXIT(
|
||||
{
|
||||
if (!succeeded)
|
||||
std::filesystem::remove(tmp_file_path);
|
||||
});
|
||||
|
||||
/// Write the file.
|
||||
WriteBufferFromFile out{tmp_file_path.string()};
|
||||
out.write(file_contents.data(), file_contents.size());
|
||||
|
||||
/// Rename.
|
||||
std::filesystem::rename(tmp_file_path, file_path);
|
||||
succeeded = true;
|
||||
}
|
||||
|
||||
|
||||
/// Calculates the path to a file named <id>.sql for saving an access entity.
|
||||
std::filesystem::path getAccessEntityFilePath(const String & directory_path, const UUID & id)
|
||||
{
|
||||
return std::filesystem::path(directory_path).append(toString(id)).replace_extension(".sql");
|
||||
}
|
||||
|
||||
|
||||
/// Reads a map of name of access entity to UUID for access entities of some type from a file.
|
||||
std::unordered_map<String, UUID> readListFile(const std::filesystem::path & file_path)
|
||||
{
|
||||
ReadBufferFromFile in(file_path);
|
||||
|
||||
size_t num;
|
||||
readVarUInt(num, in);
|
||||
std::unordered_map<String, UUID> res;
|
||||
res.reserve(num);
|
||||
|
||||
for (size_t i = 0; i != num; ++i)
|
||||
{
|
||||
String name;
|
||||
readStringBinary(name, in);
|
||||
UUID id;
|
||||
readUUIDText(id, in);
|
||||
res[name] = id;
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
/// Writes a map of name of access entity to UUID for access entities of some type to a file.
|
||||
void writeListFile(const std::filesystem::path & file_path, const std::unordered_map<String, UUID> & map)
|
||||
{
|
||||
WriteBufferFromFile out(file_path);
|
||||
writeVarUInt(map.size(), out);
|
||||
for (const auto & [name, id] : map)
|
||||
{
|
||||
writeStringBinary(name, out);
|
||||
writeUUIDText(id, out);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/// Calculates the path for storing a map of name of access entity to UUID for access entities of some type.
|
||||
std::filesystem::path getListFilePath(const String & directory_path, std::type_index type)
|
||||
{
|
||||
std::string_view file_name;
|
||||
if (type == typeid(User))
|
||||
file_name = "users";
|
||||
else if (type == typeid(Role))
|
||||
file_name = "roles";
|
||||
else if (type == typeid(Quota))
|
||||
file_name = "quotas";
|
||||
else if (type == typeid(RowPolicy))
|
||||
file_name = "row_policies";
|
||||
else
|
||||
throw Exception("Unexpected type of access entity: " + IAccessEntity::getTypeName(type),
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return std::filesystem::path(directory_path).append(file_name).replace_extension(".list");
|
||||
}
|
||||
|
||||
|
||||
/// Calculates the path to a temporary file which existence means that list files are corrupted
|
||||
/// and need to be rebuild.
|
||||
std::filesystem::path getNeedRebuildListsMarkFilePath(const String & directory_path)
|
||||
{
|
||||
return std::filesystem::path(directory_path).append("need_rebuild_lists.mark");
|
||||
}
|
||||
|
||||
|
||||
static const std::vector<std::type_index> & getAllAccessEntityTypes()
|
||||
{
|
||||
static const std::vector<std::type_index> res = {typeid(User), typeid(Role), typeid(RowPolicy), typeid(Quota)};
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
bool tryParseUUID(const String & str, UUID & id)
|
||||
{
|
||||
try
|
||||
{
|
||||
id = parseFromString<UUID>(str);
|
||||
return true;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
DiskAccessStorage::DiskAccessStorage()
|
||||
: IAccessStorage("disk")
|
||||
{
|
||||
for (const auto & type : getAllAccessEntityTypes())
|
||||
name_to_id_maps[type];
|
||||
}
|
||||
|
||||
|
||||
DiskAccessStorage::~DiskAccessStorage()
|
||||
{
|
||||
stopListsWritingThread();
|
||||
writeLists();
|
||||
}
|
||||
|
||||
|
||||
void DiskAccessStorage::setDirectory(const String & directory_path_)
|
||||
{
|
||||
Notifications notifications;
|
||||
SCOPE_EXIT({ notify(notifications); });
|
||||
|
||||
std::lock_guard lock{mutex};
|
||||
initialize(directory_path_, notifications);
|
||||
}
|
||||
|
||||
|
||||
void DiskAccessStorage::initialize(const String & directory_path_, Notifications & notifications)
|
||||
{
|
||||
auto canonical_directory_path = std::filesystem::weakly_canonical(directory_path_);
|
||||
|
||||
if (initialized)
|
||||
{
|
||||
if (directory_path == canonical_directory_path)
|
||||
return;
|
||||
throw Exception("Storage " + getStorageName() + " already initialized with another directory", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
std::filesystem::create_directories(canonical_directory_path);
|
||||
if (!std::filesystem::exists(canonical_directory_path) || !std::filesystem::is_directory(canonical_directory_path))
|
||||
throw Exception("Couldn't create directory " + canonical_directory_path.string(), ErrorCodes::DIRECTORY_DOESNT_EXIST);
|
||||
|
||||
directory_path = canonical_directory_path;
|
||||
initialized = true;
|
||||
|
||||
bool should_rebuild_lists = std::filesystem::exists(getNeedRebuildListsMarkFilePath(directory_path));
|
||||
if (!should_rebuild_lists)
|
||||
{
|
||||
if (!readLists())
|
||||
should_rebuild_lists = true;
|
||||
}
|
||||
|
||||
if (should_rebuild_lists)
|
||||
{
|
||||
rebuildLists();
|
||||
writeLists();
|
||||
}
|
||||
|
||||
for (const auto & [id, entry] : id_to_entry_map)
|
||||
prepareNotifications(id, entry, false, notifications);
|
||||
}
|
||||
|
||||
|
||||
bool DiskAccessStorage::readLists()
|
||||
{
|
||||
assert(id_to_entry_map.empty());
|
||||
assert(name_to_id_maps.size() == getAllAccessEntityTypes().size());
|
||||
bool ok = true;
|
||||
for (auto & [type, name_to_id_map] : name_to_id_maps)
|
||||
{
|
||||
auto file_path = getListFilePath(directory_path, type);
|
||||
if (!std::filesystem::exists(file_path))
|
||||
{
|
||||
LOG_WARNING(getLogger(), "File " + file_path.string() + " doesn't exist");
|
||||
ok = false;
|
||||
break;
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
name_to_id_map = readListFile(file_path);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(getLogger(), "Could not read " + file_path.string());
|
||||
ok = false;
|
||||
break;
|
||||
}
|
||||
for (const auto & [name, id] : name_to_id_map)
|
||||
id_to_entry_map.emplace(id, Entry{name, type});
|
||||
}
|
||||
|
||||
if (!ok)
|
||||
{
|
||||
id_to_entry_map.clear();
|
||||
for (auto & name_to_id_map : name_to_id_maps | boost::adaptors::map_values)
|
||||
name_to_id_map.clear();
|
||||
}
|
||||
return ok;
|
||||
}
|
||||
|
||||
|
||||
void DiskAccessStorage::writeLists()
|
||||
{
|
||||
if (failed_to_write_lists || types_of_lists_to_write.empty())
|
||||
return; /// We don't try to write list files after the first fail.
|
||||
/// The next restart of the server will invoke rebuilding of the list files.
|
||||
|
||||
for (const auto & type : types_of_lists_to_write)
|
||||
{
|
||||
const auto & name_to_id_map = name_to_id_maps.at(type);
|
||||
auto file_path = getListFilePath(directory_path, type);
|
||||
try
|
||||
{
|
||||
writeListFile(file_path, name_to_id_map);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(getLogger(), "Could not write " + file_path.string());
|
||||
failed_to_write_lists = true;
|
||||
types_of_lists_to_write.clear();
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
/// The list files was successfully written, we don't need the 'need_rebuild_lists.mark' file any longer.
|
||||
std::filesystem::remove(getNeedRebuildListsMarkFilePath(directory_path));
|
||||
types_of_lists_to_write.clear();
|
||||
}
|
||||
|
||||
|
||||
void DiskAccessStorage::scheduleWriteLists(std::type_index type)
|
||||
{
|
||||
if (failed_to_write_lists)
|
||||
return;
|
||||
|
||||
bool already_scheduled = !types_of_lists_to_write.empty();
|
||||
types_of_lists_to_write.insert(type);
|
||||
|
||||
if (already_scheduled)
|
||||
return;
|
||||
|
||||
/// Create the 'need_rebuild_lists.mark' file.
|
||||
/// This file will be used later to find out if writing lists is successful or not.
|
||||
std::ofstream{getNeedRebuildListsMarkFilePath(directory_path)};
|
||||
|
||||
startListsWritingThread();
|
||||
}
|
||||
|
||||
|
||||
void DiskAccessStorage::startListsWritingThread()
|
||||
{
|
||||
if (lists_writing_thread.joinable())
|
||||
{
|
||||
if (!lists_writing_thread_exited)
|
||||
return;
|
||||
lists_writing_thread.detach();
|
||||
}
|
||||
|
||||
lists_writing_thread_exited = false;
|
||||
lists_writing_thread = ThreadFromGlobalPool{&DiskAccessStorage::listsWritingThreadFunc, this};
|
||||
}
|
||||
|
||||
|
||||
void DiskAccessStorage::stopListsWritingThread()
|
||||
{
|
||||
if (lists_writing_thread.joinable())
|
||||
{
|
||||
lists_writing_thread_should_exit.notify_one();
|
||||
lists_writing_thread.join();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void DiskAccessStorage::listsWritingThreadFunc()
|
||||
{
|
||||
std::unique_lock lock{mutex};
|
||||
SCOPE_EXIT({ lists_writing_thread_exited = true; });
|
||||
|
||||
/// It's better not to write the lists files too often, that's why we need
|
||||
/// the following timeout.
|
||||
const auto timeout = std::chrono::minutes(1);
|
||||
if (lists_writing_thread_should_exit.wait_for(lock, timeout) != std::cv_status::timeout)
|
||||
return; /// The destructor requires us to exit.
|
||||
|
||||
writeLists();
|
||||
}
|
||||
|
||||
|
||||
/// Reads and parses all the "<id>.sql" files from a specified directory
|
||||
/// and then saves the files "users.list", "roles.list", etc. to the same directory.
|
||||
void DiskAccessStorage::rebuildLists()
|
||||
{
|
||||
LOG_WARNING(getLogger(), "Recovering lists in directory " + directory_path);
|
||||
assert(id_to_entry_map.empty());
|
||||
for (const auto & directory_entry : std::filesystem::directory_iterator(directory_path))
|
||||
{
|
||||
if (!directory_entry.is_regular_file())
|
||||
continue;
|
||||
const auto & path = directory_entry.path();
|
||||
if (path.extension() != ".sql")
|
||||
continue;
|
||||
|
||||
UUID id;
|
||||
if (!tryParseUUID(path.stem(), id))
|
||||
continue;
|
||||
|
||||
auto entity = readAccessEntityFile(getAccessEntityFilePath(directory_path, id));
|
||||
auto type = entity->getType();
|
||||
auto & name_to_id_map = name_to_id_maps[type];
|
||||
auto it_by_name = name_to_id_map.emplace(entity->getFullName(), id).first;
|
||||
id_to_entry_map.emplace(id, Entry{it_by_name->first, type});
|
||||
}
|
||||
|
||||
boost::range::copy(getAllAccessEntityTypes(), std::inserter(types_of_lists_to_write, types_of_lists_to_write.end()));
|
||||
}
|
||||
|
||||
|
||||
std::optional<UUID> DiskAccessStorage::findImpl(std::type_index type, const String & name) const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
const auto & name_to_id_map = name_to_id_maps.at(type);
|
||||
auto it = name_to_id_map.find(name);
|
||||
if (it == name_to_id_map.end())
|
||||
return {};
|
||||
return it->second;
|
||||
}
|
||||
|
||||
|
||||
std::vector<UUID> DiskAccessStorage::findAllImpl(std::type_index type) const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
const auto & name_to_id_map = name_to_id_maps.at(type);
|
||||
std::vector<UUID> res;
|
||||
res.reserve(name_to_id_map.size());
|
||||
boost::range::copy(name_to_id_map | boost::adaptors::map_values, std::back_inserter(res));
|
||||
return res;
|
||||
}
|
||||
|
||||
bool DiskAccessStorage::existsImpl(const UUID & id) const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
return id_to_entry_map.contains(id);
|
||||
}
|
||||
|
||||
|
||||
AccessEntityPtr DiskAccessStorage::readImpl(const UUID & id) const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
auto it = id_to_entry_map.find(id);
|
||||
if (it == id_to_entry_map.end())
|
||||
throwNotFound(id);
|
||||
|
||||
auto & entry = it->second;
|
||||
if (!entry.entity)
|
||||
entry.entity = readAccessEntityFromDisk(id);
|
||||
return entry.entity;
|
||||
}
|
||||
|
||||
|
||||
String DiskAccessStorage::readNameImpl(const UUID & id) const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
auto it = id_to_entry_map.find(id);
|
||||
if (it == id_to_entry_map.end())
|
||||
throwNotFound(id);
|
||||
return String{it->second.name};
|
||||
}
|
||||
|
||||
|
||||
bool DiskAccessStorage::canInsertImpl(const AccessEntityPtr &) const
|
||||
{
|
||||
return initialized;
|
||||
}
|
||||
|
||||
|
||||
UUID DiskAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists)
|
||||
{
|
||||
Notifications notifications;
|
||||
SCOPE_EXIT({ notify(notifications); });
|
||||
|
||||
UUID id = generateRandomID();
|
||||
std::lock_guard lock{mutex};
|
||||
insertNoLock(generateRandomID(), new_entity, replace_if_exists, notifications);
|
||||
return id;
|
||||
}
|
||||
|
||||
|
||||
void DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, Notifications & notifications)
|
||||
{
|
||||
const String & name = new_entity->getFullName();
|
||||
std::type_index type = new_entity->getType();
|
||||
if (!initialized)
|
||||
throw Exception(
|
||||
"Cannot insert " + new_entity->getTypeName() + " " + backQuote(name) + " to " + getStorageName()
|
||||
+ " because the output directory is not set",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
/// Check that we can insert.
|
||||
auto it_by_id = id_to_entry_map.find(id);
|
||||
if (it_by_id != id_to_entry_map.end())
|
||||
{
|
||||
const auto & existing_entry = it_by_id->second;
|
||||
throwIDCollisionCannotInsert(id, type, name, existing_entry.entity->getType(), existing_entry.entity->getFullName());
|
||||
}
|
||||
|
||||
auto & name_to_id_map = name_to_id_maps.at(type);
|
||||
auto it_by_name = name_to_id_map.find(name);
|
||||
bool name_collision = (it_by_name != name_to_id_map.end());
|
||||
|
||||
if (name_collision && !replace_if_exists)
|
||||
throwNameCollisionCannotInsert(type, name);
|
||||
|
||||
scheduleWriteLists(type);
|
||||
writeAccessEntityToDisk(id, *new_entity);
|
||||
|
||||
if (name_collision && replace_if_exists)
|
||||
removeNoLock(it_by_name->second, notifications);
|
||||
|
||||
/// Do insertion.
|
||||
it_by_name = name_to_id_map.emplace(name, id).first;
|
||||
it_by_id = id_to_entry_map.emplace(id, Entry{it_by_name->first, type}).first;
|
||||
auto & entry = it_by_id->second;
|
||||
entry.entity = new_entity;
|
||||
prepareNotifications(id, entry, false, notifications);
|
||||
}
|
||||
|
||||
|
||||
void DiskAccessStorage::removeImpl(const UUID & id)
|
||||
{
|
||||
Notifications notifications;
|
||||
SCOPE_EXIT({ notify(notifications); });
|
||||
|
||||
std::lock_guard lock{mutex};
|
||||
removeNoLock(id, notifications);
|
||||
}
|
||||
|
||||
|
||||
void DiskAccessStorage::removeNoLock(const UUID & id, Notifications & notifications)
|
||||
{
|
||||
auto it = id_to_entry_map.find(id);
|
||||
if (it == id_to_entry_map.end())
|
||||
throwNotFound(id);
|
||||
|
||||
Entry & entry = it->second;
|
||||
String name{it->second.name};
|
||||
std::type_index type = it->second.type;
|
||||
|
||||
scheduleWriteLists(type);
|
||||
deleteAccessEntityOnDisk(id);
|
||||
|
||||
/// Do removing.
|
||||
prepareNotifications(id, entry, true, notifications);
|
||||
id_to_entry_map.erase(it);
|
||||
auto & name_to_id_map = name_to_id_maps.at(type);
|
||||
name_to_id_map.erase(name);
|
||||
}
|
||||
|
||||
|
||||
void DiskAccessStorage::updateImpl(const UUID & id, const UpdateFunc & update_func)
|
||||
{
|
||||
Notifications notifications;
|
||||
SCOPE_EXIT({ notify(notifications); });
|
||||
|
||||
std::lock_guard lock{mutex};
|
||||
updateNoLock(id, update_func, notifications);
|
||||
}
|
||||
|
||||
|
||||
void DiskAccessStorage::updateNoLock(const UUID & id, const UpdateFunc & update_func, Notifications & notifications)
|
||||
{
|
||||
auto it = id_to_entry_map.find(id);
|
||||
if (it == id_to_entry_map.end())
|
||||
throwNotFound(id);
|
||||
|
||||
Entry & entry = it->second;
|
||||
if (!entry.entity)
|
||||
entry.entity = readAccessEntityFromDisk(id);
|
||||
auto old_entity = entry.entity;
|
||||
auto new_entity = update_func(old_entity);
|
||||
|
||||
if (*new_entity == *old_entity)
|
||||
return;
|
||||
|
||||
String new_name = new_entity->getFullName();
|
||||
auto old_name = entry.name;
|
||||
const std::type_index type = entry.type;
|
||||
bool name_changed = (new_name != old_name);
|
||||
if (name_changed)
|
||||
{
|
||||
const auto & name_to_id_map = name_to_id_maps.at(type);
|
||||
if (name_to_id_map.contains(new_name))
|
||||
throwNameCollisionCannotRename(type, String{old_name}, new_name);
|
||||
scheduleWriteLists(type);
|
||||
}
|
||||
|
||||
writeAccessEntityToDisk(id, *new_entity);
|
||||
entry.entity = new_entity;
|
||||
|
||||
if (name_changed)
|
||||
{
|
||||
auto & name_to_id_map = name_to_id_maps.at(type);
|
||||
name_to_id_map.erase(String{old_name});
|
||||
auto it_by_name = name_to_id_map.emplace(new_name, id).first;
|
||||
entry.name = it_by_name->first;
|
||||
}
|
||||
|
||||
prepareNotifications(id, entry, false, notifications);
|
||||
}
|
||||
|
||||
|
||||
AccessEntityPtr DiskAccessStorage::readAccessEntityFromDisk(const UUID & id) const
|
||||
{
|
||||
return readAccessEntityFile(getAccessEntityFilePath(directory_path, id));
|
||||
}
|
||||
|
||||
|
||||
void DiskAccessStorage::writeAccessEntityToDisk(const UUID & id, const IAccessEntity & entity) const
|
||||
{
|
||||
writeAccessEntityFile(getAccessEntityFilePath(directory_path, id), entity);
|
||||
}
|
||||
|
||||
|
||||
void DiskAccessStorage::deleteAccessEntityOnDisk(const UUID & id) const
|
||||
{
|
||||
auto file_path = getAccessEntityFilePath(directory_path, id);
|
||||
if (!std::filesystem::remove(file_path))
|
||||
throw Exception("Couldn't delete " + file_path.string(), ErrorCodes::FILE_DOESNT_EXIST);
|
||||
}
|
||||
|
||||
|
||||
void DiskAccessStorage::prepareNotifications(const UUID & id, const Entry & entry, bool remove, Notifications & notifications) const
|
||||
{
|
||||
if (!remove && !entry.entity)
|
||||
return;
|
||||
|
||||
const AccessEntityPtr entity = remove ? nullptr : entry.entity;
|
||||
for (const auto & handler : entry.handlers_by_id)
|
||||
notifications.push_back({handler, id, entity});
|
||||
|
||||
auto range = handlers_by_type.equal_range(entry.type);
|
||||
for (auto it = range.first; it != range.second; ++it)
|
||||
notifications.push_back({it->second, id, entity});
|
||||
}
|
||||
|
||||
|
||||
ext::scope_guard DiskAccessStorage::subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
auto it = id_to_entry_map.find(id);
|
||||
if (it == id_to_entry_map.end())
|
||||
return {};
|
||||
const Entry & entry = it->second;
|
||||
auto handler_it = entry.handlers_by_id.insert(entry.handlers_by_id.end(), handler);
|
||||
|
||||
return [this, id, handler_it]
|
||||
{
|
||||
std::lock_guard lock2{mutex};
|
||||
auto it2 = id_to_entry_map.find(id);
|
||||
if (it2 != id_to_entry_map.end())
|
||||
{
|
||||
const Entry & entry2 = it2->second;
|
||||
entry2.handlers_by_id.erase(handler_it);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
ext::scope_guard DiskAccessStorage::subscribeForChangesImpl(std::type_index type, const OnChangedHandler & handler) const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
auto handler_it = handlers_by_type.emplace(type, handler);
|
||||
|
||||
return [this, handler_it]
|
||||
{
|
||||
std::lock_guard lock2{mutex};
|
||||
handlers_by_type.erase(handler_it);
|
||||
};
|
||||
}
|
||||
|
||||
bool DiskAccessStorage::hasSubscriptionImpl(const UUID & id) const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
auto it = id_to_entry_map.find(id);
|
||||
if (it != id_to_entry_map.end())
|
||||
{
|
||||
const Entry & entry = it->second;
|
||||
return !entry.handlers_by_id.empty();
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
bool DiskAccessStorage::hasSubscriptionImpl(std::type_index type) const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
auto range = handlers_by_type.equal_range(type);
|
||||
return range.first != range.second;
|
||||
}
|
||||
|
||||
}
|
76
dbms/src/Access/DiskAccessStorage.h
Normal file
76
dbms/src/Access/DiskAccessStorage.h
Normal file
@ -0,0 +1,76 @@
|
||||
#pragma once
|
||||
|
||||
#include <Access/MemoryAccessStorage.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <boost/container/flat_set.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/// Loads and saves access entities on a local disk to a specified directory.
|
||||
class DiskAccessStorage : public IAccessStorage
|
||||
{
|
||||
public:
|
||||
DiskAccessStorage();
|
||||
~DiskAccessStorage() override;
|
||||
|
||||
void setDirectory(const String & directory_path_);
|
||||
|
||||
private:
|
||||
std::optional<UUID> findImpl(std::type_index type, const String & name) const override;
|
||||
std::vector<UUID> findAllImpl(std::type_index type) const override;
|
||||
bool existsImpl(const UUID & id) const override;
|
||||
AccessEntityPtr readImpl(const UUID & id) const override;
|
||||
String readNameImpl(const UUID & id) const override;
|
||||
bool canInsertImpl(const AccessEntityPtr & entity) const override;
|
||||
UUID insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) override;
|
||||
void removeImpl(const UUID & id) override;
|
||||
void updateImpl(const UUID & id, const UpdateFunc & update_func) override;
|
||||
ext::scope_guard subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const override;
|
||||
ext::scope_guard subscribeForChangesImpl(std::type_index type, const OnChangedHandler & handler) const override;
|
||||
bool hasSubscriptionImpl(const UUID & id) const override;
|
||||
bool hasSubscriptionImpl(std::type_index type) const override;
|
||||
|
||||
void initialize(const String & directory_path_, Notifications & notifications);
|
||||
bool readLists();
|
||||
void writeLists();
|
||||
void scheduleWriteLists(std::type_index type);
|
||||
void rebuildLists();
|
||||
|
||||
void startListsWritingThread();
|
||||
void stopListsWritingThread();
|
||||
void listsWritingThreadFunc();
|
||||
|
||||
void insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, Notifications & notifications);
|
||||
void removeNoLock(const UUID & id, Notifications & notifications);
|
||||
void updateNoLock(const UUID & id, const UpdateFunc & update_func, Notifications & notifications);
|
||||
|
||||
AccessEntityPtr readAccessEntityFromDisk(const UUID & id) const;
|
||||
void writeAccessEntityToDisk(const UUID & id, const IAccessEntity & entity) const;
|
||||
void deleteAccessEntityOnDisk(const UUID & id) const;
|
||||
|
||||
using NameToIDMap = std::unordered_map<String, UUID>;
|
||||
struct Entry
|
||||
{
|
||||
Entry(const std::string_view & name_, std::type_index type_) : name(name_), type(type_) {}
|
||||
std::string_view name; /// view points to a string in `name_to_id_maps`.
|
||||
std::type_index type;
|
||||
mutable AccessEntityPtr entity; /// may be nullptr, if the entity hasn't been loaded yet.
|
||||
mutable std::list<OnChangedHandler> handlers_by_id;
|
||||
};
|
||||
|
||||
void prepareNotifications(const UUID & id, const Entry & entry, bool remove, Notifications & notifications) const;
|
||||
|
||||
String directory_path;
|
||||
bool initialized = false;
|
||||
std::unordered_map<std::type_index, NameToIDMap> name_to_id_maps;
|
||||
std::unordered_map<UUID, Entry> id_to_entry_map;
|
||||
boost::container::flat_set<std::type_index> types_of_lists_to_write;
|
||||
bool failed_to_write_lists = false; /// Whether writing of the list files has been failed since the recent restart of the server.
|
||||
ThreadFromGlobalPool lists_writing_thread; /// List files are written in a separate thread.
|
||||
std::condition_variable lists_writing_thread_should_exit; /// Signals `lists_writing_thread` to exit.
|
||||
std::atomic<bool> lists_writing_thread_exited = false;
|
||||
mutable std::unordered_multimap<std::type_index, OnChangedHandler> handlers_by_type;
|
||||
mutable std::mutex mutex;
|
||||
};
|
||||
}
|
@ -4,6 +4,8 @@
|
||||
#include <Access/Role.h>
|
||||
#include <Parsers/ASTGenericRoleSet.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <boost/range/algorithm/set_algorithm.hpp>
|
||||
#include <boost/range/algorithm/sort.hpp>
|
||||
#include <boost/range/algorithm_ext/push_back.hpp>
|
||||
@ -45,26 +47,51 @@ GenericRoleSet::GenericRoleSet(const boost::container::flat_set<UUID> & ids_)
|
||||
}
|
||||
|
||||
|
||||
GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager, const std::optional<UUID> & current_user_id)
|
||||
GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast)
|
||||
{
|
||||
init(ast, nullptr, nullptr);
|
||||
}
|
||||
|
||||
GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast, const UUID & current_user_id)
|
||||
{
|
||||
init(ast, nullptr, ¤t_user_id);
|
||||
}
|
||||
|
||||
GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager)
|
||||
{
|
||||
init(ast, &manager, nullptr);
|
||||
}
|
||||
|
||||
GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager, const UUID & current_user_id)
|
||||
{
|
||||
init(ast, &manager, ¤t_user_id);
|
||||
}
|
||||
|
||||
void GenericRoleSet::init(const ASTGenericRoleSet & ast, const AccessControlManager * manager, const UUID * current_user_id)
|
||||
{
|
||||
all = ast.all;
|
||||
|
||||
auto name_to_id = [id_mode{ast.id_mode}, manager](const String & name) -> UUID
|
||||
{
|
||||
if (id_mode)
|
||||
return parse<UUID>(name);
|
||||
assert(manager);
|
||||
auto id = manager->find<User>(name);
|
||||
if (id)
|
||||
return *id;
|
||||
return manager->getID<Role>(name);
|
||||
};
|
||||
|
||||
if (!ast.names.empty() && !all)
|
||||
{
|
||||
ids.reserve(ast.names.size());
|
||||
for (const String & name : ast.names)
|
||||
{
|
||||
auto id = manager.find<User>(name);
|
||||
if (!id)
|
||||
id = manager.getID<Role>(name);
|
||||
ids.insert(*id);
|
||||
}
|
||||
ids.insert(name_to_id(name));
|
||||
}
|
||||
|
||||
if (ast.current_user && !all)
|
||||
{
|
||||
if (!current_user_id)
|
||||
throw Exception("Current user is unknown", ErrorCodes::LOGICAL_ERROR);
|
||||
assert(current_user_id);
|
||||
ids.insert(*current_user_id);
|
||||
}
|
||||
|
||||
@ -72,18 +99,12 @@ GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast, const AccessContro
|
||||
{
|
||||
except_ids.reserve(ast.except_names.size());
|
||||
for (const String & except_name : ast.except_names)
|
||||
{
|
||||
auto except_id = manager.find<User>(except_name);
|
||||
if (!except_id)
|
||||
except_id = manager.getID<Role>(except_name);
|
||||
except_ids.insert(*except_id);
|
||||
}
|
||||
except_ids.insert(name_to_id(except_name));
|
||||
}
|
||||
|
||||
if (ast.except_current_user)
|
||||
{
|
||||
if (!current_user_id)
|
||||
throw Exception("Current user is unknown", ErrorCodes::LOGICAL_ERROR);
|
||||
assert(current_user_id);
|
||||
except_ids.insert(*current_user_id);
|
||||
}
|
||||
|
||||
@ -91,7 +112,52 @@ GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast, const AccessContro
|
||||
ids.erase(except_id);
|
||||
}
|
||||
|
||||
std::shared_ptr<ASTGenericRoleSet> GenericRoleSet::toAST(const AccessControlManager & manager) const
|
||||
|
||||
std::shared_ptr<ASTGenericRoleSet> GenericRoleSet::toAST() const
|
||||
{
|
||||
auto ast = std::make_shared<ASTGenericRoleSet>();
|
||||
ast->id_mode = true;
|
||||
ast->all = all;
|
||||
|
||||
if (!ids.empty())
|
||||
{
|
||||
ast->names.reserve(ids.size());
|
||||
for (const UUID & id : ids)
|
||||
ast->names.emplace_back(::DB::toString(id));
|
||||
}
|
||||
|
||||
if (!except_ids.empty())
|
||||
{
|
||||
ast->except_names.reserve(except_ids.size());
|
||||
for (const UUID & except_id : except_ids)
|
||||
ast->except_names.emplace_back(::DB::toString(except_id));
|
||||
}
|
||||
|
||||
return ast;
|
||||
}
|
||||
|
||||
|
||||
String GenericRoleSet::toString() const
|
||||
{
|
||||
auto ast = toAST();
|
||||
return serializeAST(*ast);
|
||||
}
|
||||
|
||||
|
||||
Strings GenericRoleSet::toStrings() const
|
||||
{
|
||||
if (all || !except_ids.empty())
|
||||
return {toString()};
|
||||
|
||||
Strings names;
|
||||
names.reserve(ids.size());
|
||||
for (const UUID & id : ids)
|
||||
names.emplace_back(::DB::toString(id));
|
||||
return names;
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<ASTGenericRoleSet> GenericRoleSet::toASTWithNames(const AccessControlManager & manager) const
|
||||
{
|
||||
auto ast = std::make_shared<ASTGenericRoleSet>();
|
||||
ast->all = all;
|
||||
@ -124,17 +190,17 @@ std::shared_ptr<ASTGenericRoleSet> GenericRoleSet::toAST(const AccessControlMana
|
||||
}
|
||||
|
||||
|
||||
String GenericRoleSet::toString(const AccessControlManager & manager) const
|
||||
String GenericRoleSet::toStringWithNames(const AccessControlManager & manager) const
|
||||
{
|
||||
auto ast = toAST(manager);
|
||||
auto ast = toASTWithNames(manager);
|
||||
return serializeAST(*ast);
|
||||
}
|
||||
|
||||
|
||||
Strings GenericRoleSet::toStrings(const AccessControlManager & manager) const
|
||||
Strings GenericRoleSet::toStringsWithNames(const AccessControlManager & manager) const
|
||||
{
|
||||
if (all || !except_ids.empty())
|
||||
return {toString(manager)};
|
||||
return {toStringWithNames(manager)};
|
||||
|
||||
Strings names;
|
||||
names.reserve(ids.size());
|
||||
|
@ -30,11 +30,19 @@ struct GenericRoleSet
|
||||
GenericRoleSet(const std::vector<UUID> & ids_);
|
||||
GenericRoleSet(const boost::container::flat_set<UUID> & ids_);
|
||||
|
||||
GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager, const std::optional<UUID> & current_user_id = {});
|
||||
std::shared_ptr<ASTGenericRoleSet> toAST(const AccessControlManager & manager) const;
|
||||
/// The constructor from AST requires the AccessControlManager if `ast.id_mode == false`.
|
||||
GenericRoleSet(const ASTGenericRoleSet & ast);
|
||||
GenericRoleSet(const ASTGenericRoleSet & ast, const UUID & current_user_id);
|
||||
GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager);
|
||||
GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager, const UUID & current_user_id);
|
||||
|
||||
String toString(const AccessControlManager & manager) const;
|
||||
Strings toStrings(const AccessControlManager & manager) const;
|
||||
std::shared_ptr<ASTGenericRoleSet> toAST() const;
|
||||
String toString() const;
|
||||
Strings toStrings() const;
|
||||
|
||||
std::shared_ptr<ASTGenericRoleSet> toASTWithNames(const AccessControlManager & manager) const;
|
||||
String toStringWithNames(const AccessControlManager & manager) const;
|
||||
Strings toStringsWithNames(const AccessControlManager & manager) const;
|
||||
|
||||
bool empty() const;
|
||||
void clear();
|
||||
@ -61,6 +69,9 @@ struct GenericRoleSet
|
||||
boost::container::flat_set<UUID> ids;
|
||||
bool all = false;
|
||||
boost::container::flat_set<UUID> except_ids;
|
||||
|
||||
private:
|
||||
void init(const ASTGenericRoleSet & ast, const AccessControlManager * manager = nullptr, const UUID * current_user_id = nullptr);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -15,7 +15,7 @@ namespace ErrorCodes
|
||||
extern const int BAD_CAST;
|
||||
extern const int ACCESS_ENTITY_NOT_FOUND;
|
||||
extern const int ACCESS_ENTITY_ALREADY_EXISTS;
|
||||
extern const int ACCESS_ENTITY_STORAGE_READONLY;
|
||||
extern const int ACCESS_STORAGE_READONLY;
|
||||
extern const int UNKNOWN_USER;
|
||||
extern const int UNKNOWN_ROLE;
|
||||
}
|
||||
@ -419,7 +419,7 @@ void IAccessStorage::throwReadonlyCannotInsert(std::type_index type, const Strin
|
||||
{
|
||||
throw Exception(
|
||||
"Cannot insert " + getTypeName(type) + " " + backQuote(name) + " to " + getStorageName() + " because this storage is readonly",
|
||||
ErrorCodes::ACCESS_ENTITY_STORAGE_READONLY);
|
||||
ErrorCodes::ACCESS_STORAGE_READONLY);
|
||||
}
|
||||
|
||||
|
||||
@ -427,7 +427,7 @@ void IAccessStorage::throwReadonlyCannotUpdate(std::type_index type, const Strin
|
||||
{
|
||||
throw Exception(
|
||||
"Cannot update " + getTypeName(type) + " " + backQuote(name) + " in " + getStorageName() + " because this storage is readonly",
|
||||
ErrorCodes::ACCESS_ENTITY_STORAGE_READONLY);
|
||||
ErrorCodes::ACCESS_STORAGE_READONLY);
|
||||
}
|
||||
|
||||
|
||||
@ -435,6 +435,6 @@ void IAccessStorage::throwReadonlyCannotRemove(std::type_index type, const Strin
|
||||
{
|
||||
throw Exception(
|
||||
"Cannot remove " + getTypeName(type) + " " + backQuote(name) + " from " + getStorageName() + " because this storage is readonly",
|
||||
ErrorCodes::ACCESS_ENTITY_STORAGE_READONLY);
|
||||
ErrorCodes::ACCESS_STORAGE_READONLY);
|
||||
}
|
||||
}
|
||||
|
@ -74,6 +74,10 @@ public:
|
||||
String readName(const UUID & id) const;
|
||||
std::optional<String> tryReadName(const UUID & id) const;
|
||||
|
||||
/// Returns true if a specified entity can be inserted into this storage.
|
||||
/// This function doesn't check whether there are no entities with such name in the storage.
|
||||
bool canInsert(const AccessEntityPtr & entity) const { return canInsertImpl(entity); }
|
||||
|
||||
/// Inserts an entity to the storage. Returns ID of a new entry in the storage.
|
||||
/// Throws an exception if the specified name already exists.
|
||||
UUID insert(const AccessEntityPtr & entity);
|
||||
@ -133,6 +137,7 @@ protected:
|
||||
virtual bool existsImpl(const UUID & id) const = 0;
|
||||
virtual AccessEntityPtr readImpl(const UUID & id) const = 0;
|
||||
virtual String readNameImpl(const UUID & id) const = 0;
|
||||
virtual bool canInsertImpl(const AccessEntityPtr & entity) const = 0;
|
||||
virtual UUID insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) = 0;
|
||||
virtual void removeImpl(const UUID & id) = 0;
|
||||
virtual void updateImpl(const UUID & id, const UpdateFunc & update_func) = 0;
|
||||
|
@ -293,6 +293,7 @@ ext::scope_guard MemoryAccessStorage::subscribeForChangesImpl(const UUID & id, c
|
||||
|
||||
bool MemoryAccessStorage::hasSubscriptionImpl(const UUID & id) const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
auto it = entries.find(id);
|
||||
if (it != entries.end())
|
||||
{
|
||||
@ -305,6 +306,7 @@ bool MemoryAccessStorage::hasSubscriptionImpl(const UUID & id) const
|
||||
|
||||
bool MemoryAccessStorage::hasSubscriptionImpl(std::type_index type) const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
auto range = handlers_by_type.equal_range(type);
|
||||
return range.first != range.second;
|
||||
}
|
||||
|
@ -26,6 +26,7 @@ private:
|
||||
bool existsImpl(const UUID & id) const override;
|
||||
AccessEntityPtr readImpl(const UUID & id) const override;
|
||||
String readNameImpl(const UUID & id) const override;
|
||||
bool canInsertImpl(const AccessEntityPtr &) const override { return true; }
|
||||
UUID insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) override;
|
||||
void removeImpl(const UUID & id) override;
|
||||
void updateImpl(const UUID & id, const UpdateFunc & update_func) override;
|
||||
|
@ -8,6 +8,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ACCESS_ENTITY_FOUND_DUPLICATES;
|
||||
extern const int ACCESS_STORAGE_FOR_INSERTION_NOT_FOUND;
|
||||
}
|
||||
|
||||
|
||||
@ -29,10 +30,9 @@ namespace
|
||||
|
||||
|
||||
MultipleAccessStorage::MultipleAccessStorage(
|
||||
std::vector<std::unique_ptr<Storage>> nested_storages_, size_t index_of_nested_storage_for_insertion_)
|
||||
std::vector<std::unique_ptr<Storage>> nested_storages_)
|
||||
: IAccessStorage(joinStorageNames(nested_storages_))
|
||||
, nested_storages(std::move(nested_storages_))
|
||||
, nested_storage_for_insertion(nested_storages[index_of_nested_storage_for_insertion_].get())
|
||||
, ids_cache(512 /* cache size */)
|
||||
{
|
||||
}
|
||||
@ -161,13 +161,39 @@ String MultipleAccessStorage::readNameImpl(const UUID & id) const
|
||||
}
|
||||
|
||||
|
||||
bool MultipleAccessStorage::canInsertImpl(const AccessEntityPtr & entity) const
|
||||
{
|
||||
for (const auto & nested_storage : nested_storages)
|
||||
{
|
||||
if (nested_storage->canInsert(entity))
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
UUID MultipleAccessStorage::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists)
|
||||
{
|
||||
auto id = replace_if_exists ? nested_storage_for_insertion->insertOrReplace(entity) : nested_storage_for_insertion->insert(entity);
|
||||
IAccessStorage * nested_storage_for_insertion = nullptr;
|
||||
for (const auto & nested_storage : nested_storages)
|
||||
{
|
||||
if (nested_storage->canInsert(entity))
|
||||
{
|
||||
nested_storage_for_insertion = nested_storage.get();
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (!nested_storage_for_insertion)
|
||||
{
|
||||
throw Exception(
|
||||
"Not found a storage to insert " + entity->getTypeName() + backQuote(entity->getName()),
|
||||
ErrorCodes::ACCESS_STORAGE_FOR_INSERTION_NOT_FOUND);
|
||||
}
|
||||
|
||||
auto id = replace_if_exists ? nested_storage_for_insertion->insertOrReplace(entity) : nested_storage_for_insertion->insert(entity);
|
||||
std::lock_guard lock{ids_cache_mutex};
|
||||
ids_cache.set(id, std::make_shared<Storage *>(nested_storage_for_insertion));
|
||||
|
||||
return id;
|
||||
}
|
||||
|
||||
|
@ -13,7 +13,7 @@ class MultipleAccessStorage : public IAccessStorage
|
||||
public:
|
||||
using Storage = IAccessStorage;
|
||||
|
||||
MultipleAccessStorage(std::vector<std::unique_ptr<Storage>> nested_storages_, size_t index_of_nested_storage_for_insertion_ = 0);
|
||||
MultipleAccessStorage(std::vector<std::unique_ptr<Storage>> nested_storages_);
|
||||
~MultipleAccessStorage() override;
|
||||
|
||||
std::vector<UUID> findMultiple(std::type_index type, const String & name) const;
|
||||
@ -35,6 +35,7 @@ protected:
|
||||
bool existsImpl(const UUID & id) const override;
|
||||
AccessEntityPtr readImpl(const UUID & id) const override;
|
||||
String readNameImpl(const UUID &id) const override;
|
||||
bool canInsertImpl(const AccessEntityPtr & entity) const override;
|
||||
UUID insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) override;
|
||||
void removeImpl(const UUID & id) override;
|
||||
void updateImpl(const UUID & id, const UpdateFunc & update_func) override;
|
||||
@ -45,7 +46,6 @@ protected:
|
||||
|
||||
private:
|
||||
std::vector<std::unique_ptr<Storage>> nested_storages;
|
||||
IAccessStorage * nested_storage_for_insertion;
|
||||
mutable LRUCache<UUID, Storage *> ids_cache;
|
||||
mutable std::mutex ids_cache_mutex;
|
||||
};
|
||||
|
@ -342,7 +342,7 @@ UsersConfigAccessStorage::UsersConfigAccessStorage() : IAccessStorage("users.xml
|
||||
UsersConfigAccessStorage::~UsersConfigAccessStorage() {}
|
||||
|
||||
|
||||
void UsersConfigAccessStorage::loadFromConfig(const Poco::Util::AbstractConfiguration & config)
|
||||
void UsersConfigAccessStorage::setConfiguration(const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
std::vector<std::pair<UUID, AccessEntityPtr>> all_entities;
|
||||
for (const auto & entity : parseUsers(config, getLogger()))
|
||||
|
@ -21,7 +21,7 @@ public:
|
||||
UsersConfigAccessStorage();
|
||||
~UsersConfigAccessStorage() override;
|
||||
|
||||
void loadFromConfig(const Poco::Util::AbstractConfiguration & config);
|
||||
void setConfiguration(const Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
private:
|
||||
std::optional<UUID> findImpl(std::type_index type, const String & name) const override;
|
||||
@ -29,6 +29,7 @@ private:
|
||||
bool existsImpl(const UUID & id) const override;
|
||||
AccessEntityPtr readImpl(const UUID & id) const override;
|
||||
String readNameImpl(const UUID & id) const override;
|
||||
bool canInsertImpl(const AccessEntityPtr &) const override { return false; }
|
||||
UUID insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) override;
|
||||
void removeImpl(const UUID & id) override;
|
||||
void updateImpl(const UUID & id, const UpdateFunc & update_func) override;
|
||||
|
@ -467,7 +467,7 @@ namespace ErrorCodes
|
||||
extern const int ACCESS_ENTITY_NOT_FOUND = 492;
|
||||
extern const int ACCESS_ENTITY_ALREADY_EXISTS = 493;
|
||||
extern const int ACCESS_ENTITY_FOUND_DUPLICATES = 494;
|
||||
extern const int ACCESS_ENTITY_STORAGE_READONLY = 495;
|
||||
extern const int ACCESS_STORAGE_READONLY = 495;
|
||||
extern const int QUOTA_REQUIRES_CLIENT_KEY = 496;
|
||||
extern const int ACCESS_DENIED = 497;
|
||||
extern const int LIMIT_BY_WITH_TIES_IS_NOT_SUPPORTED = 498;
|
||||
@ -485,6 +485,8 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_ROLE = 511;
|
||||
extern const int SET_NON_GRANTED_ROLE = 512;
|
||||
extern const int UNKNOWN_PART_TYPE = 513;
|
||||
extern const int ACCESS_STORAGE_FOR_INSERTION_NOT_FOUND = 514;
|
||||
extern const int INCORRECT_ACCESS_ENTITY_DEFINITION = 515;
|
||||
|
||||
extern const int KEEPER_EXCEPTION = 999;
|
||||
extern const int POCO_EXCEPTION = 1000;
|
||||
|
@ -626,7 +626,7 @@ void Context::setUsersConfig(const ConfigurationPtr & config)
|
||||
{
|
||||
auto lock = getLock();
|
||||
shared->users_config = config;
|
||||
shared->access_control_manager.loadFromConfig(*shared->users_config);
|
||||
shared->access_control_manager.setUsersConfig(*shared->users_config);
|
||||
}
|
||||
|
||||
ConfigurationPtr Context::getUsersConfig()
|
||||
|
@ -12,6 +12,68 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace
|
||||
{
|
||||
void updateQuotaFromQueryImpl(Quota & quota, const ASTCreateQuotaQuery & query, const std::optional<GenericRoleSet> & roles_from_query = {})
|
||||
{
|
||||
if (query.alter)
|
||||
{
|
||||
if (!query.new_name.empty())
|
||||
quota.setName(query.new_name);
|
||||
}
|
||||
else
|
||||
quota.setName(query.name);
|
||||
|
||||
if (query.key_type)
|
||||
quota.key_type = *query.key_type;
|
||||
|
||||
auto & quota_all_limits = quota.all_limits;
|
||||
for (const auto & query_limits : query.all_limits)
|
||||
{
|
||||
auto duration = query_limits.duration;
|
||||
|
||||
auto it = boost::range::find_if(quota_all_limits, [&](const Quota::Limits & x) { return x.duration == duration; });
|
||||
if (query_limits.unset_tracking)
|
||||
{
|
||||
if (it != quota_all_limits.end())
|
||||
quota_all_limits.erase(it);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (it == quota_all_limits.end())
|
||||
{
|
||||
/// We keep `all_limits` sorted by duration.
|
||||
it = quota_all_limits.insert(
|
||||
boost::range::upper_bound(
|
||||
quota_all_limits,
|
||||
duration,
|
||||
[](const std::chrono::seconds & lhs, const Quota::Limits & rhs) { return lhs < rhs.duration; }),
|
||||
Quota::Limits{});
|
||||
it->duration = duration;
|
||||
}
|
||||
|
||||
auto & quota_limits = *it;
|
||||
quota_limits.randomize_interval = query_limits.randomize_interval;
|
||||
for (auto resource_type : ext::range(Quota::MAX_RESOURCE_TYPE))
|
||||
{
|
||||
if (query_limits.max[resource_type])
|
||||
quota_limits.max[resource_type] = *query_limits.max[resource_type];
|
||||
}
|
||||
}
|
||||
|
||||
const GenericRoleSet * roles = nullptr;
|
||||
std::optional<GenericRoleSet> temp_role_set;
|
||||
if (roles_from_query)
|
||||
roles = &*roles_from_query;
|
||||
else if (query.roles)
|
||||
roles = &temp_role_set.emplace(*query.roles);
|
||||
|
||||
if (roles)
|
||||
quota.roles = *roles;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
BlockIO InterpreterCreateQuotaQuery::execute()
|
||||
{
|
||||
const auto & query = query_ptr->as<const ASTCreateQuotaQuery &>();
|
||||
@ -27,7 +89,7 @@ BlockIO InterpreterCreateQuotaQuery::execute()
|
||||
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
|
||||
{
|
||||
auto updated_quota = typeid_cast<std::shared_ptr<Quota>>(entity->clone());
|
||||
updateQuotaFromQuery(*updated_quota, query, roles_from_query);
|
||||
updateQuotaFromQueryImpl(*updated_quota, query, roles_from_query);
|
||||
return updated_quota;
|
||||
};
|
||||
if (query.if_exists)
|
||||
@ -41,7 +103,7 @@ BlockIO InterpreterCreateQuotaQuery::execute()
|
||||
else
|
||||
{
|
||||
auto new_quota = std::make_shared<Quota>();
|
||||
updateQuotaFromQuery(*new_quota, query, roles_from_query);
|
||||
updateQuotaFromQueryImpl(*new_quota, query, roles_from_query);
|
||||
|
||||
if (query.if_not_exists)
|
||||
access_control.tryInsert(new_quota);
|
||||
@ -55,54 +117,9 @@ BlockIO InterpreterCreateQuotaQuery::execute()
|
||||
}
|
||||
|
||||
|
||||
void InterpreterCreateQuotaQuery::updateQuotaFromQuery(Quota & quota, const ASTCreateQuotaQuery & query, const std::optional<GenericRoleSet> & roles_from_query)
|
||||
void InterpreterCreateQuotaQuery::updateQuotaFromQuery(Quota & quota, const ASTCreateQuotaQuery & query)
|
||||
{
|
||||
if (query.alter)
|
||||
{
|
||||
if (!query.new_name.empty())
|
||||
quota.setName(query.new_name);
|
||||
}
|
||||
else
|
||||
quota.setName(query.name);
|
||||
|
||||
if (query.key_type)
|
||||
quota.key_type = *query.key_type;
|
||||
|
||||
auto & quota_all_limits = quota.all_limits;
|
||||
for (const auto & query_limits : query.all_limits)
|
||||
{
|
||||
auto duration = query_limits.duration;
|
||||
|
||||
auto it = boost::range::find_if(quota_all_limits, [&](const Quota::Limits & x) { return x.duration == duration; });
|
||||
if (query_limits.unset_tracking)
|
||||
{
|
||||
if (it != quota_all_limits.end())
|
||||
quota_all_limits.erase(it);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (it == quota_all_limits.end())
|
||||
{
|
||||
/// We keep `all_limits` sorted by duration.
|
||||
it = quota_all_limits.insert(
|
||||
boost::range::upper_bound(
|
||||
quota_all_limits,
|
||||
duration,
|
||||
[](const std::chrono::seconds & lhs, const Quota::Limits & rhs) { return lhs < rhs.duration; }),
|
||||
Quota::Limits{});
|
||||
it->duration = duration;
|
||||
}
|
||||
|
||||
auto & quota_limits = *it;
|
||||
quota_limits.randomize_interval = query_limits.randomize_interval;
|
||||
for (auto resource_type : ext::range(Quota::MAX_RESOURCE_TYPE))
|
||||
{
|
||||
if (query_limits.max[resource_type])
|
||||
quota_limits.max[resource_type] = *query_limits.max[resource_type];
|
||||
}
|
||||
}
|
||||
|
||||
if (roles_from_query)
|
||||
quota.roles = *roles_from_query;
|
||||
updateQuotaFromQueryImpl(quota, query);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -9,7 +9,6 @@ namespace DB
|
||||
{
|
||||
class ASTCreateQuotaQuery;
|
||||
struct Quota;
|
||||
struct GenericRoleSet;
|
||||
|
||||
|
||||
class InterpreterCreateQuotaQuery : public IInterpreter
|
||||
@ -22,9 +21,9 @@ public:
|
||||
bool ignoreQuota() const override { return true; }
|
||||
bool ignoreLimits() const override { return true; }
|
||||
|
||||
private:
|
||||
void updateQuotaFromQuery(Quota & quota, const ASTCreateQuotaQuery & query, const std::optional<GenericRoleSet> & roles_from_query);
|
||||
static void updateQuotaFromQuery(Quota & quota, const ASTCreateQuotaQuery & query);
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context & context;
|
||||
};
|
||||
|
@ -17,9 +17,9 @@ public:
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
private:
|
||||
void updateRoleFromQuery(Role & role, const ASTCreateRoleQuery & query);
|
||||
static void updateRoleFromQuery(Role & role, const ASTCreateRoleQuery & query);
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context & context;
|
||||
};
|
||||
|
@ -10,6 +10,57 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
const String & checkCurrentDatabase(const String & current_database)
|
||||
{
|
||||
if (current_database.empty())
|
||||
throw Exception("No current database", ErrorCodes::LOGICAL_ERROR);
|
||||
return current_database;
|
||||
}
|
||||
|
||||
void updateRowPolicyFromQueryImpl(
|
||||
RowPolicy & policy,
|
||||
const ASTCreateRowPolicyQuery & query,
|
||||
const std::optional<GenericRoleSet> & roles_from_query = {},
|
||||
const String & current_database = {})
|
||||
{
|
||||
if (query.alter)
|
||||
{
|
||||
if (!query.new_policy_name.empty())
|
||||
policy.setName(query.new_policy_name);
|
||||
}
|
||||
else
|
||||
{
|
||||
policy.setDatabase(!query.name_parts.database.empty() ? query.name_parts.database : checkCurrentDatabase(current_database));
|
||||
policy.setTableName(query.name_parts.table_name);
|
||||
policy.setName(query.name_parts.policy_name);
|
||||
}
|
||||
|
||||
if (query.is_restrictive)
|
||||
policy.setRestrictive(*query.is_restrictive);
|
||||
|
||||
for (const auto & [index, condition] : query.conditions)
|
||||
policy.conditions[index] = condition ? serializeAST(*condition) : String{};
|
||||
|
||||
const GenericRoleSet * roles = nullptr;
|
||||
std::optional<GenericRoleSet> temp_role_set;
|
||||
if (roles_from_query)
|
||||
roles = &*roles_from_query;
|
||||
else if (query.roles)
|
||||
roles = &temp_role_set.emplace(*query.roles);
|
||||
|
||||
if (roles)
|
||||
policy.roles = *roles;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
BlockIO InterpreterCreateRowPolicyQuery::execute()
|
||||
{
|
||||
const auto & query = query_ptr->as<const ASTCreateRowPolicyQuery &>();
|
||||
@ -20,12 +71,14 @@ BlockIO InterpreterCreateRowPolicyQuery::execute()
|
||||
if (query.roles)
|
||||
roles_from_query = GenericRoleSet{*query.roles, access_control, context.getUserID()};
|
||||
|
||||
const String current_database = context.getCurrentDatabase();
|
||||
|
||||
if (query.alter)
|
||||
{
|
||||
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
|
||||
{
|
||||
auto updated_policy = typeid_cast<std::shared_ptr<RowPolicy>>(entity->clone());
|
||||
updateRowPolicyFromQuery(*updated_policy, query, roles_from_query);
|
||||
updateRowPolicyFromQueryImpl(*updated_policy, query, roles_from_query, current_database);
|
||||
return updated_policy;
|
||||
};
|
||||
String full_name = query.name_parts.getFullName(context);
|
||||
@ -40,7 +93,7 @@ BlockIO InterpreterCreateRowPolicyQuery::execute()
|
||||
else
|
||||
{
|
||||
auto new_policy = std::make_shared<RowPolicy>();
|
||||
updateRowPolicyFromQuery(*new_policy, query, roles_from_query);
|
||||
updateRowPolicyFromQueryImpl(*new_policy, query, roles_from_query, current_database);
|
||||
|
||||
if (query.if_not_exists)
|
||||
access_control.tryInsert(new_policy);
|
||||
@ -54,27 +107,9 @@ BlockIO InterpreterCreateRowPolicyQuery::execute()
|
||||
}
|
||||
|
||||
|
||||
void InterpreterCreateRowPolicyQuery::updateRowPolicyFromQuery(RowPolicy & policy, const ASTCreateRowPolicyQuery & query, const std::optional<GenericRoleSet> & roles_from_query)
|
||||
void InterpreterCreateRowPolicyQuery::updateRowPolicyFromQuery(RowPolicy & policy, const ASTCreateRowPolicyQuery & query)
|
||||
{
|
||||
if (query.alter)
|
||||
{
|
||||
if (!query.new_policy_name.empty())
|
||||
policy.setName(query.new_policy_name);
|
||||
}
|
||||
else
|
||||
{
|
||||
policy.setDatabase(query.name_parts.database.empty() ? context.getCurrentDatabase() : query.name_parts.database);
|
||||
policy.setTableName(query.name_parts.table_name);
|
||||
policy.setName(query.name_parts.policy_name);
|
||||
}
|
||||
|
||||
if (query.is_restrictive)
|
||||
policy.setRestrictive(*query.is_restrictive);
|
||||
|
||||
for (const auto & [index, condition] : query.conditions)
|
||||
policy.conditions[index] = condition ? serializeAST(*condition) : String{};
|
||||
|
||||
if (roles_from_query)
|
||||
policy.roles = *roles_from_query;
|
||||
updateRowPolicyFromQueryImpl(policy, query);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -9,7 +9,6 @@ namespace DB
|
||||
{
|
||||
class ASTCreateRowPolicyQuery;
|
||||
struct RowPolicy;
|
||||
struct GenericRoleSet;
|
||||
|
||||
|
||||
class InterpreterCreateRowPolicyQuery : public IInterpreter
|
||||
@ -19,9 +18,9 @@ public:
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
private:
|
||||
void updateRowPolicyFromQuery(RowPolicy & policy, const ASTCreateRowPolicyQuery & query, const std::optional<GenericRoleSet> & roles_from_query);
|
||||
static void updateRowPolicyFromQuery(RowPolicy & policy, const ASTCreateRowPolicyQuery & query);
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context & context;
|
||||
};
|
||||
|
@ -11,8 +11,46 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
namespace
|
||||
{
|
||||
void updateUserFromQueryImpl(User & user, const ASTCreateUserQuery & query, const std::optional<GenericRoleSet> & default_roles_from_query = {})
|
||||
{
|
||||
if (query.alter)
|
||||
{
|
||||
if (!query.new_name.empty())
|
||||
user.setName(query.new_name);
|
||||
}
|
||||
else
|
||||
user.setName(query.name);
|
||||
|
||||
if (query.authentication)
|
||||
user.authentication = *query.authentication;
|
||||
|
||||
if (query.hosts)
|
||||
user.allowed_client_hosts = *query.hosts;
|
||||
if (query.remove_hosts)
|
||||
user.allowed_client_hosts.remove(*query.remove_hosts);
|
||||
if (query.add_hosts)
|
||||
user.allowed_client_hosts.add(*query.add_hosts);
|
||||
|
||||
const GenericRoleSet * default_roles = nullptr;
|
||||
std::optional<GenericRoleSet> temp_role_set;
|
||||
if (default_roles_from_query)
|
||||
default_roles = &*default_roles_from_query;
|
||||
else if (query.default_roles)
|
||||
default_roles = &temp_role_set.emplace(*query.default_roles);
|
||||
|
||||
if (default_roles)
|
||||
{
|
||||
if (!query.alter && !default_roles->all)
|
||||
boost::range::copy(default_roles->getMatchingIDs(), std::inserter(user.granted_roles, user.granted_roles.end()));
|
||||
|
||||
InterpreterSetRoleQuery::updateUserSetDefaultRoles(user, *default_roles);
|
||||
}
|
||||
|
||||
if (query.profile)
|
||||
user.profile = *query.profile;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -22,12 +60,10 @@ BlockIO InterpreterCreateUserQuery::execute()
|
||||
auto & access_control = context.getAccessControlManager();
|
||||
context.checkAccess(query.alter ? AccessType::ALTER_USER : AccessType::CREATE_USER);
|
||||
|
||||
GenericRoleSet * default_roles_from_query = nullptr;
|
||||
GenericRoleSet temp_role_set;
|
||||
std::optional<GenericRoleSet> default_roles_from_query;
|
||||
if (query.default_roles)
|
||||
{
|
||||
default_roles_from_query = &temp_role_set;
|
||||
*default_roles_from_query = GenericRoleSet{*query.default_roles, access_control};
|
||||
default_roles_from_query = GenericRoleSet{*query.default_roles, access_control};
|
||||
if (!query.alter && !default_roles_from_query->all)
|
||||
{
|
||||
for (const UUID & role : default_roles_from_query->getMatchingIDs())
|
||||
@ -40,7 +76,7 @@ BlockIO InterpreterCreateUserQuery::execute()
|
||||
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
|
||||
{
|
||||
auto updated_user = typeid_cast<std::shared_ptr<User>>(entity->clone());
|
||||
updateUserFromQuery(*updated_user, query, default_roles_from_query);
|
||||
updateUserFromQueryImpl(*updated_user, query, default_roles_from_query);
|
||||
return updated_user;
|
||||
};
|
||||
if (query.if_exists)
|
||||
@ -54,7 +90,7 @@ BlockIO InterpreterCreateUserQuery::execute()
|
||||
else
|
||||
{
|
||||
auto new_user = std::make_shared<User>();
|
||||
updateUserFromQuery(*new_user, query, default_roles_from_query);
|
||||
updateUserFromQueryImpl(*new_user, query, default_roles_from_query);
|
||||
|
||||
if (query.if_not_exists)
|
||||
access_control.tryInsert(new_user);
|
||||
@ -68,36 +104,9 @@ BlockIO InterpreterCreateUserQuery::execute()
|
||||
}
|
||||
|
||||
|
||||
void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreateUserQuery & query, const GenericRoleSet * default_roles_from_query)
|
||||
void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreateUserQuery & query)
|
||||
{
|
||||
if (query.alter)
|
||||
{
|
||||
if (!query.new_name.empty())
|
||||
user.setName(query.new_name);
|
||||
}
|
||||
else
|
||||
user.setName(query.name);
|
||||
|
||||
if (query.authentication)
|
||||
user.authentication = *query.authentication;
|
||||
|
||||
if (query.hosts)
|
||||
user.allowed_client_hosts = *query.hosts;
|
||||
if (query.remove_hosts)
|
||||
user.allowed_client_hosts.remove(*query.remove_hosts);
|
||||
if (query.add_hosts)
|
||||
user.allowed_client_hosts.add(*query.add_hosts);
|
||||
|
||||
if (default_roles_from_query)
|
||||
{
|
||||
if (!query.alter && !default_roles_from_query->all)
|
||||
boost::range::copy(default_roles_from_query->getMatchingIDs(), std::inserter(user.granted_roles, user.granted_roles.end()));
|
||||
|
||||
InterpreterSetRoleQuery::updateUserSetDefaultRoles(user, *default_roles_from_query);
|
||||
}
|
||||
|
||||
if (query.profile)
|
||||
user.profile = *query.profile;
|
||||
updateUserFromQueryImpl(user, query);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -7,7 +7,6 @@
|
||||
namespace DB
|
||||
{
|
||||
class ASTCreateUserQuery;
|
||||
struct GenericRoleSet;
|
||||
struct User;
|
||||
|
||||
|
||||
@ -18,9 +17,9 @@ public:
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
private:
|
||||
void updateUserFromQuery(User & user, const ASTCreateUserQuery & query, const GenericRoleSet * default_roles_from_query);
|
||||
static void updateUserFromQuery(User & user, const ASTCreateUserQuery & query);
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context & context;
|
||||
};
|
||||
|
@ -11,96 +11,91 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace
|
||||
{
|
||||
template <typename T>
|
||||
void updateFromQueryImpl(T & grantee, const ASTGrantQuery & query, const std::vector<UUID> & roles_from_query, const String & current_database, bool partial_revokes)
|
||||
{
|
||||
using Kind = ASTGrantQuery::Kind;
|
||||
if (!query.access_rights_elements.empty())
|
||||
{
|
||||
if (query.kind == Kind::GRANT)
|
||||
{
|
||||
grantee.access.grant(query.access_rights_elements, current_database);
|
||||
if (query.grant_option)
|
||||
grantee.access_with_grant_option.grant(query.access_rights_elements, current_database);
|
||||
}
|
||||
else if (partial_revokes)
|
||||
{
|
||||
grantee.access_with_grant_option.partialRevoke(query.access_rights_elements, current_database);
|
||||
if (!query.grant_option)
|
||||
grantee.access.partialRevoke(query.access_rights_elements, current_database);
|
||||
}
|
||||
else
|
||||
{
|
||||
grantee.access_with_grant_option.revoke(query.access_rights_elements, current_database);
|
||||
if (!query.grant_option)
|
||||
grantee.access.revoke(query.access_rights_elements, current_database);
|
||||
}
|
||||
}
|
||||
|
||||
if (!roles_from_query.empty())
|
||||
{
|
||||
if (query.kind == Kind::GRANT)
|
||||
{
|
||||
boost::range::copy(roles_from_query, std::inserter(grantee.granted_roles, grantee.granted_roles.end()));
|
||||
if (query.admin_option)
|
||||
boost::range::copy(roles_from_query, std::inserter(grantee.granted_roles_with_admin_option, grantee.granted_roles_with_admin_option.end()));
|
||||
}
|
||||
else
|
||||
{
|
||||
for (const UUID & role_from_query : roles_from_query)
|
||||
{
|
||||
grantee.granted_roles_with_admin_option.erase(role_from_query);
|
||||
if (!query.admin_option)
|
||||
grantee.granted_roles.erase(role_from_query);
|
||||
if constexpr (std::is_same_v<T, User>)
|
||||
grantee.default_roles.ids.erase(role_from_query);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
BlockIO InterpreterGrantQuery::execute()
|
||||
{
|
||||
const auto & query = query_ptr->as<const ASTGrantQuery &>();
|
||||
auto & access_control = context.getAccessControlManager();
|
||||
context.getAccessRights()->checkGrantOption(query.access_rights_elements);
|
||||
|
||||
using Kind = ASTGrantQuery::Kind;
|
||||
std::vector<UUID> roles;
|
||||
std::vector<UUID> roles_from_query;
|
||||
if (query.roles)
|
||||
{
|
||||
roles = GenericRoleSet{*query.roles, access_control}.getMatchingRoles(access_control);
|
||||
for (const UUID & role : roles)
|
||||
context.getAccessRights()->checkAdminOption(role);
|
||||
roles_from_query = GenericRoleSet{*query.roles, access_control}.getMatchingRoles(access_control);
|
||||
for (const UUID & role_from_query : roles_from_query)
|
||||
context.getAccessRights()->checkAdminOption(role_from_query);
|
||||
}
|
||||
|
||||
std::vector<UUID> to_roles = GenericRoleSet{*query.to_roles, access_control, context.getUserID()}.getMatchingUsersAndRoles(access_control);
|
||||
String current_database = context.getCurrentDatabase();
|
||||
using Kind = ASTGrantQuery::Kind;
|
||||
bool partial_revokes = context.getSettingsRef().partial_revokes;
|
||||
|
||||
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
|
||||
{
|
||||
auto clone = entity->clone();
|
||||
AccessRights * access = nullptr;
|
||||
AccessRights * access_with_grant_option = nullptr;
|
||||
boost::container::flat_set<UUID> * granted_roles = nullptr;
|
||||
boost::container::flat_set<UUID> * granted_roles_with_admin_option = nullptr;
|
||||
GenericRoleSet * default_roles = nullptr;
|
||||
if (auto user = typeid_cast<std::shared_ptr<User>>(clone))
|
||||
{
|
||||
access = &user->access;
|
||||
access_with_grant_option = &user->access_with_grant_option;
|
||||
granted_roles = &user->granted_roles;
|
||||
granted_roles_with_admin_option = &user->granted_roles_with_admin_option;
|
||||
default_roles = &user->default_roles;
|
||||
updateFromQueryImpl(*user, query, roles_from_query, current_database, partial_revokes);
|
||||
return user;
|
||||
}
|
||||
else if (auto role = typeid_cast<std::shared_ptr<Role>>(clone))
|
||||
{
|
||||
access = &role->access;
|
||||
access_with_grant_option = &role->access_with_grant_option;
|
||||
granted_roles = &role->granted_roles;
|
||||
granted_roles_with_admin_option = &role->granted_roles_with_admin_option;
|
||||
updateFromQueryImpl(*role, query, roles_from_query, current_database, partial_revokes);
|
||||
return role;
|
||||
}
|
||||
else
|
||||
return entity;
|
||||
|
||||
if (!query.access_rights_elements.empty())
|
||||
{
|
||||
if (query.kind == Kind::GRANT)
|
||||
{
|
||||
access->grant(query.access_rights_elements, current_database);
|
||||
if (query.grant_option)
|
||||
access_with_grant_option->grant(query.access_rights_elements, current_database);
|
||||
}
|
||||
else if (context.getSettingsRef().partial_revokes)
|
||||
{
|
||||
access_with_grant_option->partialRevoke(query.access_rights_elements, current_database);
|
||||
if (!query.grant_option)
|
||||
access->partialRevoke(query.access_rights_elements, current_database);
|
||||
}
|
||||
else
|
||||
{
|
||||
access_with_grant_option->revoke(query.access_rights_elements, current_database);
|
||||
if (!query.grant_option)
|
||||
access->revoke(query.access_rights_elements, current_database);
|
||||
}
|
||||
}
|
||||
|
||||
if (!roles.empty())
|
||||
{
|
||||
if (query.kind == Kind::GRANT)
|
||||
{
|
||||
boost::range::copy(roles, std::inserter(*granted_roles, granted_roles->end()));
|
||||
if (query.admin_option)
|
||||
boost::range::copy(roles, std::inserter(*granted_roles_with_admin_option, granted_roles_with_admin_option->end()));
|
||||
}
|
||||
else
|
||||
{
|
||||
for (const UUID & role : roles)
|
||||
{
|
||||
granted_roles_with_admin_option->erase(role);
|
||||
if (!query.admin_option)
|
||||
{
|
||||
granted_roles->erase(role);
|
||||
if (default_roles)
|
||||
default_roles->ids.erase(role);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return clone;
|
||||
};
|
||||
|
||||
access_control.update(to_roles, update_func);
|
||||
@ -108,4 +103,22 @@ BlockIO InterpreterGrantQuery::execute()
|
||||
return {};
|
||||
}
|
||||
|
||||
|
||||
void InterpreterGrantQuery::updateUserFromQuery(User & user, const ASTGrantQuery & query)
|
||||
{
|
||||
std::vector<UUID> roles_from_query;
|
||||
if (query.roles)
|
||||
roles_from_query = GenericRoleSet{*query.roles}.getMatchingIDs();
|
||||
updateFromQueryImpl(user, query, roles_from_query, {}, true);
|
||||
}
|
||||
|
||||
|
||||
void InterpreterGrantQuery::updateRoleFromQuery(Role & role, const ASTGrantQuery & query)
|
||||
{
|
||||
std::vector<UUID> roles_from_query;
|
||||
if (query.roles)
|
||||
roles_from_query = GenericRoleSet{*query.roles}.getMatchingIDs();
|
||||
updateFromQueryImpl(role, query, roles_from_query, {}, true);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,10 +2,16 @@
|
||||
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Core/UUID.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ASTGrantQuery;
|
||||
struct User;
|
||||
struct Role;
|
||||
|
||||
|
||||
class InterpreterGrantQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
@ -13,6 +19,9 @@ public:
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
static void updateUserFromQuery(User & user, const ASTGrantQuery & query);
|
||||
static void updateRoleFromQuery(Role & role, const ASTGrantQuery & query);
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context & context;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Interpreters/InterpreterShowCreateAccessEntityQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTCreateUserQuery.h>
|
||||
#include <Parsers/ASTCreateRoleQuery.h>
|
||||
#include <Parsers/ASTCreateQuotaQuery.h>
|
||||
#include <Parsers/ASTCreateRowPolicyQuery.h>
|
||||
#include <Parsers/ASTShowCreateAccessEntityQuery.h>
|
||||
@ -11,6 +12,7 @@
|
||||
#include <Access/AccessControlManager.h>
|
||||
#include <Access/QuotaContext.h>
|
||||
#include <Access/User.h>
|
||||
#include <Access/Role.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
@ -21,6 +23,141 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
ASTPtr getCreateQueryImpl(
|
||||
const User & user,
|
||||
const AccessControlManager * manager /* not used if attach_mode == true */,
|
||||
bool attach_mode = false)
|
||||
{
|
||||
auto query = std::make_shared<ASTCreateUserQuery>();
|
||||
query->name = user.getName();
|
||||
query->attach = attach_mode;
|
||||
|
||||
if (user.allowed_client_hosts != AllowedClientHosts::AnyHostTag{})
|
||||
query->hosts = user.allowed_client_hosts;
|
||||
|
||||
if (!user.profile.empty())
|
||||
query->profile = user.profile;
|
||||
|
||||
if (user.default_roles != GenericRoleSet::AllTag{})
|
||||
{
|
||||
if (attach_mode)
|
||||
query->default_roles = GenericRoleSet{user.default_roles}.toAST();
|
||||
else
|
||||
query->default_roles = GenericRoleSet{user.default_roles}.toASTWithNames(*manager);
|
||||
}
|
||||
|
||||
if (attach_mode && (user.authentication.getType() != Authentication::NO_PASSWORD))
|
||||
{
|
||||
/// We don't show password unless it's an ATTACH statement.
|
||||
query->authentication = user.authentication;
|
||||
}
|
||||
return query;
|
||||
}
|
||||
|
||||
|
||||
ASTPtr getCreateQueryImpl(const Role & role, const AccessControlManager *, bool attach_mode = false)
|
||||
{
|
||||
auto query = std::make_shared<ASTCreateRoleQuery>();
|
||||
query->name = role.getName();
|
||||
query->attach = attach_mode;
|
||||
return query;
|
||||
}
|
||||
|
||||
|
||||
ASTPtr getCreateQueryImpl(
|
||||
const Quota & quota,
|
||||
const AccessControlManager * manager /* not used if attach_mode == true */,
|
||||
bool attach_mode = false)
|
||||
{
|
||||
auto query = std::make_shared<ASTCreateQuotaQuery>();
|
||||
query->name = quota.getName();
|
||||
query->attach = attach_mode;
|
||||
|
||||
query->key_type = quota.key_type;
|
||||
query->all_limits.reserve(quota.all_limits.size());
|
||||
|
||||
for (const auto & limits : quota.all_limits)
|
||||
{
|
||||
ASTCreateQuotaQuery::Limits create_query_limits;
|
||||
create_query_limits.duration = limits.duration;
|
||||
create_query_limits.randomize_interval = limits.randomize_interval;
|
||||
for (auto resource_type : ext::range(Quota::MAX_RESOURCE_TYPE))
|
||||
if (limits.max[resource_type])
|
||||
create_query_limits.max[resource_type] = limits.max[resource_type];
|
||||
query->all_limits.push_back(create_query_limits);
|
||||
}
|
||||
|
||||
if (!quota.roles.empty())
|
||||
{
|
||||
if (attach_mode)
|
||||
query->roles = quota.roles.toAST();
|
||||
else
|
||||
query->roles = quota.roles.toASTWithNames(*manager);
|
||||
}
|
||||
|
||||
return query;
|
||||
}
|
||||
|
||||
|
||||
ASTPtr getCreateQueryImpl(
|
||||
const RowPolicy & policy,
|
||||
const AccessControlManager * manager /* not used if attach_mode == true */,
|
||||
bool attach_mode = false)
|
||||
{
|
||||
auto query = std::make_shared<ASTCreateRowPolicyQuery>();
|
||||
query->name_parts = RowPolicy::FullNameParts{policy.getDatabase(), policy.getTableName(), policy.getName()};
|
||||
query->attach = attach_mode;
|
||||
|
||||
if (policy.isRestrictive())
|
||||
query->is_restrictive = policy.isRestrictive();
|
||||
|
||||
for (auto index : ext::range_with_static_cast<RowPolicy::ConditionIndex>(RowPolicy::MAX_CONDITION_INDEX))
|
||||
{
|
||||
const auto & condition = policy.conditions[index];
|
||||
if (!condition.empty())
|
||||
{
|
||||
ParserExpression parser;
|
||||
ASTPtr expr = parseQuery(parser, condition, 0);
|
||||
query->conditions.push_back(std::pair{index, expr});
|
||||
}
|
||||
}
|
||||
|
||||
if (!policy.roles.empty())
|
||||
{
|
||||
if (attach_mode)
|
||||
query->roles = policy.roles.toAST();
|
||||
else
|
||||
query->roles = policy.roles.toASTWithNames(*manager);
|
||||
}
|
||||
|
||||
return query;
|
||||
}
|
||||
|
||||
ASTPtr getCreateQueryImpl(
|
||||
const IAccessEntity & entity,
|
||||
const AccessControlManager * manager /* not used if attach_mode == true */,
|
||||
bool attach_mode = false)
|
||||
{
|
||||
if (const User * user = typeid_cast<const User *>(&entity))
|
||||
return getCreateQueryImpl(*user, manager, attach_mode);
|
||||
if (const Role * role = typeid_cast<const Role *>(&entity))
|
||||
return getCreateQueryImpl(*role, manager, attach_mode);
|
||||
if (const RowPolicy * policy = typeid_cast<const RowPolicy *>(&entity))
|
||||
return getCreateQueryImpl(*policy, manager, attach_mode);
|
||||
if (const Quota * quota = typeid_cast<const Quota *>(&entity))
|
||||
return getCreateQueryImpl(*quota, manager, attach_mode);
|
||||
throw Exception("Unexpected type of access entity: " + entity.getTypeName(), ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
BlockIO InterpreterShowCreateAccessEntityQuery::execute()
|
||||
{
|
||||
BlockIO res;
|
||||
@ -57,98 +194,43 @@ BlockInputStreamPtr InterpreterShowCreateAccessEntityQuery::executeImpl()
|
||||
|
||||
ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuery(const ASTShowCreateAccessEntityQuery & show_query) const
|
||||
{
|
||||
const auto & access_control = context.getAccessControlManager();
|
||||
using Kind = ASTShowCreateAccessEntityQuery::Kind;
|
||||
switch (show_query.kind)
|
||||
{
|
||||
case Kind::USER: return getCreateUserQuery(show_query);
|
||||
case Kind::QUOTA: return getCreateQuotaQuery(show_query);
|
||||
case Kind::ROW_POLICY: return getCreateRowPolicyQuery(show_query);
|
||||
case Kind::USER:
|
||||
{
|
||||
UserPtr user;
|
||||
if (show_query.current_user)
|
||||
user = context.getUser();
|
||||
else
|
||||
user = access_control.read<User>(show_query.name);
|
||||
return getCreateQueryImpl(*user, &access_control);
|
||||
}
|
||||
|
||||
case Kind::QUOTA:
|
||||
{
|
||||
QuotaPtr quota;
|
||||
if (show_query.current_quota)
|
||||
quota = access_control.read<Quota>(context.getQuota()->getUsageInfo().quota_id);
|
||||
else
|
||||
quota = access_control.read<Quota>(show_query.name);
|
||||
return getCreateQueryImpl(*quota, &access_control);
|
||||
}
|
||||
|
||||
case Kind::ROW_POLICY:
|
||||
{
|
||||
RowPolicyPtr policy = access_control.read<RowPolicy>(show_query.row_policy_name.getFullName(context));
|
||||
return getCreateQueryImpl(*policy, &access_control);
|
||||
}
|
||||
}
|
||||
__builtin_unreachable();
|
||||
}
|
||||
|
||||
|
||||
ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateUserQuery(const ASTShowCreateAccessEntityQuery & show_query) const
|
||||
ASTPtr InterpreterShowCreateAccessEntityQuery::getAttachQuery(const IAccessEntity & entity)
|
||||
{
|
||||
UserPtr user;
|
||||
if (show_query.current_user)
|
||||
user = context.getUser();
|
||||
else
|
||||
user = context.getAccessControlManager().read<User>(show_query.name);
|
||||
|
||||
auto create_query = std::make_shared<ASTCreateUserQuery>();
|
||||
create_query->name = user->getName();
|
||||
|
||||
if (user->allowed_client_hosts != AllowedClientHosts::AnyHostTag{})
|
||||
create_query->hosts = user->allowed_client_hosts;
|
||||
|
||||
if (!user->profile.empty())
|
||||
create_query->profile = user->profile;
|
||||
|
||||
if (user->default_roles != GenericRoleSet::AllTag{})
|
||||
create_query->default_roles = GenericRoleSet{user->default_roles}.toAST(context.getAccessControlManager());
|
||||
|
||||
return create_query;
|
||||
return getCreateQueryImpl(entity, nullptr, true);
|
||||
}
|
||||
|
||||
|
||||
ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuotaQuery(const ASTShowCreateAccessEntityQuery & show_query) const
|
||||
{
|
||||
auto & access_control = context.getAccessControlManager();
|
||||
|
||||
QuotaPtr quota;
|
||||
if (show_query.current_quota)
|
||||
quota = access_control.read<Quota>(context.getQuota()->getUsageInfo().quota_id);
|
||||
else
|
||||
quota = access_control.read<Quota>(show_query.name);
|
||||
|
||||
auto create_query = std::make_shared<ASTCreateQuotaQuery>();
|
||||
create_query->name = quota->getName();
|
||||
create_query->key_type = quota->key_type;
|
||||
create_query->all_limits.reserve(quota->all_limits.size());
|
||||
|
||||
for (const auto & limits : quota->all_limits)
|
||||
{
|
||||
ASTCreateQuotaQuery::Limits create_query_limits;
|
||||
create_query_limits.duration = limits.duration;
|
||||
create_query_limits.randomize_interval = limits.randomize_interval;
|
||||
for (auto resource_type : ext::range(Quota::MAX_RESOURCE_TYPE))
|
||||
if (limits.max[resource_type])
|
||||
create_query_limits.max[resource_type] = limits.max[resource_type];
|
||||
create_query->all_limits.push_back(create_query_limits);
|
||||
}
|
||||
|
||||
if (!quota->roles.empty())
|
||||
create_query->roles = quota->roles.toAST(access_control);
|
||||
|
||||
return create_query;
|
||||
}
|
||||
|
||||
|
||||
ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateRowPolicyQuery(const ASTShowCreateAccessEntityQuery & show_query) const
|
||||
{
|
||||
auto & access_control = context.getAccessControlManager();
|
||||
RowPolicyPtr policy = access_control.read<RowPolicy>(show_query.row_policy_name.getFullName(context));
|
||||
|
||||
auto create_query = std::make_shared<ASTCreateRowPolicyQuery>();
|
||||
create_query->name_parts = RowPolicy::FullNameParts{policy->getDatabase(), policy->getTableName(), policy->getName()};
|
||||
if (policy->isRestrictive())
|
||||
create_query->is_restrictive = policy->isRestrictive();
|
||||
|
||||
for (auto index : ext::range_with_static_cast<RowPolicy::ConditionIndex>(RowPolicy::MAX_CONDITION_INDEX))
|
||||
{
|
||||
const auto & condition = policy->conditions[index];
|
||||
if (!condition.empty())
|
||||
{
|
||||
ParserExpression parser;
|
||||
ASTPtr expr = parseQuery(parser, condition, 0);
|
||||
create_query->conditions.push_back(std::pair{index, expr});
|
||||
}
|
||||
}
|
||||
|
||||
if (!policy->roles.empty())
|
||||
create_query->roles = policy->roles.toAST(access_control);
|
||||
|
||||
return create_query;
|
||||
}
|
||||
}
|
||||
|
@ -2,12 +2,14 @@
|
||||
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Core/UUID.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class Context;
|
||||
class ASTShowCreateAccessEntityQuery;
|
||||
struct IAccessEntity;
|
||||
|
||||
|
||||
/** Returns a single item containing a statement which could be used to create a specified role.
|
||||
@ -23,15 +25,14 @@ public:
|
||||
bool ignoreQuota() const override { return true; }
|
||||
bool ignoreLimits() const override { return true; }
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
const Context & context;
|
||||
static ASTPtr getAttachQuery(const IAccessEntity & entity);
|
||||
|
||||
private:
|
||||
BlockInputStreamPtr executeImpl();
|
||||
ASTPtr getCreateQuery(const ASTShowCreateAccessEntityQuery & show_query) const;
|
||||
ASTPtr getCreateUserQuery(const ASTShowCreateAccessEntityQuery & show_query) const;
|
||||
ASTPtr getCreateQuotaQuery(const ASTShowCreateAccessEntityQuery & show_query) const;
|
||||
ASTPtr getCreateRowPolicyQuery(const ASTShowCreateAccessEntityQuery & show_query) const;
|
||||
|
||||
ASTPtr query_ptr;
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
|
||||
|
@ -16,6 +16,11 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
std::vector<AccessRightsElements> groupByTable(AccessRightsElements && elements)
|
||||
@ -47,6 +52,79 @@ namespace
|
||||
res.partial_revokes = groupByTable(std::move(elements.partial_revokes));
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
ASTs getGrantQueriesImpl(
|
||||
const T & grantee,
|
||||
const AccessControlManager * manager /* not used if attach_mode == true */,
|
||||
bool attach_mode = false)
|
||||
{
|
||||
ASTs res;
|
||||
|
||||
std::shared_ptr<ASTGenericRoleSet> to_roles = std::make_shared<ASTGenericRoleSet>();
|
||||
to_roles->names.push_back(grantee.getName());
|
||||
|
||||
for (bool grant_option : {true, false})
|
||||
{
|
||||
if (!grant_option && (grantee.access == grantee.access_with_grant_option))
|
||||
continue;
|
||||
const auto & access_rights = grant_option ? grantee.access_with_grant_option : grantee.access;
|
||||
const auto grouped_elements = groupByTable(access_rights.getElements());
|
||||
|
||||
using Kind = ASTGrantQuery::Kind;
|
||||
for (Kind kind : {Kind::GRANT, Kind::REVOKE})
|
||||
{
|
||||
for (const auto & elements : (kind == Kind::GRANT ? grouped_elements.grants : grouped_elements.partial_revokes))
|
||||
{
|
||||
auto grant_query = std::make_shared<ASTGrantQuery>();
|
||||
grant_query->kind = kind;
|
||||
grant_query->attach = attach_mode;
|
||||
grant_query->grant_option = grant_option;
|
||||
grant_query->to_roles = to_roles;
|
||||
grant_query->access_rights_elements = elements;
|
||||
res.push_back(std::move(grant_query));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for (bool admin_option : {true, false})
|
||||
{
|
||||
if (!admin_option && (grantee.granted_roles == grantee.granted_roles_with_admin_option))
|
||||
continue;
|
||||
|
||||
const auto & roles = admin_option ? grantee.granted_roles_with_admin_option : grantee.granted_roles;
|
||||
if (roles.empty())
|
||||
continue;
|
||||
|
||||
auto grant_query = std::make_shared<ASTGrantQuery>();
|
||||
using Kind = ASTGrantQuery::Kind;
|
||||
grant_query->kind = Kind::GRANT;
|
||||
grant_query->attach = attach_mode;
|
||||
grant_query->admin_option = admin_option;
|
||||
grant_query->to_roles = to_roles;
|
||||
if (attach_mode)
|
||||
grant_query->roles = GenericRoleSet{roles}.toAST();
|
||||
else
|
||||
grant_query->roles = GenericRoleSet{roles}.toASTWithNames(*manager);
|
||||
res.push_back(std::move(grant_query));
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
ASTs getGrantQueriesImpl(
|
||||
const IAccessEntity & entity,
|
||||
const AccessControlManager * manager /* not used if attach_mode == true */,
|
||||
bool attach_mode = false)
|
||||
{
|
||||
if (const User * user = typeid_cast<const User *>(&entity))
|
||||
return getGrantQueriesImpl(*user, manager, attach_mode);
|
||||
if (const Role * role = typeid_cast<const Role *>(&entity))
|
||||
return getGrantQueriesImpl(*role, manager, attach_mode);
|
||||
throw Exception("Unexpected type of access entity: " + entity.getTypeName(), ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
@ -90,80 +168,24 @@ BlockInputStreamPtr InterpreterShowGrantsQuery::executeImpl()
|
||||
ASTs InterpreterShowGrantsQuery::getGrantQueries(const ASTShowGrantsQuery & show_query) const
|
||||
{
|
||||
const auto & access_control = context.getAccessControlManager();
|
||||
UserPtr user;
|
||||
RolePtr role;
|
||||
|
||||
AccessEntityPtr user_or_role;
|
||||
if (show_query.current_user)
|
||||
user = context.getUser();
|
||||
user_or_role = context.getUser();
|
||||
else
|
||||
{
|
||||
user = access_control.tryRead<User>(show_query.name);
|
||||
if (!user)
|
||||
role = access_control.read<Role>(show_query.name);
|
||||
user_or_role = access_control.tryRead<User>(show_query.name);
|
||||
if (!user_or_role)
|
||||
user_or_role = access_control.read<Role>(show_query.name);
|
||||
}
|
||||
|
||||
const AccessRights * access = nullptr;
|
||||
const AccessRights * access_with_grant_option = nullptr;
|
||||
const boost::container::flat_set<UUID> * granted_roles = nullptr;
|
||||
const boost::container::flat_set<UUID> * granted_roles_with_admin_option = nullptr;
|
||||
if (user)
|
||||
{
|
||||
access = &user->access;
|
||||
access_with_grant_option = &user->access_with_grant_option;
|
||||
granted_roles = &user->granted_roles;
|
||||
granted_roles_with_admin_option = &user->granted_roles_with_admin_option;
|
||||
}
|
||||
else
|
||||
{
|
||||
access = &role->access;
|
||||
access_with_grant_option = &role->access_with_grant_option;
|
||||
granted_roles = &role->granted_roles;
|
||||
granted_roles_with_admin_option = &role->granted_roles_with_admin_option;
|
||||
}
|
||||
|
||||
ASTs res;
|
||||
|
||||
for (bool grant_option : {true, false})
|
||||
{
|
||||
if (!grant_option && (*access == *access_with_grant_option))
|
||||
continue;
|
||||
const auto & access_rights = grant_option ? *access_with_grant_option : *access;
|
||||
const auto grouped_elements = groupByTable(access_rights.getElements());
|
||||
|
||||
using Kind = ASTGrantQuery::Kind;
|
||||
for (Kind kind : {Kind::GRANT, Kind::REVOKE})
|
||||
{
|
||||
for (const auto & elements : (kind == Kind::GRANT ? grouped_elements.grants : grouped_elements.partial_revokes))
|
||||
{
|
||||
auto grant_query = std::make_shared<ASTGrantQuery>();
|
||||
grant_query->kind = kind;
|
||||
grant_query->grant_option = grant_option;
|
||||
grant_query->to_roles = std::make_shared<ASTGenericRoleSet>();
|
||||
grant_query->to_roles->names.push_back(show_query.name);
|
||||
grant_query->access_rights_elements = elements;
|
||||
res.push_back(std::move(grant_query));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for (bool admin_option : {true, false})
|
||||
{
|
||||
if (!admin_option && (*granted_roles == *granted_roles_with_admin_option))
|
||||
continue;
|
||||
|
||||
const auto & roles = admin_option ? *granted_roles_with_admin_option : *granted_roles;
|
||||
if (roles.empty())
|
||||
continue;
|
||||
|
||||
auto grant_query = std::make_shared<ASTGrantQuery>();
|
||||
using Kind = ASTGrantQuery::Kind;
|
||||
grant_query->kind = Kind::GRANT;
|
||||
grant_query->admin_option = admin_option;
|
||||
grant_query->to_roles = std::make_shared<ASTGenericRoleSet>();
|
||||
grant_query->to_roles->names.push_back(show_query.name);
|
||||
grant_query->roles = GenericRoleSet{roles}.toAST(access_control);
|
||||
res.push_back(std::move(grant_query));
|
||||
}
|
||||
|
||||
return res;
|
||||
return getGrantQueriesImpl(*user_or_role, &access_control);
|
||||
}
|
||||
|
||||
|
||||
ASTs InterpreterShowGrantsQuery::getAttachGrantQueries(const IAccessEntity & user_or_role)
|
||||
{
|
||||
return getGrantQueriesImpl(user_or_role, nullptr, true);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,11 +2,13 @@
|
||||
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Core/UUID.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ASTShowGrantsQuery;
|
||||
struct IAccessEntity;
|
||||
|
||||
|
||||
class InterpreterShowGrantsQuery : public IInterpreter
|
||||
@ -16,6 +18,8 @@ public:
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
static ASTs getAttachGrantQueries(const IAccessEntity & user_or_role);
|
||||
|
||||
private:
|
||||
BlockInputStreamPtr executeImpl();
|
||||
ASTs getGrantQueries(const ASTShowGrantsQuery & show_query) const;
|
||||
|
@ -203,7 +203,7 @@ void runOneTest(const TestDescriptor & test_descriptor)
|
||||
|
||||
try
|
||||
{
|
||||
acl_manager.loadFromConfig(*config);
|
||||
acl_manager.setUsersConfig(*config);
|
||||
}
|
||||
catch (const Poco::Exception & ex)
|
||||
{
|
||||
|
@ -116,8 +116,15 @@ ASTPtr ASTCreateQuotaQuery::clone() const
|
||||
|
||||
void ASTCreateQuotaQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER QUOTA" : "CREATE QUOTA")
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
if (attach)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ATTACH QUOTA" << (settings.hilite ? hilite_none : "");
|
||||
}
|
||||
else
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER QUOTA" : "CREATE QUOTA")
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
}
|
||||
|
||||
if (if_exists)
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF EXISTS" << (settings.hilite ? hilite_none : "");
|
||||
|
@ -29,6 +29,7 @@ class ASTCreateQuotaQuery : public IAST
|
||||
{
|
||||
public:
|
||||
bool alter = false;
|
||||
bool attach = false;
|
||||
|
||||
bool if_exists = false;
|
||||
bool if_not_exists = false;
|
||||
@ -36,7 +37,6 @@ public:
|
||||
|
||||
String name;
|
||||
String new_name;
|
||||
|
||||
using KeyType = Quota::KeyType;
|
||||
std::optional<KeyType> key_type;
|
||||
|
||||
|
@ -28,8 +28,15 @@ ASTPtr ASTCreateRoleQuery::clone() const
|
||||
|
||||
void ASTCreateRoleQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER ROLE" : "CREATE ROLE")
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
if (attach)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ATTACH ROLE" << (settings.hilite ? hilite_none : "");
|
||||
}
|
||||
else
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER ROLE" : "CREATE ROLE")
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
}
|
||||
|
||||
if (if_exists)
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF EXISTS" << (settings.hilite ? hilite_none : "");
|
||||
|
@ -14,6 +14,7 @@ class ASTCreateRoleQuery : public IAST
|
||||
{
|
||||
public:
|
||||
bool alter = false;
|
||||
bool attach = false;
|
||||
|
||||
bool if_exists = false;
|
||||
bool if_not_exists = false;
|
||||
|
@ -134,8 +134,15 @@ ASTPtr ASTCreateRowPolicyQuery::clone() const
|
||||
|
||||
void ASTCreateRowPolicyQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER POLICY" : "CREATE POLICY")
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
if (attach)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ATTACH POLICY";
|
||||
}
|
||||
else
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER POLICY" : "CREATE POLICY")
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
}
|
||||
|
||||
if (if_exists)
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF EXISTS" << (settings.hilite ? hilite_none : "");
|
||||
|
@ -29,6 +29,7 @@ class ASTCreateRowPolicyQuery : public IAST
|
||||
{
|
||||
public:
|
||||
bool alter = false;
|
||||
bool attach = false;
|
||||
|
||||
bool if_exists = false;
|
||||
bool if_not_exists = false;
|
||||
|
@ -164,8 +164,15 @@ ASTPtr ASTCreateUserQuery::clone() const
|
||||
|
||||
void ASTCreateUserQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER USER" : "CREATE USER")
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
if (attach)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ATTACH USER" << (settings.hilite ? hilite_none : "");
|
||||
}
|
||||
else
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER USER" : "CREATE USER")
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
}
|
||||
|
||||
if (if_exists)
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF EXISTS" << (settings.hilite ? hilite_none : "");
|
||||
|
@ -26,6 +26,7 @@ class ASTCreateUserQuery : public IAST
|
||||
{
|
||||
public:
|
||||
bool alter = false;
|
||||
bool attach = false;
|
||||
|
||||
bool if_exists = false;
|
||||
bool if_not_exists = false;
|
||||
|
@ -4,6 +4,22 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace
|
||||
{
|
||||
void formatRoleNameOrID(const String & str, bool is_id, const IAST::FormatSettings & settings)
|
||||
{
|
||||
if (is_id)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "ID" << (settings.hilite ? IAST::hilite_none : "") << "("
|
||||
<< quoteString(str) << ")";
|
||||
}
|
||||
else
|
||||
{
|
||||
settings.ostr << backQuoteIfNeed(str);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void ASTGenericRoleSet::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
|
||||
{
|
||||
if (empty())
|
||||
@ -25,7 +41,7 @@ void ASTGenericRoleSet::formatImpl(const FormatSettings & settings, FormatState
|
||||
{
|
||||
if (std::exchange(need_comma, true))
|
||||
settings.ostr << ", ";
|
||||
settings.ostr << backQuoteIfNeed(role);
|
||||
formatRoleNameOrID(role, id_mode, settings);
|
||||
}
|
||||
|
||||
if (current_user)
|
||||
@ -45,7 +61,7 @@ void ASTGenericRoleSet::formatImpl(const FormatSettings & settings, FormatState
|
||||
{
|
||||
if (std::exchange(need_comma, true))
|
||||
settings.ostr << ", ";
|
||||
settings.ostr << backQuoteIfNeed(except_role);
|
||||
formatRoleNameOrID(except_role, id_mode, settings);
|
||||
}
|
||||
|
||||
if (except_current_user)
|
||||
|
@ -16,6 +16,7 @@ public:
|
||||
bool all = false;
|
||||
Strings except_names;
|
||||
bool except_current_user = false;
|
||||
bool id_mode = false; /// If true then `names` and `except_names` keeps UUIDs, not names.
|
||||
|
||||
bool empty() const { return names.empty() && !current_user && !all; }
|
||||
|
||||
|
@ -121,7 +121,7 @@ ASTPtr ASTGrantQuery::clone() const
|
||||
|
||||
void ASTGrantQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
|
||||
{
|
||||
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << ((kind == Kind::GRANT) ? "GRANT" : "REVOKE")
|
||||
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << (attach ? "ATTACH " : "") << ((kind == Kind::GRANT) ? "GRANT" : "REVOKE")
|
||||
<< (settings.hilite ? IAST::hilite_none : "") << " ";
|
||||
|
||||
if (kind == Kind::REVOKE)
|
||||
|
@ -24,6 +24,7 @@ public:
|
||||
REVOKE,
|
||||
};
|
||||
Kind kind = Kind::GRANT;
|
||||
bool attach = false;
|
||||
AccessRightsElements access_rights_elements;
|
||||
std::shared_ptr<ASTGenericRoleSet> roles;
|
||||
std::shared_ptr<ASTGenericRoleSet> to_roles;
|
||||
|
@ -187,12 +187,12 @@ namespace
|
||||
});
|
||||
}
|
||||
|
||||
bool parseToRoles(IParserBase::Pos & pos, Expected & expected, std::shared_ptr<ASTGenericRoleSet> & roles)
|
||||
bool parseToRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr<ASTGenericRoleSet> & roles)
|
||||
{
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
ASTPtr node;
|
||||
if (roles || !ParserKeyword{"TO"}.ignore(pos, expected) || !ParserGenericRoleSet{}.parse(pos, node, expected))
|
||||
if (roles || !ParserKeyword{"TO"}.ignore(pos, expected) || !ParserGenericRoleSet{}.enableIDMode(id_mode).parse(pos, node, expected))
|
||||
return false;
|
||||
|
||||
roles = std::static_pointer_cast<ASTGenericRoleSet>(node);
|
||||
@ -204,13 +204,21 @@ namespace
|
||||
|
||||
bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
bool alter;
|
||||
if (ParserKeyword{"CREATE QUOTA"}.ignore(pos, expected))
|
||||
alter = false;
|
||||
else if (ParserKeyword{"ALTER QUOTA"}.ignore(pos, expected))
|
||||
alter = true;
|
||||
bool alter = false;
|
||||
bool attach = false;
|
||||
if (attach_mode)
|
||||
{
|
||||
if (!ParserKeyword{"ATTACH QUOTA"}.ignore(pos, expected))
|
||||
return false;
|
||||
attach = true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
{
|
||||
if (ParserKeyword{"ALTER QUOTA"}.ignore(pos, expected))
|
||||
alter = true;
|
||||
else if (!ParserKeyword{"CREATE QUOTA"}.ignore(pos, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
bool if_exists = false;
|
||||
bool if_not_exists = false;
|
||||
@ -248,7 +256,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
if (parseAllLimits(pos, expected, alter, all_limits))
|
||||
continue;
|
||||
|
||||
if (!roles && parseToRoles(pos, expected, roles))
|
||||
if (!roles && parseToRoles(pos, expected, attach, roles))
|
||||
continue;
|
||||
|
||||
break;
|
||||
|
@ -24,8 +24,14 @@ namespace DB
|
||||
*/
|
||||
class ParserCreateQuotaQuery : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserCreateQuotaQuery & enableAttachMode(bool enable_) { attach_mode = enable_; return *this; }
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "CREATE QUOTA or ALTER QUOTA query"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
||||
private:
|
||||
bool attach_mode = false;
|
||||
};
|
||||
}
|
||||
|
@ -1,6 +1,8 @@
|
||||
#include <Parsers/ParserCreateRoleQuery.h>
|
||||
#include <Parsers/ASTCreateRoleQuery.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/parseUserName.h>
|
||||
|
||||
|
||||
@ -23,13 +25,21 @@ namespace
|
||||
|
||||
bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
bool alter;
|
||||
if (ParserKeyword{"CREATE ROLE"}.ignore(pos, expected))
|
||||
alter = false;
|
||||
else if (ParserKeyword{"ALTER ROLE"}.ignore(pos, expected))
|
||||
alter = true;
|
||||
bool attach = false;
|
||||
bool alter = false;
|
||||
if (attach_mode)
|
||||
{
|
||||
if (!ParserKeyword{"ATTACH ROLE"}.ignore(pos, expected))
|
||||
return false;
|
||||
attach = true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
{
|
||||
if (ParserKeyword{"ALTER ROLE"}.ignore(pos, expected))
|
||||
alter = true;
|
||||
else if (!ParserKeyword{"CREATE ROLE"}.ignore(pos, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
bool if_exists = false;
|
||||
bool if_not_exists = false;
|
||||
@ -59,6 +69,7 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
node = query;
|
||||
|
||||
query->alter = alter;
|
||||
query->attach = attach;
|
||||
query->if_exists = if_exists;
|
||||
query->if_not_exists = if_not_exists;
|
||||
query->or_replace = or_replace;
|
||||
|
@ -13,8 +13,14 @@ namespace DB
|
||||
*/
|
||||
class ParserCreateRoleQuery : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserCreateRoleQuery & enableAttachMode(bool enable) { attach_mode = enable; return *this; }
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "CREATE ROLE or ALTER ROLE query"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
||||
private:
|
||||
bool attach_mode = false;
|
||||
};
|
||||
}
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Parsers/parseIdentifierOrStringLiteral.h>
|
||||
#include <Parsers/parseDatabaseAndTableName.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
|
||||
@ -186,12 +187,13 @@ namespace
|
||||
});
|
||||
}
|
||||
|
||||
bool parseToRoles(IParserBase::Pos & pos, Expected & expected, std::shared_ptr<ASTGenericRoleSet> & roles)
|
||||
bool parseToRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr<ASTGenericRoleSet> & roles)
|
||||
{
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
ASTPtr ast;
|
||||
if (roles || !ParserKeyword{"TO"}.ignore(pos, expected) || !ParserGenericRoleSet{}.parse(pos, ast, expected))
|
||||
if (roles || !ParserKeyword{"TO"}.ignore(pos, expected)
|
||||
|| !ParserGenericRoleSet{}.enableIDMode(id_mode).parse(pos, ast, expected))
|
||||
return false;
|
||||
|
||||
roles = std::static_pointer_cast<ASTGenericRoleSet>(ast);
|
||||
@ -203,13 +205,21 @@ namespace
|
||||
|
||||
bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
bool alter;
|
||||
if (ParserKeyword{"CREATE POLICY"}.ignore(pos, expected) || ParserKeyword{"CREATE ROW POLICY"}.ignore(pos, expected))
|
||||
alter = false;
|
||||
else if (ParserKeyword{"ALTER POLICY"}.ignore(pos, expected) || ParserKeyword{"ALTER ROW POLICY"}.ignore(pos, expected))
|
||||
alter = true;
|
||||
bool alter = false;
|
||||
bool attach = false;
|
||||
if (attach_mode)
|
||||
{
|
||||
if (!ParserKeyword{"ATTACH POLICY"}.ignore(pos, expected) && !ParserKeyword{"ATTACH ROW POLICY"}.ignore(pos, expected))
|
||||
return false;
|
||||
attach = true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
{
|
||||
if (ParserKeyword{"ALTER POLICY"}.ignore(pos, expected) || ParserKeyword{"ALTER ROW POLICY"}.ignore(pos, expected))
|
||||
alter = true;
|
||||
else if (!ParserKeyword{"CREATE POLICY"}.ignore(pos, expected) && !ParserKeyword{"CREATE ROW POLICY"}.ignore(pos, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
bool if_exists = false;
|
||||
bool if_not_exists = false;
|
||||
@ -251,7 +261,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
if (parseMultipleConditions(pos, expected, alter, conditions))
|
||||
continue;
|
||||
|
||||
if (!roles && parseToRoles(pos, expected, roles))
|
||||
if (!roles && parseToRoles(pos, expected, attach, roles))
|
||||
continue;
|
||||
|
||||
break;
|
||||
@ -261,6 +271,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
node = query;
|
||||
|
||||
query->alter = alter;
|
||||
query->attach = attach;
|
||||
query->if_exists = if_exists;
|
||||
query->if_not_exists = if_not_exists;
|
||||
query->or_replace = or_replace;
|
||||
|
@ -23,8 +23,14 @@ namespace DB
|
||||
*/
|
||||
class ParserCreateRowPolicyQuery : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserCreateRowPolicyQuery & enableAttachMode(bool enable_) { attach_mode = enable_; return *this; }
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "CREATE ROW POLICY or ALTER ROW POLICY query"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
||||
private:
|
||||
bool attach_mode = false;
|
||||
};
|
||||
}
|
||||
|
@ -208,7 +208,7 @@ namespace
|
||||
}
|
||||
|
||||
|
||||
bool parseDefaultRoles(IParserBase::Pos & pos, Expected & expected, std::shared_ptr<ASTGenericRoleSet> & default_roles)
|
||||
bool parseDefaultRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr<ASTGenericRoleSet> & default_roles)
|
||||
{
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
@ -216,7 +216,7 @@ namespace
|
||||
return false;
|
||||
|
||||
ASTPtr ast;
|
||||
if (!ParserGenericRoleSet{}.allowCurrentUser(false).parse(pos, ast, expected))
|
||||
if (!ParserGenericRoleSet{}.enableCurrentUserKeyword(false).enableIDMode(id_mode).parse(pos, ast, expected))
|
||||
return false;
|
||||
|
||||
default_roles = typeid_cast<std::shared_ptr<ASTGenericRoleSet>>(ast);
|
||||
@ -245,13 +245,22 @@ namespace
|
||||
|
||||
bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
bool alter;
|
||||
if (ParserKeyword{"CREATE USER"}.ignore(pos, expected))
|
||||
alter = false;
|
||||
else if (ParserKeyword{"ALTER USER"}.ignore(pos, expected))
|
||||
alter = true;
|
||||
bool alter = false;
|
||||
bool attach = false;
|
||||
if (attach_mode)
|
||||
{
|
||||
if (ParserKeyword{"ATTACH USER"}.ignore(pos, expected))
|
||||
attach = true;
|
||||
else
|
||||
return false;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
{
|
||||
if (ParserKeyword{"ALTER USER"}.ignore(pos, expected))
|
||||
alter = true;
|
||||
else if (!ParserKeyword{"CREATE USER"}.ignore(pos, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
bool if_exists = false;
|
||||
bool if_not_exists = false;
|
||||
@ -294,7 +303,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
if (!profile && parseProfileName(pos, expected, profile))
|
||||
continue;
|
||||
|
||||
if (!default_roles && parseDefaultRoles(pos, expected, default_roles))
|
||||
if (!default_roles && parseDefaultRoles(pos, expected, attach, default_roles))
|
||||
continue;
|
||||
|
||||
if (alter)
|
||||
@ -321,6 +330,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
node = query;
|
||||
|
||||
query->alter = alter;
|
||||
query->attach = attach;
|
||||
query->if_exists = if_exists;
|
||||
query->if_not_exists = if_not_exists;
|
||||
query->or_replace = or_replace;
|
||||
@ -330,6 +340,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
query->hosts = std::move(hosts);
|
||||
query->add_hosts = std::move(add_hosts);
|
||||
query->remove_hosts = std::move(remove_hosts);
|
||||
query->default_roles = std::move(default_roles);
|
||||
query->profile = std::move(profile);
|
||||
|
||||
return true;
|
||||
|
@ -19,8 +19,14 @@ namespace DB
|
||||
*/
|
||||
class ParserCreateUserQuery : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserCreateUserQuery & enableAttachMode(bool enable) { attach_mode = enable; return *this; }
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "CREATE USER or ALTER USER query"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
||||
private:
|
||||
bool attach_mode = false;
|
||||
};
|
||||
}
|
||||
|
@ -1,5 +1,7 @@
|
||||
#include <Parsers/ParserGenericRoleSet.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTGenericRoleSet.h>
|
||||
#include <Parsers/parseUserName.h>
|
||||
#include <boost/range/algorithm/find.hpp>
|
||||
@ -9,7 +11,39 @@ namespace DB
|
||||
{
|
||||
namespace
|
||||
{
|
||||
bool parseBeforeExcept(IParserBase::Pos & pos, Expected & expected, bool * all, bool * current_user, Strings & names)
|
||||
bool parseRoleNameOrID(IParserBase::Pos & pos, Expected & expected, bool parse_id, String & res)
|
||||
{
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
if (!parse_id)
|
||||
return parseRoleName(pos, expected, res);
|
||||
|
||||
if (!ParserKeyword{"ID"}.ignore(pos, expected))
|
||||
return false;
|
||||
if (!ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected))
|
||||
return false;
|
||||
ASTPtr ast;
|
||||
if (!ParserStringLiteral{}.parse(pos, ast, expected))
|
||||
return false;
|
||||
String id = ast->as<ASTLiteral &>().value.safeGet<String>();
|
||||
if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
res = std::move(id);
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
bool parseBeforeExcept(
|
||||
IParserBase::Pos & pos,
|
||||
Expected & expected,
|
||||
bool id_mode,
|
||||
bool all_keyword_enabled,
|
||||
bool current_user_keyword_enabled,
|
||||
Strings & names,
|
||||
bool & all,
|
||||
bool & current_user)
|
||||
{
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
@ -22,7 +56,8 @@ namespace
|
||||
{
|
||||
}
|
||||
else if (
|
||||
current_user && (ParserKeyword{"CURRENT_USER"}.ignore(pos, expected) || ParserKeyword{"currentUser"}.ignore(pos, expected)))
|
||||
current_user_keyword_enabled
|
||||
&& (ParserKeyword{"CURRENT_USER"}.ignore(pos, expected) || ParserKeyword{"currentUser"}.ignore(pos, expected)))
|
||||
{
|
||||
if (ParserToken{TokenType::OpeningRoundBracket}.ignore(pos, expected))
|
||||
{
|
||||
@ -31,14 +66,14 @@ namespace
|
||||
}
|
||||
res_current_user = true;
|
||||
}
|
||||
else if (all && ParserKeyword{"ALL"}.ignore(pos, expected))
|
||||
else if (all_keyword_enabled && ParserKeyword{"ALL"}.ignore(pos, expected))
|
||||
{
|
||||
res_all = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
String name;
|
||||
if (!parseUserName(pos, expected, name))
|
||||
if (!parseRoleNameOrID(pos, expected, id_mode, name))
|
||||
return false;
|
||||
res_names.push_back(name);
|
||||
}
|
||||
@ -47,23 +82,28 @@ namespace
|
||||
break;
|
||||
}
|
||||
|
||||
if (all)
|
||||
*all = res_all;
|
||||
if (current_user)
|
||||
*current_user = res_current_user;
|
||||
all = res_all;
|
||||
current_user = res_current_user;
|
||||
names = std::move(res_names);
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
bool parseExcept(IParserBase::Pos & pos, Expected & expected, bool * except_current_user, Strings & except_names)
|
||||
bool parseExceptAndAfterExcept(
|
||||
IParserBase::Pos & pos,
|
||||
Expected & expected,
|
||||
bool id_mode,
|
||||
bool current_user_keyword_enabled,
|
||||
Strings & except_names,
|
||||
bool & except_current_user)
|
||||
{
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
if (!ParserKeyword{"EXCEPT"}.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
return parseBeforeExcept(pos, expected, nullptr, except_current_user, except_names);
|
||||
bool dummy;
|
||||
return parseBeforeExcept(pos, expected, id_mode, false, current_user_keyword_enabled, except_names, dummy, except_current_user);
|
||||
});
|
||||
}
|
||||
}
|
||||
@ -77,10 +117,10 @@ bool ParserGenericRoleSet::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
|
||||
Strings except_names;
|
||||
bool except_current_user = false;
|
||||
|
||||
if (!parseBeforeExcept(pos, expected, (allow_all ? &all : nullptr), (allow_current_user ? ¤t_user : nullptr), names))
|
||||
if (!parseBeforeExcept(pos, expected, id_mode, all_keyword, current_user_keyword, names, all, current_user))
|
||||
return false;
|
||||
|
||||
parseExcept(pos, expected, (allow_current_user ? &except_current_user : nullptr), except_names);
|
||||
parseExceptAndAfterExcept(pos, expected, id_mode, current_user_keyword, except_names, except_current_user);
|
||||
|
||||
if (all)
|
||||
names.clear();
|
||||
@ -91,6 +131,7 @@ bool ParserGenericRoleSet::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
|
||||
result->all = all;
|
||||
result->except_names = std::move(except_names);
|
||||
result->except_current_user = except_current_user;
|
||||
result->id_mode = id_mode;
|
||||
node = result;
|
||||
return true;
|
||||
}
|
||||
|
@ -11,16 +11,18 @@ namespace DB
|
||||
class ParserGenericRoleSet : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserGenericRoleSet & allowAll(bool allow_) { allow_all = allow_; return *this; }
|
||||
ParserGenericRoleSet & allowCurrentUser(bool allow_) { allow_current_user = allow_; return *this; }
|
||||
ParserGenericRoleSet & enableAllKeyword(bool enable_) { all_keyword = enable_; return *this; }
|
||||
ParserGenericRoleSet & enableCurrentUserKeyword(bool enable_) { current_user_keyword = enable_; return *this; }
|
||||
ParserGenericRoleSet & enableIDMode(bool enable_) { id_mode = enable_; return *this; }
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "GenericRoleSet"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
||||
private:
|
||||
bool allow_all = true;
|
||||
bool allow_current_user = true;
|
||||
bool all_keyword = true;
|
||||
bool current_user_keyword = true;
|
||||
bool id_mode = false;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -211,12 +211,12 @@ namespace
|
||||
}
|
||||
|
||||
|
||||
bool parseRoles(IParser::Pos & pos, Expected & expected, std::shared_ptr<ASTGenericRoleSet> & roles)
|
||||
bool parseRoles(IParser::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr<ASTGenericRoleSet> & roles)
|
||||
{
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
ASTPtr ast;
|
||||
if (!ParserGenericRoleSet{}.allowAll(false).allowCurrentUser(false).parse(pos, ast, expected))
|
||||
if (!ParserGenericRoleSet{}.enableAllKeyword(false).enableCurrentUserKeyword(false).enableIDMode(id_mode).parse(pos, ast, expected))
|
||||
return false;
|
||||
|
||||
roles = typeid_cast<std::shared_ptr<ASTGenericRoleSet>>(ast);
|
||||
@ -242,7 +242,7 @@ namespace
|
||||
}
|
||||
|
||||
ASTPtr ast;
|
||||
if (!ParserGenericRoleSet{}.allowAll(kind == Kind::REVOKE).parse(pos, ast, expected))
|
||||
if (!ParserGenericRoleSet{}.enableAllKeyword(kind == Kind::REVOKE).parse(pos, ast, expected))
|
||||
return false;
|
||||
|
||||
to_roles = typeid_cast<std::shared_ptr<ASTGenericRoleSet>>(ast);
|
||||
@ -254,6 +254,14 @@ namespace
|
||||
|
||||
bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
bool attach = false;
|
||||
if (attach_mode)
|
||||
{
|
||||
if (!ParserKeyword{"ATTACH"}.ignore(pos, expected))
|
||||
return false;
|
||||
attach = true;
|
||||
}
|
||||
|
||||
using Kind = ASTGrantQuery::Kind;
|
||||
Kind kind;
|
||||
if (ParserKeyword{"GRANT"}.ignore(pos, expected))
|
||||
@ -275,7 +283,7 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
|
||||
AccessRightsElements elements;
|
||||
std::shared_ptr<ASTGenericRoleSet> roles;
|
||||
if (!parseAccessRightsElements(pos, expected, elements) && !parseRoles(pos, expected, roles))
|
||||
if (!parseAccessRightsElements(pos, expected, elements) && !parseRoles(pos, expected, attach, roles))
|
||||
return false;
|
||||
|
||||
std::shared_ptr<ASTGenericRoleSet> to_roles;
|
||||
@ -299,6 +307,7 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
node = query;
|
||||
|
||||
query->kind = kind;
|
||||
query->attach = attach;
|
||||
query->access_rights_elements = std::move(elements);
|
||||
query->roles = std::move(roles);
|
||||
query->to_roles = std::move(to_roles);
|
||||
|
@ -11,8 +11,14 @@ namespace DB
|
||||
*/
|
||||
class ParserGrantQuery : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserGrantQuery & enableAttachMode(bool enable) { attach_mode = enable; return *this; }
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "GRANT or REVOKE query"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
||||
private:
|
||||
bool attach_mode = false;
|
||||
};
|
||||
}
|
||||
|
@ -14,7 +14,7 @@ namespace
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
ASTPtr ast;
|
||||
if (!ParserGenericRoleSet{}.allowCurrentUser(false).parse(pos, ast, expected))
|
||||
if (!ParserGenericRoleSet{}.enableCurrentUserKeyword(false).parse(pos, ast, expected))
|
||||
return false;
|
||||
|
||||
roles = typeid_cast<std::shared_ptr<ASTGenericRoleSet>>(ast);
|
||||
@ -30,7 +30,7 @@ namespace
|
||||
return false;
|
||||
|
||||
ASTPtr ast;
|
||||
if (!ParserGenericRoleSet{}.allowAll(false).parse(pos, ast, expected))
|
||||
if (!ParserGenericRoleSet{}.enableAllKeyword(false).parse(pos, ast, expected))
|
||||
return false;
|
||||
|
||||
to_users = typeid_cast<std::shared_ptr<ASTGenericRoleSet>>(ast);
|
||||
|
@ -87,7 +87,7 @@ void StorageSystemQuotas::fillData(MutableColumns & res_columns, const Context &
|
||||
storage_name_column.insert(storage_name);
|
||||
key_type_column.insert(static_cast<UInt8>(quota->key_type));
|
||||
|
||||
for (const String & role : quota->roles.toStrings(access_control))
|
||||
for (const String & role : quota->roles.toStringsWithNames(access_control))
|
||||
roles_data.insert(role);
|
||||
roles_offsets.push_back(roles_data.size());
|
||||
|
||||
|
@ -401,77 +401,82 @@ class ClickHouseCluster:
|
||||
|
||||
if not subprocess_call(['docker-compose', 'kill']):
|
||||
subprocess_call(['docker-compose', 'down', '--volumes'])
|
||||
logging.info("Unstopped containers killed")
|
||||
except:
|
||||
pass
|
||||
|
||||
logging.info("Unstopped containers killed")
|
||||
try:
|
||||
if destroy_dirs and p.exists(self.instances_dir):
|
||||
logging.info("Removing instances dir %s", self.instances_dir)
|
||||
shutil.rmtree(self.instances_dir)
|
||||
|
||||
if destroy_dirs and p.exists(self.instances_dir):
|
||||
logging.info("Removing instances dir %s", self.instances_dir)
|
||||
shutil.rmtree(self.instances_dir)
|
||||
for instance in self.instances.values():
|
||||
instance.create_dir(destroy_dir=destroy_dirs)
|
||||
|
||||
for instance in self.instances.values():
|
||||
instance.create_dir(destroy_dir=destroy_dirs)
|
||||
self.docker_client = docker.from_env(version=self.docker_api_version)
|
||||
|
||||
self.docker_client = docker.from_env(version=self.docker_api_version)
|
||||
common_opts = ['up', '-d', '--force-recreate']
|
||||
|
||||
common_opts = ['up', '-d', '--force-recreate']
|
||||
if self.with_zookeeper and self.base_zookeeper_cmd:
|
||||
subprocess_check_call(self.base_zookeeper_cmd + common_opts)
|
||||
for command in self.pre_zookeeper_commands:
|
||||
self.run_kazoo_commands_with_retries(command, repeats=5)
|
||||
self.wait_zookeeper_to_start(120)
|
||||
|
||||
if self.with_zookeeper and self.base_zookeeper_cmd:
|
||||
subprocess_check_call(self.base_zookeeper_cmd + common_opts)
|
||||
for command in self.pre_zookeeper_commands:
|
||||
self.run_kazoo_commands_with_retries(command, repeats=5)
|
||||
self.wait_zookeeper_to_start(120)
|
||||
if self.with_mysql and self.base_mysql_cmd:
|
||||
subprocess_check_call(self.base_mysql_cmd + common_opts)
|
||||
self.wait_mysql_to_start(120)
|
||||
|
||||
if self.with_mysql and self.base_mysql_cmd:
|
||||
subprocess_check_call(self.base_mysql_cmd + common_opts)
|
||||
self.wait_mysql_to_start(120)
|
||||
if self.with_postgres and self.base_postgres_cmd:
|
||||
subprocess_check_call(self.base_postgres_cmd + common_opts)
|
||||
self.wait_postgres_to_start(120)
|
||||
|
||||
if self.with_postgres and self.base_postgres_cmd:
|
||||
subprocess_check_call(self.base_postgres_cmd + common_opts)
|
||||
self.wait_postgres_to_start(120)
|
||||
if self.with_kafka and self.base_kafka_cmd:
|
||||
subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes'])
|
||||
self.kafka_docker_id = self.get_instance_docker_id('kafka1')
|
||||
self.wait_schema_registry_to_start(120)
|
||||
|
||||
if self.with_kafka and self.base_kafka_cmd:
|
||||
subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes'])
|
||||
self.kafka_docker_id = self.get_instance_docker_id('kafka1')
|
||||
self.wait_schema_registry_to_start(120)
|
||||
if self.with_hdfs and self.base_hdfs_cmd:
|
||||
subprocess_check_call(self.base_hdfs_cmd + common_opts)
|
||||
self.wait_hdfs_to_start(120)
|
||||
|
||||
if self.with_hdfs and self.base_hdfs_cmd:
|
||||
subprocess_check_call(self.base_hdfs_cmd + common_opts)
|
||||
self.wait_hdfs_to_start(120)
|
||||
if self.with_mongo and self.base_mongo_cmd:
|
||||
subprocess_check_call(self.base_mongo_cmd + common_opts)
|
||||
self.wait_mongo_to_start(30)
|
||||
|
||||
if self.with_mongo and self.base_mongo_cmd:
|
||||
subprocess_check_call(self.base_mongo_cmd + common_opts)
|
||||
self.wait_mongo_to_start(30)
|
||||
if self.with_redis and self.base_redis_cmd:
|
||||
subprocess_check_call(self.base_redis_cmd + ['up', '-d', '--force-recreate'])
|
||||
time.sleep(10)
|
||||
|
||||
if self.with_redis and self.base_redis_cmd:
|
||||
subprocess_check_call(self.base_redis_cmd + ['up', '-d', '--force-recreate'])
|
||||
time.sleep(10)
|
||||
if self.with_minio and self.base_minio_cmd:
|
||||
minio_start_cmd = self.base_minio_cmd + common_opts
|
||||
logging.info("Trying to create Minio instance by command %s", ' '.join(map(str, minio_start_cmd)))
|
||||
subprocess_check_call(minio_start_cmd)
|
||||
logging.info("Trying to connect to Minio...")
|
||||
self.wait_minio_to_start()
|
||||
|
||||
if self.with_minio and self.base_minio_cmd:
|
||||
minio_start_cmd = self.base_minio_cmd + common_opts
|
||||
logging.info("Trying to create Minio instance by command %s", ' '.join(map(str, minio_start_cmd)))
|
||||
subprocess_check_call(minio_start_cmd)
|
||||
logging.info("Trying to connect to Minio...")
|
||||
self.wait_minio_to_start()
|
||||
clickhouse_start_cmd = self.base_cmd + ['up', '-d', '--no-recreate']
|
||||
logging.info("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd)))
|
||||
subprocess_check_call(clickhouse_start_cmd)
|
||||
logging.info("ClickHouse instance created")
|
||||
|
||||
clickhouse_start_cmd = self.base_cmd + ['up', '-d', '--no-recreate']
|
||||
logging.info("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd)))
|
||||
subprocess_check_call(clickhouse_start_cmd)
|
||||
logging.info("ClickHouse instance created")
|
||||
start_deadline = time.time() + 20.0 # seconds
|
||||
for instance in self.instances.itervalues():
|
||||
instance.docker_client = self.docker_client
|
||||
instance.ip_address = self.get_instance_ip(instance.name)
|
||||
|
||||
start_deadline = time.time() + 20.0 # seconds
|
||||
for instance in self.instances.itervalues():
|
||||
instance.docker_client = self.docker_client
|
||||
instance.ip_address = self.get_instance_ip(instance.name)
|
||||
logging.info("Waiting for ClickHouse start...")
|
||||
instance.wait_for_start(start_deadline)
|
||||
logging.info("ClickHouse started")
|
||||
|
||||
logging.info("Waiting for ClickHouse start...")
|
||||
instance.wait_for_start(start_deadline)
|
||||
logging.info("ClickHouse started")
|
||||
instance.client = Client(instance.ip_address, command=self.client_bin_path)
|
||||
|
||||
instance.client = Client(instance.ip_address, command=self.client_bin_path)
|
||||
|
||||
self.is_up = True
|
||||
self.is_up = True
|
||||
|
||||
except BaseException, e:
|
||||
print "Failed to start cluster: "
|
||||
print str(e)
|
||||
raise
|
||||
|
||||
def shutdown(self, kill=True):
|
||||
sanitizer_assert_instance = None
|
||||
@ -686,6 +691,9 @@ class ClickHouseInstance:
|
||||
self.exec_in_container(["bash", "-c", "pkill {} clickhouse".format("-9" if kill else "")], user='root')
|
||||
time.sleep(stop_start_wait_sec)
|
||||
self.exec_in_container(["bash", "-c", "{} --daemon".format(CLICKHOUSE_START_COMMAND)], user=str(os.getuid()))
|
||||
# wait start
|
||||
from helpers.test_tools import assert_eq_with_retry
|
||||
assert_eq_with_retry(self, "select 1", "1", retry_count=int(stop_start_wait_sec / 0.5), sleep_time=0.5)
|
||||
|
||||
def exec_in_container(self, cmd, detach=False, **kwargs):
|
||||
container = self.get_docker_handle()
|
||||
|
@ -0,0 +1,4 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<access_control_path>/var/lib/clickhouse/access</access_control_path>
|
||||
</yandex>
|
92
dbms/tests/integration/test_disk_access_storage/test.py
Normal file
92
dbms/tests/integration/test_disk_access_storage/test.py
Normal file
@ -0,0 +1,92 @@
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance('instance', config_dir='configs', main_configs=['configs/access_control_path.xml'], stay_alive=True)
|
||||
|
||||
|
||||
@pytest.fixture(scope="module", autouse=True)
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def create_entities():
|
||||
instance.query("CREATE USER u1")
|
||||
instance.query("CREATE ROLE rx")
|
||||
instance.query("CREATE USER u2 IDENTIFIED BY 'qwerty' HOST LOCAL DEFAULT ROLE rx")
|
||||
instance.query("CREATE ROW POLICY p ON mydb.mytable FOR SELECT USING a<1000 TO u1, u2")
|
||||
instance.query("CREATE QUOTA q FOR INTERVAL 1 HOUR SET MAX QUERIES = 100 TO ALL EXCEPT rx")
|
||||
|
||||
|
||||
@pytest.fixture(autouse=True)
|
||||
def drop_entities():
|
||||
instance.query("DROP USER IF EXISTS u1, u2")
|
||||
instance.query("DROP ROLE IF EXISTS rx, ry")
|
||||
instance.query("DROP ROW POLICY IF EXISTS p ON mydb.mytable")
|
||||
instance.query("DROP QUOTA IF EXISTS q")
|
||||
|
||||
|
||||
def test_create():
|
||||
create_entities()
|
||||
|
||||
def check():
|
||||
assert instance.query("SHOW CREATE USER u1") == "CREATE USER u1\n"
|
||||
assert instance.query("SHOW CREATE USER u2") == "CREATE USER u2 HOST LOCAL DEFAULT ROLE rx\n"
|
||||
assert instance.query("SHOW CREATE ROW POLICY p ON mydb.mytable") == "CREATE POLICY p ON mydb.mytable FOR SELECT USING a < 1000 TO u1, u2\n"
|
||||
assert instance.query("SHOW CREATE QUOTA q") == "CREATE QUOTA q KEYED BY \\'none\\' FOR INTERVAL 1 HOUR MAX QUERIES = 100 TO ALL EXCEPT rx\n"
|
||||
assert instance.query("SHOW GRANTS FOR u1") == ""
|
||||
assert instance.query("SHOW GRANTS FOR u2") == "GRANT rx TO u2\n"
|
||||
assert instance.query("SHOW GRANTS FOR rx") == ""
|
||||
|
||||
check()
|
||||
instance.restart_clickhouse() # Check persistency
|
||||
check()
|
||||
|
||||
|
||||
def test_alter():
|
||||
create_entities()
|
||||
instance.restart_clickhouse()
|
||||
|
||||
instance.query("CREATE ROLE ry")
|
||||
instance.query("GRANT ry TO u2")
|
||||
instance.query("ALTER USER u2 DEFAULT ROLE ry")
|
||||
instance.query("GRANT rx TO ry WITH ADMIN OPTION")
|
||||
instance.query("GRANT SELECT ON mydb.mytable TO u1")
|
||||
instance.query("GRANT SELECT ON mydb.* TO rx WITH GRANT OPTION")
|
||||
|
||||
def check():
|
||||
assert instance.query("SHOW CREATE USER u1") == "CREATE USER u1\n"
|
||||
assert instance.query("SHOW CREATE USER u2") == "CREATE USER u2 HOST LOCAL DEFAULT ROLE ry\n"
|
||||
assert instance.query("SHOW GRANTS FOR u1") == "GRANT SELECT ON mydb.mytable TO u1\n"
|
||||
assert instance.query("SHOW GRANTS FOR u2") == "GRANT rx, ry TO u2\n"
|
||||
assert instance.query("SHOW GRANTS FOR rx") == "GRANT SELECT ON mydb.* TO rx WITH GRANT OPTION\n"
|
||||
assert instance.query("SHOW GRANTS FOR ry") == "GRANT rx TO ry WITH ADMIN OPTION\n"
|
||||
|
||||
check()
|
||||
instance.restart_clickhouse() # Check persistency
|
||||
check()
|
||||
|
||||
|
||||
def test_drop():
|
||||
create_entities()
|
||||
instance.restart_clickhouse()
|
||||
|
||||
instance.query("DROP USER u2")
|
||||
instance.query("DROP ROLE rx")
|
||||
instance.query("DROP ROW POLICY p ON mydb.mytable")
|
||||
instance.query("DROP QUOTA q")
|
||||
|
||||
def check():
|
||||
assert instance.query("SHOW CREATE USER u1") == "CREATE USER u1\n"
|
||||
assert "User `u2` not found" in instance.query_and_get_error("SHOW CREATE USER u2")
|
||||
assert "Row policy `p ON mydb.mytable` not found" in instance.query_and_get_error("SHOW CREATE ROW POLICY p ON mydb.mytable")
|
||||
assert "Quota `q` not found" in instance.query_and_get_error("SHOW CREATE QUOTA q")
|
||||
|
||||
check()
|
||||
instance.restart_clickhouse() # Check persistency
|
||||
check()
|
@ -1164,7 +1164,6 @@ def test_kill_while_insert(start_cluster):
|
||||
except:
|
||||
""""""
|
||||
|
||||
time.sleep(0.5)
|
||||
assert node1.query("SELECT count() FROM {name}".format(name=name)).splitlines() == ["10"]
|
||||
|
||||
finally:
|
||||
|
@ -47,7 +47,6 @@ def test_startup_without_zookeeper(start_cluster):
|
||||
node1.query("INSERT INTO test_table VALUES ('2018-10-01', 1), ('2018-10-02', 2), ('2018-10-03', 3)")
|
||||
|
||||
node1.restart_clickhouse()
|
||||
time.sleep(5)
|
||||
|
||||
assert node1.query("SELECT COUNT(*) from test_table") == "3\n"
|
||||
assert node1.query("SELECT is_readonly from system.replicas where table='test_table'") == "1\n"
|
||||
|
@ -44,5 +44,3 @@ def test_live_view_dependency(started_cluster):
|
||||
instance.query("CREATE TABLE b_load_second.mt (a Int32) Engine=MergeTree order by tuple()")
|
||||
instance.query("CREATE LIVE VIEW a_load_first.lv AS SELECT sum(a) FROM b_load_second.mt", settings={'allow_experimental_live_view': 1})
|
||||
instance.restart_clickhouse()
|
||||
time.sleep(5)
|
||||
instance.query("SELECT 1")
|
||||
|
Loading…
Reference in New Issue
Block a user