Merge pull request #30998 from vitlibar/rename-access-control-manager

Rename AccessControlManager
This commit is contained in:
Vitaly Baranov 2021-11-03 12:20:17 +03:00 committed by GitHub
commit 1f217aeb5a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
70 changed files with 1117 additions and 1067 deletions

View File

@ -53,7 +53,7 @@
#include <Interpreters/loadMetadata.h>
#include <Interpreters/UserDefinedSQLObjectsLoader.h>
#include <Interpreters/JIT/CompiledExpressionCache.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/System/attachSystemTables.h>
#include <Storages/System/attachInformationSchemaTables.h>
@ -883,7 +883,7 @@ if (ThreadFuzzer::instance().isEffective())
},
/* already_loaded = */ false); /// Reload it right now (initial loading)
auto & access_control = global_context->getAccessControlManager();
auto & access_control = global_context->getAccessControl();
if (config().has("custom_settings_prefixes"))
access_control.setCustomSettingsPrefixes(config().getString("custom_settings_prefixes"));

View File

@ -1,4 +1,4 @@
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/MultipleAccessStorage.h>
#include <Access/MemoryAccessStorage.h>
#include <Access/ReplicatedAccessStorage.h>
@ -55,10 +55,10 @@ namespace
}
class AccessControlManager::ContextAccessCache
class AccessControl::ContextAccessCache
{
public:
explicit ContextAccessCache(const AccessControlManager & manager_) : manager(manager_) {}
explicit ContextAccessCache(const AccessControl & access_control_) : access_control(access_control_) {}
std::shared_ptr<const ContextAccess> getContextAccess(const ContextAccessParams & params)
{
@ -71,19 +71,19 @@ public:
/// No user, probably the user has been dropped while it was in the cache.
cache.remove(params);
}
auto res = std::shared_ptr<ContextAccess>(new ContextAccess(manager, params));
auto res = std::shared_ptr<ContextAccess>(new ContextAccess(access_control, params));
cache.add(params, res);
return res;
}
private:
const AccessControlManager & manager;
const AccessControl & access_control;
Poco::ExpireCache<ContextAccess::Params, std::shared_ptr<const ContextAccess>> cache;
std::mutex mutex;
};
class AccessControlManager::CustomSettingsPrefixes
class AccessControl::CustomSettingsPrefixes
{
public:
void registerPrefixes(const Strings & prefixes_)
@ -130,7 +130,7 @@ private:
};
AccessControlManager::AccessControlManager()
AccessControl::AccessControl()
: MultipleAccessStorage("user directories"),
context_access_cache(std::make_unique<ContextAccessCache>(*this)),
role_cache(std::make_unique<RoleCache>(*this)),
@ -143,9 +143,9 @@ AccessControlManager::AccessControlManager()
}
AccessControlManager::~AccessControlManager() = default;
AccessControl::~AccessControl() = default;
void AccessControlManager::setUsersConfig(const Poco::Util::AbstractConfiguration & users_config_)
void AccessControl::setUsersConfig(const Poco::Util::AbstractConfiguration & users_config_)
{
auto storages = getStoragesPtr();
for (const auto & storage : *storages)
@ -159,12 +159,12 @@ void AccessControlManager::setUsersConfig(const Poco::Util::AbstractConfiguratio
addUsersConfigStorage(users_config_);
}
void AccessControlManager::addUsersConfigStorage(const Poco::Util::AbstractConfiguration & users_config_)
void AccessControl::addUsersConfigStorage(const Poco::Util::AbstractConfiguration & users_config_)
{
addUsersConfigStorage(UsersConfigAccessStorage::STORAGE_TYPE, users_config_);
}
void AccessControlManager::addUsersConfigStorage(const String & storage_name_, const Poco::Util::AbstractConfiguration & users_config_)
void AccessControl::addUsersConfigStorage(const String & storage_name_, const Poco::Util::AbstractConfiguration & users_config_)
{
auto check_setting_name_function = [this](const std::string_view & setting_name) { checkSettingNameIsAllowed(setting_name); };
auto new_storage = std::make_shared<UsersConfigAccessStorage>(storage_name_, check_setting_name_function);
@ -173,7 +173,7 @@ void AccessControlManager::addUsersConfigStorage(const String & storage_name_, c
LOG_DEBUG(getLogger(), "Added {} access storage '{}', path: {}", String(new_storage->getStorageType()), new_storage->getStorageName(), new_storage->getPath());
}
void AccessControlManager::addUsersConfigStorage(
void AccessControl::addUsersConfigStorage(
const String & users_config_path_,
const String & include_from_path_,
const String & preprocessed_dir_,
@ -183,7 +183,7 @@ void AccessControlManager::addUsersConfigStorage(
UsersConfigAccessStorage::STORAGE_TYPE, users_config_path_, include_from_path_, preprocessed_dir_, get_zookeeper_function_);
}
void AccessControlManager::addUsersConfigStorage(
void AccessControl::addUsersConfigStorage(
const String & storage_name_,
const String & users_config_path_,
const String & include_from_path_,
@ -206,7 +206,7 @@ void AccessControlManager::addUsersConfigStorage(
LOG_DEBUG(getLogger(), "Added {} access storage '{}', path: {}", String(new_storage->getStorageType()), new_storage->getStorageName(), new_storage->getPath());
}
void AccessControlManager::reloadUsersConfigs()
void AccessControl::reloadUsersConfigs()
{
auto storages = getStoragesPtr();
for (const auto & storage : *storages)
@ -216,7 +216,7 @@ void AccessControlManager::reloadUsersConfigs()
}
}
void AccessControlManager::startPeriodicReloadingUsersConfigs()
void AccessControl::startPeriodicReloadingUsersConfigs()
{
auto storages = getStoragesPtr();
for (const auto & storage : *storages)
@ -226,7 +226,7 @@ void AccessControlManager::startPeriodicReloadingUsersConfigs()
}
}
void AccessControlManager::addReplicatedStorage(
void AccessControl::addReplicatedStorage(
const String & storage_name_,
const String & zookeeper_path_,
const zkutil::GetZooKeeper & get_zookeeper_function_)
@ -243,12 +243,12 @@ void AccessControlManager::addReplicatedStorage(
new_storage->startup();
}
void AccessControlManager::addDiskStorage(const String & directory_, bool readonly_)
void AccessControl::addDiskStorage(const String & directory_, bool readonly_)
{
addDiskStorage(DiskAccessStorage::STORAGE_TYPE, directory_, readonly_);
}
void AccessControlManager::addDiskStorage(const String & storage_name_, const String & directory_, bool readonly_)
void AccessControl::addDiskStorage(const String & storage_name_, const String & directory_, bool readonly_)
{
auto storages = getStoragesPtr();
for (const auto & storage : *storages)
@ -269,7 +269,7 @@ void AccessControlManager::addDiskStorage(const String & storage_name_, const St
}
void AccessControlManager::addMemoryStorage(const String & storage_name_)
void AccessControl::addMemoryStorage(const String & storage_name_)
{
auto storages = getStoragesPtr();
for (const auto & storage : *storages)
@ -283,7 +283,7 @@ void AccessControlManager::addMemoryStorage(const String & storage_name_)
}
void AccessControlManager::addLDAPStorage(const String & storage_name_, const Poco::Util::AbstractConfiguration & config_, const String & prefix_)
void AccessControl::addLDAPStorage(const String & storage_name_, const Poco::Util::AbstractConfiguration & config_, const String & prefix_)
{
auto new_storage = std::make_shared<LDAPAccessStorage>(storage_name_, this, config_, prefix_);
addStorage(new_storage);
@ -291,7 +291,7 @@ void AccessControlManager::addLDAPStorage(const String & storage_name_, const Po
}
void AccessControlManager::addStoragesFromUserDirectoriesConfig(
void AccessControl::addStoragesFromUserDirectoriesConfig(
const Poco::Util::AbstractConfiguration & config,
const String & key,
const String & config_dir,
@ -350,7 +350,7 @@ void AccessControlManager::addStoragesFromUserDirectoriesConfig(
}
void AccessControlManager::addStoragesFromMainConfig(
void AccessControl::addStoragesFromMainConfig(
const Poco::Util::AbstractConfiguration & config,
const String & config_path,
const zkutil::GetZooKeeper & get_zookeeper_function)
@ -388,47 +388,47 @@ void AccessControlManager::addStoragesFromMainConfig(
}
UUID AccessControlManager::login(const Credentials & credentials, const Poco::Net::IPAddress & address) const
UUID AccessControl::login(const Credentials & credentials, const Poco::Net::IPAddress & address) const
{
return MultipleAccessStorage::login(credentials, address, *external_authenticators);
}
void AccessControlManager::setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config)
void AccessControl::setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config)
{
external_authenticators->setConfiguration(config, getLogger());
}
void AccessControlManager::setDefaultProfileName(const String & default_profile_name)
void AccessControl::setDefaultProfileName(const String & default_profile_name)
{
settings_profiles_cache->setDefaultProfileName(default_profile_name);
}
void AccessControlManager::setCustomSettingsPrefixes(const Strings & prefixes)
void AccessControl::setCustomSettingsPrefixes(const Strings & prefixes)
{
custom_settings_prefixes->registerPrefixes(prefixes);
}
void AccessControlManager::setCustomSettingsPrefixes(const String & comma_separated_prefixes)
void AccessControl::setCustomSettingsPrefixes(const String & comma_separated_prefixes)
{
Strings prefixes;
splitInto<','>(prefixes, comma_separated_prefixes);
setCustomSettingsPrefixes(prefixes);
}
bool AccessControlManager::isSettingNameAllowed(const std::string_view & setting_name) const
bool AccessControl::isSettingNameAllowed(const std::string_view & setting_name) const
{
return custom_settings_prefixes->isSettingNameAllowed(setting_name);
}
void AccessControlManager::checkSettingNameIsAllowed(const std::string_view & setting_name) const
void AccessControl::checkSettingNameIsAllowed(const std::string_view & setting_name) const
{
custom_settings_prefixes->checkSettingNameIsAllowed(setting_name);
}
std::shared_ptr<const ContextAccess> AccessControlManager::getContextAccess(
std::shared_ptr<const ContextAccess> AccessControl::getContextAccess(
const UUID & user_id,
const std::vector<UUID> & current_roles,
bool use_default_roles,
@ -464,13 +464,13 @@ std::shared_ptr<const ContextAccess> AccessControlManager::getContextAccess(
}
std::shared_ptr<const ContextAccess> AccessControlManager::getContextAccess(const ContextAccessParams & params) const
std::shared_ptr<const ContextAccess> AccessControl::getContextAccess(const ContextAccessParams & params) const
{
return context_access_cache->getContextAccess(params);
}
std::shared_ptr<const EnabledRoles> AccessControlManager::getEnabledRoles(
std::shared_ptr<const EnabledRoles> AccessControl::getEnabledRoles(
const std::vector<UUID> & current_roles,
const std::vector<UUID> & current_roles_with_admin_option) const
{
@ -478,13 +478,13 @@ std::shared_ptr<const EnabledRoles> AccessControlManager::getEnabledRoles(
}
std::shared_ptr<const EnabledRowPolicies> AccessControlManager::getEnabledRowPolicies(const UUID & user_id, const boost::container::flat_set<UUID> & enabled_roles) const
std::shared_ptr<const EnabledRowPolicies> AccessControl::getEnabledRowPolicies(const UUID & user_id, const boost::container::flat_set<UUID> & enabled_roles) const
{
return row_policy_cache->getEnabledRowPolicies(user_id, enabled_roles);
}
std::shared_ptr<const EnabledQuota> AccessControlManager::getEnabledQuota(
std::shared_ptr<const EnabledQuota> AccessControl::getEnabledQuota(
const UUID & user_id,
const String & user_name,
const boost::container::flat_set<UUID> & enabled_roles,
@ -496,13 +496,13 @@ std::shared_ptr<const EnabledQuota> AccessControlManager::getEnabledQuota(
}
std::vector<QuotaUsage> AccessControlManager::getAllQuotasUsage() const
std::vector<QuotaUsage> AccessControl::getAllQuotasUsage() const
{
return quota_cache->getAllQuotasUsage();
}
std::shared_ptr<const EnabledSettings> AccessControlManager::getEnabledSettings(
std::shared_ptr<const EnabledSettings> AccessControl::getEnabledSettings(
const UUID & user_id,
const SettingsProfileElements & settings_from_user,
const boost::container::flat_set<UUID> & enabled_roles,
@ -511,13 +511,13 @@ std::shared_ptr<const EnabledSettings> AccessControlManager::getEnabledSettings(
return settings_profiles_cache->getEnabledSettings(user_id, settings_from_user, enabled_roles, settings_from_enabled_roles);
}
std::shared_ptr<const SettingsProfilesInfo> AccessControlManager::getSettingsProfileInfo(const UUID & profile_id)
std::shared_ptr<const SettingsProfilesInfo> AccessControl::getSettingsProfileInfo(const UUID & profile_id)
{
return settings_profiles_cache->getSettingsProfileInfo(profile_id);
}
const ExternalAuthenticators & AccessControlManager::getExternalAuthenticators() const
const ExternalAuthenticators & AccessControl::getExternalAuthenticators() const
{
return *external_authenticators;
}

View File

@ -42,11 +42,11 @@ struct Settings;
/// Manages access control entities.
class AccessControlManager : public MultipleAccessStorage
class AccessControl : public MultipleAccessStorage
{
public:
AccessControlManager();
~AccessControlManager() override;
AccessControl();
~AccessControl() override;
/// Parses access entities from a configuration loaded from users.xml.
/// This function add UsersConfigAccessStorage if it wasn't added before.

View File

@ -2,6 +2,7 @@
#include <base/types.h>
#include <Access/Common/AccessRightsElement.h>
#include <functional>
#include <memory>
#include <vector>

View File

@ -23,7 +23,7 @@ namespace
bool checkPasswordPlainText(const String & password, const Digest & password_plaintext)
{
return (Util::encodePlainText(password) == password_plaintext);
return (Util::stringToDigest(password) == password_plaintext);
}
bool checkPasswordDoubleSHA1(const std::string_view & password, const Digest & password_double_sha1)
@ -87,7 +87,7 @@ bool Authentication::areCredentialsValid(const Credentials & credentials, const
case AuthenticationType::KERBEROS:
return external_authenticators.checkKerberosCredentials(auth_data.getKerberosRealm(), *gss_acceptor_context);
case AuthenticationType::MAX_TYPE:
case AuthenticationType::MAX:
break;
}
}
@ -110,7 +110,7 @@ bool Authentication::areCredentialsValid(const Credentials & credentials, const
case AuthenticationType::KERBEROS:
throw Authentication::Require<BasicCredentials>("ClickHouse Basic Authentication");
case AuthenticationType::MAX_TYPE:
case AuthenticationType::MAX:
break;
}
}
@ -137,7 +137,7 @@ bool Authentication::areCredentialsValid(const Credentials & credentials, const
case AuthenticationType::KERBEROS:
throw Authentication::Require<GSSAcceptorContext>(auth_data.getKerberosRealm());
case AuthenticationType::MAX_TYPE:
case AuthenticationType::MAX:
break;
}
}

View File

@ -0,0 +1,378 @@
#include <Access/Common/AccessFlags.h>
#include <Access/Common/AccessType.h>
#include <Common/Exception.h>
#include <base/types.h>
#include <boost/algorithm/string/case_conv.hpp>
#include <boost/algorithm/string/replace.hpp>
#include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/trim.hpp>
#include <unordered_map>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_ACCESS_TYPE;
extern const int LOGICAL_ERROR;
}
namespace
{
using Flags = std::bitset<AccessFlags::SIZE>;
class Helper
{
public:
static const Helper & instance()
{
static const Helper res;
return res;
}
Flags accessTypeToFlags(AccessType type) const
{
return access_type_to_flags_mapping[static_cast<size_t>(type)];
}
Flags keywordToFlags(const std::string_view & keyword) const
{
auto it = keyword_to_flags_map.find(keyword);
if (it == keyword_to_flags_map.end())
{
String uppercased_keyword{keyword};
boost::to_upper(uppercased_keyword);
it = keyword_to_flags_map.find(uppercased_keyword);
if (it == keyword_to_flags_map.end())
throw Exception("Unknown access type: " + String(keyword), ErrorCodes::UNKNOWN_ACCESS_TYPE);
}
return it->second;
}
Flags keywordsToFlags(const std::vector<std::string_view> & keywords) const
{
Flags res;
for (const auto & keyword : keywords)
res |= keywordToFlags(keyword);
return res;
}
Flags keywordsToFlags(const Strings & keywords) const
{
Flags res;
for (const auto & keyword : keywords)
res |= keywordToFlags(keyword);
return res;
}
std::vector<AccessType> flagsToAccessTypes(const Flags & flags_) const
{
std::vector<AccessType> access_types;
flagsToAccessTypesRec(flags_, access_types, *all_node);
return access_types;
}
std::vector<std::string_view> flagsToKeywords(const Flags & flags_) const
{
std::vector<std::string_view> keywords;
flagsToKeywordsRec(flags_, keywords, *all_node);
return keywords;
}
String flagsToString(const Flags & flags_) const
{
auto keywords = flagsToKeywords(flags_);
if (keywords.empty())
return "USAGE";
String str;
for (const auto & keyword : keywords)
{
if (!str.empty())
str += ", ";
str += keyword;
}
return str;
}
const Flags & getAllFlags() const { return all_flags; }
const Flags & getGlobalFlags() const { return all_flags_for_target[GLOBAL]; }
const Flags & getDatabaseFlags() const { return all_flags_for_target[DATABASE]; }
const Flags & getTableFlags() const { return all_flags_for_target[TABLE]; }
const Flags & getColumnFlags() const { return all_flags_for_target[COLUMN]; }
const Flags & getDictionaryFlags() const { return all_flags_for_target[DICTIONARY]; }
const Flags & getAllFlagsGrantableOnGlobalLevel() const { return getAllFlags(); }
const Flags & getAllFlagsGrantableOnDatabaseLevel() const { return all_flags_grantable_on_database_level; }
const Flags & getAllFlagsGrantableOnTableLevel() const { return all_flags_grantable_on_table_level; }
const Flags & getAllFlagsGrantableOnColumnLevel() const { return getColumnFlags(); }
private:
enum NodeType
{
UNKNOWN = -2,
GROUP = -1,
GLOBAL,
DATABASE,
TABLE,
VIEW = TABLE,
COLUMN,
DICTIONARY,
};
struct Node;
using NodePtr = std::unique_ptr<Node>;
struct Node
{
const String keyword;
NodeType node_type = UNKNOWN;
AccessType access_type = AccessType::NONE;
Strings aliases;
Flags flags;
std::vector<NodePtr> children;
explicit Node(String keyword_) : keyword(std::move(keyword_)) {}
Node(String keyword_, NodeType node_type_) : keyword(std::move(keyword_)), node_type(node_type_) {}
void setFlag(size_t flag) { flags.set(flag); }
void addChild(NodePtr child)
{
flags |= child->flags;
children.push_back(std::move(child));
}
};
static String replaceUnderscoreWithSpace(const std::string_view & str)
{
String res{str};
boost::replace_all(res, "_", " ");
return res;
}
static Strings splitAliases(const std::string_view & str)
{
Strings aliases;
boost::split(aliases, str, boost::is_any_of(","));
for (auto & alias : aliases)
boost::trim(alias);
return aliases;
}
static void makeNode(
AccessType access_type,
const std::string_view & name,
const std::string_view & aliases,
NodeType node_type,
const std::string_view & parent_group_name,
std::unordered_map<std::string_view, Node *> & nodes,
std::unordered_map<std::string_view, NodePtr> & owned_nodes,
size_t & next_flag)
{
NodePtr node;
auto keyword = replaceUnderscoreWithSpace(name);
auto it = owned_nodes.find(keyword);
if (it != owned_nodes.end())
{
node = std::move(it->second);
owned_nodes.erase(it);
}
else
{
if (nodes.count(keyword))
throw Exception(keyword + " declared twice", ErrorCodes::LOGICAL_ERROR);
node = std::make_unique<Node>(keyword, node_type);
nodes[node->keyword] = node.get();
}
node->access_type = access_type;
node->node_type = node_type;
node->aliases = splitAliases(aliases);
if (node_type != GROUP)
node->setFlag(next_flag++);
bool has_parent_group = (parent_group_name != std::string_view{"NONE"});
if (!has_parent_group)
{
std::string_view keyword_as_string_view = node->keyword;
owned_nodes[keyword_as_string_view] = std::move(node);
return;
}
auto parent_keyword = replaceUnderscoreWithSpace(parent_group_name);
auto it_parent = nodes.find(parent_keyword);
if (it_parent == nodes.end())
{
auto parent_node = std::make_unique<Node>(parent_keyword);
it_parent = nodes.emplace(parent_node->keyword, parent_node.get()).first;
assert(!owned_nodes.count(parent_node->keyword));
std::string_view parent_keyword_as_string_view = parent_node->keyword;
owned_nodes[parent_keyword_as_string_view] = std::move(parent_node);
}
it_parent->second->addChild(std::move(node));
}
void makeNodes()
{
std::unordered_map<std::string_view, NodePtr> owned_nodes;
std::unordered_map<std::string_view, Node *> nodes;
size_t next_flag = 0;
# define MAKE_ACCESS_FLAGS_NODE(name, aliases, node_type, parent_group_name) \
makeNode(AccessType::name, #name, aliases, node_type, #parent_group_name, nodes, owned_nodes, next_flag);
APPLY_FOR_ACCESS_TYPES(MAKE_ACCESS_FLAGS_NODE)
# undef MAKE_ACCESS_FLAGS_NODE
if (!owned_nodes.count("NONE"))
throw Exception("'NONE' not declared", ErrorCodes::LOGICAL_ERROR);
if (!owned_nodes.count("ALL"))
throw Exception("'ALL' not declared", ErrorCodes::LOGICAL_ERROR);
all_node = std::move(owned_nodes["ALL"]);
none_node = std::move(owned_nodes["NONE"]);
owned_nodes.erase("ALL");
owned_nodes.erase("NONE");
if (!owned_nodes.empty())
{
const auto & unused_node = *(owned_nodes.begin()->second);
if (unused_node.node_type == UNKNOWN)
throw Exception("Parent group '" + unused_node.keyword + "' not found", ErrorCodes::LOGICAL_ERROR);
else
throw Exception("Access type '" + unused_node.keyword + "' should have parent group", ErrorCodes::LOGICAL_ERROR);
}
}
void makeKeywordToFlagsMap(Node * start_node = nullptr)
{
if (!start_node)
{
makeKeywordToFlagsMap(none_node.get());
start_node = all_node.get();
}
start_node->aliases.emplace_back(start_node->keyword);
for (auto & alias : start_node->aliases)
{
boost::to_upper(alias);
keyword_to_flags_map[alias] = start_node->flags;
}
for (auto & child : start_node->children)
makeKeywordToFlagsMap(child.get());
}
void makeAccessTypeToFlagsMapping(Node * start_node = nullptr)
{
if (!start_node)
{
makeAccessTypeToFlagsMapping(none_node.get());
start_node = all_node.get();
}
size_t index = static_cast<size_t>(start_node->access_type);
access_type_to_flags_mapping.resize(std::max(index + 1, access_type_to_flags_mapping.size()));
access_type_to_flags_mapping[index] = start_node->flags;
for (auto & child : start_node->children)
makeAccessTypeToFlagsMapping(child.get());
}
void collectAllFlags(const Node * start_node = nullptr)
{
if (!start_node)
{
start_node = all_node.get();
all_flags = start_node->flags;
}
if (start_node->node_type != GROUP)
{
assert(static_cast<size_t>(start_node->node_type) < std::size(all_flags_for_target));
all_flags_for_target[start_node->node_type] |= start_node->flags;
}
for (const auto & child : start_node->children)
collectAllFlags(child.get());
all_flags_grantable_on_table_level = all_flags_for_target[TABLE] | all_flags_for_target[DICTIONARY] | all_flags_for_target[COLUMN];
all_flags_grantable_on_database_level = all_flags_for_target[DATABASE] | all_flags_grantable_on_table_level;
}
Helper()
{
makeNodes();
makeKeywordToFlagsMap();
makeAccessTypeToFlagsMapping();
collectAllFlags();
}
static void flagsToAccessTypesRec(const Flags & flags_, std::vector<AccessType> & access_types, const Node & start_node)
{
Flags matching_flags = (flags_ & start_node.flags);
if (matching_flags.any())
{
if (matching_flags == start_node.flags)
{
access_types.push_back(start_node.access_type);
}
else
{
for (const auto & child : start_node.children)
flagsToAccessTypesRec(flags_, access_types, *child);
}
}
}
static void flagsToKeywordsRec(const Flags & flags_, std::vector<std::string_view> & keywords, const Node & start_node)
{
Flags matching_flags = (flags_ & start_node.flags);
if (matching_flags.any())
{
if (matching_flags == start_node.flags)
{
keywords.push_back(start_node.keyword);
}
else
{
for (const auto & child : start_node.children)
flagsToKeywordsRec(flags_, keywords, *child);
}
}
}
NodePtr all_node;
NodePtr none_node;
std::unordered_map<std::string_view, Flags> keyword_to_flags_map;
std::vector<Flags> access_type_to_flags_mapping;
Flags all_flags;
Flags all_flags_for_target[static_cast<size_t>(DICTIONARY) + 1];
Flags all_flags_grantable_on_database_level;
Flags all_flags_grantable_on_table_level;
};
}
AccessFlags::AccessFlags(AccessType type) : flags(Helper::instance().accessTypeToFlags(type)) {}
AccessFlags::AccessFlags(const std::string_view & keyword) : flags(Helper::instance().keywordToFlags(keyword)) {}
AccessFlags::AccessFlags(const std::vector<std::string_view> & keywords) : flags(Helper::instance().keywordsToFlags(keywords)) {}
AccessFlags::AccessFlags(const Strings & keywords) : flags(Helper::instance().keywordsToFlags(keywords)) {}
String AccessFlags::toString() const { return Helper::instance().flagsToString(flags); }
std::vector<AccessType> AccessFlags::toAccessTypes() const { return Helper::instance().flagsToAccessTypes(flags); }
std::vector<std::string_view> AccessFlags::toKeywords() const { return Helper::instance().flagsToKeywords(flags); }
AccessFlags AccessFlags::allFlags() { return Helper::instance().getAllFlags(); }
AccessFlags AccessFlags::allGlobalFlags() { return Helper::instance().getGlobalFlags(); }
AccessFlags AccessFlags::allDatabaseFlags() { return Helper::instance().getDatabaseFlags(); }
AccessFlags AccessFlags::allTableFlags() { return Helper::instance().getTableFlags(); }
AccessFlags AccessFlags::allColumnFlags() { return Helper::instance().getColumnFlags(); }
AccessFlags AccessFlags::allDictionaryFlags() { return Helper::instance().getDictionaryFlags(); }
AccessFlags AccessFlags::allFlagsGrantableOnGlobalLevel() { return Helper::instance().getAllFlagsGrantableOnGlobalLevel(); }
AccessFlags AccessFlags::allFlagsGrantableOnDatabaseLevel() { return Helper::instance().getAllFlagsGrantableOnDatabaseLevel(); }
AccessFlags AccessFlags::allFlagsGrantableOnTableLevel() { return Helper::instance().getAllFlagsGrantableOnTableLevel(); }
AccessFlags AccessFlags::allFlagsGrantableOnColumnLevel() { return Helper::instance().getAllFlagsGrantableOnColumnLevel(); }
AccessFlags operator |(AccessType left, AccessType right) { return AccessFlags(left) | right; }
AccessFlags operator &(AccessType left, AccessType right) { return AccessFlags(left) & right; }
AccessFlags operator -(AccessType left, AccessType right) { return AccessFlags(left) - right; }
AccessFlags operator ~(AccessType x) { return ~AccessFlags(x); }
}

View File

@ -1,21 +1,14 @@
#pragma once
#include <Access/Common/AccessType.h>
#include <base/types.h>
#include <base/range.h>
#include <Common/Exception.h>
#include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/trim.hpp>
#include <bitset>
#include <unordered_map>
#include <cstring>
#include <vector>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
using Strings = std::vector<String>;
/// Represents a combination of access types which can be granted globally, on databases, tables, columns, etc.
/// For example "SELECT, CREATE USER" is an access type.
@ -111,374 +104,17 @@ public:
/// The same as allColumnFlags().
static AccessFlags allFlagsGrantableOnColumnLevel();
static constexpr size_t SIZE = 128;
private:
static constexpr size_t NUM_FLAGS = 128;
using Flags = std::bitset<NUM_FLAGS>;
using Flags = std::bitset<SIZE>;
Flags flags;
AccessFlags(const Flags & flags_) : flags(flags_) {}
template <typename = void>
class Impl;
};
namespace ErrorCodes
{
extern const int UNKNOWN_ACCESS_TYPE;
}
template <typename>
class AccessFlags::Impl
{
public:
static const Impl & instance()
{
static const Impl res;
return res;
}
Flags accessTypeToFlags(AccessType type) const
{
return access_type_to_flags_mapping[static_cast<size_t>(type)];
}
Flags keywordToFlags(const std::string_view & keyword) const
{
auto it = keyword_to_flags_map.find(keyword);
if (it == keyword_to_flags_map.end())
{
String uppercased_keyword{keyword};
boost::to_upper(uppercased_keyword);
it = keyword_to_flags_map.find(uppercased_keyword);
if (it == keyword_to_flags_map.end())
throw Exception("Unknown access type: " + String(keyword), ErrorCodes::UNKNOWN_ACCESS_TYPE);
}
return it->second;
}
Flags keywordsToFlags(const std::vector<std::string_view> & keywords) const
{
Flags res;
for (const auto & keyword : keywords)
res |= keywordToFlags(keyword);
return res;
}
Flags keywordsToFlags(const Strings & keywords) const
{
Flags res;
for (const auto & keyword : keywords)
res |= keywordToFlags(keyword);
return res;
}
std::vector<AccessType> flagsToAccessTypes(const Flags & flags_) const
{
std::vector<AccessType> access_types;
flagsToAccessTypesRec(flags_, access_types, *all_node);
return access_types;
}
std::vector<std::string_view> flagsToKeywords(const Flags & flags_) const
{
std::vector<std::string_view> keywords;
flagsToKeywordsRec(flags_, keywords, *all_node);
return keywords;
}
String flagsToString(const Flags & flags_) const
{
auto keywords = flagsToKeywords(flags_);
if (keywords.empty())
return "USAGE";
String str;
for (const auto & keyword : keywords)
{
if (!str.empty())
str += ", ";
str += keyword;
}
return str;
}
const Flags & getAllFlags() const { return all_flags; }
const Flags & getGlobalFlags() const { return all_flags_for_target[GLOBAL]; }
const Flags & getDatabaseFlags() const { return all_flags_for_target[DATABASE]; }
const Flags & getTableFlags() const { return all_flags_for_target[TABLE]; }
const Flags & getColumnFlags() const { return all_flags_for_target[COLUMN]; }
const Flags & getDictionaryFlags() const { return all_flags_for_target[DICTIONARY]; }
const Flags & getAllFlagsGrantableOnGlobalLevel() const { return getAllFlags(); }
const Flags & getAllFlagsGrantableOnDatabaseLevel() const { return all_flags_grantable_on_database_level; }
const Flags & getAllFlagsGrantableOnTableLevel() const { return all_flags_grantable_on_table_level; }
const Flags & getAllFlagsGrantableOnColumnLevel() const { return getColumnFlags(); }
private:
enum NodeType
{
UNKNOWN = -2,
GROUP = -1,
GLOBAL,
DATABASE,
TABLE,
VIEW = TABLE,
COLUMN,
DICTIONARY,
};
struct Node;
using NodePtr = std::unique_ptr<Node>;
struct Node
{
const String keyword;
NodeType node_type;
AccessType access_type = AccessType::NONE;
Strings aliases;
Flags flags;
std::vector<NodePtr> children;
Node(String keyword_, NodeType node_type_ = UNKNOWN) : keyword(std::move(keyword_)), node_type(node_type_) {}
void setFlag(size_t flag) { flags.set(flag); }
void addChild(NodePtr child)
{
flags |= child->flags;
children.push_back(std::move(child));
}
};
static String replaceUnderscoreWithSpace(const std::string_view & str)
{
String res{str};
boost::replace_all(res, "_", " ");
return res;
}
static Strings splitAliases(const std::string_view & str)
{
Strings aliases;
boost::split(aliases, str, boost::is_any_of(","));
for (auto & alias : aliases)
boost::trim(alias);
return aliases;
}
static void makeNode(
AccessType access_type,
const std::string_view & name,
const std::string_view & aliases,
NodeType node_type,
const std::string_view & parent_group_name,
std::unordered_map<std::string_view, Node *> & nodes,
std::unordered_map<std::string_view, NodePtr> & owned_nodes,
size_t & next_flag)
{
NodePtr node;
auto keyword = replaceUnderscoreWithSpace(name);
auto it = owned_nodes.find(keyword);
if (it != owned_nodes.end())
{
node = std::move(it->second);
owned_nodes.erase(it);
}
else
{
if (nodes.count(keyword))
throw Exception(keyword + " declared twice", ErrorCodes::LOGICAL_ERROR);
node = std::make_unique<Node>(keyword, node_type);
nodes[node->keyword] = node.get();
}
node->access_type = access_type;
node->node_type = node_type;
node->aliases = splitAliases(aliases);
if (node_type != GROUP)
node->setFlag(next_flag++);
bool has_parent_group = (parent_group_name != std::string_view{"NONE"});
if (!has_parent_group)
{
std::string_view keyword_as_string_view = node->keyword;
owned_nodes[keyword_as_string_view] = std::move(node);
return;
}
auto parent_keyword = replaceUnderscoreWithSpace(parent_group_name);
auto it_parent = nodes.find(parent_keyword);
if (it_parent == nodes.end())
{
auto parent_node = std::make_unique<Node>(parent_keyword);
it_parent = nodes.emplace(parent_node->keyword, parent_node.get()).first;
assert(!owned_nodes.count(parent_node->keyword));
std::string_view parent_keyword_as_string_view = parent_node->keyword;
owned_nodes[parent_keyword_as_string_view] = std::move(parent_node);
}
it_parent->second->addChild(std::move(node));
}
void makeNodes()
{
std::unordered_map<std::string_view, NodePtr> owned_nodes;
std::unordered_map<std::string_view, Node *> nodes;
size_t next_flag = 0;
#define MAKE_ACCESS_FLAGS_NODE(name, aliases, node_type, parent_group_name) \
makeNode(AccessType::name, #name, aliases, node_type, #parent_group_name, nodes, owned_nodes, next_flag);
APPLY_FOR_ACCESS_TYPES(MAKE_ACCESS_FLAGS_NODE)
#undef MAKE_ACCESS_FLAGS_NODE
if (!owned_nodes.count("NONE"))
throw Exception("'NONE' not declared", ErrorCodes::LOGICAL_ERROR);
if (!owned_nodes.count("ALL"))
throw Exception("'ALL' not declared", ErrorCodes::LOGICAL_ERROR);
all_node = std::move(owned_nodes["ALL"]);
none_node = std::move(owned_nodes["NONE"]);
owned_nodes.erase("ALL");
owned_nodes.erase("NONE");
if (!owned_nodes.empty())
{
const auto & unused_node = *(owned_nodes.begin()->second);
if (unused_node.node_type == UNKNOWN)
throw Exception("Parent group '" + unused_node.keyword + "' not found", ErrorCodes::LOGICAL_ERROR);
else
throw Exception("Access type '" + unused_node.keyword + "' should have parent group", ErrorCodes::LOGICAL_ERROR);
}
}
void makeKeywordToFlagsMap(Node * start_node = nullptr)
{
if (!start_node)
{
makeKeywordToFlagsMap(none_node.get());
start_node = all_node.get();
}
start_node->aliases.emplace_back(start_node->keyword);
for (auto & alias : start_node->aliases)
{
boost::to_upper(alias);
keyword_to_flags_map[alias] = start_node->flags;
}
for (auto & child : start_node->children)
makeKeywordToFlagsMap(child.get());
}
void makeAccessTypeToFlagsMapping(Node * start_node = nullptr)
{
if (!start_node)
{
makeAccessTypeToFlagsMapping(none_node.get());
start_node = all_node.get();
}
size_t index = static_cast<size_t>(start_node->access_type);
access_type_to_flags_mapping.resize(std::max(index + 1, access_type_to_flags_mapping.size()));
access_type_to_flags_mapping[index] = start_node->flags;
for (auto & child : start_node->children)
makeAccessTypeToFlagsMapping(child.get());
}
void collectAllFlags(const Node * start_node = nullptr)
{
if (!start_node)
{
start_node = all_node.get();
all_flags = start_node->flags;
}
if (start_node->node_type != GROUP)
{
assert(static_cast<size_t>(start_node->node_type) < std::size(all_flags_for_target));
all_flags_for_target[start_node->node_type] |= start_node->flags;
}
for (const auto & child : start_node->children)
collectAllFlags(child.get());
all_flags_grantable_on_table_level = all_flags_for_target[TABLE] | all_flags_for_target[DICTIONARY] | all_flags_for_target[COLUMN];
all_flags_grantable_on_database_level = all_flags_for_target[DATABASE] | all_flags_grantable_on_table_level;
}
Impl()
{
makeNodes();
makeKeywordToFlagsMap();
makeAccessTypeToFlagsMapping();
collectAllFlags();
}
static void flagsToAccessTypesRec(const Flags & flags_, std::vector<AccessType> & access_types, const Node & start_node)
{
Flags matching_flags = (flags_ & start_node.flags);
if (matching_flags.any())
{
if (matching_flags == start_node.flags)
{
access_types.push_back(start_node.access_type);
}
else
{
for (const auto & child : start_node.children)
flagsToAccessTypesRec(flags_, access_types, *child);
}
}
}
static void flagsToKeywordsRec(const Flags & flags_, std::vector<std::string_view> & keywords, const Node & start_node)
{
Flags matching_flags = (flags_ & start_node.flags);
if (matching_flags.any())
{
if (matching_flags == start_node.flags)
{
keywords.push_back(start_node.keyword);
}
else
{
for (const auto & child : start_node.children)
flagsToKeywordsRec(flags_, keywords, *child);
}
}
}
NodePtr all_node;
NodePtr none_node;
std::unordered_map<std::string_view, Flags> keyword_to_flags_map;
std::vector<Flags> access_type_to_flags_mapping;
Flags all_flags;
Flags all_flags_for_target[static_cast<size_t>(DICTIONARY) + 1];
Flags all_flags_grantable_on_database_level;
Flags all_flags_grantable_on_table_level;
};
inline AccessFlags::AccessFlags(AccessType type) : flags(Impl<>::instance().accessTypeToFlags(type)) {}
inline AccessFlags::AccessFlags(const std::string_view & keyword) : flags(Impl<>::instance().keywordToFlags(keyword)) {}
inline AccessFlags::AccessFlags(const std::vector<std::string_view> & keywords) : flags(Impl<>::instance().keywordsToFlags(keywords)) {}
inline AccessFlags::AccessFlags(const Strings & keywords) : flags(Impl<>::instance().keywordsToFlags(keywords)) {}
inline String AccessFlags::toString() const { return Impl<>::instance().flagsToString(flags); }
inline std::vector<AccessType> AccessFlags::toAccessTypes() const { return Impl<>::instance().flagsToAccessTypes(flags); }
inline std::vector<std::string_view> AccessFlags::toKeywords() const { return Impl<>::instance().flagsToKeywords(flags); }
inline AccessFlags AccessFlags::allFlags() { return Impl<>::instance().getAllFlags(); }
inline AccessFlags AccessFlags::allGlobalFlags() { return Impl<>::instance().getGlobalFlags(); }
inline AccessFlags AccessFlags::allDatabaseFlags() { return Impl<>::instance().getDatabaseFlags(); }
inline AccessFlags AccessFlags::allTableFlags() { return Impl<>::instance().getTableFlags(); }
inline AccessFlags AccessFlags::allColumnFlags() { return Impl<>::instance().getColumnFlags(); }
inline AccessFlags AccessFlags::allDictionaryFlags() { return Impl<>::instance().getDictionaryFlags(); }
inline AccessFlags AccessFlags::allFlagsGrantableOnGlobalLevel() { return Impl<>::instance().getAllFlagsGrantableOnGlobalLevel(); }
inline AccessFlags AccessFlags::allFlagsGrantableOnDatabaseLevel() { return Impl<>::instance().getAllFlagsGrantableOnDatabaseLevel(); }
inline AccessFlags AccessFlags::allFlagsGrantableOnTableLevel() { return Impl<>::instance().getAllFlagsGrantableOnTableLevel(); }
inline AccessFlags AccessFlags::allFlagsGrantableOnColumnLevel() { return Impl<>::instance().getAllFlagsGrantableOnColumnLevel(); }
inline AccessFlags operator |(AccessType left, AccessType right) { return AccessFlags(left) | right; }
inline AccessFlags operator &(AccessType left, AccessType right) { return AccessFlags(left) & right; }
inline AccessFlags operator -(AccessType left, AccessType right) { return AccessFlags(left) - right; }
inline AccessFlags operator ~(AccessType x) { return ~AccessFlags(x); }
AccessFlags operator |(AccessType left, AccessType right);
AccessFlags operator &(AccessType left, AccessType right);
AccessFlags operator -(AccessType left, AccessType right);
AccessFlags operator ~(AccessType x);
}

View File

@ -145,10 +145,85 @@ namespace
}
AccessRightsElement::AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_)
: access_flags(access_flags_), database(database_), any_database(false)
{
}
AccessRightsElement::AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_)
: access_flags(access_flags_), database(database_), table(table_), any_database(false), any_table(false)
{
}
AccessRightsElement::AccessRightsElement(
AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const std::string_view & column_)
: access_flags(access_flags_)
, database(database_)
, table(table_)
, columns({String{column_}})
, any_database(false)
, any_table(false)
, any_column(false)
{
}
AccessRightsElement::AccessRightsElement(
AccessFlags access_flags_,
const std::string_view & database_,
const std::string_view & table_,
const std::vector<std::string_view> & columns_)
: access_flags(access_flags_), database(database_), table(table_), any_database(false), any_table(false), any_column(false)
{
columns.resize(columns_.size());
for (size_t i = 0; i != columns_.size(); ++i)
columns[i] = String{columns_[i]};
}
AccessRightsElement::AccessRightsElement(
AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const Strings & columns_)
: access_flags(access_flags_)
, database(database_)
, table(table_)
, columns(columns_)
, any_database(false)
, any_table(false)
, any_column(false)
{
}
void AccessRightsElement::eraseNonGrantable()
{
if (!any_column)
access_flags &= AccessFlags::allFlagsGrantableOnColumnLevel();
else if (!any_table)
access_flags &= AccessFlags::allFlagsGrantableOnTableLevel();
else if (!any_database)
access_flags &= AccessFlags::allFlagsGrantableOnDatabaseLevel();
else
access_flags &= AccessFlags::allFlagsGrantableOnGlobalLevel();
}
void AccessRightsElement::replaceEmptyDatabase(const String & current_database)
{
if (isEmptyDatabase())
database = current_database;
}
String AccessRightsElement::toString() const { return toStringImpl(*this, true); }
String AccessRightsElement::toStringWithoutOptions() const { return toStringImpl(*this, false); }
String AccessRightsElements::toString() const { return toStringImpl(*this, true); }
String AccessRightsElements::toStringWithoutOptions() const { return toStringImpl(*this, false); }
bool AccessRightsElements::empty() const { return std::all_of(begin(), end(), [](const AccessRightsElement & e) { return e.empty(); }); }
bool AccessRightsElements::sameDatabaseAndTable() const
{
return (size() < 2) || std::all_of(std::next(begin()), end(), [this](const AccessRightsElement & e) { return e.sameDatabaseAndTable(front()); });
}
bool AccessRightsElements::sameOptions() const
{
return (size() < 2) || std::all_of(std::next(begin()), end(), [this](const AccessRightsElement & e) { return e.sameOptions(front()); });
}
void AccessRightsElements::eraseNonGrantable()
{
@ -159,4 +234,13 @@ void AccessRightsElements::eraseNonGrantable()
});
}
void AccessRightsElements::replaceEmptyDatabase(const String & current_database)
{
for (auto & element : *this)
element.replaceEmptyDatabase(current_database);
}
String AccessRightsElements::toString() const { return toStringImpl(*this, true); }
String AccessRightsElements::toStringWithoutOptions() const { return toStringImpl(*this, false); }
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <Access/Common/AccessFlags.h>
#include <tuple>
namespace DB
@ -27,51 +28,19 @@ struct AccessRightsElement
AccessRightsElement(AccessFlags access_flags_) : access_flags(access_flags_) {}
AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_)
: access_flags(access_flags_), database(database_), any_database(false)
{
}
AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_)
: access_flags(access_flags_), database(database_), table(table_), any_database(false), any_table(false)
{
}
AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_);
AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_);
AccessRightsElement(
AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const std::string_view & column_)
: access_flags(access_flags_)
, database(database_)
, table(table_)
, columns({String{column_}})
, any_database(false)
, any_table(false)
, any_column(false)
{
}
AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const std::string_view & column_);
AccessRightsElement(
AccessFlags access_flags_,
const std::string_view & database_,
const std::string_view & table_,
const std::vector<std::string_view> & columns_)
: access_flags(access_flags_), database(database_), table(table_), any_database(false), any_table(false), any_column(false)
{
columns.resize(columns_.size());
for (size_t i = 0; i != columns_.size(); ++i)
columns[i] = String{columns_[i]};
}
const std::vector<std::string_view> & columns_);
AccessRightsElement(
AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const Strings & columns_)
: access_flags(access_flags_)
, database(database_)
, table(table_)
, columns(columns_)
, any_database(false)
, any_table(false)
, any_column(false)
{
}
AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const Strings & columns_);
bool empty() const { return !access_flags || (!any_column && columns.empty()); }
@ -91,26 +60,12 @@ struct AccessRightsElement
}
/// Resets flags which cannot be granted.
void eraseNonGrantable()
{
if (!any_column)
access_flags &= AccessFlags::allFlagsGrantableOnColumnLevel();
else if (!any_table)
access_flags &= AccessFlags::allFlagsGrantableOnTableLevel();
else if (!any_database)
access_flags &= AccessFlags::allFlagsGrantableOnDatabaseLevel();
else
access_flags &= AccessFlags::allFlagsGrantableOnGlobalLevel();
}
void eraseNonGrantable();
bool isEmptyDatabase() const { return !any_database && database.empty(); }
/// If the database is empty, replaces it with `current_database`. Otherwise does nothing.
void replaceEmptyDatabase(const String & current_database)
{
if (isEmptyDatabase())
database = current_database;
}
void replaceEmptyDatabase(const String & current_database);
/// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table".
String toString() const;
@ -125,27 +80,15 @@ public:
using Base = std::vector<AccessRightsElement>;
using Base::Base;
bool empty() const { return std::all_of(begin(), end(), [](const AccessRightsElement & e) { return e.empty(); }); }
bool sameDatabaseAndTable() const
{
return (size() < 2) || std::all_of(std::next(begin()), end(), [this](const AccessRightsElement & e) { return e.sameDatabaseAndTable(front()); });
}
bool sameOptions() const
{
return (size() < 2) || std::all_of(std::next(begin()), end(), [this](const AccessRightsElement & e) { return e.sameOptions(front()); });
}
bool empty() const;
bool sameDatabaseAndTable() const;
bool sameOptions() const;
/// Resets flags which cannot be granted.
void eraseNonGrantable();
/// If the database is empty, replaces it with `current_database`. Otherwise does nothing.
void replaceEmptyDatabase(const String & current_database)
{
for (auto & element : *this)
element.replaceEmptyDatabase(current_database);
}
void replaceEmptyDatabase(const String & current_database);
/// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table".
String toString() const;

View File

@ -0,0 +1,56 @@
#include <Access/Common/AccessType.h>
#include <boost/algorithm/string/replace.hpp>
#include <vector>
namespace DB
{
namespace
{
using Strings = std::vector<String>;
class AccessTypeToStringConverter
{
public:
static const AccessTypeToStringConverter & instance()
{
static const AccessTypeToStringConverter res;
return res;
}
std::string_view convert(AccessType type) const
{
return access_type_to_string_mapping[static_cast<size_t>(type)];
}
private:
AccessTypeToStringConverter()
{
#define ACCESS_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING(name, aliases, node_type, parent_group_name) \
addToMapping(AccessType::name, #name);
APPLY_FOR_ACCESS_TYPES(ACCESS_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING)
#undef ACCESS_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING
}
void addToMapping(AccessType type, const std::string_view & str)
{
String str2{str};
boost::replace_all(str2, "_", " ");
size_t index = static_cast<size_t>(type);
access_type_to_string_mapping.resize(std::max(index + 1, access_type_to_string_mapping.size()));
access_type_to_string_mapping[index] = str2;
}
Strings access_type_to_string_mapping;
};
}
std::string_view toString(AccessType type)
{
return AccessTypeToStringConverter::instance().convert(type);
}
}

View File

@ -1,17 +1,11 @@
#pragma once
#include <base/types.h>
#include <boost/algorithm/string/case_conv.hpp>
#include <boost/algorithm/string/replace.hpp>
#include <array>
#include <vector>
namespace DB
{
using Strings = std::vector<String>;
/// Represents an access type which can be granted on databases, tables, columns, etc.
enum class AccessType
{
@ -198,48 +192,6 @@ enum class AccessType
#undef DECLARE_ACCESS_TYPE_ENUM_CONST
};
namespace impl
{
template <typename = void>
class AccessTypeToStringConverter
{
public:
static const AccessTypeToStringConverter & instance()
{
static const AccessTypeToStringConverter res;
return res;
}
std::string_view convert(AccessType type) const
{
return access_type_to_string_mapping[static_cast<size_t>(type)];
}
private:
AccessTypeToStringConverter()
{
#define ACCESS_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING(name, aliases, node_type, parent_group_name) \
addToMapping(AccessType::name, #name);
APPLY_FOR_ACCESS_TYPES(ACCESS_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING)
#undef ACCESS_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING
}
void addToMapping(AccessType type, const std::string_view & str)
{
String str2{str};
boost::replace_all(str2, "_", " ");
size_t index = static_cast<size_t>(type);
access_type_to_string_mapping.resize(std::max(index + 1, access_type_to_string_mapping.size()));
access_type_to_string_mapping[index] = str2;
}
Strings access_type_to_string_mapping;
};
}
inline std::string_view toString(AccessType type) { return impl::AccessTypeToStringConverter<>::instance().convert(type); }
std::string_view toString(AccessType type);
}

View File

@ -5,9 +5,15 @@
#include <Functions/likePatternToRegexp.h>
#include <Poco/Net/SocketAddress.h>
#include <Poco/RegularExpression.h>
#include <boost/algorithm/string/predicate.hpp>
#include <boost/algorithm/string/replace.hpp>
#include <ifaddrs.h>
#include <boost/range/algorithm/find.hpp>
#include <boost/range/algorithm_ext/erase.hpp>
#include <Common/DNSResolver.h>
#include <ifaddrs.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
@ -186,6 +192,262 @@ namespace
}
void AllowedClientHosts::IPSubnet::set(const IPAddress & prefix_, const IPAddress & mask_)
{
prefix = prefix_;
mask = mask_;
if (prefix.family() != mask.family())
{
if (prefix.family() == IPAddress::IPv4)
prefix = IPAddress("::ffff:" + prefix.toString());
if (mask.family() == IPAddress::IPv4)
mask = IPAddress(96, IPAddress::IPv6) | IPAddress("::ffff:" + mask.toString());
}
prefix = prefix & mask;
if (prefix.family() == IPAddress::IPv4)
{
if ((prefix & IPAddress{8, IPAddress::IPv4}) == IPAddress{"127.0.0.0"})
{
// 127.XX.XX.XX -> 127.0.0.1
prefix = IPAddress{"127.0.0.1"};
mask = IPAddress{32, IPAddress::IPv4};
}
}
else
{
if ((prefix & IPAddress{104, IPAddress::IPv6}) == IPAddress{"::ffff:127.0.0.0"})
{
// ::ffff:127.XX.XX.XX -> ::1
prefix = IPAddress{"::1"};
mask = IPAddress{128, IPAddress::IPv6};
}
}
}
void AllowedClientHosts::IPSubnet::set(const IPAddress & prefix_, size_t num_prefix_bits)
{
set(prefix_, IPAddress(num_prefix_bits, prefix_.family()));
}
void AllowedClientHosts::IPSubnet::set(const IPAddress & address)
{
set(address, address.length() * 8);
}
AllowedClientHosts::IPSubnet::IPSubnet(const String & str)
{
size_t slash = str.find('/');
if (slash == String::npos)
{
set(IPAddress(str));
return;
}
IPAddress new_prefix{String{str, 0, slash}};
String mask_str(str, slash + 1, str.length() - slash - 1);
bool only_digits = (mask_str.find_first_not_of("0123456789") == std::string::npos);
if (only_digits)
set(new_prefix, std::stoul(mask_str));
else
set(new_prefix, IPAddress{mask_str});
}
String AllowedClientHosts::IPSubnet::toString() const
{
unsigned int prefix_length = mask.prefixLength();
if (isMaskAllBitsOne())
return prefix.toString();
else if (IPAddress{prefix_length, mask.family()} == mask)
return fs::path(prefix.toString()) / std::to_string(prefix_length);
else
return fs::path(prefix.toString()) / mask.toString();
}
bool AllowedClientHosts::IPSubnet::isMaskAllBitsOne() const
{
return mask == IPAddress(mask.length() * 8, mask.family());
}
void AllowedClientHosts::clear()
{
addresses = {};
subnets = {};
names = {};
name_regexps = {};
like_patterns = {};
any_host = false;
local_host = false;
}
bool AllowedClientHosts::empty() const
{
return !any_host && !local_host && addresses.empty() && subnets.empty() && names.empty() && name_regexps.empty() && like_patterns.empty();
}
void AllowedClientHosts::addAddress(const IPAddress & address)
{
if (address.isLoopback())
local_host = true;
else if (boost::range::find(addresses, address) == addresses.end())
addresses.push_back(address);
}
void AllowedClientHosts::removeAddress(const IPAddress & address)
{
if (address.isLoopback())
local_host = false;
else
boost::range::remove_erase(addresses, address);
}
void AllowedClientHosts::addSubnet(const IPSubnet & subnet)
{
if (subnet.getMask().isWildcard())
any_host = true;
else if (subnet.isMaskAllBitsOne())
addAddress(subnet.getPrefix());
else if (boost::range::find(subnets, subnet) == subnets.end())
subnets.push_back(subnet);
}
void AllowedClientHosts::removeSubnet(const IPSubnet & subnet)
{
if (subnet.getMask().isWildcard())
any_host = false;
else if (subnet.isMaskAllBitsOne())
removeAddress(subnet.getPrefix());
else
boost::range::remove_erase(subnets, subnet);
}
void AllowedClientHosts::addName(const String & name)
{
if (boost::iequals(name, "localhost"))
local_host = true;
else if (boost::range::find(names, name) == names.end())
names.push_back(name);
}
void AllowedClientHosts::removeName(const String & name)
{
if (boost::iequals(name, "localhost"))
local_host = false;
else
boost::range::remove_erase(names, name);
}
void AllowedClientHosts::addNameRegexp(const String & name_regexp)
{
if (boost::iequals(name_regexp, "localhost"))
local_host = true;
else if (name_regexp == ".*")
any_host = true;
else if (boost::range::find(name_regexps, name_regexp) == name_regexps.end())
name_regexps.push_back(name_regexp);
}
void AllowedClientHosts::removeNameRegexp(const String & name_regexp)
{
if (boost::iequals(name_regexp, "localhost"))
local_host = false;
else if (name_regexp == ".*")
any_host = false;
else
boost::range::remove_erase(name_regexps, name_regexp);
}
void AllowedClientHosts::addLikePattern(const String & pattern)
{
if (boost::iequals(pattern, "localhost") || (pattern == "127.0.0.1") || (pattern == "::1"))
local_host = true;
else if ((pattern == "%") || (pattern == "0.0.0.0/0") || (pattern == "::/0"))
any_host = true;
else if (boost::range::find(like_patterns, pattern) == name_regexps.end())
like_patterns.push_back(pattern);
}
void AllowedClientHosts::removeLikePattern(const String & pattern)
{
if (boost::iequals(pattern, "localhost") || (pattern == "127.0.0.1") || (pattern == "::1"))
local_host = false;
else if ((pattern == "%") || (pattern == "0.0.0.0/0") || (pattern == "::/0"))
any_host = false;
else
boost::range::remove_erase(like_patterns, pattern);
}
void AllowedClientHosts::addLocalHost()
{
local_host = true;
}
void AllowedClientHosts::removeLocalHost()
{
local_host = false;
}
void AllowedClientHosts::addAnyHost()
{
clear();
any_host = true;
}
void AllowedClientHosts::add(const AllowedClientHosts & other)
{
if (other.containsAnyHost())
{
addAnyHost();
return;
}
if (other.containsLocalHost())
addLocalHost();
for (const IPAddress & address : other.getAddresses())
addAddress(address);
for (const IPSubnet & subnet : other.getSubnets())
addSubnet(subnet);
for (const String & name : other.getNames())
addName(name);
for (const String & name_regexp : other.getNameRegexps())
addNameRegexp(name_regexp);
for (const String & like_pattern : other.getLikePatterns())
addLikePattern(like_pattern);
}
void AllowedClientHosts::remove(const AllowedClientHosts & other)
{
if (other.containsAnyHost())
{
clear();
return;
}
if (other.containsLocalHost())
removeLocalHost();
for (const IPAddress & address : other.getAddresses())
removeAddress(address);
for (const IPSubnet & subnet : other.getSubnets())
removeSubnet(subnet);
for (const String & name : other.getNames())
removeName(name);
for (const String & name_regexp : other.getNameRegexps())
removeNameRegexp(name_regexp);
for (const String & like_pattern : other.getLikePatterns())
removeLikePattern(like_pattern);
}
bool operator ==(const AllowedClientHosts & lhs, const AllowedClientHosts & rhs)
{
return (lhs.any_host == rhs.any_host) && (lhs.local_host == rhs.local_host) && (lhs.addresses == rhs.addresses)
&& (lhs.subnets == rhs.subnets) && (lhs.names == rhs.names) && (lhs.name_regexps == rhs.name_regexps)
&& (lhs.like_patterns == rhs.like_patterns);
}
bool AllowedClientHosts::contains(const IPAddress & client_address) const
{
if (any_host)

View File

@ -2,18 +2,11 @@
#include <base/types.h>
#include <Poco/Net/IPAddress.h>
#include <memory>
#include <vector>
#include <boost/range/algorithm/find.hpp>
#include <boost/range/algorithm_ext/erase.hpp>
#include <boost/algorithm/string/predicate.hpp>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
using Strings = std::vector<String>;
/// Represents lists of hosts a user is allowed to connect to server from.
@ -129,260 +122,4 @@ private:
bool local_host = false;
};
inline void AllowedClientHosts::IPSubnet::set(const IPAddress & prefix_, const IPAddress & mask_)
{
prefix = prefix_;
mask = mask_;
if (prefix.family() != mask.family())
{
if (prefix.family() == IPAddress::IPv4)
prefix = IPAddress("::ffff:" + prefix.toString());
if (mask.family() == IPAddress::IPv4)
mask = IPAddress(96, IPAddress::IPv6) | IPAddress("::ffff:" + mask.toString());
}
prefix = prefix & mask;
if (prefix.family() == IPAddress::IPv4)
{
if ((prefix & IPAddress{8, IPAddress::IPv4}) == IPAddress{"127.0.0.0"})
{
// 127.XX.XX.XX -> 127.0.0.1
prefix = IPAddress{"127.0.0.1"};
mask = IPAddress{32, IPAddress::IPv4};
}
}
else
{
if ((prefix & IPAddress{104, IPAddress::IPv6}) == IPAddress{"::ffff:127.0.0.0"})
{
// ::ffff:127.XX.XX.XX -> ::1
prefix = IPAddress{"::1"};
mask = IPAddress{128, IPAddress::IPv6};
}
}
}
inline void AllowedClientHosts::IPSubnet::set(const IPAddress & prefix_, size_t num_prefix_bits)
{
set(prefix_, IPAddress(num_prefix_bits, prefix_.family()));
}
inline void AllowedClientHosts::IPSubnet::set(const IPAddress & address)
{
set(address, address.length() * 8);
}
inline AllowedClientHosts::IPSubnet::IPSubnet(const String & str)
{
size_t slash = str.find('/');
if (slash == String::npos)
{
set(IPAddress(str));
return;
}
IPAddress new_prefix{String{str, 0, slash}};
String mask_str(str, slash + 1, str.length() - slash - 1);
bool only_digits = (mask_str.find_first_not_of("0123456789") == std::string::npos);
if (only_digits)
set(new_prefix, std::stoul(mask_str));
else
set(new_prefix, IPAddress{mask_str});
}
inline String AllowedClientHosts::IPSubnet::toString() const
{
unsigned int prefix_length = mask.prefixLength();
if (isMaskAllBitsOne())
return prefix.toString();
else if (IPAddress{prefix_length, mask.family()} == mask)
return fs::path(prefix.toString()) / std::to_string(prefix_length);
else
return fs::path(prefix.toString()) / mask.toString();
}
inline bool AllowedClientHosts::IPSubnet::isMaskAllBitsOne() const
{
return mask == IPAddress(mask.length() * 8, mask.family());
}
inline void AllowedClientHosts::clear()
{
addresses = {};
subnets = {};
names = {};
name_regexps = {};
like_patterns = {};
any_host = false;
local_host = false;
}
inline bool AllowedClientHosts::empty() const
{
return !any_host && !local_host && addresses.empty() && subnets.empty() && names.empty() && name_regexps.empty() && like_patterns.empty();
}
inline void AllowedClientHosts::addAddress(const IPAddress & address)
{
if (address.isLoopback())
local_host = true;
else if (boost::range::find(addresses, address) == addresses.end())
addresses.push_back(address);
}
inline void AllowedClientHosts::removeAddress(const IPAddress & address)
{
if (address.isLoopback())
local_host = false;
else
boost::range::remove_erase(addresses, address);
}
inline void AllowedClientHosts::addSubnet(const IPSubnet & subnet)
{
if (subnet.getMask().isWildcard())
any_host = true;
else if (subnet.isMaskAllBitsOne())
addAddress(subnet.getPrefix());
else if (boost::range::find(subnets, subnet) == subnets.end())
subnets.push_back(subnet);
}
inline void AllowedClientHosts::removeSubnet(const IPSubnet & subnet)
{
if (subnet.getMask().isWildcard())
any_host = false;
else if (subnet.isMaskAllBitsOne())
removeAddress(subnet.getPrefix());
else
boost::range::remove_erase(subnets, subnet);
}
inline void AllowedClientHosts::addName(const String & name)
{
if (boost::iequals(name, "localhost"))
local_host = true;
else if (boost::range::find(names, name) == names.end())
names.push_back(name);
}
inline void AllowedClientHosts::removeName(const String & name)
{
if (boost::iequals(name, "localhost"))
local_host = false;
else
boost::range::remove_erase(names, name);
}
inline void AllowedClientHosts::addNameRegexp(const String & name_regexp)
{
if (boost::iequals(name_regexp, "localhost"))
local_host = true;
else if (name_regexp == ".*")
any_host = true;
else if (boost::range::find(name_regexps, name_regexp) == name_regexps.end())
name_regexps.push_back(name_regexp);
}
inline void AllowedClientHosts::removeNameRegexp(const String & name_regexp)
{
if (boost::iequals(name_regexp, "localhost"))
local_host = false;
else if (name_regexp == ".*")
any_host = false;
else
boost::range::remove_erase(name_regexps, name_regexp);
}
inline void AllowedClientHosts::addLikePattern(const String & pattern)
{
if (boost::iequals(pattern, "localhost") || (pattern == "127.0.0.1") || (pattern == "::1"))
local_host = true;
else if ((pattern == "%") || (pattern == "0.0.0.0/0") || (pattern == "::/0"))
any_host = true;
else if (boost::range::find(like_patterns, pattern) == name_regexps.end())
like_patterns.push_back(pattern);
}
inline void AllowedClientHosts::removeLikePattern(const String & pattern)
{
if (boost::iequals(pattern, "localhost") || (pattern == "127.0.0.1") || (pattern == "::1"))
local_host = false;
else if ((pattern == "%") || (pattern == "0.0.0.0/0") || (pattern == "::/0"))
any_host = false;
else
boost::range::remove_erase(like_patterns, pattern);
}
inline void AllowedClientHosts::addLocalHost()
{
local_host = true;
}
inline void AllowedClientHosts::removeLocalHost()
{
local_host = false;
}
inline void AllowedClientHosts::addAnyHost()
{
clear();
any_host = true;
}
inline void AllowedClientHosts::add(const AllowedClientHosts & other)
{
if (other.containsAnyHost())
{
addAnyHost();
return;
}
if (other.containsLocalHost())
addLocalHost();
for (const IPAddress & address : other.getAddresses())
addAddress(address);
for (const IPSubnet & subnet : other.getSubnets())
addSubnet(subnet);
for (const String & name : other.getNames())
addName(name);
for (const String & name_regexp : other.getNameRegexps())
addNameRegexp(name_regexp);
for (const String & like_pattern : other.getLikePatterns())
addLikePattern(like_pattern);
}
inline void AllowedClientHosts::remove(const AllowedClientHosts & other)
{
if (other.containsAnyHost())
{
clear();
return;
}
if (other.containsLocalHost())
removeLocalHost();
for (const IPAddress & address : other.getAddresses())
removeAddress(address);
for (const IPSubnet & subnet : other.getSubnets())
removeSubnet(subnet);
for (const String & name : other.getNames())
removeName(name);
for (const String & name_regexp : other.getNameRegexps())
removeNameRegexp(name_regexp);
for (const String & like_pattern : other.getLikePatterns())
removeLikePattern(like_pattern);
}
inline bool operator ==(const AllowedClientHosts & lhs, const AllowedClientHosts & rhs)
{
return (lhs.any_host == rhs.any_host) && (lhs.local_host == rhs.local_host) && (lhs.addresses == rhs.addresses)
&& (lhs.subnets == rhs.subnets) && (lhs.names == rhs.names) && (lhs.name_regexps == rhs.name_regexps)
&& (lhs.like_patterns == rhs.like_patterns);
}
}

View File

@ -59,7 +59,7 @@ const AuthenticationTypeInfo & AuthenticationTypeInfo::get(AuthenticationType ty
static const auto info = make_info("KERBEROS");
return info;
}
case AuthenticationType::MAX_TYPE:
case AuthenticationType::MAX:
break;
}
throw Exception("Unknown authentication type: " + std::to_string(static_cast<int>(type_)), ErrorCodes::LOGICAL_ERROR);
@ -101,7 +101,7 @@ void AuthenticationData::setPassword(const String & password_)
switch (type)
{
case AuthenticationType::PLAINTEXT_PASSWORD:
return setPasswordHashBinary(Util::encodePlainText(password_));
return setPasswordHashBinary(Util::stringToDigest(password_));
case AuthenticationType::SHA256_PASSWORD:
return setPasswordHashBinary(Util::encodeSHA256(password_));
@ -114,7 +114,7 @@ void AuthenticationData::setPassword(const String & password_)
case AuthenticationType::KERBEROS:
throw Exception("Cannot specify password for authentication type " + toString(type), ErrorCodes::LOGICAL_ERROR);
case AuthenticationType::MAX_TYPE:
case AuthenticationType::MAX:
break;
}
throw Exception("setPassword(): authentication type " + toString(type) + " not supported", ErrorCodes::NOT_IMPLEMENTED);
@ -187,7 +187,7 @@ void AuthenticationData::setPasswordHashBinary(const Digest & hash)
case AuthenticationType::KERBEROS:
throw Exception("Cannot specify password binary hash for authentication type " + toString(type), ErrorCodes::LOGICAL_ERROR);
case AuthenticationType::MAX_TYPE:
case AuthenticationType::MAX:
break;
}
throw Exception("setPasswordHashBinary(): authentication type " + toString(type) + " not supported", ErrorCodes::NOT_IMPLEMENTED);

View File

@ -27,7 +27,7 @@ enum class AuthenticationType
/// Kerberos authentication performed through GSS-API negotiation loop.
KERBEROS,
MAX_TYPE,
MAX,
};
struct AuthenticationTypeInfo
@ -37,6 +37,11 @@ struct AuthenticationTypeInfo
static const AuthenticationTypeInfo & get(AuthenticationType type_);
};
inline String toString(AuthenticationType type_)
{
return AuthenticationTypeInfo::get(type_).raw_name;
}
/// Stores data for checking password when a user logins.
class AuthenticationData
@ -79,7 +84,7 @@ public:
struct Util
{
static Digest encodePlainText(const std::string_view & text) { return Digest(text.data(), text.data() + text.size()); }
static Digest stringToDigest(const std::string_view & text) { return Digest(text.data(), text.data() + text.size()); }
static Digest encodeSHA256(const std::string_view & text);
static Digest encodeSHA1(const std::string_view & text);
static Digest encodeSHA1(const Digest & text) { return encodeSHA1(std::string_view{reinterpret_cast<const char *>(text.data()), text.size()}); }
@ -94,10 +99,4 @@ private:
String kerberos_realm;
};
inline String toString(AuthenticationType type_)
{
return AuthenticationTypeInfo::get(type_).raw_name;
}
}

View File

@ -1,5 +1,5 @@
#include <Access/ContextAccess.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/EnabledRoles.h>
#include <Access/EnabledRowPolicies.h>
#include <Access/EnabledQuota.h>
@ -142,13 +142,13 @@ namespace
}
ContextAccess::ContextAccess(const AccessControlManager & manager_, const Params & params_)
: manager(&manager_)
ContextAccess::ContextAccess(const AccessControl & access_control_, const Params & params_)
: access_control(&access_control_)
, params(params_)
{
std::lock_guard lock{mutex};
subscription_for_user_change = manager->subscribeForChanges(
subscription_for_user_change = access_control->subscribeForChanges(
*params.user_id, [this](const UUID &, const AccessEntityPtr & entity)
{
UserPtr changed_user = entity ? typeid_cast<UserPtr>(entity) : nullptr;
@ -156,7 +156,7 @@ ContextAccess::ContextAccess(const AccessControlManager & manager_, const Params
setUser(changed_user);
});
setUser(manager->read<User>(*params.user_id));
setUser(access_control->read<User>(*params.user_id));
}
@ -194,7 +194,7 @@ void ContextAccess::setUser(const UserPtr & user_) const
}
subscription_for_roles_changes.reset();
enabled_roles = manager->getEnabledRoles(current_roles, current_roles_with_admin_option);
enabled_roles = access_control->getEnabledRoles(current_roles, current_roles_with_admin_option);
subscription_for_roles_changes = enabled_roles->subscribeForChanges([this](const std::shared_ptr<const EnabledRolesInfo> & roles_info_)
{
std::lock_guard lock{mutex};
@ -209,11 +209,11 @@ void ContextAccess::setRolesInfo(const std::shared_ptr<const EnabledRolesInfo> &
{
assert(roles_info_);
roles_info = roles_info_;
enabled_row_policies = manager->getEnabledRowPolicies(
enabled_row_policies = access_control->getEnabledRowPolicies(
*params.user_id, roles_info->enabled_roles);
enabled_quota = manager->getEnabledQuota(
enabled_quota = access_control->getEnabledQuota(
*params.user_id, user_name, roles_info->enabled_roles, params.address, params.forwarded_address, params.quota_key);
enabled_settings = manager->getEnabledSettings(
enabled_settings = access_control->getEnabledSettings(
*params.user_id, user->settings, roles_info->enabled_roles, roles_info->settings_from_enabled_roles);
calculateAccessRights();
}
@ -327,7 +327,7 @@ std::shared_ptr<const SettingsProfilesInfo> ContextAccess::getDefaultProfileInfo
std::lock_guard lock{mutex};
if (enabled_settings)
return enabled_settings->getInfo();
static const auto everything_by_default = std::make_shared<SettingsProfilesInfo>(*manager);
static const auto everything_by_default = std::make_shared<SettingsProfilesInfo>(*access_control);
return everything_by_default;
}
@ -609,7 +609,7 @@ bool ContextAccess::checkAdminOptionImplHelper(const Container & role_ids, const
template <bool throw_if_denied>
bool ContextAccess::checkAdminOptionImpl(const UUID & role_id) const
{
return checkAdminOptionImplHelper<throw_if_denied>(to_array(role_id), [this](const UUID & id, size_t) { return manager->tryReadName(id); });
return checkAdminOptionImplHelper<throw_if_denied>(to_array(role_id), [this](const UUID & id, size_t) { return access_control->tryReadName(id); });
}
template <bool throw_if_denied>
@ -627,7 +627,7 @@ bool ContextAccess::checkAdminOptionImpl(const UUID & role_id, const std::unorde
template <bool throw_if_denied>
bool ContextAccess::checkAdminOptionImpl(const std::vector<UUID> & role_ids) const
{
return checkAdminOptionImplHelper<throw_if_denied>(role_ids, [this](const UUID & id, size_t) { return manager->tryReadName(id); });
return checkAdminOptionImplHelper<throw_if_denied>(role_ids, [this](const UUID & id, size_t) { return access_control->tryReadName(id); });
}
template <bool throw_if_denied>

View File

@ -8,6 +8,7 @@
#include <base/shared_ptr_helper.h>
#include <boost/container/flat_set.hpp>
#include <mutex>
#include <unordered_map>
namespace Poco { class Logger; }
@ -25,7 +26,7 @@ struct QuotaUsage;
struct Settings;
struct SettingsProfilesInfo;
class SettingsChanges;
class AccessControlManager;
class AccessControl;
class IAST;
using ASTPtr = std::shared_ptr<IAST>;
@ -155,9 +156,9 @@ public:
static std::shared_ptr<const ContextAccess> getFullAccess();
private:
friend class AccessControlManager;
friend class AccessControl;
ContextAccess() {}
ContextAccess(const AccessControlManager & manager_, const Params & params_);
ContextAccess(const AccessControl & access_control_, const Params & params_);
void setUser(const UserPtr & user_) const;
void setRolesInfo(const std::shared_ptr<const EnabledRolesInfo> & roles_info_) const;
@ -203,7 +204,7 @@ private:
template <bool throw_if_denied, typename Container, typename GetNameFunction>
bool checkAdminOptionImplHelper(const Container & role_ids, const GetNameFunction & get_name_function) const;
const AccessControlManager * manager = nullptr;
const AccessControl * access_control = nullptr;
const Params params;
bool is_full_access = false;
mutable Poco::Logger * trace_log = nullptr;

View File

@ -1,5 +1,5 @@
#include <Access/LDAPAccessStorage.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/ExternalAuthenticators.h>
#include <Access/User.h>
#include <Access/Role.h>
@ -27,10 +27,10 @@ namespace ErrorCodes
}
LDAPAccessStorage::LDAPAccessStorage(const String & storage_name_, AccessControlManager * access_control_manager_, const Poco::Util::AbstractConfiguration & config, const String & prefix)
LDAPAccessStorage::LDAPAccessStorage(const String & storage_name_, AccessControl * access_control_, const Poco::Util::AbstractConfiguration & config, const String & prefix)
: IAccessStorage(storage_name_)
{
setConfiguration(access_control_manager_, config, prefix);
setConfiguration(access_control_, config, prefix);
}
@ -40,7 +40,7 @@ String LDAPAccessStorage::getLDAPServerName() const
}
void LDAPAccessStorage::setConfiguration(AccessControlManager * access_control_manager_, const Poco::Util::AbstractConfiguration & config, const String & prefix)
void LDAPAccessStorage::setConfiguration(AccessControl * access_control_, const Poco::Util::AbstractConfiguration & config, const String & prefix)
{
std::scoped_lock lock(mutex);
@ -80,7 +80,7 @@ void LDAPAccessStorage::setConfiguration(AccessControlManager * access_control_m
}
}
access_control_manager = access_control_manager_;
access_control = access_control_;
ldap_server_name = ldap_server_name_cfg;
role_search_params.swap(role_search_params_cfg);
common_role_names.swap(common_roles_cfg);
@ -91,7 +91,7 @@ void LDAPAccessStorage::setConfiguration(AccessControlManager * access_control_m
granted_role_names.clear();
granted_role_ids.clear();
role_change_subscription = access_control_manager->subscribeForChanges<Role>(
role_change_subscription = access_control->subscribeForChanges<Role>(
[this] (const UUID & id, const AccessEntityPtr & entity)
{
return this->processRoleChange(id, entity);
@ -215,7 +215,7 @@ void LDAPAccessStorage::assignRolesNoLock(User & user, const LDAPClient::SearchR
auto it = granted_role_ids.find(role_name);
if (it == granted_role_ids.end())
{
if (const auto role_id = access_control_manager->find<Role>(role_name))
if (const auto role_id = access_control->find<Role>(role_name))
{
granted_role_names.insert_or_assign(*role_id, role_name);
it = granted_role_ids.insert_or_assign(role_name, *role_id).first;

View File

@ -22,7 +22,7 @@ namespace Poco
namespace DB
{
class AccessControlManager;
class AccessControl;
/// Implementation of IAccessStorage which allows attaching users from a remote LDAP server.
/// Currently, any user name will be treated as a name of an existing remote user,
@ -32,7 +32,7 @@ class LDAPAccessStorage : public IAccessStorage
public:
static constexpr char STORAGE_TYPE[] = "ldap";
explicit LDAPAccessStorage(const String & storage_name_, AccessControlManager * access_control_manager_, const Poco::Util::AbstractConfiguration & config, const String & prefix);
explicit LDAPAccessStorage(const String & storage_name_, AccessControl * access_control_, const Poco::Util::AbstractConfiguration & config, const String & prefix);
virtual ~LDAPAccessStorage() override = default;
String getLDAPServerName() const;
@ -59,7 +59,7 @@ private: // IAccessStorage implementations.
virtual UUID getIDOfLoggedUserImpl(const String & user_name) const override;
private:
void setConfiguration(AccessControlManager * access_control_manager_, const Poco::Util::AbstractConfiguration & config, const String & prefix);
void setConfiguration(AccessControl * access_control_, const Poco::Util::AbstractConfiguration & config, const String & prefix);
void processRoleChange(const UUID & id, const AccessEntityPtr & entity);
void applyRoleChangeNoLock(bool grant, const UUID & role_id, const String & role_name);
@ -71,7 +71,7 @@ private:
const ExternalAuthenticators & external_authenticators, LDAPClient::SearchResultsList & role_search_results) const;
mutable std::recursive_mutex mutex;
AccessControlManager * access_control_manager = nullptr;
AccessControl * access_control = nullptr;
String ldap_server_name;
LDAPClient::RoleSearchParamsList role_search_params;
std::set<String> common_role_names; // role name that should be granted to all users at all times

View File

@ -1,7 +1,7 @@
#include <Access/EnabledQuota.h>
#include <Access/QuotaCache.h>
#include <Access/QuotaUsage.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Common/Exception.h>
#include <Common/thread_local_rng.h>
#include <base/range.h>
@ -172,8 +172,8 @@ boost::shared_ptr<const EnabledQuota::Intervals> QuotaCache::QuotaInfo::rebuildI
}
QuotaCache::QuotaCache(const AccessControlManager & access_control_manager_)
: access_control_manager(access_control_manager_)
QuotaCache::QuotaCache(const AccessControl & access_control_)
: access_control(access_control_)
{
}
@ -215,7 +215,7 @@ void QuotaCache::ensureAllQuotasRead()
return;
all_quotas_read = true;
subscription = access_control_manager.subscribeForChanges<Quota>(
subscription = access_control.subscribeForChanges<Quota>(
[&](const UUID & id, const AccessEntityPtr & entity)
{
if (entity)
@ -224,9 +224,9 @@ void QuotaCache::ensureAllQuotasRead()
quotaRemoved(id);
});
for (const UUID & quota_id : access_control_manager.findAll<Quota>())
for (const UUID & quota_id : access_control.findAll<Quota>())
{
auto quota = access_control_manager.tryRead<Quota>(quota_id);
auto quota = access_control.tryRead<Quota>(quota_id);
if (quota)
all_quotas.emplace(quota_id, QuotaInfo(quota, quota_id));
}

View File

@ -10,14 +10,14 @@
namespace DB
{
class AccessControlManager;
class AccessControl;
/// Stores information how much amount of resources have been consumed and how much are left.
class QuotaCache
{
public:
QuotaCache(const AccessControlManager & access_control_manager_);
QuotaCache(const AccessControl & access_control_);
~QuotaCache();
std::shared_ptr<const EnabledQuota> getEnabledQuota(
@ -56,7 +56,7 @@ private:
void chooseQuotaToConsume();
void chooseQuotaToConsumeFor(EnabledQuota & enabled_quota);
const AccessControlManager & access_control_manager;
const AccessControl & access_control;
mutable std::mutex mutex;
std::unordered_map<UUID /* quota id */, QuotaInfo> all_quotas;
bool all_quotas_read = false;

View File

@ -1,7 +1,7 @@
#include <Access/RoleCache.h>
#include <Access/Role.h>
#include <Access/EnabledRolesInfo.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <boost/container/flat_set.hpp>
#include <base/FnTraits.h>
@ -56,8 +56,8 @@ namespace
}
RoleCache::RoleCache(const AccessControlManager & manager_)
: manager(manager_), cache(600000 /* 10 minutes */) {}
RoleCache::RoleCache(const AccessControl & access_control_)
: access_control(access_control_), cache(600000 /* 10 minutes */) {}
RoleCache::~RoleCache() = default;
@ -136,7 +136,7 @@ RolePtr RoleCache::getRole(const UUID & role_id)
if (role_from_cache)
return role_from_cache->first;
auto subscription = manager.subscribeForChanges(role_id,
auto subscription = access_control.subscribeForChanges(role_id,
[this, role_id](const UUID &, const AccessEntityPtr & entity)
{
auto changed_role = entity ? typeid_cast<RolePtr>(entity) : nullptr;
@ -146,7 +146,7 @@ RolePtr RoleCache::getRole(const UUID & role_id)
roleRemoved(role_id);
});
auto role = manager.tryRead<Role>(role_id);
auto role = access_control.tryRead<Role>(role_id);
if (role)
{
auto cache_value = Poco::SharedPtr<std::pair<RolePtr, scope_guard>>(

View File

@ -9,14 +9,14 @@
namespace DB
{
class AccessControlManager;
class AccessControl;
struct Role;
using RolePtr = std::shared_ptr<const Role>;
class RoleCache
{
public:
RoleCache(const AccessControlManager & manager_);
RoleCache(const AccessControl & access_control_);
~RoleCache();
std::shared_ptr<const EnabledRoles> getEnabledRoles(
@ -30,7 +30,7 @@ private:
void roleChanged(const UUID & role_id, const RolePtr & changed_role);
void roleRemoved(const UUID & role_id);
const AccessControlManager & manager;
const AccessControl & access_control;
Poco::ExpireCache<UUID, std::pair<RolePtr, scope_guard>> cache;
std::map<EnabledRoles::Params, std::weak_ptr<EnabledRoles>> enabled_roles;
mutable std::mutex mutex;

View File

@ -1,7 +1,7 @@
#include <Access/RolesOrUsersSet.h>
#include <Parsers/Access/ASTRolesOrUsersSet.h>
#include <Parsers/formatAST.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/User.h>
#include <Access/Role.h>
#include <IO/ReadHelpers.h>
@ -53,40 +53,40 @@ RolesOrUsersSet::RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const std::opti
init(ast, nullptr, current_user_id);
}
RolesOrUsersSet::RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const AccessControlManager & manager)
RolesOrUsersSet::RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const AccessControl & access_control)
{
init(ast, &manager);
init(ast, &access_control);
}
RolesOrUsersSet::RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const AccessControlManager & manager, const std::optional<UUID> & current_user_id)
RolesOrUsersSet::RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const AccessControl & access_control, const std::optional<UUID> & current_user_id)
{
init(ast, &manager, current_user_id);
init(ast, &access_control, current_user_id);
}
void RolesOrUsersSet::init(const ASTRolesOrUsersSet & ast, const AccessControlManager * manager, const std::optional<UUID> & current_user_id)
void RolesOrUsersSet::init(const ASTRolesOrUsersSet & ast, const AccessControl * access_control, const std::optional<UUID> & current_user_id)
{
all = ast.all;
auto name_to_id = [&ast, manager](const String & name) -> UUID
auto name_to_id = [&ast, access_control](const String & name) -> UUID
{
if (ast.id_mode)
return parse<UUID>(name);
assert(manager);
assert(access_control);
if (ast.allow_users && ast.allow_roles)
{
auto id = manager->find<User>(name);
auto id = access_control->find<User>(name);
if (id)
return *id;
return manager->getID<Role>(name);
return access_control->getID<Role>(name);
}
else if (ast.allow_users)
{
return manager->getID<User>(name);
return access_control->getID<User>(name);
}
else
{
assert(ast.allow_roles);
return manager->getID<Role>(name);
return access_control->getID<Role>(name);
}
};
@ -147,7 +147,7 @@ std::shared_ptr<ASTRolesOrUsersSet> RolesOrUsersSet::toAST() const
}
std::shared_ptr<ASTRolesOrUsersSet> RolesOrUsersSet::toASTWithNames(const AccessControlManager & manager) const
std::shared_ptr<ASTRolesOrUsersSet> RolesOrUsersSet::toASTWithNames(const AccessControl & access_control) const
{
auto ast = std::make_shared<ASTRolesOrUsersSet>();
ast->all = all;
@ -157,7 +157,7 @@ std::shared_ptr<ASTRolesOrUsersSet> RolesOrUsersSet::toASTWithNames(const Access
ast->names.reserve(ids.size());
for (const UUID & id : ids)
{
auto name = manager.tryReadName(id);
auto name = access_control.tryReadName(id);
if (name)
ast->names.emplace_back(std::move(*name));
}
@ -169,7 +169,7 @@ std::shared_ptr<ASTRolesOrUsersSet> RolesOrUsersSet::toASTWithNames(const Access
ast->except_names.reserve(except_ids.size());
for (const UUID & except_id : except_ids)
{
auto except_name = manager.tryReadName(except_id);
auto except_name = access_control.tryReadName(except_id);
if (except_name)
ast->except_names.emplace_back(std::move(*except_name));
}
@ -187,9 +187,9 @@ String RolesOrUsersSet::toString() const
}
String RolesOrUsersSet::toStringWithNames(const AccessControlManager & manager) const
String RolesOrUsersSet::toStringWithNames(const AccessControl & access_control) const
{
auto ast = toASTWithNames(manager);
auto ast = toASTWithNames(access_control);
return serializeAST(*ast);
}
@ -253,25 +253,25 @@ bool RolesOrUsersSet::match(const UUID & user_id, const boost::container::flat_s
std::vector<UUID> RolesOrUsersSet::getMatchingIDs() const
{
if (all)
throw Exception("getAllMatchingIDs() can't get ALL ids without manager", ErrorCodes::LOGICAL_ERROR);
throw Exception("getAllMatchingIDs() can't get ALL ids without access_control", ErrorCodes::LOGICAL_ERROR);
std::vector<UUID> res;
boost::range::set_difference(ids, except_ids, std::back_inserter(res));
return res;
}
std::vector<UUID> RolesOrUsersSet::getMatchingIDs(const AccessControlManager & manager) const
std::vector<UUID> RolesOrUsersSet::getMatchingIDs(const AccessControl & access_control) const
{
if (!all)
return getMatchingIDs();
std::vector<UUID> res;
for (const UUID & id : manager.findAll<User>())
for (const UUID & id : access_control.findAll<User>())
{
if (match(id))
res.push_back(id);
}
for (const UUID & id : manager.findAll<Role>())
for (const UUID & id : access_control.findAll<Role>())
{
if (match(id))
res.push_back(id);

View File

@ -10,7 +10,7 @@
namespace DB
{
class ASTRolesOrUsersSet;
class AccessControlManager;
class AccessControl;
/// Represents a set of users/roles like
@ -31,18 +31,18 @@ struct RolesOrUsersSet
RolesOrUsersSet(const UUID & id);
RolesOrUsersSet(const std::vector<UUID> & ids_);
/// The constructor from AST requires the AccessControlManager if `ast.id_mode == false`.
/// The constructor from AST requires the AccessControl if `ast.id_mode == false`.
RolesOrUsersSet(const ASTRolesOrUsersSet & ast);
RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const std::optional<UUID> & current_user_id);
RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const AccessControlManager & manager);
RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const AccessControlManager & manager, const std::optional<UUID> & current_user_id);
RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const AccessControl & access_control);
RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const AccessControl & access_control, const std::optional<UUID> & current_user_id);
std::shared_ptr<ASTRolesOrUsersSet> toAST() const;
std::shared_ptr<ASTRolesOrUsersSet> toASTWithNames(const AccessControlManager & manager) const;
std::shared_ptr<ASTRolesOrUsersSet> toASTWithNames(const AccessControl & access_control) const;
String toString() const;
String toStringWithNames(const AccessControlManager & manager) const;
Strings toStringsWithNames(const AccessControlManager & manager) const;
String toStringWithNames(const AccessControl & access_control) const;
Strings toStringsWithNames(const AccessControl & access_control) const;
bool empty() const;
void clear();
@ -57,7 +57,7 @@ struct RolesOrUsersSet
std::vector<UUID> getMatchingIDs() const;
/// Returns a list of matching users and roles.
std::vector<UUID> getMatchingIDs(const AccessControlManager & manager) const;
std::vector<UUID> getMatchingIDs(const AccessControl & access_control) const;
friend bool operator ==(const RolesOrUsersSet & lhs, const RolesOrUsersSet & rhs);
friend bool operator !=(const RolesOrUsersSet & lhs, const RolesOrUsersSet & rhs) { return !(lhs == rhs); }
@ -67,7 +67,7 @@ struct RolesOrUsersSet
boost::container::flat_set<UUID> except_ids;
private:
void init(const ASTRolesOrUsersSet & ast, const AccessControlManager * manager = nullptr, const std::optional<UUID> & current_user_id = {});
void init(const ASTRolesOrUsersSet & ast, const AccessControl * access_control = nullptr, const std::optional<UUID> & current_user_id = {});
};
}

View File

@ -1,6 +1,6 @@
#include <Access/RowPolicyCache.h>
#include <Access/EnabledRowPolicies.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/parseQuery.h>
#include <Parsers/makeASTForLogicalFunction.h>
@ -92,8 +92,8 @@ void RowPolicyCache::PolicyInfo::setPolicy(const RowPolicyPtr & policy_)
}
RowPolicyCache::RowPolicyCache(const AccessControlManager & access_control_manager_)
: access_control_manager(access_control_manager_)
RowPolicyCache::RowPolicyCache(const AccessControl & access_control_)
: access_control(access_control_)
{
}
@ -131,7 +131,7 @@ void RowPolicyCache::ensureAllRowPoliciesRead()
return;
all_policies_read = true;
subscription = access_control_manager.subscribeForChanges<RowPolicy>(
subscription = access_control.subscribeForChanges<RowPolicy>(
[&](const UUID & id, const AccessEntityPtr & entity)
{
if (entity)
@ -140,9 +140,9 @@ void RowPolicyCache::ensureAllRowPoliciesRead()
rowPolicyRemoved(id);
});
for (const UUID & id : access_control_manager.findAll<RowPolicy>())
for (const UUID & id : access_control.findAll<RowPolicy>())
{
auto quota = access_control_manager.tryRead<RowPolicy>(id);
auto quota = access_control.tryRead<RowPolicy>(id);
if (quota)
all_policies.emplace(id, PolicyInfo(quota));
}

View File

@ -9,13 +9,13 @@
namespace DB
{
class AccessControlManager;
class AccessControl;
/// Stores read and parsed row policies.
class RowPolicyCache
{
public:
RowPolicyCache(const AccessControlManager & access_control_manager_);
RowPolicyCache(const AccessControl & access_control_);
~RowPolicyCache();
std::shared_ptr<const EnabledRowPolicies> getEnabledRowPolicies(const UUID & user_id, const boost::container::flat_set<UUID> & enabled_roles);
@ -38,7 +38,7 @@ private:
void mixConditions();
void mixConditionsFor(EnabledRowPolicies & enabled);
const AccessControlManager & access_control_manager;
const AccessControl & access_control;
std::unordered_map<UUID, PolicyInfo> all_policies;
bool all_policies_read = false;
scope_guard subscription;

View File

@ -1,5 +1,5 @@
#include <Access/SettingsConstraints.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Core/Settings.h>
#include <Common/FieldVisitorToString.h>
#include <Common/FieldVisitorsAccurateComparison.h>
@ -18,7 +18,7 @@ namespace ErrorCodes
}
SettingsConstraints::SettingsConstraints(const AccessControlManager & manager_) : manager(&manager_)
SettingsConstraints::SettingsConstraints(const AccessControl & access_control_) : access_control(&access_control_)
{
}
@ -200,8 +200,8 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh
};
if (reaction == THROW_ON_VIOLATION)
manager->checkSettingNameIsAllowed(setting_name);
else if (!manager->isSettingNameAllowed(setting_name))
access_control->checkSettingNameIsAllowed(setting_name);
else if (!access_control->isSettingNameAllowed(setting_name))
return false;
Field current_value, new_value;

View File

@ -14,7 +14,7 @@ namespace DB
struct Settings;
struct SettingChange;
class SettingsChanges;
class AccessControlManager;
class AccessControl;
/** Checks if specified changes of settings are allowed or not.
@ -51,7 +51,7 @@ class AccessControlManager;
class SettingsConstraints
{
public:
SettingsConstraints(const AccessControlManager & manager_);
SettingsConstraints(const AccessControl & access_control_);
SettingsConstraints(const SettingsConstraints & src);
SettingsConstraints & operator =(const SettingsConstraints & src);
SettingsConstraints(SettingsConstraints && src);
@ -109,7 +109,7 @@ private:
const Constraint * tryGetConstraint(const std::string_view & setting_name) const;
std::unordered_map<std::string_view, Constraint> constraints;
const AccessControlManager * manager = nullptr;
const AccessControl * access_control = nullptr;
};
}

View File

@ -15,7 +15,7 @@ struct SettingsConstraintsAndProfileIDs
std::vector<UUID> current_profiles;
std::vector<UUID> enabled_profiles;
SettingsConstraintsAndProfileIDs(const AccessControlManager & manager_) : constraints(manager_) {}
SettingsConstraintsAndProfileIDs(const AccessControl & access_control_) : constraints(access_control_) {}
};
}

View File

@ -1,6 +1,6 @@
#include <Access/SettingsProfileElement.h>
#include <Access/SettingsConstraints.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/SettingsProfile.h>
#include <Core/Settings.h>
#include <Common/SettingsChanges.h>
@ -17,19 +17,19 @@ SettingsProfileElement::SettingsProfileElement(const ASTSettingsProfileElement &
init(ast, nullptr);
}
SettingsProfileElement::SettingsProfileElement(const ASTSettingsProfileElement & ast, const AccessControlManager & manager)
SettingsProfileElement::SettingsProfileElement(const ASTSettingsProfileElement & ast, const AccessControl & access_control)
{
init(ast, &manager);
init(ast, &access_control);
}
void SettingsProfileElement::init(const ASTSettingsProfileElement & ast, const AccessControlManager * manager)
void SettingsProfileElement::init(const ASTSettingsProfileElement & ast, const AccessControl * access_control)
{
auto name_to_id = [id_mode{ast.id_mode}, manager](const String & name_) -> UUID
auto name_to_id = [id_mode{ast.id_mode}, access_control](const String & name_) -> UUID
{
if (id_mode)
return parse<UUID>(name_);
assert(manager);
return manager->getID<SettingsProfile>(name_);
assert(access_control);
return access_control->getID<SettingsProfile>(name_);
};
if (!ast.parent_profile.empty())
@ -40,8 +40,8 @@ void SettingsProfileElement::init(const ASTSettingsProfileElement & ast, const A
setting_name = ast.setting_name;
/// Optionally check if a setting with that name is allowed.
if (manager)
manager->checkSettingNameIsAllowed(setting_name);
if (access_control)
access_control->checkSettingNameIsAllowed(setting_name);
value = ast.value;
min_value = ast.min_value;
@ -76,13 +76,13 @@ std::shared_ptr<ASTSettingsProfileElement> SettingsProfileElement::toAST() const
}
std::shared_ptr<ASTSettingsProfileElement> SettingsProfileElement::toASTWithNames(const AccessControlManager & manager) const
std::shared_ptr<ASTSettingsProfileElement> SettingsProfileElement::toASTWithNames(const AccessControl & access_control) const
{
auto ast = std::make_shared<ASTSettingsProfileElement>();
if (parent_profile)
{
auto parent_profile_name = manager.tryReadName(*parent_profile);
auto parent_profile_name = access_control.tryReadName(*parent_profile);
if (parent_profile_name)
ast->parent_profile = *parent_profile_name;
}
@ -103,10 +103,10 @@ SettingsProfileElements::SettingsProfileElements(const ASTSettingsProfileElement
emplace_back(*ast_element);
}
SettingsProfileElements::SettingsProfileElements(const ASTSettingsProfileElements & ast, const AccessControlManager & manager)
SettingsProfileElements::SettingsProfileElements(const ASTSettingsProfileElements & ast, const AccessControl & access_control)
{
for (const auto & ast_element : ast.elements)
emplace_back(*ast_element, manager);
emplace_back(*ast_element, access_control);
}
@ -118,11 +118,11 @@ std::shared_ptr<ASTSettingsProfileElements> SettingsProfileElements::toAST() con
return res;
}
std::shared_ptr<ASTSettingsProfileElements> SettingsProfileElements::toASTWithNames(const AccessControlManager & manager) const
std::shared_ptr<ASTSettingsProfileElements> SettingsProfileElements::toASTWithNames(const AccessControl & access_control) const
{
auto res = std::make_shared<ASTSettingsProfileElements>();
for (const auto & element : *this)
res->elements.push_back(element.toASTWithNames(manager));
res->elements.push_back(element.toASTWithNames(access_control));
return res;
}
@ -155,9 +155,9 @@ SettingsChanges SettingsProfileElements::toSettingsChanges() const
return res;
}
SettingsConstraints SettingsProfileElements::toSettingsConstraints(const AccessControlManager & manager) const
SettingsConstraints SettingsProfileElements::toSettingsConstraints(const AccessControl & access_control) const
{
SettingsConstraints res{manager};
SettingsConstraints res{access_control};
for (const auto & elem : *this)
{
if (!elem.setting_name.empty())

View File

@ -13,7 +13,7 @@ class SettingsChanges;
class SettingsConstraints;
class ASTSettingsProfileElement;
class ASTSettingsProfileElements;
class AccessControlManager;
class AccessControl;
struct SettingsProfileElement
@ -35,14 +35,14 @@ struct SettingsProfileElement
SettingsProfileElement() {}
/// The constructor from AST requires the AccessControlManager if `ast.id_mode == false`.
/// The constructor from AST requires the AccessControl if `ast.id_mode == false`.
SettingsProfileElement(const ASTSettingsProfileElement & ast);
SettingsProfileElement(const ASTSettingsProfileElement & ast, const AccessControlManager & manager);
SettingsProfileElement(const ASTSettingsProfileElement & ast, const AccessControl & access_control);
std::shared_ptr<ASTSettingsProfileElement> toAST() const;
std::shared_ptr<ASTSettingsProfileElement> toASTWithNames(const AccessControlManager & manager) const;
std::shared_ptr<ASTSettingsProfileElement> toASTWithNames(const AccessControl & access_control) const;
private:
void init(const ASTSettingsProfileElement & ast, const AccessControlManager * manager);
void init(const ASTSettingsProfileElement & ast, const AccessControl * access_control);
};
@ -51,17 +51,17 @@ class SettingsProfileElements : public std::vector<SettingsProfileElement>
public:
SettingsProfileElements() {}
/// The constructor from AST requires the AccessControlManager if `ast.id_mode == false`.
/// The constructor from AST requires the AccessControl if `ast.id_mode == false`.
SettingsProfileElements(const ASTSettingsProfileElements & ast);
SettingsProfileElements(const ASTSettingsProfileElements & ast, const AccessControlManager & manager);
SettingsProfileElements(const ASTSettingsProfileElements & ast, const AccessControl & access_control);
std::shared_ptr<ASTSettingsProfileElements> toAST() const;
std::shared_ptr<ASTSettingsProfileElements> toASTWithNames(const AccessControlManager & manager) const;
std::shared_ptr<ASTSettingsProfileElements> toASTWithNames(const AccessControl & access_control) const;
void merge(const SettingsProfileElements & other);
Settings toSettings() const;
SettingsChanges toSettingsChanges() const;
SettingsConstraints toSettingsConstraints(const AccessControlManager & manager) const;
SettingsConstraints toSettingsConstraints(const AccessControl & access_control) const;
std::vector<UUID> toProfileIDs() const;
};

View File

@ -1,5 +1,5 @@
#include <Access/SettingsProfilesCache.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/SettingsProfile.h>
#include <Access/SettingsProfilesInfo.h>
#include <Common/quoteString.h>
@ -12,8 +12,8 @@ namespace ErrorCodes
extern const int THERE_IS_NO_PROFILE;
}
SettingsProfilesCache::SettingsProfilesCache(const AccessControlManager & manager_)
: manager(manager_) {}
SettingsProfilesCache::SettingsProfilesCache(const AccessControl & access_control_)
: access_control(access_control_) {}
SettingsProfilesCache::~SettingsProfilesCache() = default;
@ -25,7 +25,7 @@ void SettingsProfilesCache::ensureAllProfilesRead()
return;
all_profiles_read = true;
subscription = manager.subscribeForChanges<SettingsProfile>(
subscription = access_control.subscribeForChanges<SettingsProfile>(
[&](const UUID & id, const AccessEntityPtr & entity)
{
if (entity)
@ -34,9 +34,9 @@ void SettingsProfilesCache::ensureAllProfilesRead()
profileRemoved(id);
});
for (const UUID & id : manager.findAll<SettingsProfile>())
for (const UUID & id : access_control.findAll<SettingsProfile>())
{
auto profile = manager.tryRead<SettingsProfile>(id);
auto profile = access_control.tryRead<SettingsProfile>(id);
if (profile)
{
all_profiles.emplace(id, profile);
@ -138,11 +138,11 @@ void SettingsProfilesCache::mergeSettingsAndConstraintsFor(EnabledSettings & ena
merged_settings.merge(enabled.params.settings_from_enabled_roles);
merged_settings.merge(enabled.params.settings_from_user);
auto info = std::make_shared<SettingsProfilesInfo>(manager);
auto info = std::make_shared<SettingsProfilesInfo>(access_control);
info->profiles = enabled.params.settings_from_user.toProfileIDs();
substituteProfiles(merged_settings, info->profiles_with_implicit, info->names_of_profiles);
info->settings = merged_settings.toSettingsChanges();
info->constraints = merged_settings.toSettingsConstraints(manager);
info->constraints = merged_settings.toSettingsConstraints(access_control);
enabled.setInfo(std::move(info));
}
@ -225,13 +225,13 @@ std::shared_ptr<const SettingsProfilesInfo> SettingsProfilesCache::getSettingsPr
SettingsProfileElements elements = all_profiles[profile_id]->elements;
auto info = std::make_shared<SettingsProfilesInfo>(manager);
auto info = std::make_shared<SettingsProfilesInfo>(access_control);
info->profiles.push_back(profile_id);
info->profiles_with_implicit.push_back(profile_id);
substituteProfiles(elements, info->profiles_with_implicit, info->names_of_profiles);
info->settings = elements.toSettingsChanges();
info->constraints.merge(elements.toSettingsConstraints(manager));
info->constraints.merge(elements.toSettingsConstraints(access_control));
profile_infos_cache.add(profile_id, info);
return info;

View File

@ -9,7 +9,7 @@
namespace DB
{
class AccessControlManager;
class AccessControl;
struct SettingsProfile;
using SettingsProfilePtr = std::shared_ptr<const SettingsProfile>;
struct SettingsProfilesInfo;
@ -18,7 +18,7 @@ struct SettingsProfilesInfo;
class SettingsProfilesCache
{
public:
SettingsProfilesCache(const AccessControlManager & manager_);
SettingsProfilesCache(const AccessControl & access_control_);
~SettingsProfilesCache();
void setDefaultProfileName(const String & default_profile_name);
@ -39,7 +39,7 @@ private:
void mergeSettingsAndConstraintsFor(EnabledSettings & enabled) const;
void substituteProfiles(SettingsProfileElements & elements, std::vector<UUID> & substituted_profiles, std::unordered_map<UUID, String> & names_of_substituted_profiles) const;
const AccessControlManager & manager;
const AccessControl & access_control;
std::unordered_map<UUID, SettingsProfilePtr> all_profiles;
std::unordered_map<String, UUID> profiles_by_name;
bool all_profiles_read = false;

View File

@ -29,7 +29,7 @@ bool operator==(const SettingsProfilesInfo & lhs, const SettingsProfilesInfo & r
std::shared_ptr<const SettingsConstraintsAndProfileIDs>
SettingsProfilesInfo::getConstraintsAndProfileIDs(const std::shared_ptr<const SettingsConstraintsAndProfileIDs> & previous) const
{
auto res = std::make_shared<SettingsConstraintsAndProfileIDs>(manager);
auto res = std::make_shared<SettingsConstraintsAndProfileIDs>(access_control);
res->current_profiles = profiles;
if (previous)

View File

@ -29,7 +29,7 @@ struct SettingsProfilesInfo
/// Names of all the profiles in `profiles`.
std::unordered_map<UUID, String> names_of_profiles;
SettingsProfilesInfo(const AccessControlManager & manager_) : constraints(manager_), manager(manager_) {}
SettingsProfilesInfo(const AccessControl & access_control_) : constraints(access_control_), access_control(access_control_) {}
std::shared_ptr<const SettingsConstraintsAndProfileIDs> getConstraintsAndProfileIDs(
const std::shared_ptr<const SettingsConstraintsAndProfileIDs> & previous = nullptr) const;
@ -47,7 +47,7 @@ struct SettingsProfilesInfo
}
private:
const AccessControlManager & manager;
const AccessControl & access_control;
};
}

View File

@ -1,7 +1,7 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Interpreters/Context.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/User.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
@ -35,7 +35,7 @@ namespace
explicit FunctionCurrentProfiles(const ContextPtr & context)
{
const auto & manager = context->getAccessControlManager();
const auto & manager = context->getAccessControl();
std::vector<UUID> profile_ids;
if constexpr (kind == Kind::CURRENT_PROFILES)

View File

@ -1,7 +1,7 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Interpreters/Context.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/EnabledRolesInfo.h>
#include <Access/User.h>
#include <Columns/ColumnArray.h>
@ -47,7 +47,7 @@ namespace
else
{
static_assert(kind == Kind::DEFAULT_ROLES);
const auto & manager = context->getAccessControlManager();
const auto & manager = context->getAccessControl();
if (auto user = context->getUser())
role_names = manager.tryReadNames(user->granted_roles.findGranted(user->default_roles));
}

View File

@ -1,7 +1,7 @@
#include <Interpreters/Access/InterpreterCreateQuotaQuery.h>
#include <Parsers/Access/ASTCreateQuotaQuery.h>
#include <Parsers/Access/ASTRolesOrUsersSet.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/Common/AccessFlags.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
@ -73,7 +73,7 @@ namespace
BlockIO InterpreterCreateQuotaQuery::execute()
{
auto & query = query_ptr->as<ASTCreateQuotaQuery &>();
auto & access_control = getContext()->getAccessControlManager();
auto & access_control = getContext()->getAccessControl();
getContext()->checkAccess(query.alter ? AccessType::ALTER_QUOTA : AccessType::CREATE_QUOTA);
if (!query.cluster.empty())

View File

@ -1,6 +1,6 @@
#include <Interpreters/Access/InterpreterCreateRoleQuery.h>
#include <Parsers/Access/ASTCreateRoleQuery.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/Role.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
@ -34,7 +34,7 @@ namespace
BlockIO InterpreterCreateRoleQuery::execute()
{
const auto & query = query_ptr->as<const ASTCreateRoleQuery &>();
auto & access_control = getContext()->getAccessControlManager();
auto & access_control = getContext()->getAccessControl();
if (query.alter)
getContext()->checkAccess(AccessType::ALTER_ROLE);
else

View File

@ -3,7 +3,7 @@
#include <Parsers/Access/ASTRowPolicyName.h>
#include <Parsers/Access/ASTRolesOrUsersSet.h>
#include <Parsers/formatAST.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/Common/AccessFlags.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
@ -44,7 +44,7 @@ namespace
BlockIO InterpreterCreateRowPolicyQuery::execute()
{
auto & query = query_ptr->as<ASTCreateRowPolicyQuery &>();
auto & access_control = getContext()->getAccessControlManager();
auto & access_control = getContext()->getAccessControl();
getContext()->checkAccess(query.alter ? AccessType::ALTER_ROW_POLICY : AccessType::CREATE_ROW_POLICY);
if (!query.cluster.empty())

View File

@ -1,7 +1,7 @@
#include <Interpreters/Access/InterpreterCreateSettingsProfileQuery.h>
#include <Parsers/Access/ASTCreateSettingsProfileQuery.h>
#include <Parsers/Access/ASTRolesOrUsersSet.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/SettingsProfile.h>
#include <Access/Common/AccessFlags.h>
#include <Interpreters/Context.h>
@ -42,7 +42,7 @@ namespace
BlockIO InterpreterCreateSettingsProfileQuery::execute()
{
auto & query = query_ptr->as<ASTCreateSettingsProfileQuery &>();
auto & access_control = getContext()->getAccessControlManager();
auto & access_control = getContext()->getAccessControl();
if (query.alter)
getContext()->checkAccess(AccessType::ALTER_SETTINGS_PROFILE);
else

View File

@ -3,7 +3,7 @@
#include <Parsers/Access/ASTRolesOrUsersSet.h>
#include <Parsers/Access/ASTUserNameWithHost.h>
#include <Parsers/ASTDatabaseOrNone.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/ContextAccess.h>
#include <Access/User.h>
#include <Interpreters/Access/InterpreterSetRoleQuery.h>
@ -79,7 +79,7 @@ namespace
BlockIO InterpreterCreateUserQuery::execute()
{
const auto & query = query_ptr->as<const ASTCreateUserQuery &>();
auto & access_control = getContext()->getAccessControlManager();
auto & access_control = getContext()->getAccessControl();
auto access = getContext()->getAccess();
access->checkAccess(query.alter ? AccessType::ALTER_USER : AccessType::CREATE_USER);

View File

@ -1,7 +1,7 @@
#include <Interpreters/Access/InterpreterDropAccessEntityQuery.h>
#include <Parsers/Access/ASTDropAccessEntityQuery.h>
#include <Parsers/Access/ASTRowPolicyName.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/Common/AccessFlags.h>
#include <Access/User.h>
#include <Access/Role.h>
@ -25,7 +25,7 @@ using EntityType = IAccessEntity::Type;
BlockIO InterpreterDropAccessEntityQuery::execute()
{
auto & query = query_ptr->as<ASTDropAccessEntityQuery &>();
auto & access_control = getContext()->getAccessControlManager();
auto & access_control = getContext()->getAccessControl();
getContext()->checkAccess(getRequiredAccess());
if (!query.cluster.empty())

View File

@ -1,7 +1,7 @@
#include <Interpreters/Access/InterpreterGrantQuery.h>
#include <Parsers/Access/ASTGrantQuery.h>
#include <Parsers/Access/ASTRolesOrUsersSet.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/ContextAccess.h>
#include <Access/Role.h>
#include <Access/RolesOrUsersSet.h>
@ -11,6 +11,7 @@
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <boost/range/algorithm/copy.hpp>
#include <boost/range/algorithm/set_algorithm.hpp>
#include <boost/range/algorithm_ext/erase.hpp>
namespace DB
{
@ -51,7 +52,7 @@ namespace
/// Extracts roles which are going to be granted or revoked from a query.
void collectRolesToGrantOrRevoke(
const AccessControlManager & access_control,
const AccessControl & access_control,
const ASTGrantQuery & query,
std::vector<UUID> & roles_to_grant,
RolesOrUsersSet & roles_to_revoke)
@ -121,7 +122,7 @@ namespace
}
/// Checks if grantees are allowed for the current user, throws an exception if not.
void checkGranteesAreAllowed(const AccessControlManager & access_control, const ContextAccess & current_user_access, const std::vector<UUID> & grantee_ids)
void checkGranteesAreAllowed(const AccessControl & access_control, const ContextAccess & current_user_access, const std::vector<UUID> & grantee_ids)
{
auto current_user = current_user_access.getUser();
if (!current_user || (current_user->grantees == RolesOrUsersSet::AllTag{}))
@ -139,7 +140,7 @@ namespace
/// Checks if the current user has enough access rights granted with grant option to grant or revoke specified access rights.
void checkGrantOption(
const AccessControlManager & access_control,
const AccessControl & access_control,
const ContextAccess & current_user_access,
const std::vector<UUID> & grantees_from_query,
bool & need_check_grantees_are_allowed,
@ -205,7 +206,7 @@ namespace
/// Checks if the current user has enough roles granted with admin option to grant or revoke specified roles.
void checkAdminOption(
const AccessControlManager & access_control,
const AccessControl & access_control,
const ContextAccess & current_user_access,
const std::vector<UUID> & grantees_from_query,
bool & need_check_grantees_are_allowed,
@ -382,7 +383,7 @@ BlockIO InterpreterGrantQuery::execute()
if (!query.access_rights_elements.empty() && query.access_rights_elements[0].is_partial_revoke && !query.is_revoke)
throw Exception("A partial revoke should be revoked, not granted", ErrorCodes::LOGICAL_ERROR);
auto & access_control = getContext()->getAccessControlManager();
auto & access_control = getContext()->getAccessControl();
auto current_user_access = getContext()->getAccess();
std::vector<UUID> grantees = RolesOrUsersSet{*query.grantees, access_control, getContext()->getUserID()}.getMatchingIDs(access_control);

View File

@ -2,7 +2,7 @@
#include <Parsers/Access/ASTSetRoleQuery.h>
#include <Parsers/Access/ASTRolesOrUsersSet.h>
#include <Access/RolesOrUsersSet.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/User.h>
#include <Interpreters/Context.h>
@ -28,7 +28,7 @@ BlockIO InterpreterSetRoleQuery::execute()
void InterpreterSetRoleQuery::setRole(const ASTSetRoleQuery & query)
{
auto & access_control = getContext()->getAccessControlManager();
auto & access_control = getContext()->getAccessControl();
auto session_context = getContext()->getSessionContext();
auto user = session_context->getUser();
@ -62,7 +62,7 @@ void InterpreterSetRoleQuery::setDefaultRole(const ASTSetRoleQuery & query)
{
getContext()->checkAccess(AccessType::ALTER_USER);
auto & access_control = getContext()->getAccessControlManager();
auto & access_control = getContext()->getAccessControl();
std::vector<UUID> to_users = RolesOrUsersSet{*query.to_users, access_control, getContext()->getUserID()}.getMatchingIDs(access_control);
RolesOrUsersSet roles_from_query{*query.roles, access_control};

View File

@ -8,7 +8,7 @@
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <DataTypes/DataTypeString.h>
#include <Access/Common/AccessFlags.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <base/range.h>
#include <boost/range/algorithm/sort.hpp>
#include <boost/range/algorithm_ext/push_back.hpp>
@ -49,7 +49,7 @@ QueryPipeline InterpreterShowAccessQuery::executeImpl() const
std::vector<AccessEntityPtr> InterpreterShowAccessQuery::getEntities() const
{
const auto & access_control = getContext()->getAccessControlManager();
const auto & access_control = getContext()->getAccessControl();
getContext()->checkAccess(AccessType::SHOW_ACCESS);
std::vector<AccessEntityPtr> entities;
@ -71,7 +71,7 @@ std::vector<AccessEntityPtr> InterpreterShowAccessQuery::getEntities() const
ASTs InterpreterShowAccessQuery::getCreateAndGrantQueries() const
{
auto entities = getEntities();
const auto & access_control = getContext()->getAccessControlManager();
const auto & access_control = getContext()->getAccessControl();
ASTs create_queries, grant_queries;
for (const auto & entity : entities)

View File

@ -12,7 +12,7 @@
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/EnabledQuota.h>
#include <Access/QuotaUsage.h>
#include <Access/User.h>
@ -40,7 +40,7 @@ namespace
{
ASTPtr getCreateQueryImpl(
const User & user,
const AccessControlManager * manager /* not used if attach_mode == true */,
const AccessControl * access_control /* not used if attach_mode == true */,
bool attach_mode)
{
auto query = std::make_shared<ASTCreateUserQuery>();
@ -56,7 +56,7 @@ namespace
if (attach_mode)
query->default_roles = user.default_roles.toAST();
else
query->default_roles = user.default_roles.toASTWithNames(*manager);
query->default_roles = user.default_roles.toASTWithNames(*access_control);
}
if (user.auth_data.getType() != AuthenticationType::NO_PASSWORD)
@ -70,7 +70,7 @@ namespace
if (attach_mode)
query->settings = user.settings.toAST();
else
query->settings = user.settings.toASTWithNames(*manager);
query->settings = user.settings.toASTWithNames(*access_control);
}
if (user.grantees != RolesOrUsersSet::AllTag{})
@ -78,7 +78,7 @@ namespace
if (attach_mode)
query->grantees = user.grantees.toAST();
else
query->grantees = user.grantees.toASTWithNames(*manager);
query->grantees = user.grantees.toASTWithNames(*access_control);
query->grantees->use_keyword_any = true;
}
@ -93,7 +93,7 @@ namespace
}
ASTPtr getCreateQueryImpl(const Role & role, const AccessControlManager * manager, bool attach_mode)
ASTPtr getCreateQueryImpl(const Role & role, const AccessControl * access_control, bool attach_mode)
{
auto query = std::make_shared<ASTCreateRoleQuery>();
query->names.emplace_back(role.getName());
@ -104,14 +104,14 @@ namespace
if (attach_mode)
query->settings = role.settings.toAST();
else
query->settings = role.settings.toASTWithNames(*manager);
query->settings = role.settings.toASTWithNames(*access_control);
}
return query;
}
ASTPtr getCreateQueryImpl(const SettingsProfile & profile, const AccessControlManager * manager, bool attach_mode)
ASTPtr getCreateQueryImpl(const SettingsProfile & profile, const AccessControl * access_control, bool attach_mode)
{
auto query = std::make_shared<ASTCreateSettingsProfileQuery>();
query->names.emplace_back(profile.getName());
@ -122,7 +122,7 @@ namespace
if (attach_mode)
query->settings = profile.elements.toAST();
else
query->settings = profile.elements.toASTWithNames(*manager);
query->settings = profile.elements.toASTWithNames(*access_control);
if (query->settings)
query->settings->setUseInheritKeyword(true);
}
@ -132,7 +132,7 @@ namespace
if (attach_mode)
query->to_roles = profile.to_roles.toAST();
else
query->to_roles = profile.to_roles.toASTWithNames(*manager);
query->to_roles = profile.to_roles.toASTWithNames(*access_control);
}
return query;
@ -141,7 +141,7 @@ namespace
ASTPtr getCreateQueryImpl(
const Quota & quota,
const AccessControlManager * manager /* not used if attach_mode == true */,
const AccessControl * access_control /* not used if attach_mode == true */,
bool attach_mode)
{
auto query = std::make_shared<ASTCreateQuotaQuery>();
@ -168,7 +168,7 @@ namespace
if (attach_mode)
query->roles = quota.to_roles.toAST();
else
query->roles = quota.to_roles.toASTWithNames(*manager);
query->roles = quota.to_roles.toASTWithNames(*access_control);
}
return query;
@ -177,7 +177,7 @@ namespace
ASTPtr getCreateQueryImpl(
const RowPolicy & policy,
const AccessControlManager * manager /* not used if attach_mode == true */,
const AccessControl * access_control /* not used if attach_mode == true */,
bool attach_mode)
{
auto query = std::make_shared<ASTCreateRowPolicyQuery>();
@ -204,7 +204,7 @@ namespace
if (attach_mode)
query->roles = policy.to_roles.toAST();
else
query->roles = policy.to_roles.toASTWithNames(*manager);
query->roles = policy.to_roles.toASTWithNames(*access_control);
}
return query;
@ -212,19 +212,19 @@ namespace
ASTPtr getCreateQueryImpl(
const IAccessEntity & entity,
const AccessControlManager * manager /* not used if attach_mode == true */,
const AccessControl * access_control /* not used if attach_mode == true */,
bool attach_mode)
{
if (const User * user = typeid_cast<const User *>(&entity))
return getCreateQueryImpl(*user, manager, attach_mode);
return getCreateQueryImpl(*user, access_control, attach_mode);
if (const Role * role = typeid_cast<const Role *>(&entity))
return getCreateQueryImpl(*role, manager, attach_mode);
return getCreateQueryImpl(*role, access_control, attach_mode);
if (const RowPolicy * policy = typeid_cast<const RowPolicy *>(&entity))
return getCreateQueryImpl(*policy, manager, attach_mode);
return getCreateQueryImpl(*policy, access_control, attach_mode);
if (const Quota * quota = typeid_cast<const Quota *>(&entity))
return getCreateQueryImpl(*quota, manager, attach_mode);
return getCreateQueryImpl(*quota, access_control, attach_mode);
if (const SettingsProfile * profile = typeid_cast<const SettingsProfile *>(&entity))
return getCreateQueryImpl(*profile, manager, attach_mode);
return getCreateQueryImpl(*profile, access_control, attach_mode);
throw Exception(entity.outputTypeAndName() + ": type is not supported by SHOW CREATE query", ErrorCodes::NOT_IMPLEMENTED);
}
@ -277,7 +277,7 @@ QueryPipeline InterpreterShowCreateAccessEntityQuery::executeImpl()
std::vector<AccessEntityPtr> InterpreterShowCreateAccessEntityQuery::getEntities() const
{
auto & show_query = query_ptr->as<ASTShowCreateAccessEntityQuery &>();
const auto & access_control = getContext()->getAccessControlManager();
const auto & access_control = getContext()->getAccessControl();
getContext()->checkAccess(getRequiredAccess());
show_query.replaceEmptyDatabase(getContext()->getCurrentDatabase());
std::vector<AccessEntityPtr> entities;
@ -348,7 +348,7 @@ ASTs InterpreterShowCreateAccessEntityQuery::getCreateQueries() const
auto entities = getEntities();
ASTs list;
const auto & access_control = getContext()->getAccessControlManager();
const auto & access_control = getContext()->getAccessControl();
for (const auto & entity : entities)
list.push_back(getCreateQuery(*entity, access_control));
@ -356,7 +356,7 @@ ASTs InterpreterShowCreateAccessEntityQuery::getCreateQueries() const
}
ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuery(const IAccessEntity & entity, const AccessControlManager & access_control)
ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuery(const IAccessEntity & entity, const AccessControl & access_control)
{
return getCreateQueryImpl(entity, &access_control, false);
}

View File

@ -7,7 +7,7 @@
namespace DB
{
class AccessControlManager;
class AccessControl;
class Context;
class AccessRightsElements;
struct IAccessEntity;
@ -26,7 +26,7 @@ public:
bool ignoreQuota() const override { return true; }
bool ignoreLimits() const override { return true; }
static ASTPtr getCreateQuery(const IAccessEntity & entity, const AccessControlManager & access_control);
static ASTPtr getCreateQuery(const IAccessEntity & entity, const AccessControl & access_control);
static ASTPtr getAttachQuery(const IAccessEntity & entity);
private:

View File

@ -3,7 +3,7 @@
#include <Parsers/Access/ASTRolesOrUsersSet.h>
#include <Parsers/Access/ASTShowGrantsQuery.h>
#include <Parsers/formatAST.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/Role.h>
#include <Access/RolesOrUsersSet.h>
#include <Access/User.h>
@ -27,7 +27,7 @@ namespace
template <typename T>
ASTs getGrantQueriesImpl(
const T & grantee,
const AccessControlManager * manager /* not used if attach_mode == true */,
const AccessControl * access_control /* not used if attach_mode == true */,
bool attach_mode = false)
{
ASTs res;
@ -75,7 +75,7 @@ namespace
if (attach_mode)
grant_query->roles = RolesOrUsersSet{element.ids}.toAST();
else
grant_query->roles = RolesOrUsersSet{element.ids}.toASTWithNames(*manager);
grant_query->roles = RolesOrUsersSet{element.ids}.toASTWithNames(*access_control);
res.push_back(std::move(grant_query));
}
@ -84,13 +84,13 @@ namespace
ASTs getGrantQueriesImpl(
const IAccessEntity & entity,
const AccessControlManager * manager /* not used if attach_mode == true */,
const AccessControl * access_control /* 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);
return getGrantQueriesImpl(*user, access_control, attach_mode);
if (const Role * role = typeid_cast<const Role *>(&entity))
return getGrantQueriesImpl(*role, manager, attach_mode);
return getGrantQueriesImpl(*role, access_control, attach_mode);
throw Exception(entity.outputTypeAndName() + " is expected to be user or role", ErrorCodes::LOGICAL_ERROR);
}
@ -136,7 +136,7 @@ QueryPipeline InterpreterShowGrantsQuery::executeImpl()
std::vector<AccessEntityPtr> InterpreterShowGrantsQuery::getEntities() const
{
const auto & show_query = query_ptr->as<ASTShowGrantsQuery &>();
const auto & access_control = getContext()->getAccessControlManager();
const auto & access_control = getContext()->getAccessControl();
auto ids = RolesOrUsersSet{*show_query.for_roles, access_control, getContext()->getUserID()}.getMatchingIDs(access_control);
std::vector<AccessEntityPtr> entities;
@ -155,7 +155,7 @@ std::vector<AccessEntityPtr> InterpreterShowGrantsQuery::getEntities() const
ASTs InterpreterShowGrantsQuery::getGrantQueries() const
{
auto entities = getEntities();
const auto & access_control = getContext()->getAccessControlManager();
const auto & access_control = getContext()->getAccessControl();
ASTs grant_queries;
for (const auto & entity : entities)
@ -165,7 +165,7 @@ ASTs InterpreterShowGrantsQuery::getGrantQueries() const
}
ASTs InterpreterShowGrantsQuery::getGrantQueries(const IAccessEntity & user_or_role, const AccessControlManager & access_control)
ASTs InterpreterShowGrantsQuery::getGrantQueries(const IAccessEntity & user_or_role, const AccessControl & access_control)
{
return getGrantQueriesImpl(user_or_role, &access_control, false);
}

View File

@ -8,7 +8,7 @@
namespace DB
{
class AccessControlManager;
class AccessControl;
class ASTShowGrantsQuery;
struct IAccessEntity;
using AccessEntityPtr = std::shared_ptr<const IAccessEntity>;
@ -20,7 +20,7 @@ public:
BlockIO execute() override;
static ASTs getGrantQueries(const IAccessEntity & user_or_role, const AccessControlManager & access_control);
static ASTs getGrantQueries(const IAccessEntity & user_or_role, const AccessControl & access_control);
static ASTs getAttachGrantQueries(const IAccessEntity & user_or_role);
bool ignoreQuota() const override { return true; }

View File

@ -34,7 +34,7 @@
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
#include <Core/Settings.h>
#include <Core/SettingsQuirks.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/ContextAccess.h>
#include <Access/EnabledRolesInfo.h>
#include <Access/EnabledRowPolicies.h>
@ -206,7 +206,7 @@ struct ContextSharedPart
String default_profile_name; /// Default profile name used for default values.
String system_profile_name; /// Profile used by system processes
String buffer_profile_name; /// Profile used by Buffer engine for flushing to the underlying
std::unique_ptr<AccessControlManager> access_control_manager;
std::unique_ptr<AccessControl> access_control;
mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks.
mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files.
mutable UncompressedCachePtr index_uncompressed_cache; /// The cache of decompressed blocks for MergeTree indices.
@ -279,7 +279,7 @@ struct ContextSharedPart
Context::ConfigReloadCallback config_reload_callback;
ContextSharedPart()
: access_control_manager(std::make_unique<AccessControlManager>()), macros(std::make_unique<Macros>())
: access_control(std::make_unique<AccessControl>()), macros(std::make_unique<Macros>())
{
/// TODO: make it singleton (?)
static std::atomic<size_t> num_calls{0};
@ -371,7 +371,7 @@ struct ContextSharedPart
distributed_schedule_pool.reset();
message_broker_schedule_pool.reset();
ddl_worker.reset();
access_control_manager.reset();
access_control.reset();
/// Stop trace collector if any
trace_collector.reset();
@ -635,7 +635,7 @@ void Context::setConfig(const ConfigurationPtr & config)
{
auto lock = getLock();
shared->config = config;
shared->access_control_manager->setExternalAuthenticatorsConfig(*shared->config);
shared->access_control->setExternalAuthenticatorsConfig(*shared->config);
}
const Poco::Util::AbstractConfiguration & Context::getConfigRef() const
@ -645,33 +645,33 @@ const Poco::Util::AbstractConfiguration & Context::getConfigRef() const
}
AccessControlManager & Context::getAccessControlManager()
AccessControl & Context::getAccessControl()
{
return *shared->access_control_manager;
return *shared->access_control;
}
const AccessControlManager & Context::getAccessControlManager() const
const AccessControl & Context::getAccessControl() const
{
return *shared->access_control_manager;
return *shared->access_control;
}
void Context::setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config)
{
auto lock = getLock();
shared->access_control_manager->setExternalAuthenticatorsConfig(config);
shared->access_control->setExternalAuthenticatorsConfig(config);
}
std::unique_ptr<GSSAcceptorContext> Context::makeGSSAcceptorContext() const
{
auto lock = getLock();
return std::make_unique<GSSAcceptorContext>(shared->access_control_manager->getExternalAuthenticators().getKerberosParams());
return std::make_unique<GSSAcceptorContext>(shared->access_control->getExternalAuthenticators().getKerberosParams());
}
void Context::setUsersConfig(const ConfigurationPtr & config)
{
auto lock = getLock();
shared->users_config = config;
shared->access_control_manager->setUsersConfig(*shared->users_config);
shared->access_control->setUsersConfig(*shared->users_config);
}
ConfigurationPtr Context::getUsersConfig()
@ -686,7 +686,7 @@ void Context::setUser(const UUID & user_id_)
user_id = user_id_;
access = getAccessControlManager().getContextAccess(
access = getAccessControl().getContextAccess(
user_id_, /* current_roles = */ {}, /* use_default_roles = */ true, settings, current_database, client_info);
auto user = access->getUser();
@ -759,7 +759,7 @@ void Context::calculateAccessRights()
{
auto lock = getLock();
if (user_id)
access = getAccessControlManager().getContextAccess(
access = getAccessControl().getContextAccess(
*user_id,
current_roles ? *current_roles : std::vector<UUID>{},
/* use_default_roles = */ false,
@ -808,10 +808,10 @@ void Context::setInitialRowPolicy()
initial_row_policy = nullptr;
if (client_info.initial_user == client_info.current_user)
return;
auto initial_user_id = getAccessControlManager().find<User>(client_info.initial_user);
auto initial_user_id = getAccessControl().find<User>(client_info.initial_user);
if (!initial_user_id)
return;
initial_row_policy = getAccessControlManager().getEnabledRowPolicies(*initial_user_id, {});
initial_row_policy = getAccessControl().getEnabledRowPolicies(*initial_user_id, {});
}
@ -832,7 +832,7 @@ void Context::setCurrentProfile(const String & profile_name)
auto lock = getLock();
try
{
UUID profile_id = getAccessControlManager().getID<SettingsProfile>(profile_name);
UUID profile_id = getAccessControl().getID<SettingsProfile>(profile_name);
setCurrentProfile(profile_id);
}
catch (Exception & e)
@ -845,7 +845,7 @@ void Context::setCurrentProfile(const String & profile_name)
void Context::setCurrentProfile(const UUID & profile_id)
{
auto lock = getLock();
auto profile_info = getAccessControlManager().getSettingsProfileInfo(profile_id);
auto profile_info = getAccessControl().getSettingsProfileInfo(profile_id);
checkSettingsConstraints(profile_info->settings);
applySettingsChanges(profile_info->settings);
settings_constraints_and_current_profiles = profile_info->getConstraintsAndProfileIDs(settings_constraints_and_current_profiles);
@ -1153,7 +1153,7 @@ std::shared_ptr<const SettingsConstraintsAndProfileIDs> Context::getSettingsCons
auto lock = getLock();
if (settings_constraints_and_current_profiles)
return settings_constraints_and_current_profiles;
static auto no_constraints_or_profiles = std::make_shared<SettingsConstraintsAndProfileIDs>(getAccessControlManager());
static auto no_constraints_or_profiles = std::make_shared<SettingsConstraintsAndProfileIDs>(getAccessControl());
return no_constraints_or_profiles;
}
@ -2641,7 +2641,7 @@ void Context::setApplicationType(ApplicationType type)
void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & config)
{
shared->default_profile_name = config.getString("default_profile", "default");
getAccessControlManager().setDefaultProfileName(shared->default_profile_name);
getAccessControl().setDefaultProfileName(shared->default_profile_name);
shared->system_profile_name = config.getString("system_profile", shared->default_profile_name);
setCurrentProfile(shared->system_profile_name);

View File

@ -85,7 +85,7 @@ class ActionLocksManager;
using ActionLocksManagerPtr = std::shared_ptr<ActionLocksManager>;
class ShellCommand;
class ICompressionCodec;
class AccessControlManager;
class AccessControl;
class Credentials;
class GSSAcceptorContext;
struct SettingsConstraintsAndProfileIDs;
@ -363,8 +363,8 @@ public:
void setConfig(const ConfigurationPtr & config);
const Poco::Util::AbstractConfiguration & getConfigRef() const;
AccessControlManager & getAccessControlManager();
const AccessControlManager & getAccessControlManager() const;
AccessControl & getAccessControl();
const AccessControl & getAccessControl() const;
/// Sets external authenticators config (LDAP, Kerberos).
void setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config);

View File

@ -1,6 +1,6 @@
#include <Interpreters/Session.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/Credentials.h>
#include <Access/ContextAccess.h>
#include <Access/User.h>
@ -273,7 +273,7 @@ Session::~Session()
AuthenticationType Session::getAuthenticationType(const String & user_name) const
{
return global_context->getAccessControlManager().read<User>(user_name)->auth_data.getType();
return global_context->getAccessControl().read<User>(user_name)->auth_data.getType();
}
AuthenticationType Session::getAuthenticationTypeOrLogInFailure(const String & user_name) const
@ -310,7 +310,7 @@ void Session::authenticate(const Credentials & credentials_, const Poco::Net::So
try
{
user_id = global_context->getAccessControlManager().login(credentials_, address.host());
user_id = global_context->getAccessControl().login(credentials_, address.host());
LOG_DEBUG(log, "{} Authenticated with global context as user {}",
toString(auth_id), user_id ? toString(*user_id) : "<EMPTY>");
}

View File

@ -152,7 +152,7 @@ NamesAndTypesList SessionLogElement::getNamesAndTypes()
void SessionLogElement::appendToBlock(MutableColumns & columns) const
{
assert(type >= SESSION_LOGIN_FAILURE && type <= SESSION_LOGOUT);
assert(user_identified_with >= AuthenticationType::NO_PASSWORD && user_identified_with <= AuthenticationType::MAX_TYPE);
assert(user_identified_with >= AuthenticationType::NO_PASSWORD && user_identified_with <= AuthenticationType::MAX);
size_t i = 0;

View File

@ -78,7 +78,7 @@ namespace
}
case AuthenticationType::NO_PASSWORD: [[fallthrough]];
case AuthenticationType::MAX_TYPE:
case AuthenticationType::MAX:
throw Exception("AST: Unexpected authentication type " + toString(auth_type), ErrorCodes::LOGICAL_ERROR);
}
}

View File

@ -55,7 +55,7 @@ namespace
if (ParserKeyword{"WITH"}.ignore(pos, expected))
{
for (auto check_type : collections::range(AuthenticationType::MAX_TYPE))
for (auto check_type : collections::range(AuthenticationType::MAX))
{
if (ParserKeyword{AuthenticationTypeInfo::get(check_type).raw_name}.ignore(pos, expected))
{

View File

@ -7,7 +7,7 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnsNumber.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/Common/AccessRightsElement.h>
#include <Access/Role.h>
#include <Access/User.h>
@ -38,7 +38,7 @@ NamesAndTypesList StorageSystemGrants::getNamesAndTypes()
void StorageSystemGrants::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_USERS | AccessType::SHOW_ROLES);
const auto & access_control = context->getAccessControlManager();
const auto & access_control = context->getAccessControl();
std::vector<UUID> ids = access_control.findAll<User>();
boost::range::push_back(ids, access_control.findAll<Role>());

View File

@ -1,5 +1,5 @@
#include <Storages/System/StorageSystemPrivileges.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/Common/AccessFlags.h>
#include <Access/SettingsProfile.h>
#include <Columns/ColumnArray.h>

View File

@ -6,7 +6,7 @@
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnNullable.h>
#include <Interpreters/Context.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/Quota.h>
#include <Access/Common/AccessFlags.h>
#include <base/range.h>
@ -72,7 +72,7 @@ NamesAndTypesList StorageSystemQuotaLimits::getNamesAndTypes()
void StorageSystemQuotaLimits::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_QUOTAS);
const auto & access_control = context->getAccessControlManager();
const auto & access_control = context->getAccessControl();
std::vector<UUID> ids = access_control.findAll<Quota>();
size_t column_index = 0;

View File

@ -7,7 +7,7 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnNullable.h>
#include <Interpreters/Context.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/QuotaUsage.h>
#include <Access/Common/AccessFlags.h>
#include <base/range.h>

View File

@ -1,5 +1,5 @@
#include <Storages/System/StorageSystemQuotas.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/Common/AccessFlags.h>
#include <Access/Quota.h>
#include <Columns/ColumnArray.h>
@ -55,7 +55,7 @@ NamesAndTypesList StorageSystemQuotas::getNamesAndTypes()
void StorageSystemQuotas::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_QUOTAS);
const auto & access_control = context->getAccessControlManager();
const auto & access_control = context->getAccessControl();
std::vector<UUID> ids = access_control.findAll<Quota>();
size_t column_index = 0;

View File

@ -1,7 +1,7 @@
#include <Storages/System/StorageSystemQuotasUsage.h>
#include <Storages/System/StorageSystemQuotaUsage.h>
#include <Interpreters/Context.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/Common/AccessFlags.h>
#include <Access/QuotaUsage.h>
@ -16,7 +16,7 @@ NamesAndTypesList StorageSystemQuotasUsage::getNamesAndTypes()
void StorageSystemQuotasUsage::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_QUOTAS);
auto all_quotas_usage = context->getAccessControlManager().getAllQuotasUsage();
auto all_quotas_usage = context->getAccessControl().getAllQuotasUsage();
StorageSystemQuotaUsage::fillDataImpl(res_columns, context, /* add_column_is_current = */ true, all_quotas_usage);
}
}

View File

@ -6,7 +6,7 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnsNumber.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/Role.h>
#include <Access/User.h>
#include <Interpreters/Context.h>
@ -34,7 +34,7 @@ NamesAndTypesList StorageSystemRoleGrants::getNamesAndTypes()
void StorageSystemRoleGrants::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_USERS | AccessType::SHOW_ROLES);
const auto & access_control = context->getAccessControlManager();
const auto & access_control = context->getAccessControl();
std::vector<UUID> ids = access_control.findAll<User>();
boost::range::push_back(ids, access_control.findAll<Role>());

View File

@ -3,7 +3,7 @@
#include <DataTypes/DataTypeUUID.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/Role.h>
#include <Access/Common/AccessFlags.h>
#include <Interpreters/Context.h>
@ -26,7 +26,7 @@ NamesAndTypesList StorageSystemRoles::getNamesAndTypes()
void StorageSystemRoles::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_ROLES);
const auto & access_control = context->getAccessControlManager();
const auto & access_control = context->getAccessControl();
std::vector<UUID> ids = access_control.findAll<Role>();
size_t column_index = 0;

View File

@ -1,5 +1,5 @@
#include <Storages/System/StorageSystemRowPolicies.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/Common/AccessFlags.h>
#include <Access/RowPolicy.h>
#include <Columns/ColumnString.h>
@ -55,7 +55,7 @@ NamesAndTypesList StorageSystemRowPolicies::getNamesAndTypes()
void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_ROW_POLICIES);
const auto & access_control = context->getAccessControlManager();
const auto & access_control = context->getAccessControl();
std::vector<UUID> ids = access_control.findAll<RowPolicy>();
size_t column_index = 0;

View File

@ -6,7 +6,7 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnsNumber.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/Role.h>
#include <Access/User.h>
#include <Access/SettingsProfile.h>
@ -40,7 +40,7 @@ NamesAndTypesList StorageSystemSettingsProfileElements::getNamesAndTypes()
void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_SETTINGS_PROFILES);
const auto & access_control = context->getAccessControlManager();
const auto & access_control = context->getAccessControl();
std::vector<UUID> ids = access_control.findAll<User>();
boost::range::push_back(ids, access_control.findAll<Role>());
boost::range::push_back(ids, access_control.findAll<SettingsProfile>());

View File

@ -1,5 +1,5 @@
#include <Storages/System/StorageSystemSettingsProfiles.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/Common/AccessFlags.h>
#include <Access/SettingsProfile.h>
#include <Columns/ColumnArray.h>
@ -33,7 +33,7 @@ NamesAndTypesList StorageSystemSettingsProfiles::getNamesAndTypes()
void StorageSystemSettingsProfiles::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_SETTINGS_PROFILES);
const auto & access_control = context->getAccessControlManager();
const auto & access_control = context->getAccessControl();
std::vector<UUID> ids = access_control.findAll<SettingsProfile>();
size_t column_index = 0;

View File

@ -4,7 +4,7 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <Interpreters/Context.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
namespace DB
@ -23,7 +23,7 @@ NamesAndTypesList StorageSystemUserDirectories::getNamesAndTypes()
void StorageSystemUserDirectories::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
const auto & access_control = context->getAccessControlManager();
const auto & access_control = context->getAccessControl();
auto storages = access_control.getStorages();
size_t column_index = 0;

View File

@ -1,5 +1,5 @@
#include <Storages/System/StorageSystemUsers.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessControl.h>
#include <Access/Common/AccessFlags.h>
#include <Access/User.h>
#include <DataTypes/DataTypeString.h>
@ -25,7 +25,7 @@ namespace
DataTypeEnum8::Values getAuthenticationTypeEnumValues()
{
DataTypeEnum8::Values enum_values;
for (auto type : collections::range(AuthenticationType::MAX_TYPE))
for (auto type : collections::range(AuthenticationType::MAX))
enum_values.emplace_back(AuthenticationTypeInfo::get(type).name, static_cast<Int8>(type));
return enum_values;
}
@ -59,7 +59,7 @@ NamesAndTypesList StorageSystemUsers::getNamesAndTypes()
void StorageSystemUsers::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_USERS);
const auto & access_control = context->getAccessControlManager();
const auto & access_control = context->getAccessControl();
std::vector<UUID> ids = access_control.findAll<User>();
size_t column_index = 0;