Mass rename: AccessRightsContext -> ContextAccess, QuotaContext -> EnabledQuota, RoleContext -> EnabledRoles, and so on.

This commit is contained in:
Vitaly Baranov 2020-03-07 20:37:38 +03:00
parent f1e9e3dec0
commit 18d5f63b31
97 changed files with 1918 additions and 1988 deletions

View File

@ -3,10 +3,14 @@
#include <Access/MemoryAccessStorage.h>
#include <Access/UsersConfigAccessStorage.h>
#include <Access/DiskAccessStorage.h>
#include <Access/AccessRightsContextFactory.h>
#include <Access/RoleContextFactory.h>
#include <Access/RowPolicyContextFactory.h>
#include <Access/QuotaContextFactory.h>
#include <Access/ContextAccess.h>
#include <Access/RoleCache.h>
#include <Access/RowPolicyCache.h>
#include <Access/QuotaCache.h>
#include <Access/QuotaUsageInfo.h>
#include <Core/Settings.h>
#include <Poco/ExpireCache.h>
#include <mutex>
namespace DB
@ -27,12 +31,48 @@ namespace
}
class AccessControlManager::ContextAccessCache
{
public:
explicit ContextAccessCache(const AccessControlManager & manager_) : manager(manager_) {}
std::shared_ptr<const ContextAccess> getContextAccess(const UUID & user_id, const std::vector<UUID> & current_roles, bool use_default_roles, const Settings & settings, const String & current_database, const ClientInfo & client_info)
{
ContextAccess::Params params;
params.user_id = user_id;
params.current_roles = current_roles;
params.use_default_roles = use_default_roles;
params.current_database = current_database;
params.readonly = settings.readonly;
params.allow_ddl = settings.allow_ddl;
params.allow_introspection = settings.allow_introspection_functions;
params.interface = client_info.interface;
params.http_method = client_info.http_method;
params.address = client_info.current_address.host();
params.quota_key = client_info.quota_key;
std::lock_guard lock{mutex};
auto x = cache.get(params);
if (x)
return *x;
auto res = std::shared_ptr<ContextAccess>(new ContextAccess(manager, params));
cache.add(params, res);
return res;
}
private:
const AccessControlManager & manager;
Poco::ExpireCache<ContextAccess::Params, std::shared_ptr<const ContextAccess>> cache;
std::mutex mutex;
};
AccessControlManager::AccessControlManager()
: MultipleAccessStorage(createStorages()),
access_rights_context_factory(std::make_unique<AccessRightsContextFactory>(*this)),
role_context_factory(std::make_unique<RoleContextFactory>(*this)),
row_policy_context_factory(std::make_unique<RowPolicyContextFactory>(*this)),
quota_context_factory(std::make_unique<QuotaContextFactory>(*this))
context_access_cache(std::make_unique<ContextAccessCache>(*this)),
role_cache(std::make_unique<RoleCache>(*this)),
row_policy_cache(std::make_unique<RowPolicyCache>(*this)),
quota_cache(std::make_unique<QuotaCache>(*this))
{
}
@ -54,7 +94,7 @@ void AccessControlManager::setUsersConfig(const Poco::Util::AbstractConfiguratio
}
AccessRightsContextPtr AccessControlManager::getAccessRightsContext(
std::shared_ptr<const ContextAccess> AccessControlManager::getContextAccess(
const UUID & user_id,
const std::vector<UUID> & current_roles,
bool use_default_roles,
@ -62,34 +102,34 @@ AccessRightsContextPtr AccessControlManager::getAccessRightsContext(
const String & current_database,
const ClientInfo & client_info) const
{
return access_rights_context_factory->createContext(user_id, current_roles, use_default_roles, settings, current_database, client_info);
return context_access_cache->getContextAccess(user_id, current_roles, use_default_roles, settings, current_database, client_info);
}
RoleContextPtr AccessControlManager::getRoleContext(
std::shared_ptr<const EnabledRoles> AccessControlManager::getEnabledRoles(
const std::vector<UUID> & current_roles,
const std::vector<UUID> & current_roles_with_admin_option) const
{
return role_context_factory->createContext(current_roles, current_roles_with_admin_option);
return role_cache->getEnabledRoles(current_roles, current_roles_with_admin_option);
}
RowPolicyContextPtr AccessControlManager::getRowPolicyContext(const UUID & user_id, const std::vector<UUID> & enabled_roles) const
std::shared_ptr<const EnabledRowPolicies> AccessControlManager::getEnabledRowPolicies(const UUID & user_id, const std::vector<UUID> & enabled_roles) const
{
return row_policy_context_factory->createContext(user_id, enabled_roles);
return row_policy_cache->getEnabledRowPolicies(user_id, enabled_roles);
}
QuotaContextPtr AccessControlManager::getQuotaContext(
const String & user_name, const UUID & user_id, const std::vector<UUID> & enabled_roles, const Poco::Net::IPAddress & address, const String & custom_quota_key) const
std::shared_ptr<const EnabledQuota> AccessControlManager::getEnabledQuota(
const UUID & user_id, const String & user_name, const std::vector<UUID> & enabled_roles, const Poco::Net::IPAddress & address, const String & custom_quota_key) const
{
return quota_context_factory->createContext(user_name, user_id, enabled_roles, address, custom_quota_key);
return quota_cache->getEnabledQuota(user_id, user_name, enabled_roles, address, custom_quota_key);
}
std::vector<QuotaUsageInfo> AccessControlManager::getQuotaUsageInfo() const
{
return quota_context_factory->getUsageInfo();
return quota_cache->getUsageInfo();
}
}

View File

@ -19,20 +19,15 @@ namespace Poco
namespace DB
{
class AccessRightsContext;
using AccessRightsContextPtr = std::shared_ptr<const AccessRightsContext>;
class AccessRightsContextFactory;
class ContextAccess;
struct User;
using UserPtr = std::shared_ptr<const User>;
class RoleContext;
using RoleContextPtr = std::shared_ptr<const RoleContext>;
class RoleContextFactory;
class RowPolicyContext;
using RowPolicyContextPtr = std::shared_ptr<const RowPolicyContext>;
class RowPolicyContextFactory;
class QuotaContext;
using QuotaContextPtr = std::shared_ptr<const QuotaContext>;
class QuotaContextFactory;
class EnabledRoles;
class RoleCache;
class EnabledRowPolicies;
class RowPolicyCache;
class EnabledQuota;
class QuotaCache;
struct QuotaUsageInfo;
class ClientInfo;
struct Settings;
@ -48,7 +43,7 @@ public:
void setLocalDirectory(const String & directory);
void setUsersConfig(const Poco::Util::AbstractConfiguration & users_config);
AccessRightsContextPtr getAccessRightsContext(
std::shared_ptr<const ContextAccess> getContextAccess(
const UUID & user_id,
const std::vector<UUID> & current_roles,
bool use_default_roles,
@ -56,17 +51,17 @@ public:
const String & current_database,
const ClientInfo & client_info) const;
RoleContextPtr getRoleContext(
std::shared_ptr<const EnabledRoles> getEnabledRoles(
const std::vector<UUID> & current_roles,
const std::vector<UUID> & current_roles_with_admin_option) const;
RowPolicyContextPtr getRowPolicyContext(
std::shared_ptr<const EnabledRowPolicies> getEnabledRowPolicies(
const UUID & user_id,
const std::vector<UUID> & enabled_roles) const;
QuotaContextPtr getQuotaContext(
const String & user_name,
std::shared_ptr<const EnabledQuota> getEnabledQuota(
const UUID & user_id,
const String & user_name,
const std::vector<UUID> & enabled_roles,
const Poco::Net::IPAddress & address,
const String & custom_quota_key) const;
@ -74,10 +69,11 @@ public:
std::vector<QuotaUsageInfo> getQuotaUsageInfo() const;
private:
std::unique_ptr<AccessRightsContextFactory> access_rights_context_factory;
std::unique_ptr<RoleContextFactory> role_context_factory;
std::unique_ptr<RowPolicyContextFactory> row_policy_context_factory;
std::unique_ptr<QuotaContextFactory> quota_context_factory;
class ContextAccessCache;
std::unique_ptr<ContextAccessCache> context_access_cache;
std::unique_ptr<RoleCache> role_cache;
std::unique_ptr<RowPolicyCache> row_policy_cache;
std::unique_ptr<QuotaCache> quota_cache;
};
}

View File

@ -28,23 +28,23 @@ public:
/// Grants access on a specified database/table/column.
/// Does nothing if the specified access has been already granted.
void grant(const AccessFlags & access);
void grant(const AccessFlags & access, const std::string_view & database);
void grant(const AccessFlags & access, const std::string_view & database, const std::string_view & table);
void grant(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column);
void grant(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns);
void grant(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns);
void grant(const AccessFlags & flags);
void grant(const AccessFlags & flags, const std::string_view & database);
void grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table);
void grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column);
void grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns);
void grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns);
void grant(const AccessRightsElement & element, std::string_view current_database = {});
void grant(const AccessRightsElements & elements, std::string_view current_database = {});
/// Revokes a specified access granted earlier on a specified database/table/column.
/// For example, revoke(AccessType::ALL) revokes all grants at all, just like clear();
void revoke(const AccessFlags & access);
void revoke(const AccessFlags & access, const std::string_view & database);
void revoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table);
void revoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column);
void revoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns);
void revoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns);
void revoke(const AccessFlags & flags);
void revoke(const AccessFlags & flags, const std::string_view & database);
void revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table);
void revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column);
void revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns);
void revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns);
void revoke(const AccessRightsElement & element, std::string_view current_database = {});
void revoke(const AccessRightsElements & elements, std::string_view current_database = {});

View File

@ -1,585 +0,0 @@
#include <Access/AccessRightsContext.h>
#include <Access/AccessControlManager.h>
#include <Access/RoleContext.h>
#include <Access/RowPolicyContext.h>
#include <Access/QuotaContext.h>
#include <Access/User.h>
#include <Access/CurrentRolesInfo.h>
#include <Common/Exception.h>
#include <Common/quoteString.h>
#include <Core/Settings.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Poco/Logger.h>
#include <common/logger_useful.h>
#include <boost/algorithm/string/join.hpp>
#include <boost/smart_ptr/make_shared_object.hpp>
#include <boost/range/algorithm/fill.hpp>
#include <boost/range/algorithm/set_algorithm.hpp>
#include <assert.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ACCESS_DENIED;
extern const int READONLY;
extern const int QUERY_IS_PROHIBITED;
extern const int FUNCTION_NOT_ALLOWED;
extern const int UNKNOWN_USER;
}
namespace
{
enum CheckAccessRightsMode
{
RETURN_FALSE_IF_ACCESS_DENIED,
LOG_WARNING_IF_ACCESS_DENIED,
THROW_IF_ACCESS_DENIED,
};
String formatSkippedMessage()
{
return "";
}
String formatSkippedMessage(const std::string_view & database)
{
return ". Skipped database " + backQuoteIfNeed(database);
}
String formatSkippedMessage(const std::string_view & database, const std::string_view & table)
{
String str = ". Skipped table ";
if (!database.empty())
str += backQuoteIfNeed(database) + ".";
str += backQuoteIfNeed(table);
return str;
}
String formatSkippedMessage(const std::string_view & database, const std::string_view & table, const std::string_view & column)
{
String str = ". Skipped column " + backQuoteIfNeed(column) + " ON ";
if (!database.empty())
str += backQuoteIfNeed(database) + ".";
str += backQuoteIfNeed(table);
return str;
}
template <typename StringT>
String formatSkippedMessage(const std::string_view & database, const std::string_view & table, const std::vector<StringT> & columns)
{
if (columns.size() == 1)
return formatSkippedMessage(database, table, columns[0]);
String str = ". Skipped columns ";
bool need_comma = false;
for (const auto & column : columns)
{
if (std::exchange(need_comma, true))
str += ", ";
str += backQuoteIfNeed(column);
}
str += " ON ";
if (!database.empty())
str += backQuoteIfNeed(database) + ".";
str += backQuoteIfNeed(table);
return str;
}
}
AccessRightsContext::AccessRightsContext()
{
auto everything_granted = boost::make_shared<AccessRights>();
everything_granted->grant(AccessType::ALL);
boost::range::fill(result_access_cache, everything_granted);
enabled_roles_with_admin_option = boost::make_shared<boost::container::flat_set<UUID>>();
row_policy_context = std::make_shared<RowPolicyContext>();
quota_context = std::make_shared<QuotaContext>();
}
AccessRightsContext::AccessRightsContext(const AccessControlManager & manager_, const Params & params_)
: manager(&manager_)
, params(params_)
{
subscription_for_user_change = manager->subscribeForChanges(
*params.user_id, [this](const UUID &, const AccessEntityPtr & entity)
{
UserPtr changed_user = entity ? typeid_cast<UserPtr>(entity) : nullptr;
std::lock_guard lock{mutex};
setUser(changed_user);
});
setUser(manager->read<User>(*params.user_id));
}
void AccessRightsContext::setUser(const UserPtr & user_) const
{
user = user_;
if (!user)
{
/// User has been dropped.
auto nothing_granted = boost::make_shared<AccessRights>();
boost::range::fill(result_access_cache, nothing_granted);
subscription_for_user_change = {};
subscription_for_roles_info_change = {};
role_context = nullptr;
enabled_roles_with_admin_option = boost::make_shared<boost::container::flat_set<UUID>>();
row_policy_context = std::make_shared<RowPolicyContext>();
quota_context = std::make_shared<QuotaContext>();
return;
}
user_name = user->getName();
trace_log = &Poco::Logger::get("AccessRightsContext (" + user_name + ")");
std::vector<UUID> current_roles, current_roles_with_admin_option;
if (params.use_default_roles)
{
for (const UUID & id : user->granted_roles)
{
if (user->default_roles.match(id))
current_roles.push_back(id);
}
boost::range::set_intersection(current_roles, user->granted_roles_with_admin_option,
std::back_inserter(current_roles_with_admin_option));
}
else
{
current_roles.reserve(params.current_roles.size());
for (const auto & id : params.current_roles)
{
if (user->granted_roles.contains(id))
current_roles.push_back(id);
if (user->granted_roles_with_admin_option.contains(id))
current_roles_with_admin_option.push_back(id);
}
}
subscription_for_roles_info_change = {};
role_context = manager->getRoleContext(current_roles, current_roles_with_admin_option);
subscription_for_roles_info_change = role_context->subscribeForChanges([this](const CurrentRolesInfoPtr & roles_info_)
{
std::lock_guard lock{mutex};
setRolesInfo(roles_info_);
});
setRolesInfo(role_context->getInfo());
}
void AccessRightsContext::setRolesInfo(const CurrentRolesInfoPtr & roles_info_) const
{
assert(roles_info_);
roles_info = roles_info_;
enabled_roles_with_admin_option.store(nullptr /* need to recalculate */);
boost::range::fill(result_access_cache, nullptr /* need recalculate */);
row_policy_context = manager->getRowPolicyContext(*params.user_id, roles_info->enabled_roles);
quota_context = manager->getQuotaContext(user_name, *params.user_id, roles_info->enabled_roles, params.address, params.quota_key);
}
bool AccessRightsContext::isCorrectPassword(const String & password) const
{
std::lock_guard lock{mutex};
if (!user)
return false;
return user->authentication.isCorrectPassword(password);
}
bool AccessRightsContext::isClientHostAllowed() const
{
std::lock_guard lock{mutex};
if (!user)
return false;
return user->allowed_client_hosts.contains(params.address);
}
template <int mode, bool grant_option, typename... Args>
bool AccessRightsContext::checkAccessImpl(Poco::Logger * log_, const AccessFlags & access, const Args &... args) const
{
auto result_access = calculateResultAccess(grant_option);
bool is_granted = result_access->isGranted(access, args...);
if (trace_log)
LOG_TRACE(trace_log, "Access " << (is_granted ? "granted" : "denied") << ": " << (AccessRightsElement{access, args...}.toString()));
if (is_granted)
return true;
if constexpr (mode == RETURN_FALSE_IF_ACCESS_DENIED)
return false;
if constexpr (mode == LOG_WARNING_IF_ACCESS_DENIED)
{
if (!log_)
return false;
}
auto show_error = [&](const String & msg, [[maybe_unused]] int error_code)
{
if constexpr (mode == THROW_IF_ACCESS_DENIED)
throw Exception(user_name + ": " + msg, error_code);
else if constexpr (mode == LOG_WARNING_IF_ACCESS_DENIED)
LOG_WARNING(log_, user_name + ": " + msg + formatSkippedMessage(args...));
};
if (!user)
{
show_error("User has been dropped", ErrorCodes::UNKNOWN_USER);
}
else if (grant_option && calculateResultAccess(false, params.readonly, params.allow_ddl, params.allow_introspection)->isGranted(access, args...))
{
show_error(
"Not enough privileges. "
"The required privileges have been granted, but without grant option. "
"To execute this query it's necessary to have the grant "
+ AccessRightsElement{access, args...}.toString() + " WITH GRANT OPTION",
ErrorCodes::ACCESS_DENIED);
}
else if (params.readonly && calculateResultAccess(false, false, params.allow_ddl, params.allow_introspection)->isGranted(access, args...))
{
if (params.interface == ClientInfo::Interface::HTTP && params.http_method == ClientInfo::HTTPMethod::GET)
show_error(
"Cannot execute query in readonly mode. "
"For queries over HTTP, method GET implies readonly. You should use method POST for modifying queries",
ErrorCodes::READONLY);
else
show_error("Cannot execute query in readonly mode", ErrorCodes::READONLY);
}
else if (!params.allow_ddl && calculateResultAccess(false, params.readonly, true, params.allow_introspection)->isGranted(access, args...))
{
show_error("Cannot execute query. DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
}
else if (!params.allow_introspection && calculateResultAccess(false, params.readonly, params.allow_ddl, true)->isGranted(access, args...))
{
show_error("Introspection functions are disabled, because setting 'allow_introspection_functions' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED);
}
else
{
show_error(
"Not enough privileges. To execute this query it's necessary to have the grant "
+ AccessRightsElement{access, args...}.toString() + (grant_option ? " WITH GRANT OPTION" : ""),
ErrorCodes::ACCESS_DENIED);
}
return false;
}
template <int mode, bool grant_option>
bool AccessRightsContext::checkAccessImpl(Poco::Logger * log_, const AccessRightsElement & element) const
{
if (element.any_database)
{
return checkAccessImpl<mode, grant_option>(log_, element.access_flags);
}
else if (element.any_table)
{
if (element.database.empty())
return checkAccessImpl<mode, grant_option>(log_, element.access_flags, params.current_database);
else
return checkAccessImpl<mode, grant_option>(log_, element.access_flags, element.database);
}
else if (element.any_column)
{
if (element.database.empty())
return checkAccessImpl<mode, grant_option>(log_, element.access_flags, params.current_database, element.table);
else
return checkAccessImpl<mode, grant_option>(log_, element.access_flags, element.database, element.table);
}
else
{
if (element.database.empty())
return checkAccessImpl<mode, grant_option>(log_, element.access_flags, params.current_database, element.table, element.columns);
else
return checkAccessImpl<mode, grant_option>(log_, element.access_flags, element.database, element.table, element.columns);
}
}
template <int mode, bool grant_option>
bool AccessRightsContext::checkAccessImpl(Poco::Logger * log_, const AccessRightsElements & elements) const
{
for (const auto & element : elements)
if (!checkAccessImpl<mode, grant_option>(log_, element))
return false;
return true;
}
void AccessRightsContext::checkAccess(const AccessFlags & access) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, false>(nullptr, access); }
void AccessRightsContext::checkAccess(const AccessFlags & access, const std::string_view & database) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, false>(nullptr, access, database); }
void AccessRightsContext::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, false>(nullptr, access, database, table); }
void AccessRightsContext::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, false>(nullptr, access, database, table, column); }
void AccessRightsContext::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, false>(nullptr, access, database, table, columns); }
void AccessRightsContext::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, false>(nullptr, access, database, table, columns); }
void AccessRightsContext::checkAccess(const AccessRightsElement & access) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, false>(nullptr, access); }
void AccessRightsContext::checkAccess(const AccessRightsElements & access) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, false>(nullptr, access); }
bool AccessRightsContext::isGranted(const AccessFlags & access) const { return checkAccessImpl<RETURN_FALSE_IF_ACCESS_DENIED, false>(nullptr, access); }
bool AccessRightsContext::isGranted(const AccessFlags & access, const std::string_view & database) const { return checkAccessImpl<RETURN_FALSE_IF_ACCESS_DENIED, false>(nullptr, access, database); }
bool AccessRightsContext::isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl<RETURN_FALSE_IF_ACCESS_DENIED, false>(nullptr, access, database, table); }
bool AccessRightsContext::isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl<RETURN_FALSE_IF_ACCESS_DENIED, false>(nullptr, access, database, table, column); }
bool AccessRightsContext::isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const { return checkAccessImpl<RETURN_FALSE_IF_ACCESS_DENIED, false>(nullptr, access, database, table, columns); }
bool AccessRightsContext::isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl<RETURN_FALSE_IF_ACCESS_DENIED, false>(nullptr, access, database, table, columns); }
bool AccessRightsContext::isGranted(const AccessRightsElement & access) const { return checkAccessImpl<RETURN_FALSE_IF_ACCESS_DENIED, false>(nullptr, access); }
bool AccessRightsContext::isGranted(const AccessRightsElements & access) const { return checkAccessImpl<RETURN_FALSE_IF_ACCESS_DENIED, false>(nullptr, access); }
bool AccessRightsContext::isGranted(Poco::Logger * log_, const AccessFlags & access) const { return checkAccessImpl<LOG_WARNING_IF_ACCESS_DENIED, false>(log_, access); }
bool AccessRightsContext::isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database) const { return checkAccessImpl<LOG_WARNING_IF_ACCESS_DENIED, false>(log_, access, database); }
bool AccessRightsContext::isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl<LOG_WARNING_IF_ACCESS_DENIED, false>(log_, access, database, table); }
bool AccessRightsContext::isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl<LOG_WARNING_IF_ACCESS_DENIED, false>(log_, access, database, table, column); }
bool AccessRightsContext::isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const { return checkAccessImpl<LOG_WARNING_IF_ACCESS_DENIED, false>(log_, access, database, table, columns); }
bool AccessRightsContext::isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl<LOG_WARNING_IF_ACCESS_DENIED, false>(log_, access, database, table, columns); }
bool AccessRightsContext::isGranted(Poco::Logger * log_, const AccessRightsElement & access) const { return checkAccessImpl<LOG_WARNING_IF_ACCESS_DENIED, false>(log_, access); }
bool AccessRightsContext::isGranted(Poco::Logger * log_, const AccessRightsElements & access) const { return checkAccessImpl<LOG_WARNING_IF_ACCESS_DENIED, false>(log_, access); }
void AccessRightsContext::checkGrantOption(const AccessFlags & access) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, true>(nullptr, access); }
void AccessRightsContext::checkGrantOption(const AccessFlags & access, const std::string_view & database) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, true>(nullptr, access, database); }
void AccessRightsContext::checkGrantOption(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, true>(nullptr, access, database, table); }
void AccessRightsContext::checkGrantOption(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, true>(nullptr, access, database, table, column); }
void AccessRightsContext::checkGrantOption(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, true>(nullptr, access, database, table, columns); }
void AccessRightsContext::checkGrantOption(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, true>(nullptr, access, database, table, columns); }
void AccessRightsContext::checkGrantOption(const AccessRightsElement & access) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, true>(nullptr, access); }
void AccessRightsContext::checkGrantOption(const AccessRightsElements & access) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, true>(nullptr, access); }
void AccessRightsContext::checkAdminOption(const UUID & role_id) const
{
if (isGranted(AccessType::ROLE_ADMIN))
return;
boost::shared_ptr<const boost::container::flat_set<UUID>> enabled_roles = enabled_roles_with_admin_option.load();
if (!enabled_roles)
{
std::lock_guard lock{mutex};
enabled_roles = enabled_roles_with_admin_option.load();
if (!enabled_roles)
{
if (roles_info)
enabled_roles = boost::make_shared<boost::container::flat_set<UUID>>(roles_info->enabled_roles_with_admin_option.begin(), roles_info->enabled_roles_with_admin_option.end());
else
enabled_roles = boost::make_shared<boost::container::flat_set<UUID>>();
enabled_roles_with_admin_option.store(enabled_roles);
}
}
if (enabled_roles->contains(role_id))
return;
std::optional<String> role_name = manager->readName(role_id);
if (!role_name)
role_name = "ID {" + toString(role_id) + "}";
throw Exception(
getUserName() + ": Not enough privileges. To execute this query it's necessary to have the grant " + backQuoteIfNeed(*role_name)
+ " WITH ADMIN OPTION ",
ErrorCodes::ACCESS_DENIED);
}
boost::shared_ptr<const AccessRights> AccessRightsContext::calculateResultAccess(bool grant_option) const
{
return calculateResultAccess(grant_option, params.readonly, params.allow_ddl, params.allow_introspection);
}
boost::shared_ptr<const AccessRights> AccessRightsContext::calculateResultAccess(bool grant_option, UInt64 readonly_, bool allow_ddl_, bool allow_introspection_) const
{
size_t cache_index = static_cast<size_t>(readonly_ != params.readonly)
+ static_cast<size_t>(allow_ddl_ != params.allow_ddl) * 2 +
+ static_cast<size_t>(allow_introspection_ != params.allow_introspection) * 3
+ static_cast<size_t>(grant_option) * 4;
assert(cache_index < std::size(result_access_cache));
auto cached = result_access_cache[cache_index].load();
if (cached)
return cached;
std::lock_guard lock{mutex};
cached = result_access_cache[cache_index].load();
if (cached)
return cached;
auto result_ptr = boost::make_shared<AccessRights>();
auto & result = *result_ptr;
if (grant_option)
{
result = user->access_with_grant_option;
if (roles_info)
result.merge(roles_info->access_with_grant_option);
}
else
{
result = user->access;
if (roles_info)
result.merge(roles_info->access);
}
static const AccessFlags table_ddl = AccessType::CREATE_DATABASE | AccessType::CREATE_TABLE | AccessType::CREATE_VIEW
| AccessType::ALTER_TABLE | AccessType::ALTER_VIEW | AccessType::DROP_DATABASE | AccessType::DROP_TABLE | AccessType::DROP_VIEW
| AccessType::TRUNCATE;
static const AccessFlags dictionary_ddl = AccessType::CREATE_DICTIONARY | AccessType::DROP_DICTIONARY;
static const AccessFlags table_and_dictionary_ddl = table_ddl | dictionary_ddl;
static const AccessFlags write_table_access = AccessType::INSERT | AccessType::OPTIMIZE;
static const AccessFlags all_dcl = AccessType::CREATE_USER | AccessType::CREATE_ROLE | AccessType::CREATE_POLICY
| AccessType::CREATE_QUOTA | AccessType::ALTER_USER | AccessType::ALTER_POLICY | AccessType::ALTER_QUOTA | AccessType::DROP_USER
| AccessType::DROP_ROLE | AccessType::DROP_POLICY | AccessType::DROP_QUOTA | AccessType::ROLE_ADMIN;
if (readonly_)
result.revoke(write_table_access | all_dcl | AccessType::SYSTEM | AccessType::KILL_QUERY);
if (readonly_ || !allow_ddl_)
result.revoke(table_and_dictionary_ddl);
if (readonly_ == 1)
{
/// Table functions are forbidden in readonly mode.
/// For example, for readonly = 2 - allowed.
result.revoke(AccessType::CREATE_TEMPORARY_TABLE | AccessType::TABLE_FUNCTIONS);
}
if (!allow_introspection_)
result.revoke(AccessType::INTROSPECTION);
/// Anyone has access to the "system" database.
result.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE);
if (readonly_ != 1)
{
/// User has access to temporary or external table if such table was resolved in session or query context
result.grant(AccessFlags::allTableFlags() | AccessFlags::allColumnFlags(), DatabaseCatalog::TEMPORARY_DATABASE);
}
if (readonly_ && grant_option)
{
/// No grant option in readonly mode.
result.revoke(AccessType::ALL);
}
result_access_cache[cache_index].store(result_ptr);
if (trace_log && (params.readonly == readonly_) && (params.allow_ddl == allow_ddl_) && (params.allow_introspection == allow_introspection_))
{
LOG_TRACE(trace_log, "List of all grants: " << result_ptr->toString() << (grant_option ? " WITH GRANT OPTION" : ""));
if (roles_info && !roles_info->getCurrentRolesNames().empty())
{
LOG_TRACE(
trace_log,
"Current_roles: " << boost::algorithm::join(roles_info->getCurrentRolesNames(), ", ")
<< ", enabled_roles: " << boost::algorithm::join(roles_info->getEnabledRolesNames(), ", "));
}
}
return result_ptr;
}
UserPtr AccessRightsContext::getUser() const
{
std::lock_guard lock{mutex};
return user;
}
String AccessRightsContext::getUserName() const
{
std::lock_guard lock{mutex};
return user_name;
}
CurrentRolesInfoPtr AccessRightsContext::getRolesInfo() const
{
std::lock_guard lock{mutex};
return roles_info;
}
std::vector<UUID> AccessRightsContext::getCurrentRoles() const
{
std::lock_guard lock{mutex};
return roles_info ? roles_info->current_roles : std::vector<UUID>{};
}
Strings AccessRightsContext::getCurrentRolesNames() const
{
std::lock_guard lock{mutex};
return roles_info ? roles_info->getCurrentRolesNames() : Strings{};
}
std::vector<UUID> AccessRightsContext::getEnabledRoles() const
{
std::lock_guard lock{mutex};
return roles_info ? roles_info->enabled_roles : std::vector<UUID>{};
}
Strings AccessRightsContext::getEnabledRolesNames() const
{
std::lock_guard lock{mutex};
return roles_info ? roles_info->getEnabledRolesNames() : Strings{};
}
RowPolicyContextPtr AccessRightsContext::getRowPolicy() const
{
std::lock_guard lock{mutex};
return row_policy_context;
}
QuotaContextPtr AccessRightsContext::getQuota() const
{
std::lock_guard lock{mutex};
return quota_context;
}
bool operator <(const AccessRightsContext::Params & lhs, const AccessRightsContext::Params & rhs)
{
#define ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(field) \
if (lhs.field < rhs.field) \
return true; \
if (lhs.field > rhs.field) \
return false
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(user_id);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(current_roles);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(use_default_roles);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(address);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(quota_key);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(current_database);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(readonly);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(allow_ddl);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(allow_introspection);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(interface);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(http_method);
return false;
#undef ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER
}
bool operator ==(const AccessRightsContext::Params & lhs, const AccessRightsContext::Params & rhs)
{
#define ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(field) \
if (lhs.field != rhs.field) \
return false
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(user_id);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(current_roles);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(use_default_roles);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(address);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(quota_key);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(current_database);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(readonly);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(allow_ddl);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(allow_introspection);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(interface);
ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(http_method);
return true;
#undef ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER
}
}

View File

@ -1,157 +0,0 @@
#pragma once
#include <Access/AccessRights.h>
#include <Interpreters/ClientInfo.h>
#include <Core/UUID.h>
#include <ext/scope_guard.h>
#include <ext/shared_ptr_helper.h>
#include <boost/smart_ptr/atomic_shared_ptr.hpp>
#include <boost/container/flat_set.hpp>
#include <mutex>
namespace Poco { class Logger; }
namespace DB
{
struct User;
using UserPtr = std::shared_ptr<const User>;
struct CurrentRolesInfo;
using CurrentRolesInfoPtr = std::shared_ptr<const CurrentRolesInfo>;
class RoleContext;
using RoleContextPtr = std::shared_ptr<const RoleContext>;
class RowPolicyContext;
using RowPolicyContextPtr = std::shared_ptr<const RowPolicyContext>;
class QuotaContext;
using QuotaContextPtr = std::shared_ptr<const QuotaContext>;
struct Settings;
class AccessControlManager;
class AccessRightsContext
{
public:
struct Params
{
std::optional<UUID> user_id;
std::vector<UUID> current_roles;
bool use_default_roles = false;
UInt64 readonly = 0;
bool allow_ddl = false;
bool allow_introspection = false;
String current_database;
ClientInfo::Interface interface = ClientInfo::Interface::TCP;
ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN;
Poco::Net::IPAddress address;
String quota_key;
friend bool operator ==(const Params & lhs, const Params & rhs);
friend bool operator !=(const Params & lhs, const Params & rhs) { return !(lhs == rhs); }
friend bool operator <(const Params & lhs, const Params & rhs);
friend bool operator >(const Params & lhs, const Params & rhs) { return rhs < lhs; }
friend bool operator <=(const Params & lhs, const Params & rhs) { return !(rhs < lhs); }
friend bool operator >=(const Params & lhs, const Params & rhs) { return !(lhs < rhs); }
};
/// Default constructor creates access rights' context which allows everything.
AccessRightsContext();
const Params & getParams() const { return params; }
UserPtr getUser() const;
String getUserName() const;
bool isCorrectPassword(const String & password) const;
bool isClientHostAllowed() const;
CurrentRolesInfoPtr getRolesInfo() const;
std::vector<UUID> getCurrentRoles() const;
Strings getCurrentRolesNames() const;
std::vector<UUID> getEnabledRoles() const;
Strings getEnabledRolesNames() const;
RowPolicyContextPtr getRowPolicy() const;
QuotaContextPtr getQuota() const;
/// Checks if a specified access is granted, and throws an exception if not.
/// Empty database means the current database.
void checkAccess(const AccessFlags & access) const;
void checkAccess(const AccessFlags & access, const std::string_view & database) const;
void checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const;
void checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const;
void checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const;
void checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const;
void checkAccess(const AccessRightsElement & access) const;
void checkAccess(const AccessRightsElements & access) const;
/// Checks if a specified access is granted.
bool isGranted(const AccessFlags & access) const;
bool isGranted(const AccessFlags & access, const std::string_view & database) const;
bool isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const;
bool isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const;
bool isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const;
bool isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const;
bool isGranted(const AccessRightsElement & access) const;
bool isGranted(const AccessRightsElements & access) const;
/// Checks if a specified access is granted, and logs a warning if not.
bool isGranted(Poco::Logger * log_, const AccessFlags & access) const;
bool isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database) const;
bool isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database, const std::string_view & table) const;
bool isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const;
bool isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const;
bool isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const;
bool isGranted(Poco::Logger * log_, const AccessRightsElement & access) const;
bool isGranted(Poco::Logger * log_, const AccessRightsElements & access) const;
/// Checks if a specified access is granted with grant option, and throws an exception if not.
void checkGrantOption(const AccessFlags & access) const;
void checkGrantOption(const AccessFlags & access, const std::string_view & database) const;
void checkGrantOption(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const;
void checkGrantOption(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const;
void checkGrantOption(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const;
void checkGrantOption(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const;
void checkGrantOption(const AccessRightsElement & access) const;
void checkGrantOption(const AccessRightsElements & access) const;
/// Checks if a specified role is granted with admin option, and throws an exception if not.
void checkAdminOption(const UUID & role_id) const;
private:
friend class AccessRightsContextFactory;
friend struct ext::shared_ptr_helper<AccessRightsContext>;
AccessRightsContext(const AccessControlManager & manager_, const Params & params_); /// AccessRightsContext should be created by AccessRightsContextFactory.
void setUser(const UserPtr & user_) const;
void setRolesInfo(const CurrentRolesInfoPtr & roles_info_) const;
template <int mode, bool grant_option, typename... Args>
bool checkAccessImpl(Poco::Logger * log_, const AccessFlags & access, const Args &... args) const;
template <int mode, bool grant_option>
bool checkAccessImpl(Poco::Logger * log_, const AccessRightsElement & element) const;
template <int mode, bool grant_option>
bool checkAccessImpl(Poco::Logger * log_, const AccessRightsElements & elements) const;
boost::shared_ptr<const AccessRights> calculateResultAccess(bool grant_option) const;
boost::shared_ptr<const AccessRights> calculateResultAccess(bool grant_option, UInt64 readonly_, bool allow_ddl_, bool allow_introspection_) const;
const AccessControlManager * manager = nullptr;
const Params params;
mutable Poco::Logger * trace_log = nullptr;
mutable UserPtr user;
mutable String user_name;
mutable ext::scope_guard subscription_for_user_change;
mutable RoleContextPtr role_context;
mutable ext::scope_guard subscription_for_roles_info_change;
mutable CurrentRolesInfoPtr roles_info;
mutable boost::atomic_shared_ptr<const boost::container::flat_set<UUID>> enabled_roles_with_admin_option;
mutable boost::atomic_shared_ptr<const AccessRights> result_access_cache[7];
mutable RowPolicyContextPtr row_policy_context;
mutable QuotaContextPtr quota_context;
mutable std::mutex mutex;
};
using AccessRightsContextPtr = std::shared_ptr<const AccessRightsContext>;
}

View File

@ -1,48 +0,0 @@
#include <Access/AccessRightsContextFactory.h>
#include <Access/AccessControlManager.h>
#include <Core/Settings.h>
namespace DB
{
AccessRightsContextFactory::AccessRightsContextFactory(const AccessControlManager & manager_)
: manager(manager_), cache(600000 /* 10 minutes */) {}
AccessRightsContextFactory::~AccessRightsContextFactory() = default;
AccessRightsContextPtr AccessRightsContextFactory::createContext(const Params & params)
{
std::lock_guard lock{mutex};
auto x = cache.get(params);
if (x)
return *x;
auto res = ext::shared_ptr_helper<AccessRightsContext>::create(manager, params);
cache.add(params, res);
return res;
}
AccessRightsContextPtr AccessRightsContextFactory::createContext(
const UUID & user_id,
const std::vector<UUID> & current_roles,
bool use_default_roles,
const Settings & settings,
const String & current_database,
const ClientInfo & client_info)
{
Params params;
params.user_id = user_id;
params.current_roles = current_roles;
params.use_default_roles = use_default_roles;
params.current_database = current_database;
params.readonly = settings.readonly;
params.allow_ddl = settings.allow_ddl;
params.allow_introspection = settings.allow_introspection_functions;
params.interface = client_info.interface;
params.http_method = client_info.http_method;
params.address = client_info.current_address.host();
params.quota_key = client_info.quota_key;
return createContext(params);
}
}

View File

@ -1,29 +0,0 @@
#pragma once
#include <Access/AccessRightsContext.h>
#include <Poco/ExpireCache.h>
#include <mutex>
namespace DB
{
class AccessControlManager;
class AccessRightsContextFactory
{
public:
AccessRightsContextFactory(const AccessControlManager & manager_);
~AccessRightsContextFactory();
using Params = AccessRightsContext::Params;
AccessRightsContextPtr createContext(const Params & params);
AccessRightsContextPtr createContext(const UUID & user_id, const std::vector<UUID> & current_roles, bool use_default_roles, const Settings & settings, const String & current_database, const ClientInfo & client_info);
private:
const AccessControlManager & manager;
Poco::ExpireCache<Params, AccessRightsContextPtr> cache;
std::mutex mutex;
};
}

View File

@ -0,0 +1,535 @@
#include <Access/ContextAccess.h>
#include <Access/AccessControlManager.h>
#include <Access/EnabledRoles.h>
#include <Access/EnabledRowPolicies.h>
#include <Access/EnabledQuota.h>
#include <Access/User.h>
#include <Access/EnabledRolesInfo.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Common/Exception.h>
#include <Common/quoteString.h>
#include <Core/Settings.h>
#include <IO/WriteHelpers.h>
#include <Poco/Logger.h>
#include <common/logger_useful.h>
#include <boost/algorithm/string/join.hpp>
#include <boost/smart_ptr/make_shared_object.hpp>
#include <boost/range/algorithm/fill.hpp>
#include <boost/range/algorithm/set_algorithm.hpp>
#include <assert.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ACCESS_DENIED;
extern const int READONLY;
extern const int QUERY_IS_PROHIBITED;
extern const int FUNCTION_NOT_ALLOWED;
extern const int UNKNOWN_USER;
}
namespace
{
enum CheckAccessRightsMode
{
RETURN_FALSE_IF_ACCESS_DENIED,
LOG_WARNING_IF_ACCESS_DENIED,
THROW_IF_ACCESS_DENIED,
};
String formatSkippedMessage()
{
return "";
}
String formatSkippedMessage(const std::string_view & database)
{
return ". Skipped database " + backQuoteIfNeed(database);
}
String formatSkippedMessage(const std::string_view & database, const std::string_view & table)
{
String str = ". Skipped table ";
if (!database.empty())
str += backQuoteIfNeed(database) + ".";
str += backQuoteIfNeed(table);
return str;
}
String formatSkippedMessage(const std::string_view & database, const std::string_view & table, const std::string_view & column)
{
String str = ". Skipped column " + backQuoteIfNeed(column) + " ON ";
if (!database.empty())
str += backQuoteIfNeed(database) + ".";
str += backQuoteIfNeed(table);
return str;
}
template <typename StringT>
String formatSkippedMessage(const std::string_view & database, const std::string_view & table, const std::vector<StringT> & columns)
{
if (columns.size() == 1)
return formatSkippedMessage(database, table, columns[0]);
String str = ". Skipped columns ";
bool need_comma = false;
for (const auto & column : columns)
{
if (std::exchange(need_comma, true))
str += ", ";
str += backQuoteIfNeed(column);
}
str += " ON ";
if (!database.empty())
str += backQuoteIfNeed(database) + ".";
str += backQuoteIfNeed(table);
return str;
}
}
ContextAccess::ContextAccess(const AccessControlManager & manager_, const Params & params_)
: manager(&manager_)
, params(params_)
{
subscription_for_user_change = manager->subscribeForChanges(
*params.user_id, [this](const UUID &, const AccessEntityPtr & entity)
{
UserPtr changed_user = entity ? typeid_cast<UserPtr>(entity) : nullptr;
std::lock_guard lock{mutex};
setUser(changed_user);
});
setUser(manager->read<User>(*params.user_id));
}
void ContextAccess::setUser(const UserPtr & user_) const
{
user = user_;
if (!user)
{
/// User has been dropped.
auto nothing_granted = boost::make_shared<AccessRights>();
boost::range::fill(result_access, nothing_granted);
subscription_for_user_change = {};
subscription_for_roles_changes = {};
enabled_roles = nullptr;
roles_info = nullptr;
roles_with_admin_option = nullptr;
enabled_row_policies = nullptr;
enabled_quota = nullptr;
return;
}
user_name = user->getName();
trace_log = &Poco::Logger::get("ContextAccess (" + user_name + ")");
std::vector<UUID> current_roles, current_roles_with_admin_option;
if (params.use_default_roles)
{
for (const UUID & id : user->granted_roles)
{
if (user->default_roles.match(id))
current_roles.push_back(id);
}
boost::range::set_intersection(current_roles, user->granted_roles_with_admin_option,
std::back_inserter(current_roles_with_admin_option));
}
else
{
current_roles.reserve(params.current_roles.size());
for (const auto & id : params.current_roles)
{
if (user->granted_roles.contains(id))
current_roles.push_back(id);
if (user->granted_roles_with_admin_option.contains(id))
current_roles_with_admin_option.push_back(id);
}
}
subscription_for_roles_changes = {};
enabled_roles = manager->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};
setRolesInfo(roles_info_);
});
setRolesInfo(enabled_roles->getRolesInfo());
}
void ContextAccess::setRolesInfo(const std::shared_ptr<const EnabledRolesInfo> & roles_info_) const
{
assert(roles_info_);
roles_info = roles_info_;
roles_with_admin_option.store(boost::make_shared<boost::container::flat_set<UUID>>(roles_info->enabled_roles_with_admin_option.begin(), roles_info->enabled_roles_with_admin_option.end()));
boost::range::fill(result_access, nullptr /* need recalculate */);
enabled_row_policies = manager->getEnabledRowPolicies(*params.user_id, roles_info->enabled_roles);
enabled_quota = manager->getEnabledQuota(*params.user_id, user_name, roles_info->enabled_roles, params.address, params.quota_key);
}
bool ContextAccess::isCorrectPassword(const String & password) const
{
std::lock_guard lock{mutex};
if (!user)
return false;
return user->authentication.isCorrectPassword(password);
}
bool ContextAccess::isClientHostAllowed() const
{
std::lock_guard lock{mutex};
if (!user)
return false;
return user->allowed_client_hosts.contains(params.address);
}
template <int mode, bool grant_option, typename... Args>
bool ContextAccess::checkAccessImpl(Poco::Logger * log_, const AccessFlags & flags, const Args &... args) const
{
auto access = calculateResultAccess(grant_option);
bool is_granted = access->isGranted(flags, args...);
if (trace_log)
LOG_TRACE(trace_log, "Access " << (is_granted ? "granted" : "denied") << ": " << (AccessRightsElement{flags, args...}.toString()));
if (is_granted)
return true;
if constexpr (mode == RETURN_FALSE_IF_ACCESS_DENIED)
return false;
if constexpr (mode == LOG_WARNING_IF_ACCESS_DENIED)
{
if (!log_)
return false;
}
auto show_error = [&](const String & msg, [[maybe_unused]] int error_code)
{
if constexpr (mode == THROW_IF_ACCESS_DENIED)
throw Exception(user_name + ": " + msg, error_code);
else if constexpr (mode == LOG_WARNING_IF_ACCESS_DENIED)
LOG_WARNING(log_, user_name + ": " + msg + formatSkippedMessage(args...));
};
if (!user)
{
show_error("User has been dropped", ErrorCodes::UNKNOWN_USER);
}
else if (grant_option && calculateResultAccess(false, params.readonly, params.allow_ddl, params.allow_introspection)->isGranted(flags, args...))
{
show_error(
"Not enough privileges. "
"The required privileges have been granted, but without grant option. "
"To execute this query it's necessary to have the grant "
+ AccessRightsElement{flags, args...}.toString() + " WITH GRANT OPTION",
ErrorCodes::ACCESS_DENIED);
}
else if (params.readonly && calculateResultAccess(false, false, params.allow_ddl, params.allow_introspection)->isGranted(flags, args...))
{
if (params.interface == ClientInfo::Interface::HTTP && params.http_method == ClientInfo::HTTPMethod::GET)
show_error(
"Cannot execute query in readonly mode. "
"For queries over HTTP, method GET implies readonly. You should use method POST for modifying queries",
ErrorCodes::READONLY);
else
show_error("Cannot execute query in readonly mode", ErrorCodes::READONLY);
}
else if (!params.allow_ddl && calculateResultAccess(false, params.readonly, true, params.allow_introspection)->isGranted(flags, args...))
{
show_error("Cannot execute query. DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
}
else if (!params.allow_introspection && calculateResultAccess(false, params.readonly, params.allow_ddl, true)->isGranted(flags, args...))
{
show_error("Introspection functions are disabled, because setting 'allow_introspection_functions' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED);
}
else
{
show_error(
"Not enough privileges. To execute this query it's necessary to have the grant "
+ AccessRightsElement{flags, args...}.toString() + (grant_option ? " WITH GRANT OPTION" : ""),
ErrorCodes::ACCESS_DENIED);
}
return false;
}
template <int mode, bool grant_option>
bool ContextAccess::checkAccessImpl(Poco::Logger * log_, const AccessRightsElement & element) const
{
if (element.any_database)
{
return checkAccessImpl<mode, grant_option>(log_, element.access_flags);
}
else if (element.any_table)
{
if (element.database.empty())
return checkAccessImpl<mode, grant_option>(log_, element.access_flags, params.current_database);
else
return checkAccessImpl<mode, grant_option>(log_, element.access_flags, element.database);
}
else if (element.any_column)
{
if (element.database.empty())
return checkAccessImpl<mode, grant_option>(log_, element.access_flags, params.current_database, element.table);
else
return checkAccessImpl<mode, grant_option>(log_, element.access_flags, element.database, element.table);
}
else
{
if (element.database.empty())
return checkAccessImpl<mode, grant_option>(log_, element.access_flags, params.current_database, element.table, element.columns);
else
return checkAccessImpl<mode, grant_option>(log_, element.access_flags, element.database, element.table, element.columns);
}
}
template <int mode, bool grant_option>
bool ContextAccess::checkAccessImpl(Poco::Logger * log_, const AccessRightsElements & elements) const
{
for (const auto & element : elements)
if (!checkAccessImpl<mode, grant_option>(log_, element))
return false;
return true;
}
void ContextAccess::checkAccess(const AccessFlags & flags) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, false>(nullptr, flags); }
void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, false>(nullptr, flags, database); }
void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, false>(nullptr, flags, database, table); }
void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, false>(nullptr, flags, database, table, column); }
void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, false>(nullptr, flags, database, table, columns); }
void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, false>(nullptr, flags, database, table, columns); }
void ContextAccess::checkAccess(const AccessRightsElement & element) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, false>(nullptr, element); }
void ContextAccess::checkAccess(const AccessRightsElements & elements) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, false>(nullptr, elements); }
bool ContextAccess::isGranted(const AccessFlags & flags) const { return checkAccessImpl<RETURN_FALSE_IF_ACCESS_DENIED, false>(nullptr, flags); }
bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database) const { return checkAccessImpl<RETURN_FALSE_IF_ACCESS_DENIED, false>(nullptr, flags, database); }
bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl<RETURN_FALSE_IF_ACCESS_DENIED, false>(nullptr, flags, database, table); }
bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl<RETURN_FALSE_IF_ACCESS_DENIED, false>(nullptr, flags, database, table, column); }
bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const { return checkAccessImpl<RETURN_FALSE_IF_ACCESS_DENIED, false>(nullptr, flags, database, table, columns); }
bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl<RETURN_FALSE_IF_ACCESS_DENIED, false>(nullptr, flags, database, table, columns); }
bool ContextAccess::isGranted(const AccessRightsElement & element) const { return checkAccessImpl<RETURN_FALSE_IF_ACCESS_DENIED, false>(nullptr, element); }
bool ContextAccess::isGranted(const AccessRightsElements & elements) const { return checkAccessImpl<RETURN_FALSE_IF_ACCESS_DENIED, false>(nullptr, elements); }
bool ContextAccess::isGranted(Poco::Logger * log_, const AccessFlags & flags) const { return checkAccessImpl<LOG_WARNING_IF_ACCESS_DENIED, false>(log_, flags); }
bool ContextAccess::isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database) const { return checkAccessImpl<LOG_WARNING_IF_ACCESS_DENIED, false>(log_, flags, database); }
bool ContextAccess::isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl<LOG_WARNING_IF_ACCESS_DENIED, false>(log_, flags, database, table); }
bool ContextAccess::isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl<LOG_WARNING_IF_ACCESS_DENIED, false>(log_, flags, database, table, column); }
bool ContextAccess::isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const { return checkAccessImpl<LOG_WARNING_IF_ACCESS_DENIED, false>(log_, flags, database, table, columns); }
bool ContextAccess::isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl<LOG_WARNING_IF_ACCESS_DENIED, false>(log_, flags, database, table, columns); }
bool ContextAccess::isGranted(Poco::Logger * log_, const AccessRightsElement & element) const { return checkAccessImpl<LOG_WARNING_IF_ACCESS_DENIED, false>(log_, element); }
bool ContextAccess::isGranted(Poco::Logger * log_, const AccessRightsElements & elements) const { return checkAccessImpl<LOG_WARNING_IF_ACCESS_DENIED, false>(log_, elements); }
void ContextAccess::checkGrantOption(const AccessFlags & flags) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, true>(nullptr, flags); }
void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, true>(nullptr, flags, database); }
void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, true>(nullptr, flags, database, table); }
void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, true>(nullptr, flags, database, table, column); }
void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, true>(nullptr, flags, database, table, columns); }
void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, true>(nullptr, flags, database, table, columns); }
void ContextAccess::checkGrantOption(const AccessRightsElement & element) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, true>(nullptr, element); }
void ContextAccess::checkGrantOption(const AccessRightsElements & elements) const { checkAccessImpl<THROW_IF_ACCESS_DENIED, true>(nullptr, elements); }
void ContextAccess::checkAdminOption(const UUID & role_id) const
{
if (isGranted(AccessType::ROLE_ADMIN))
return;
auto roles_with_admin_option_loaded = roles_with_admin_option.load();
if (roles_with_admin_option_loaded && roles_with_admin_option_loaded->contains(role_id))
return;
std::optional<String> role_name = manager->readName(role_id);
if (!role_name)
role_name = "ID {" + toString(role_id) + "}";
throw Exception(
getUserName() + ": Not enough privileges. To execute this query it's necessary to have the grant " + backQuoteIfNeed(*role_name)
+ " WITH ADMIN OPTION ",
ErrorCodes::ACCESS_DENIED);
}
boost::shared_ptr<const AccessRights> ContextAccess::calculateResultAccess(bool grant_option) const
{
return calculateResultAccess(grant_option, params.readonly, params.allow_ddl, params.allow_introspection);
}
boost::shared_ptr<const AccessRights> ContextAccess::calculateResultAccess(bool grant_option, UInt64 readonly_, bool allow_ddl_, bool allow_introspection_) const
{
size_t cache_index = static_cast<size_t>(readonly_ != params.readonly)
+ static_cast<size_t>(allow_ddl_ != params.allow_ddl) * 2 +
+ static_cast<size_t>(allow_introspection_ != params.allow_introspection) * 3
+ static_cast<size_t>(grant_option) * 4;
assert(cache_index < std::size(result_access));
auto res = result_access[cache_index].load();
if (res)
return res;
std::lock_guard lock{mutex};
res = result_access[cache_index].load();
if (res)
return res;
auto merged_access = boost::make_shared<AccessRights>();
if (grant_option)
{
*merged_access = user->access_with_grant_option;
if (roles_info)
merged_access->merge(roles_info->access_with_grant_option);
}
else
{
*merged_access = user->access;
if (roles_info)
merged_access->merge(roles_info->access);
}
static const AccessFlags table_ddl = AccessType::CREATE_DATABASE | AccessType::CREATE_TABLE | AccessType::CREATE_VIEW
| AccessType::ALTER_TABLE | AccessType::ALTER_VIEW | AccessType::DROP_DATABASE | AccessType::DROP_TABLE | AccessType::DROP_VIEW
| AccessType::TRUNCATE;
static const AccessFlags dictionary_ddl = AccessType::CREATE_DICTIONARY | AccessType::DROP_DICTIONARY;
static const AccessFlags table_and_dictionary_ddl = table_ddl | dictionary_ddl;
static const AccessFlags write_table_access = AccessType::INSERT | AccessType::OPTIMIZE;
static const AccessFlags all_dcl = AccessType::CREATE_USER | AccessType::CREATE_ROLE | AccessType::CREATE_POLICY
| AccessType::CREATE_QUOTA | AccessType::ALTER_USER | AccessType::ALTER_POLICY | AccessType::ALTER_QUOTA | AccessType::DROP_USER
| AccessType::DROP_ROLE | AccessType::DROP_POLICY | AccessType::DROP_QUOTA | AccessType::ROLE_ADMIN;
if (readonly_)
merged_access->revoke(write_table_access | all_dcl | AccessType::SYSTEM | AccessType::KILL_QUERY);
if (readonly_ || !allow_ddl_)
merged_access->revoke(table_and_dictionary_ddl);
if (readonly_ == 1)
{
/// Table functions are forbidden in readonly mode.
/// For example, for readonly = 2 - allowed.
merged_access->revoke(AccessType::CREATE_TEMPORARY_TABLE | AccessType::TABLE_FUNCTIONS);
}
if (!allow_introspection_)
merged_access->revoke(AccessType::INTROSPECTION);
/// Anyone has access to the "system" database.
merged_access->grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE);
if (readonly_ != 1)
{
/// User has access to temporary or external table if such table was resolved in session or query context
merged_access->grant(AccessFlags::allTableFlags() | AccessFlags::allColumnFlags(), DatabaseCatalog::TEMPORARY_DATABASE);
}
if (readonly_ && grant_option)
{
/// No grant option in readonly mode.
merged_access->revoke(AccessType::ALL);
}
if (trace_log && (params.readonly == readonly_) && (params.allow_ddl == allow_ddl_) && (params.allow_introspection == allow_introspection_))
{
LOG_TRACE(trace_log, "List of all grants: " << merged_access->toString() << (grant_option ? " WITH GRANT OPTION" : ""));
if (roles_info && !roles_info->getCurrentRolesNames().empty())
{
LOG_TRACE(
trace_log,
"Current_roles: " << boost::algorithm::join(roles_info->getCurrentRolesNames(), ", ")
<< ", enabled_roles: " << boost::algorithm::join(roles_info->getEnabledRolesNames(), ", "));
}
}
res = std::move(merged_access);
result_access[cache_index].store(res);
return res;
}
UserPtr ContextAccess::getUser() const
{
std::lock_guard lock{mutex};
return user;
}
String ContextAccess::getUserName() const
{
std::lock_guard lock{mutex};
return user_name;
}
std::shared_ptr<const EnabledRolesInfo> ContextAccess::getRolesInfo() const
{
std::lock_guard lock{mutex};
return roles_info;
}
std::vector<UUID> ContextAccess::getCurrentRoles() const
{
std::lock_guard lock{mutex};
return roles_info ? roles_info->current_roles : std::vector<UUID>{};
}
Strings ContextAccess::getCurrentRolesNames() const
{
std::lock_guard lock{mutex};
return roles_info ? roles_info->getCurrentRolesNames() : Strings{};
}
std::vector<UUID> ContextAccess::getEnabledRoles() const
{
std::lock_guard lock{mutex};
return roles_info ? roles_info->enabled_roles : std::vector<UUID>{};
}
Strings ContextAccess::getEnabledRolesNames() const
{
std::lock_guard lock{mutex};
return roles_info ? roles_info->getEnabledRolesNames() : Strings{};
}
std::shared_ptr<const EnabledRowPolicies> ContextAccess::getRowPolicies() const
{
std::lock_guard lock{mutex};
return enabled_row_policies;
}
ASTPtr ContextAccess::getRowPolicyCondition(const String & database, const String & table_name, RowPolicy::ConditionType index, const ASTPtr & extra_condition) const
{
std::lock_guard lock{mutex};
return enabled_row_policies ? enabled_row_policies->getCondition(database, table_name, index, extra_condition) : nullptr;
}
std::shared_ptr<const EnabledQuota> ContextAccess::getQuota() const
{
std::lock_guard lock{mutex};
return enabled_quota;
}
std::shared_ptr<const ContextAccess> ContextAccess::getFullAccess()
{
static const std::shared_ptr<const ContextAccess> res = []
{
auto full_access = std::shared_ptr<ContextAccess>(new ContextAccess);
auto everything_granted = boost::make_shared<AccessRights>();
everything_granted->grant(AccessType::ALL);
boost::range::fill(full_access->result_access, everything_granted);
full_access->enabled_quota = EnabledQuota::getUnlimitedQuota();
return full_access;
}();
return res;
}
}

View File

@ -0,0 +1,156 @@
#pragma once
#include <Access/AccessRights.h>
#include <Access/RowPolicy.h>
#include <Interpreters/ClientInfo.h>
#include <Core/UUID.h>
#include <ext/scope_guard.h>
#include <ext/shared_ptr_helper.h>
#include <boost/smart_ptr/atomic_shared_ptr.hpp>
#include <boost/container/flat_set.hpp>
#include <mutex>
namespace Poco { class Logger; }
namespace DB
{
struct User;
using UserPtr = std::shared_ptr<const User>;
struct EnabledRolesInfo;
class EnabledRoles;
class EnabledRowPolicies;
class EnabledQuota;
struct Settings;
class AccessControlManager;
class IAST;
using ASTPtr = std::shared_ptr<IAST>;
class ContextAccess
{
public:
struct Params
{
std::optional<UUID> user_id;
std::vector<UUID> current_roles;
bool use_default_roles = false;
UInt64 readonly = 0;
bool allow_ddl = false;
bool allow_introspection = false;
String current_database;
ClientInfo::Interface interface = ClientInfo::Interface::TCP;
ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN;
Poco::Net::IPAddress address;
String quota_key;
auto toTuple() const { return std::tie(user_id, current_roles, use_default_roles, readonly, allow_ddl, allow_introspection, current_database, interface, http_method, address, quota_key); }
friend bool operator ==(const Params & lhs, const Params & rhs) { return lhs.toTuple() == rhs.toTuple(); }
friend bool operator !=(const Params & lhs, const Params & rhs) { return !(lhs == rhs); }
friend bool operator <(const Params & lhs, const Params & rhs) { return lhs.toTuple() < rhs.toTuple(); }
friend bool operator >(const Params & lhs, const Params & rhs) { return rhs < lhs; }
friend bool operator <=(const Params & lhs, const Params & rhs) { return !(rhs < lhs); }
friend bool operator >=(const Params & lhs, const Params & rhs) { return !(lhs < rhs); }
};
const Params & getParams() const { return params; }
UserPtr getUser() const;
String getUserName() const;
bool isCorrectPassword(const String & password) const;
bool isClientHostAllowed() const;
std::shared_ptr<const EnabledRolesInfo> getRolesInfo() const;
std::vector<UUID> getCurrentRoles() const;
Strings getCurrentRolesNames() const;
std::vector<UUID> getEnabledRoles() const;
Strings getEnabledRolesNames() const;
std::shared_ptr<const EnabledRowPolicies> getRowPolicies() const;
ASTPtr getRowPolicyCondition(const String & database, const String & table_name, RowPolicy::ConditionType index, const ASTPtr & extra_condition = nullptr) const;
std::shared_ptr<const EnabledQuota> getQuota() const;
/// Checks if a specified access is granted, and throws an exception if not.
/// Empty database means the current database.
void checkAccess(const AccessFlags & flags) const;
void checkAccess(const AccessFlags & flags, const std::string_view & database) const;
void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const;
void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const;
void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const;
void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const;
void checkAccess(const AccessRightsElement & element) const;
void checkAccess(const AccessRightsElements & elements) const;
/// Checks if a specified access is granted.
bool isGranted(const AccessFlags & flags) const;
bool isGranted(const AccessFlags & flags, const std::string_view & database) const;
bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const;
bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const;
bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const;
bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const;
bool isGranted(const AccessRightsElement & element) const;
bool isGranted(const AccessRightsElements & elements) const;
/// Checks if a specified access is granted, and logs a warning if not.
bool isGranted(Poco::Logger * log_, const AccessFlags & flags) const;
bool isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database) const;
bool isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const;
bool isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const;
bool isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const;
bool isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const;
bool isGranted(Poco::Logger * log_, const AccessRightsElement & element) const;
bool isGranted(Poco::Logger * log_, const AccessRightsElements & elements) const;
/// Checks if a specified access is granted with grant option, and throws an exception if not.
void checkGrantOption(const AccessFlags & flags) const;
void checkGrantOption(const AccessFlags & flags, const std::string_view & database) const;
void checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const;
void checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const;
void checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const;
void checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const;
void checkGrantOption(const AccessRightsElement & element) const;
void checkGrantOption(const AccessRightsElements & elements) const;
/// Checks if a specified role is granted with admin option, and throws an exception if not.
void checkAdminOption(const UUID & role_id) const;
/// Returns an instance of ContextAccess which has full access to everything.
static std::shared_ptr<const ContextAccess> getFullAccess();
private:
friend class AccessControlManager;
ContextAccess() {}
ContextAccess(const AccessControlManager & manager_, const Params & params_);
void setUser(const UserPtr & user_) const;
void setRolesInfo(const std::shared_ptr<const EnabledRolesInfo> & roles_info_) const;
template <int mode, bool grant_option, typename... Args>
bool checkAccessImpl(Poco::Logger * log_, const AccessFlags & flags, const Args &... args) const;
template <int mode, bool grant_option>
bool checkAccessImpl(Poco::Logger * log_, const AccessRightsElement & element) const;
template <int mode, bool grant_option>
bool checkAccessImpl(Poco::Logger * log_, const AccessRightsElements & elements) const;
boost::shared_ptr<const AccessRights> calculateResultAccess(bool grant_option) const;
boost::shared_ptr<const AccessRights> calculateResultAccess(bool grant_option, UInt64 readonly_, bool allow_ddl_, bool allow_introspection_) const;
const AccessControlManager * manager = nullptr;
const Params params;
mutable Poco::Logger * trace_log = nullptr;
mutable UserPtr user;
mutable String user_name;
mutable ext::scope_guard subscription_for_user_change;
mutable std::shared_ptr<const EnabledRoles> enabled_roles;
mutable ext::scope_guard subscription_for_roles_changes;
mutable std::shared_ptr<const EnabledRolesInfo> roles_info;
mutable boost::atomic_shared_ptr<const boost::container::flat_set<UUID>> roles_with_admin_option;
mutable boost::atomic_shared_ptr<const AccessRights> result_access[7];
mutable std::shared_ptr<const EnabledRowPolicies> enabled_row_policies;
mutable std::shared_ptr<const EnabledQuota> enabled_quota;
mutable std::mutex mutex;
};
}

View File

@ -1,4 +1,5 @@
#include <Access/QuotaContext.h>
#include <Access/EnabledQuota.h>
#include <Access/QuotaUsageInfo.h>
#include <Common/Exception.h>
#include <Common/quoteString.h>
#include <ext/chrono_io.h>
@ -14,7 +15,7 @@ namespace ErrorCodes
extern const int QUOTA_EXPIRED;
}
struct QuotaContext::Impl
struct EnabledQuota::Impl
{
[[noreturn]] static void throwQuotaExceed(
const String & user_name,
@ -133,7 +134,7 @@ struct QuotaContext::Impl
};
QuotaContext::Interval & QuotaContext::Interval::operator =(const Interval & src)
EnabledQuota::Interval & EnabledQuota::Interval::operator =(const Interval & src)
{
if (this == &src)
return *this;
@ -150,7 +151,7 @@ QuotaContext::Interval & QuotaContext::Interval::operator =(const Interval & src
}
QuotaUsageInfo QuotaContext::Intervals::getUsageInfo(std::chrono::system_clock::time_point current_time) const
QuotaUsageInfo EnabledQuota::Intervals::getUsageInfo(std::chrono::system_clock::time_point current_time) const
{
QuotaUsageInfo info;
info.quota_id = quota_id;
@ -174,97 +175,85 @@ QuotaUsageInfo QuotaContext::Intervals::getUsageInfo(std::chrono::system_clock::
}
QuotaContext::QuotaContext()
: intervals(boost::make_shared<Intervals>()) /// Unlimited quota.
EnabledQuota::EnabledQuota(const Params & params_) : params(params_)
{
}
QuotaContext::QuotaContext(
const String & user_name_,
const UUID & user_id_,
const std::vector<UUID> & enabled_roles_,
const Poco::Net::IPAddress & address_,
const String & client_key_)
: user_name(user_name_), user_id(user_id_), enabled_roles(enabled_roles_), address(address_), client_key(client_key_)
{
}
EnabledQuota::~EnabledQuota() = default;
QuotaContext::~QuotaContext() = default;
void QuotaContext::used(ResourceType resource_type, ResourceAmount amount, bool check_exceeded) const
void EnabledQuota::used(ResourceType resource_type, ResourceAmount amount, bool check_exceeded) const
{
used({resource_type, amount}, check_exceeded);
}
void QuotaContext::used(const std::pair<ResourceType, ResourceAmount> & resource, bool check_exceeded) const
void EnabledQuota::used(const std::pair<ResourceType, ResourceAmount> & resource, bool check_exceeded) const
{
auto loaded = intervals.load();
auto current_time = std::chrono::system_clock::now();
Impl::used(user_name, *loaded, resource.first, resource.second, current_time, check_exceeded);
Impl::used(getUserName(), *loaded, resource.first, resource.second, current_time, check_exceeded);
}
void QuotaContext::used(const std::pair<ResourceType, ResourceAmount> & resource1, const std::pair<ResourceType, ResourceAmount> & resource2, bool check_exceeded) const
void EnabledQuota::used(const std::pair<ResourceType, ResourceAmount> & resource1, const std::pair<ResourceType, ResourceAmount> & resource2, bool check_exceeded) const
{
auto loaded = intervals.load();
auto current_time = std::chrono::system_clock::now();
Impl::used(user_name, *loaded, resource1.first, resource1.second, current_time, check_exceeded);
Impl::used(user_name, *loaded, resource2.first, resource2.second, current_time, check_exceeded);
Impl::used(getUserName(), *loaded, resource1.first, resource1.second, current_time, check_exceeded);
Impl::used(getUserName(), *loaded, resource2.first, resource2.second, current_time, check_exceeded);
}
void QuotaContext::used(const std::pair<ResourceType, ResourceAmount> & resource1, const std::pair<ResourceType, ResourceAmount> & resource2, const std::pair<ResourceType, ResourceAmount> & resource3, bool check_exceeded) const
void EnabledQuota::used(const std::pair<ResourceType, ResourceAmount> & resource1, const std::pair<ResourceType, ResourceAmount> & resource2, const std::pair<ResourceType, ResourceAmount> & resource3, bool check_exceeded) const
{
auto loaded = intervals.load();
auto current_time = std::chrono::system_clock::now();
Impl::used(user_name, *loaded, resource1.first, resource1.second, current_time, check_exceeded);
Impl::used(user_name, *loaded, resource2.first, resource2.second, current_time, check_exceeded);
Impl::used(user_name, *loaded, resource3.first, resource3.second, current_time, check_exceeded);
Impl::used(getUserName(), *loaded, resource1.first, resource1.second, current_time, check_exceeded);
Impl::used(getUserName(), *loaded, resource2.first, resource2.second, current_time, check_exceeded);
Impl::used(getUserName(), *loaded, resource3.first, resource3.second, current_time, check_exceeded);
}
void QuotaContext::used(const std::vector<std::pair<ResourceType, ResourceAmount>> & resources, bool check_exceeded) const
void EnabledQuota::used(const std::vector<std::pair<ResourceType, ResourceAmount>> & resources, bool check_exceeded) const
{
auto loaded = intervals.load();
auto current_time = std::chrono::system_clock::now();
for (const auto & resource : resources)
Impl::used(user_name, *loaded, resource.first, resource.second, current_time, check_exceeded);
Impl::used(getUserName(), *loaded, resource.first, resource.second, current_time, check_exceeded);
}
void QuotaContext::checkExceeded() const
void EnabledQuota::checkExceeded() const
{
auto loaded = intervals.load();
Impl::checkExceeded(user_name, *loaded, std::chrono::system_clock::now());
Impl::checkExceeded(getUserName(), *loaded, std::chrono::system_clock::now());
}
void QuotaContext::checkExceeded(ResourceType resource_type) const
void EnabledQuota::checkExceeded(ResourceType resource_type) const
{
auto loaded = intervals.load();
Impl::checkExceeded(user_name, *loaded, resource_type, std::chrono::system_clock::now());
Impl::checkExceeded(getUserName(), *loaded, resource_type, std::chrono::system_clock::now());
}
QuotaUsageInfo QuotaContext::getUsageInfo() const
QuotaUsageInfo EnabledQuota::getUsageInfo() const
{
auto loaded = intervals.load();
return loaded->getUsageInfo(std::chrono::system_clock::now());
}
QuotaUsageInfo::QuotaUsageInfo() : quota_id(UUID(UInt128(0)))
std::shared_ptr<const EnabledQuota> EnabledQuota::getUnlimitedQuota()
{
static const std::shared_ptr<const EnabledQuota> res = []
{
auto unlimited_quota = std::shared_ptr<EnabledQuota>(new EnabledQuota);
unlimited_quota->intervals = boost::make_shared<Intervals>();
return unlimited_quota;
}();
return res;
}
QuotaUsageInfo::Interval::Interval()
{
boost::range::fill(used, 0);
boost::range::fill(max, 0);
}
}

View File

@ -3,7 +3,6 @@
#include <Access/Quota.h>
#include <Core/UUID.h>
#include <Poco/Net/IPAddress.h>
#include <ext/shared_ptr_helper.h>
#include <boost/noncopyable.hpp>
#include <boost/smart_ptr/atomic_shared_ptr.hpp>
#include <atomic>
@ -16,17 +15,31 @@ namespace DB
struct QuotaUsageInfo;
/// Instances of `QuotaContext` are used to track resource consumption.
class QuotaContext : public boost::noncopyable
/// Instances of `EnabledQuota` are used to track resource consumption.
class EnabledQuota : public boost::noncopyable
{
public:
struct Params
{
UUID user_id;
String user_name;
std::vector<UUID> enabled_roles;
Poco::Net::IPAddress client_address;
String client_key;
auto toTuple() const { return std::tie(user_id, enabled_roles, user_name, client_address, client_key); }
friend bool operator ==(const Params & lhs, const Params & rhs) { return lhs.toTuple() == rhs.toTuple(); }
friend bool operator !=(const Params & lhs, const Params & rhs) { return !(lhs == rhs); }
friend bool operator <(const Params & lhs, const Params & rhs) { return lhs.toTuple() < rhs.toTuple(); }
friend bool operator >(const Params & lhs, const Params & rhs) { return rhs < lhs; }
friend bool operator <=(const Params & lhs, const Params & rhs) { return !(rhs < lhs); }
friend bool operator >=(const Params & lhs, const Params & rhs) { return !(lhs < rhs); }
};
using ResourceType = Quota::ResourceType;
using ResourceAmount = Quota::ResourceAmount;
/// Default constructors makes an unlimited quota.
QuotaContext();
~QuotaContext();
~EnabledQuota();
/// Tracks resource consumption. If the quota exceeded and `check_exceeded == true`, throws an exception.
void used(ResourceType resource_type, ResourceAmount amount, bool check_exceeded = true) const;
@ -39,15 +52,18 @@ public:
void checkExceeded() const;
void checkExceeded(ResourceType resource_type) const;
/// Returns the information about this quota context.
/// Returns the information about quota consumption.
QuotaUsageInfo getUsageInfo() const;
private:
friend class QuotaContextFactory;
friend struct ext::shared_ptr_helper<QuotaContext>;
/// Returns an instance of EnabledQuota which is never exceeded.
static std::shared_ptr<const EnabledQuota> getUnlimitedQuota();
/// Instances of this class are created by QuotaContextFactory.
QuotaContext(const String & user_name_, const UUID & user_id_, const std::vector<UUID> & enabled_roles_, const Poco::Net::IPAddress & address_, const String & client_key_);
private:
friend class QuotaCache;
EnabledQuota(const Params & params_);
EnabledQuota() {}
const String & getUserName() const { return params.user_name; }
static constexpr size_t MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE;
@ -76,38 +92,8 @@ private:
struct Impl;
const String user_name;
const UUID user_id;
const std::vector<UUID> enabled_roles;
const Poco::Net::IPAddress address;
const String client_key;
const Params params;
boost::atomic_shared_ptr<const Intervals> intervals; /// atomically changed by QuotaUsageManager
};
using QuotaContextPtr = std::shared_ptr<const QuotaContext>;
/// The information about a quota context.
struct QuotaUsageInfo
{
using ResourceType = Quota::ResourceType;
using ResourceAmount = Quota::ResourceAmount;
static constexpr size_t MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE;
struct Interval
{
ResourceAmount used[MAX_RESOURCE_TYPE];
ResourceAmount max[MAX_RESOURCE_TYPE];
std::chrono::seconds duration = std::chrono::seconds::zero();
bool randomize_interval = false;
std::chrono::system_clock::time_point end_of_interval;
Interval();
};
std::vector<Interval> intervals;
UUID quota_id;
String quota_name;
String quota_key;
QuotaUsageInfo();
};
}

View File

@ -0,0 +1,51 @@
#include <Access/EnabledRoles.h>
#include <Access/Role.h>
#include <Access/EnabledRolesInfo.h>
#include <boost/range/algorithm/copy.hpp>
namespace DB
{
EnabledRoles::EnabledRoles(const Params & params_) : params(params_)
{
}
EnabledRoles::~EnabledRoles() = default;
std::shared_ptr<const EnabledRolesInfo> EnabledRoles::getRolesInfo() const
{
std::lock_guard lock{mutex};
return info;
}
ext::scope_guard EnabledRoles::subscribeForChanges(const OnChangeHandler & handler) const
{
std::lock_guard lock{mutex};
handlers.push_back(handler);
auto it = std::prev(handlers.end());
return [this, it]
{
std::lock_guard lock2{mutex};
handlers.erase(it);
};
}
void EnabledRoles::setRolesInfo(const std::shared_ptr<const EnabledRolesInfo> & info_)
{
std::vector<OnChangeHandler> handlers_to_notify;
SCOPE_EXIT({ for (const auto & handler : handlers_to_notify) handler(info_); });
std::lock_guard lock{mutex};
if (info && info_ && *info == *info_)
return;
info = info_;
boost::range::copy(handlers, std::back_inserter(handlers_to_notify));
}
}

View File

@ -0,0 +1,53 @@
#pragma once
#include <Core/UUID.h>
#include <ext/scope_guard.h>
#include <list>
#include <mutex>
#include <vector>
namespace DB
{
struct EnabledRolesInfo;
class EnabledRoles
{
public:
struct Params
{
std::vector<UUID> current_roles;
std::vector<UUID> current_roles_with_admin_option;
auto toTuple() const { return std::tie(current_roles, current_roles_with_admin_option); }
friend bool operator ==(const Params & lhs, const Params & rhs) { return lhs.toTuple() == rhs.toTuple(); }
friend bool operator !=(const Params & lhs, const Params & rhs) { return !(lhs == rhs); }
friend bool operator <(const Params & lhs, const Params & rhs) { return lhs.toTuple() < rhs.toTuple(); }
friend bool operator >(const Params & lhs, const Params & rhs) { return rhs < lhs; }
friend bool operator <=(const Params & lhs, const Params & rhs) { return !(rhs < lhs); }
friend bool operator >=(const Params & lhs, const Params & rhs) { return !(lhs < rhs); }
};
~EnabledRoles();
/// Returns all the roles specified in the constructor.
std::shared_ptr<const EnabledRolesInfo> getRolesInfo() const;
using OnChangeHandler = std::function<void(const std::shared_ptr<const EnabledRolesInfo> & info)>;
/// Called when either the specified roles or the roles granted to the specified roles are changed.
ext::scope_guard subscribeForChanges(const OnChangeHandler & handler) const;
private:
friend class RoleCache;
EnabledRoles(const Params & params_);
void setRolesInfo(const std::shared_ptr<const EnabledRolesInfo> & info_);
const Params params;
mutable std::shared_ptr<const EnabledRolesInfo> info;
mutable std::list<OnChangeHandler> handlers;
mutable std::mutex mutex;
};
}

View File

@ -1,10 +1,10 @@
#include <Access/CurrentRolesInfo.h>
#include <Access/EnabledRolesInfo.h>
namespace DB
{
Strings CurrentRolesInfo::getCurrentRolesNames() const
Strings EnabledRolesInfo::getCurrentRolesNames() const
{
Strings result;
result.reserve(current_roles.size());
@ -14,7 +14,7 @@ Strings CurrentRolesInfo::getCurrentRolesNames() const
}
Strings CurrentRolesInfo::getEnabledRolesNames() const
Strings EnabledRolesInfo::getEnabledRolesNames() const
{
Strings result;
result.reserve(enabled_roles.size());
@ -24,7 +24,7 @@ Strings CurrentRolesInfo::getEnabledRolesNames() const
}
bool operator==(const CurrentRolesInfo & lhs, const CurrentRolesInfo & rhs)
bool operator==(const EnabledRolesInfo & lhs, const EnabledRolesInfo & rhs)
{
return (lhs.current_roles == rhs.current_roles) && (lhs.enabled_roles == rhs.enabled_roles)
&& (lhs.enabled_roles_with_admin_option == rhs.enabled_roles_with_admin_option) && (lhs.names_of_roles == rhs.names_of_roles)

View File

@ -10,7 +10,7 @@ namespace DB
{
/// Information about a role.
struct CurrentRolesInfo
struct EnabledRolesInfo
{
std::vector<UUID> current_roles;
std::vector<UUID> enabled_roles;
@ -22,10 +22,8 @@ struct CurrentRolesInfo
Strings getCurrentRolesNames() const;
Strings getEnabledRolesNames() const;
friend bool operator ==(const CurrentRolesInfo & lhs, const CurrentRolesInfo & rhs);
friend bool operator !=(const CurrentRolesInfo & lhs, const CurrentRolesInfo & rhs) { return !(lhs == rhs); }
friend bool operator ==(const EnabledRolesInfo & lhs, const EnabledRolesInfo & rhs);
friend bool operator !=(const EnabledRolesInfo & lhs, const EnabledRolesInfo & rhs) { return !(lhs == rhs); }
};
using CurrentRolesInfoPtr = std::shared_ptr<const CurrentRolesInfo>;
}

View File

@ -1,4 +1,4 @@
#include <Access/RowPolicyContext.h>
#include <Access/EnabledRowPolicies.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTExpressionList.h>
#include <boost/smart_ptr/make_shared.hpp>
@ -8,55 +8,50 @@
namespace DB
{
size_t RowPolicyContext::Hash::operator()(const DatabaseAndTableNameRef & database_and_table_name) const
size_t EnabledRowPolicies::Hash::operator()(const DatabaseAndTableNameRef & database_and_table_name) const
{
return std::hash<std::string_view>{}(database_and_table_name.first) - std::hash<std::string_view>{}(database_and_table_name.second);
}
RowPolicyContext::RowPolicyContext()
: map_of_mixed_conditions(boost::make_shared<MapOfMixedConditions>())
EnabledRowPolicies::EnabledRowPolicies(const Params & params_)
: params(params_)
{
}
RowPolicyContext::~RowPolicyContext() = default;
EnabledRowPolicies::~EnabledRowPolicies() = default;
RowPolicyContext::RowPolicyContext(const UUID & user_id_, const std::vector<UUID> & enabled_roles_)
: user_id(user_id_), enabled_roles(enabled_roles_)
{}
ASTPtr RowPolicyContext::getCondition(const String & database, const String & table_name, ConditionIndex index) const
ASTPtr EnabledRowPolicies::getCondition(const String & database, const String & table_name, ConditionType type) const
{
/// We don't lock `mutex` here.
auto loaded = map_of_mixed_conditions.load();
auto it = loaded->find({database, table_name});
if (it == loaded->end())
return {};
return it->second.mixed_conditions[index];
return it->second.mixed_conditions[type];
}
ASTPtr RowPolicyContext::combineConditionsUsingAnd(const ASTPtr & lhs, const ASTPtr & rhs)
ASTPtr EnabledRowPolicies::getCondition(const String & database, const String & table_name, ConditionType type, const ASTPtr & extra_condition) const
{
if (!lhs)
return rhs;
if (!rhs)
return lhs;
ASTPtr main_condition = getCondition(database, table_name, type);
if (!main_condition)
return extra_condition;
if (!extra_condition)
return main_condition;
auto function = std::make_shared<ASTFunction>();
auto exp_list = std::make_shared<ASTExpressionList>();
function->name = "and";
function->arguments = exp_list;
function->children.push_back(exp_list);
exp_list->children.push_back(lhs);
exp_list->children.push_back(rhs);
exp_list->children.push_back(main_condition);
exp_list->children.push_back(extra_condition);
return function;
}
std::vector<UUID> RowPolicyContext::getCurrentPolicyIDs() const
std::vector<UUID> EnabledRowPolicies::getCurrentPolicyIDs() const
{
/// We don't lock `mutex` here.
auto loaded = map_of_mixed_conditions.load();
@ -67,7 +62,7 @@ std::vector<UUID> RowPolicyContext::getCurrentPolicyIDs() const
}
std::vector<UUID> RowPolicyContext::getCurrentPolicyIDs(const String & database, const String & table_name) const
std::vector<UUID> EnabledRowPolicies::getCurrentPolicyIDs(const String & database, const String & table_name) const
{
/// We don't lock `mutex` here.
auto loaded = map_of_mixed_conditions.load();
@ -76,4 +71,5 @@ std::vector<UUID> RowPolicyContext::getCurrentPolicyIDs(const String & database,
return {};
return it->second.policy_ids;
}
}

View File

@ -15,23 +15,32 @@ using ASTPtr = std::shared_ptr<IAST>;
/// Provides fast access to row policies' conditions for a specific user and tables.
class RowPolicyContext
class EnabledRowPolicies
{
public:
/// Default constructor makes a row policy usage context which restricts nothing.
RowPolicyContext();
struct Params
{
UUID user_id;
std::vector<UUID> enabled_roles;
~RowPolicyContext();
auto toTuple() const { return std::tie(user_id, enabled_roles); }
friend bool operator ==(const Params & lhs, const Params & rhs) { return lhs.toTuple() == rhs.toTuple(); }
friend bool operator !=(const Params & lhs, const Params & rhs) { return !(lhs == rhs); }
friend bool operator <(const Params & lhs, const Params & rhs) { return lhs.toTuple() < rhs.toTuple(); }
friend bool operator >(const Params & lhs, const Params & rhs) { return rhs < lhs; }
friend bool operator <=(const Params & lhs, const Params & rhs) { return !(rhs < lhs); }
friend bool operator >=(const Params & lhs, const Params & rhs) { return !(lhs < rhs); }
};
using ConditionIndex = RowPolicy::ConditionIndex;
~EnabledRowPolicies();
using ConditionType = RowPolicy::ConditionType;
/// Returns prepared filter for a specific table and operations.
/// The function can return nullptr, that means there is no filters applied.
/// The returned filter can be a combination of the filters defined by multiple row policies.
ASTPtr getCondition(const String & database, const String & table_name, ConditionIndex index) const;
/// Combines two conditions into one by using the logical AND operator.
static ASTPtr combineConditionsUsingAnd(const ASTPtr & lhs, const ASTPtr & rhs);
ASTPtr getCondition(const String & database, const String & table_name, ConditionType type) const;
ASTPtr getCondition(const String & database, const String & table_name, ConditionType type, const ASTPtr & extra_condition) const;
/// Returns IDs of all the policies used by the current user.
std::vector<UUID> getCurrentPolicyIDs() const;
@ -40,9 +49,8 @@ public:
std::vector<UUID> getCurrentPolicyIDs(const String & database, const String & table_name) const;
private:
friend class RowPolicyContextFactory;
friend struct ext::shared_ptr_helper<RowPolicyContext>;
RowPolicyContext(const UUID & user_id_, const std::vector<UUID> & enabled_roles_); /// RowPolicyContext should be created by RowPolicyContextFactory.
friend class RowPolicyCache;
EnabledRowPolicies(const Params & params_);
using DatabaseAndTableName = std::pair<String, String>;
using DatabaseAndTableNameRef = std::pair<std::string_view, std::string_view>;
@ -50,8 +58,8 @@ private:
{
size_t operator()(const DatabaseAndTableNameRef & database_and_table_name) const;
};
static constexpr size_t MAX_CONDITION_INDEX = RowPolicy::MAX_CONDITION_INDEX;
using ParsedConditions = std::array<ASTPtr, MAX_CONDITION_INDEX>;
static constexpr size_t MAX_CONDITION_TYPE = RowPolicy::MAX_CONDITION_TYPE;
using ParsedConditions = std::array<ASTPtr, MAX_CONDITION_TYPE>;
struct MixedConditions
{
std::unique_ptr<DatabaseAndTableName> database_and_table_name_keeper;
@ -60,11 +68,8 @@ private:
};
using MapOfMixedConditions = std::unordered_map<DatabaseAndTableNameRef, MixedConditions, Hash>;
const UUID user_id;
const std::vector<UUID> enabled_roles;
const Params params;
mutable boost::atomic_shared_ptr<const MapOfMixedConditions> map_of_mixed_conditions;
};
using RowPolicyContextPtr = std::shared_ptr<const RowPolicyContext>;
}

View File

@ -1,8 +1,8 @@
#include <Access/GenericRoleSet.h>
#include <Access/ExtendedRoleSet.h>
#include <Access/AccessControlManager.h>
#include <Access/User.h>
#include <Access/Role.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Parsers/formatAST.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
@ -17,57 +17,59 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
GenericRoleSet::GenericRoleSet() = default;
GenericRoleSet::GenericRoleSet(const GenericRoleSet & src) = default;
GenericRoleSet & GenericRoleSet::operator =(const GenericRoleSet & src) = default;
GenericRoleSet::GenericRoleSet(GenericRoleSet && src) = default;
GenericRoleSet & GenericRoleSet::operator =(GenericRoleSet && src) = default;
GenericRoleSet::GenericRoleSet(AllTag)
ExtendedRoleSet::ExtendedRoleSet() = default;
ExtendedRoleSet::ExtendedRoleSet(const ExtendedRoleSet & src) = default;
ExtendedRoleSet & ExtendedRoleSet::operator =(const ExtendedRoleSet & src) = default;
ExtendedRoleSet::ExtendedRoleSet(ExtendedRoleSet && src) = default;
ExtendedRoleSet & ExtendedRoleSet::operator =(ExtendedRoleSet && src) = default;
ExtendedRoleSet::ExtendedRoleSet(AllTag)
{
all = true;
}
GenericRoleSet::GenericRoleSet(const UUID & id)
ExtendedRoleSet::ExtendedRoleSet(const UUID & id)
{
add(id);
}
GenericRoleSet::GenericRoleSet(const std::vector<UUID> & ids_)
ExtendedRoleSet::ExtendedRoleSet(const std::vector<UUID> & ids_)
{
add(ids_);
}
GenericRoleSet::GenericRoleSet(const boost::container::flat_set<UUID> & ids_)
ExtendedRoleSet::ExtendedRoleSet(const boost::container::flat_set<UUID> & ids_)
{
add(ids_);
}
GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast)
ExtendedRoleSet::ExtendedRoleSet(const ASTExtendedRoleSet & ast)
{
init(ast, nullptr, nullptr);
}
GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast, const UUID & current_user_id)
ExtendedRoleSet::ExtendedRoleSet(const ASTExtendedRoleSet & ast, const UUID & current_user_id)
{
init(ast, nullptr, &current_user_id);
}
GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager)
ExtendedRoleSet::ExtendedRoleSet(const ASTExtendedRoleSet & ast, const AccessControlManager & manager)
{
init(ast, &manager, nullptr);
}
GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager, const UUID & current_user_id)
ExtendedRoleSet::ExtendedRoleSet(const ASTExtendedRoleSet & ast, const AccessControlManager & manager, const UUID & current_user_id)
{
init(ast, &manager, &current_user_id);
}
void GenericRoleSet::init(const ASTGenericRoleSet & ast, const AccessControlManager * manager, const UUID * current_user_id)
void ExtendedRoleSet::init(const ASTExtendedRoleSet & ast, const AccessControlManager * manager, const UUID * current_user_id)
{
all = ast.all;
@ -113,9 +115,9 @@ void GenericRoleSet::init(const ASTGenericRoleSet & ast, const AccessControlMana
}
std::shared_ptr<ASTGenericRoleSet> GenericRoleSet::toAST() const
std::shared_ptr<ASTExtendedRoleSet> ExtendedRoleSet::toAST() const
{
auto ast = std::make_shared<ASTGenericRoleSet>();
auto ast = std::make_shared<ASTExtendedRoleSet>();
ast->id_mode = true;
ast->all = all;
@ -137,14 +139,14 @@ std::shared_ptr<ASTGenericRoleSet> GenericRoleSet::toAST() const
}
String GenericRoleSet::toString() const
String ExtendedRoleSet::toString() const
{
auto ast = toAST();
return serializeAST(*ast);
}
Strings GenericRoleSet::toStrings() const
Strings ExtendedRoleSet::toStrings() const
{
if (all || !except_ids.empty())
return {toString()};
@ -157,9 +159,9 @@ Strings GenericRoleSet::toStrings() const
}
std::shared_ptr<ASTGenericRoleSet> GenericRoleSet::toASTWithNames(const AccessControlManager & manager) const
std::shared_ptr<ASTExtendedRoleSet> ExtendedRoleSet::toASTWithNames(const AccessControlManager & manager) const
{
auto ast = std::make_shared<ASTGenericRoleSet>();
auto ast = std::make_shared<ASTExtendedRoleSet>();
ast->all = all;
if (!ids.empty())
@ -190,14 +192,14 @@ std::shared_ptr<ASTGenericRoleSet> GenericRoleSet::toASTWithNames(const AccessCo
}
String GenericRoleSet::toStringWithNames(const AccessControlManager & manager) const
String ExtendedRoleSet::toStringWithNames(const AccessControlManager & manager) const
{
auto ast = toASTWithNames(manager);
return serializeAST(*ast);
}
Strings GenericRoleSet::toStringsWithNames(const AccessControlManager & manager) const
Strings ExtendedRoleSet::toStringsWithNames(const AccessControlManager & manager) const
{
if (all || !except_ids.empty())
return {toStringWithNames(manager)};
@ -215,13 +217,13 @@ Strings GenericRoleSet::toStringsWithNames(const AccessControlManager & manager)
}
bool GenericRoleSet::empty() const
bool ExtendedRoleSet::empty() const
{
return ids.empty() && !all;
}
void GenericRoleSet::clear()
void ExtendedRoleSet::clear()
{
ids.clear();
all = false;
@ -229,33 +231,33 @@ void GenericRoleSet::clear()
}
void GenericRoleSet::add(const UUID & id)
void ExtendedRoleSet::add(const UUID & id)
{
ids.insert(id);
}
void GenericRoleSet::add(const std::vector<UUID> & ids_)
void ExtendedRoleSet::add(const std::vector<UUID> & ids_)
{
for (const auto & id : ids_)
add(id);
}
void GenericRoleSet::add(const boost::container::flat_set<UUID> & ids_)
void ExtendedRoleSet::add(const boost::container::flat_set<UUID> & ids_)
{
for (const auto & id : ids_)
add(id);
}
bool GenericRoleSet::match(const UUID & id) const
bool ExtendedRoleSet::match(const UUID & id) const
{
return (all || ids.contains(id)) && !except_ids.contains(id);
}
bool GenericRoleSet::match(const UUID & user_id, const std::vector<UUID> & enabled_roles) const
bool ExtendedRoleSet::match(const UUID & user_id, const std::vector<UUID> & enabled_roles) const
{
if (!all && !ids.contains(user_id))
{
@ -274,7 +276,7 @@ bool GenericRoleSet::match(const UUID & user_id, const std::vector<UUID> & enabl
}
bool GenericRoleSet::match(const UUID & user_id, const boost::container::flat_set<UUID> & enabled_roles) const
bool ExtendedRoleSet::match(const UUID & user_id, const boost::container::flat_set<UUID> & enabled_roles) const
{
if (!all && !ids.contains(user_id))
{
@ -293,17 +295,17 @@ bool GenericRoleSet::match(const UUID & user_id, const boost::container::flat_se
}
std::vector<UUID> GenericRoleSet::getMatchingIDs() const
std::vector<UUID> ExtendedRoleSet::getMatchingIDs() const
{
if (all)
throw Exception("getAllMatchingIDs() can't get ALL ids", ErrorCodes::LOGICAL_ERROR);
throw Exception("getAllMatchingIDs() can't get ALL ids without manager", ErrorCodes::LOGICAL_ERROR);
std::vector<UUID> res;
boost::range::set_difference(ids, except_ids, std::back_inserter(res));
return res;
}
std::vector<UUID> GenericRoleSet::getMatchingUsers(const AccessControlManager & manager) const
std::vector<UUID> ExtendedRoleSet::getMatchingIDs(const AccessControlManager & manager) const
{
if (!all)
return getMatchingIDs();
@ -314,37 +316,17 @@ std::vector<UUID> GenericRoleSet::getMatchingUsers(const AccessControlManager &
if (match(id))
res.push_back(id);
}
return res;
}
std::vector<UUID> GenericRoleSet::getMatchingRoles(const AccessControlManager & manager) const
{
if (!all)
return getMatchingIDs();
std::vector<UUID> res;
for (const UUID & id : manager.findAll<Role>())
{
if (match(id))
res.push_back(id);
}
return res;
}
std::vector<UUID> GenericRoleSet::getMatchingUsersAndRoles(const AccessControlManager & manager) const
{
if (!all)
return getMatchingIDs();
std::vector<UUID> vec = getMatchingUsers(manager);
boost::range::push_back(vec, getMatchingRoles(manager));
return vec;
}
bool operator ==(const GenericRoleSet & lhs, const GenericRoleSet & rhs)
bool operator ==(const ExtendedRoleSet & lhs, const ExtendedRoleSet & rhs)
{
return (lhs.all == rhs.all) && (lhs.ids == rhs.ids) && (lhs.except_ids == rhs.except_ids);
}

View File

@ -0,0 +1,75 @@
#pragma once
#include <Core/UUID.h>
#include <boost/container/flat_set.hpp>
#include <memory>
#include <optional>
namespace DB
{
class ASTExtendedRoleSet;
class AccessControlManager;
/// Represents a set of users/roles like
/// {user_name | role_name | CURRENT_USER} [,...] | NONE | ALL | ALL EXCEPT {user_name | role_name | CURRENT_USER} [,...]
/// Similar to ASTExtendedRoleSet, but with IDs instead of names.
struct ExtendedRoleSet
{
ExtendedRoleSet();
ExtendedRoleSet(const ExtendedRoleSet & src);
ExtendedRoleSet & operator =(const ExtendedRoleSet & src);
ExtendedRoleSet(ExtendedRoleSet && src);
ExtendedRoleSet & operator =(ExtendedRoleSet && src);
struct AllTag {};
ExtendedRoleSet(AllTag);
ExtendedRoleSet(const UUID & id);
ExtendedRoleSet(const std::vector<UUID> & ids_);
ExtendedRoleSet(const boost::container::flat_set<UUID> & ids_);
/// The constructor from AST requires the AccessControlManager if `ast.id_mode == false`.
ExtendedRoleSet(const ASTExtendedRoleSet & ast);
ExtendedRoleSet(const ASTExtendedRoleSet & ast, const UUID & current_user_id);
ExtendedRoleSet(const ASTExtendedRoleSet & ast, const AccessControlManager & manager);
ExtendedRoleSet(const ASTExtendedRoleSet & ast, const AccessControlManager & manager, const UUID & current_user_id);
std::shared_ptr<ASTExtendedRoleSet> toAST() const;
String toString() const;
Strings toStrings() const;
std::shared_ptr<ASTExtendedRoleSet> toASTWithNames(const AccessControlManager & manager) const;
String toStringWithNames(const AccessControlManager & manager) const;
Strings toStringsWithNames(const AccessControlManager & manager) const;
bool empty() const;
void clear();
void add(const UUID & id);
void add(const std::vector<UUID> & ids_);
void add(const boost::container::flat_set<UUID> & ids_);
/// Checks if a specified ID matches this ExtendedRoleSet.
bool match(const UUID & id) const;
bool match(const UUID & user_id, const std::vector<UUID> & enabled_roles) const;
bool match(const UUID & user_id, const boost::container::flat_set<UUID> & enabled_roles) const;
/// Returns a list of matching IDs. The function must not be called if `all` == `true`.
std::vector<UUID> getMatchingIDs() const;
/// Returns a list of matching users and roles.
std::vector<UUID> getMatchingIDs(const AccessControlManager & manager) const;
friend bool operator ==(const ExtendedRoleSet & lhs, const ExtendedRoleSet & rhs);
friend bool operator !=(const ExtendedRoleSet & lhs, const ExtendedRoleSet & rhs) { return !(lhs == rhs); }
boost::container::flat_set<UUID> ids;
bool all = false;
boost::container::flat_set<UUID> except_ids;
private:
void init(const ASTExtendedRoleSet & ast, const AccessControlManager * manager = nullptr, const UUID * current_user_id = nullptr);
};
}

View File

@ -1,77 +0,0 @@
#pragma once
#include <Core/UUID.h>
#include <boost/container/flat_set.hpp>
#include <memory>
#include <optional>
namespace DB
{
class ASTGenericRoleSet;
class AccessControlManager;
/// Represents a set of users/roles like
/// {user_name | role_name | CURRENT_USER} [,...] | NONE | ALL | ALL EXCEPT {user_name | role_name | CURRENT_USER} [,...]
/// Similar to ASTGenericRoleSet, but with IDs instead of names.
struct GenericRoleSet
{
GenericRoleSet();
GenericRoleSet(const GenericRoleSet & src);
GenericRoleSet & operator =(const GenericRoleSet & src);
GenericRoleSet(GenericRoleSet && src);
GenericRoleSet & operator =(GenericRoleSet && src);
struct AllTag {};
GenericRoleSet(AllTag);
GenericRoleSet(const UUID & id);
GenericRoleSet(const std::vector<UUID> & ids_);
GenericRoleSet(const boost::container::flat_set<UUID> & ids_);
/// The constructor from AST requires the AccessControlManager if `ast.id_mode == false`.
GenericRoleSet(const ASTGenericRoleSet & ast);
GenericRoleSet(const ASTGenericRoleSet & ast, const UUID & current_user_id);
GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager);
GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager, const UUID & current_user_id);
std::shared_ptr<ASTGenericRoleSet> toAST() const;
String toString() const;
Strings toStrings() const;
std::shared_ptr<ASTGenericRoleSet> toASTWithNames(const AccessControlManager & manager) const;
String toStringWithNames(const AccessControlManager & manager) const;
Strings toStringsWithNames(const AccessControlManager & manager) const;
bool empty() const;
void clear();
void add(const UUID & id);
void add(const std::vector<UUID> & ids_);
void add(const boost::container::flat_set<UUID> & ids_);
/// Checks if a specified ID matches this GenericRoleSet.
bool match(const UUID & id) const;
bool match(const UUID & user_id, const std::vector<UUID> & enabled_roles) const;
bool match(const UUID & user_id, const boost::container::flat_set<UUID> & enabled_roles) const;
/// Returns a list of matching IDs. The function must not be called if `all` == `true`.
std::vector<UUID> getMatchingIDs() const;
/// Returns a list of matching users.
std::vector<UUID> getMatchingUsers(const AccessControlManager & manager) const;
std::vector<UUID> getMatchingRoles(const AccessControlManager & manager) const;
std::vector<UUID> getMatchingUsersAndRoles(const AccessControlManager & manager) const;
friend bool operator ==(const GenericRoleSet & lhs, const GenericRoleSet & rhs);
friend bool operator !=(const GenericRoleSet & lhs, const GenericRoleSet & rhs) { return !(lhs == rhs); }
boost::container::flat_set<UUID> ids;
bool all = false;
boost::container::flat_set<UUID> except_ids;
private:
void init(const ASTGenericRoleSet & ast, const AccessControlManager * manager = nullptr, const UUID * current_user_id = nullptr);
};
}

View File

@ -23,7 +23,7 @@ bool Quota::equal(const IAccessEntity & other) const
if (!IAccessEntity::equal(other))
return false;
const auto & other_quota = typeid_cast<const Quota &>(other);
return (all_limits == other_quota.all_limits) && (key_type == other_quota.key_type) && (roles == other_quota.roles);
return (all_limits == other_quota.all_limits) && (key_type == other_quota.key_type) && (to_roles == other_quota.to_roles);
}

View File

@ -1,7 +1,7 @@
#pragma once
#pragma once
#include <Access/IAccessEntity.h>
#include <Access/GenericRoleSet.h>
#include <Access/ExtendedRoleSet.h>
#include <chrono>
@ -63,7 +63,7 @@ struct Quota : public IAccessEntity
KeyType key_type = KeyType::NONE;
/// Which roles or users should use this quota.
GenericRoleSet roles;
ExtendedRoleSet to_roles;
bool equal(const IAccessEntity & other) const override;
std::shared_ptr<IAccessEntity> clone() const override { return cloneImpl<Quota>(); }

View File

@ -1,5 +1,6 @@
#include <Access/QuotaContext.h>
#include <Access/QuotaContextFactory.h>
#include <Access/EnabledQuota.h>
#include <Access/QuotaCache.h>
#include <Access/QuotaUsageInfo.h>
#include <Access/AccessControlManager.h>
#include <Common/Exception.h>
#include <Common/thread_local_rng.h>
@ -8,7 +9,6 @@
#include <boost/range/algorithm/copy.hpp>
#include <boost/range/algorithm/lower_bound.hpp>
#include <boost/range/algorithm/stable_sort.hpp>
#include <boost/range/algorithm_ext/erase.hpp>
#include <boost/smart_ptr/make_shared.hpp>
@ -31,58 +31,53 @@ namespace
}
void QuotaContextFactory::QuotaInfo::setQuota(const QuotaPtr & quota_, const UUID & quota_id_)
void QuotaCache::QuotaInfo::setQuota(const QuotaPtr & quota_, const UUID & quota_id_)
{
quota = quota_;
quota_id = quota_id_;
roles = &quota->roles;
roles = &quota->to_roles;
rebuildAllIntervals();
}
bool QuotaContextFactory::QuotaInfo::canUseWithContext(const QuotaContext & context) const
{
return roles->match(context.user_id, context.enabled_roles);
}
String QuotaContextFactory::QuotaInfo::calculateKey(const QuotaContext & context) const
String QuotaCache::QuotaInfo::calculateKey(const EnabledQuota & enabled) const
{
const auto & params = enabled.params;
using KeyType = Quota::KeyType;
switch (quota->key_type)
{
case KeyType::NONE:
return "";
case KeyType::USER_NAME:
return context.user_name;
return params.user_name;
case KeyType::IP_ADDRESS:
return context.address.toString();
return params.client_address.toString();
case KeyType::CLIENT_KEY:
{
if (!context.client_key.empty())
return context.client_key;
if (!params.client_key.empty())
return params.client_key;
throw Exception(
"Quota " + quota->getName() + " (for user " + context.user_name + ") requires a client supplied key.",
"Quota " + quota->getName() + " (for user " + params.user_name + ") requires a client supplied key.",
ErrorCodes::QUOTA_REQUIRES_CLIENT_KEY);
}
case KeyType::CLIENT_KEY_OR_USER_NAME:
{
if (!context.client_key.empty())
return context.client_key;
return context.user_name;
if (!params.client_key.empty())
return params.client_key;
return params.user_name;
}
case KeyType::CLIENT_KEY_OR_IP_ADDRESS:
{
if (!context.client_key.empty())
return context.client_key;
return context.address.toString();
if (!params.client_key.empty())
return params.client_key;
return params.client_address.toString();
}
}
__builtin_unreachable();
}
boost::shared_ptr<const QuotaContext::Intervals> QuotaContextFactory::QuotaInfo::getOrBuildIntervals(const String & key)
boost::shared_ptr<const EnabledQuota::Intervals> QuotaCache::QuotaInfo::getOrBuildIntervals(const String & key)
{
auto it = key_to_intervals.find(key);
if (it != key_to_intervals.end())
@ -91,14 +86,14 @@ boost::shared_ptr<const QuotaContext::Intervals> QuotaContextFactory::QuotaInfo:
}
void QuotaContextFactory::QuotaInfo::rebuildAllIntervals()
void QuotaCache::QuotaInfo::rebuildAllIntervals()
{
for (const String & key : key_to_intervals | boost::adaptors::map_keys)
rebuildIntervals(key);
}
boost::shared_ptr<const QuotaContext::Intervals> QuotaContextFactory::QuotaInfo::rebuildIntervals(const String & key)
boost::shared_ptr<const EnabledQuota::Intervals> QuotaCache::QuotaInfo::rebuildIntervals(const String & key)
{
auto new_intervals = boost::make_shared<Intervals>();
new_intervals->quota_name = quota->getName();
@ -164,27 +159,42 @@ boost::shared_ptr<const QuotaContext::Intervals> QuotaContextFactory::QuotaInfo:
}
QuotaContextFactory::QuotaContextFactory(const AccessControlManager & access_control_manager_)
QuotaCache::QuotaCache(const AccessControlManager & access_control_manager_)
: access_control_manager(access_control_manager_)
{
}
QuotaContextFactory::~QuotaContextFactory() = default;
QuotaCache::~QuotaCache() = default;
QuotaContextPtr QuotaContextFactory::createContext(const String & user_name, const UUID & user_id, const std::vector<UUID> & enabled_roles, const Poco::Net::IPAddress & address, const String & client_key)
std::shared_ptr<const EnabledQuota> QuotaCache::getEnabledQuota(const UUID & user_id, const String & user_name, const std::vector<UUID> & enabled_roles, const Poco::Net::IPAddress & client_address, const String & client_key)
{
std::lock_guard lock{mutex};
ensureAllQuotasRead();
auto context = ext::shared_ptr_helper<QuotaContext>::create(user_name, user_id, enabled_roles, address, client_key);
contexts.push_back(context);
chooseQuotaForContext(context);
return context;
EnabledQuota::Params params;
params.user_id = user_id;
params.user_name = user_name;
params.enabled_roles = enabled_roles;
params.client_address = client_address;
params.client_key = client_key;
auto it = enabled_quotas.find(params);
if (it != enabled_quotas.end())
{
auto from_cache = it->second.lock();
if (from_cache)
return from_cache;
enabled_quotas.erase(it);
}
auto res = std::shared_ptr<EnabledQuota>(new EnabledQuota(params));
enabled_quotas.emplace(std::move(params), res);
chooseQuotaToConsumeFor(*res);
return res;
}
void QuotaContextFactory::ensureAllQuotasRead()
void QuotaCache::ensureAllQuotasRead()
{
/// `mutex` is already locked.
if (all_quotas_read)
@ -209,7 +219,7 @@ void QuotaContextFactory::ensureAllQuotasRead()
}
void QuotaContextFactory::quotaAddedOrChanged(const UUID & quota_id, const std::shared_ptr<const Quota> & new_quota)
void QuotaCache::quotaAddedOrChanged(const UUID & quota_id, const std::shared_ptr<const Quota> & new_quota)
{
std::lock_guard lock{mutex};
auto it = all_quotas.find(quota_id);
@ -225,42 +235,42 @@ void QuotaContextFactory::quotaAddedOrChanged(const UUID & quota_id, const std::
auto & info = it->second;
info.setQuota(new_quota, quota_id);
chooseQuotaForAllContexts();
chooseQuotaToConsume();
}
void QuotaContextFactory::quotaRemoved(const UUID & quota_id)
void QuotaCache::quotaRemoved(const UUID & quota_id)
{
std::lock_guard lock{mutex};
all_quotas.erase(quota_id);
chooseQuotaForAllContexts();
chooseQuotaToConsume();
}
void QuotaContextFactory::chooseQuotaForAllContexts()
void QuotaCache::chooseQuotaToConsume()
{
/// `mutex` is already locked.
boost::range::remove_erase_if(
contexts,
[&](const std::weak_ptr<QuotaContext> & weak)
std::erase_if(
enabled_quotas,
[&](const std::pair<EnabledQuota::Params, std::weak_ptr<EnabledQuota>> & pr)
{
auto context = weak.lock();
if (!context)
return true; // remove from the `contexts` list.
chooseQuotaForContext(context);
return false; // keep in the `contexts` list.
auto elem = pr.second.lock();
if (!elem)
return true; // remove from the `enabled_quotas` list.
chooseQuotaToConsumeFor(*elem);
return false; // keep in the `enabled_quotas` list.
});
}
void QuotaContextFactory::chooseQuotaForContext(const std::shared_ptr<QuotaContext> & context)
void QuotaCache::chooseQuotaToConsumeFor(EnabledQuota & enabled)
{
/// `mutex` is already locked.
boost::shared_ptr<const Intervals> intervals;
for (auto & info : all_quotas | boost::adaptors::map_values)
{
if (info.canUseWithContext(*context))
if (info.roles->match(enabled.params.user_id, enabled.params.enabled_roles))
{
String key = info.calculateKey(*context);
String key = info.calculateKey(enabled);
intervals = info.getOrBuildIntervals(key);
break;
}
@ -269,11 +279,11 @@ void QuotaContextFactory::chooseQuotaForContext(const std::shared_ptr<QuotaConte
if (!intervals)
intervals = boost::make_shared<Intervals>(); /// No quota == no limits.
context->intervals.store(intervals);
enabled.intervals.store(intervals);
}
std::vector<QuotaUsageInfo> QuotaContextFactory::getUsageInfo() const
std::vector<QuotaUsageInfo> QuotaCache::getUsageInfo() const
{
std::lock_guard lock{mutex};
std::vector<QuotaUsageInfo> all_infos;

View File

@ -1,11 +1,11 @@
#pragma once
#include <Access/QuotaContext.h>
#include <Access/EnabledQuota.h>
#include <ext/scope_guard.h>
#include <memory>
#include <mutex>
#include <map>
#include <unordered_map>
#include <unordered_set>
namespace DB
@ -14,47 +14,46 @@ class AccessControlManager;
/// Stores information how much amount of resources have been consumed and how much are left.
class QuotaContextFactory
class QuotaCache
{
public:
QuotaContextFactory(const AccessControlManager & access_control_manager_);
~QuotaContextFactory();
QuotaCache(const AccessControlManager & access_control_manager_);
~QuotaCache();
QuotaContextPtr createContext(const String & user_name, const UUID & user_id, const std::vector<UUID> & enabled_roles, const Poco::Net::IPAddress & address, const String & client_key);
std::shared_ptr<const EnabledQuota> getEnabledQuota(const UUID & user_id, const String & user_name, const std::vector<UUID> & enabled_roles, const Poco::Net::IPAddress & address, const String & client_key);
std::vector<QuotaUsageInfo> getUsageInfo() const;
private:
using Interval = QuotaContext::Interval;
using Intervals = QuotaContext::Intervals;
using Interval = EnabledQuota::Interval;
using Intervals = EnabledQuota::Intervals;
struct QuotaInfo
{
QuotaInfo(const QuotaPtr & quota_, const UUID & quota_id_) { setQuota(quota_, quota_id_); }
void setQuota(const QuotaPtr & quota_, const UUID & quota_id_);
bool canUseWithContext(const QuotaContext & context) const;
String calculateKey(const QuotaContext & context) const;
String calculateKey(const EnabledQuota & enabled_quota) const;
boost::shared_ptr<const Intervals> getOrBuildIntervals(const String & key);
boost::shared_ptr<const Intervals> rebuildIntervals(const String & key);
void rebuildAllIntervals();
QuotaPtr quota;
UUID quota_id;
const GenericRoleSet * roles = nullptr;
const ExtendedRoleSet * roles = nullptr;
std::unordered_map<String /* quota key */, boost::shared_ptr<const Intervals>> key_to_intervals;
};
void ensureAllQuotasRead();
void quotaAddedOrChanged(const UUID & quota_id, const std::shared_ptr<const Quota> & new_quota);
void quotaRemoved(const UUID & quota_id);
void chooseQuotaForAllContexts();
void chooseQuotaForContext(const std::shared_ptr<QuotaContext> & context);
void chooseQuotaToConsume();
void chooseQuotaToConsumeFor(EnabledQuota & enabled_quota);
const AccessControlManager & access_control_manager;
mutable std::mutex mutex;
std::unordered_map<UUID /* quota id */, QuotaInfo> all_quotas;
bool all_quotas_read = false;
ext::scope_guard subscription;
std::vector<std::weak_ptr<QuotaContext>> contexts;
std::map<EnabledQuota::Params, std::weak_ptr<EnabledQuota>> enabled_quotas;
};
}

View File

@ -0,0 +1,17 @@
#include <Access/QuotaUsageInfo.h>
#include <boost/range/algorithm/fill.hpp>
namespace DB
{
QuotaUsageInfo::QuotaUsageInfo() : quota_id(UUID(UInt128(0)))
{
}
QuotaUsageInfo::Interval::Interval()
{
boost::range::fill(used, 0);
boost::range::fill(max, 0);
}
}

View File

@ -0,0 +1,32 @@
#pragma once
#include <Access/Quota.h>
#include <chrono>
namespace DB
{
/// The information about a quota consumption.
struct QuotaUsageInfo
{
using ResourceType = Quota::ResourceType;
using ResourceAmount = Quota::ResourceAmount;
static constexpr size_t MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE;
struct Interval
{
ResourceAmount used[MAX_RESOURCE_TYPE];
ResourceAmount max[MAX_RESOURCE_TYPE];
std::chrono::seconds duration = std::chrono::seconds::zero();
bool randomize_interval = false;
std::chrono::system_clock::time_point end_of_interval;
Interval();
};
std::vector<Interval> intervals;
UUID quota_id;
String quota_name;
String quota_key;
QuotaUsageInfo();
};
}

View File

@ -0,0 +1,186 @@
#include <Access/RoleCache.h>
#include <Access/Role.h>
#include <Access/EnabledRolesInfo.h>
#include <Access/AccessControlManager.h>
#include <boost/container/flat_map.hpp>
namespace DB
{
namespace
{
struct CollectedRoleInfo
{
RolePtr role;
bool is_current_role = false;
bool with_admin_option = false;
};
void collectRoles(boost::container::flat_map<UUID, CollectedRoleInfo> & collected_roles,
const std::function<RolePtr(const UUID &)> & get_role_function,
const UUID & role_id,
bool is_current_role,
bool with_admin_option)
{
auto it = collected_roles.find(role_id);
if (it != collected_roles.end())
{
it->second.is_current_role |= is_current_role;
it->second.with_admin_option |= with_admin_option;
return;
}
auto role = get_role_function(role_id);
collected_roles[role_id] = CollectedRoleInfo{role, is_current_role, with_admin_option};
if (!role)
return;
for (const auto & granted_role : role->granted_roles)
collectRoles(collected_roles, get_role_function, granted_role, false, false);
for (const auto & granted_role : role->granted_roles_with_admin_option)
collectRoles(collected_roles, get_role_function, granted_role, false, true);
}
std::shared_ptr<EnabledRolesInfo> collectInfoForRoles(const boost::container::flat_map<UUID, CollectedRoleInfo> & roles)
{
auto new_info = std::make_shared<EnabledRolesInfo>();
for (const auto & [role_id, collect_info] : roles)
{
const auto & role = collect_info.role;
if (!role)
continue;
if (collect_info.is_current_role)
new_info->current_roles.emplace_back(role_id);
new_info->enabled_roles.emplace_back(role_id);
if (collect_info.with_admin_option)
new_info->enabled_roles_with_admin_option.emplace_back(role_id);
new_info->names_of_roles[role_id] = role->getName();
new_info->access.merge(role->access);
new_info->access_with_grant_option.merge(role->access_with_grant_option);
}
return new_info;
}
}
RoleCache::RoleCache(const AccessControlManager & manager_)
: manager(manager_), cache(600000 /* 10 minutes */) {}
RoleCache::~RoleCache() = default;
std::shared_ptr<const EnabledRoles> RoleCache::getEnabledRoles(
const std::vector<UUID> & roles, const std::vector<UUID> & roles_with_admin_option)
{
std::lock_guard lock{mutex};
EnabledRoles::Params params;
params.current_roles = roles;
params.current_roles_with_admin_option = roles_with_admin_option;
auto it = enabled_roles.find(params);
if (it != enabled_roles.end())
{
auto from_cache = it->second.lock();
if (from_cache)
return from_cache;
enabled_roles.erase(it);
}
auto res = std::shared_ptr<EnabledRoles>(new EnabledRoles(params));
collectRolesInfoFor(*res);
enabled_roles.emplace(std::move(params), res);
return res;
}
void RoleCache::collectRolesInfo()
{
/// `mutex` is already locked.
std::erase_if(
enabled_roles,
[&](const std::pair<EnabledRoles::Params, std::weak_ptr<EnabledRoles>> & pr)
{
auto elem = pr.second.lock();
if (!elem)
return true; // remove from the `enabled_roles` map.
collectRolesInfoFor(*elem);
return false; // keep in the `enabled_roles` map.
});
}
void RoleCache::collectRolesInfoFor(EnabledRoles & enabled)
{
/// `mutex` is already locked.
/// Collect roles in use. That includes the current roles, the roles granted to the current roles, and so on.
boost::container::flat_map<UUID, CollectedRoleInfo> collected_roles;
auto get_role_function = [this](const UUID & id) { return getRole(id); };
for (const auto & current_role : enabled.params.current_roles)
collectRoles(collected_roles, get_role_function, current_role, true, false);
for (const auto & current_role : enabled.params.current_roles_with_admin_option)
collectRoles(collected_roles, get_role_function, current_role, true, true);
/// Collect data from the collected roles.
enabled.setRolesInfo(collectInfoForRoles(collected_roles));
}
RolePtr RoleCache::getRole(const UUID & role_id)
{
/// `mutex` is already locked.
auto role_from_cache = cache.get(role_id);
if (role_from_cache)
return role_from_cache->first;
auto subscription = manager.subscribeForChanges(role_id,
[this, role_id](const UUID &, const AccessEntityPtr & entity)
{
auto changed_role = entity ? typeid_cast<RolePtr>(entity) : nullptr;
if (changed_role)
roleChanged(role_id, changed_role);
else
roleRemoved(role_id);
});
auto role = manager.tryRead<Role>(role_id);
if (role)
{
auto cache_value = Poco::SharedPtr<std::pair<RolePtr, ext::scope_guard>>(
new std::pair<RolePtr, ext::scope_guard>{role, std::move(subscription)});
cache.add(role_id, cache_value);
return role;
}
return nullptr;
}
void RoleCache::roleChanged(const UUID & role_id, const RolePtr & changed_role)
{
std::lock_guard lock{mutex};
auto role_from_cache = cache.get(role_id);
if (!role_from_cache)
return;
role_from_cache->first = changed_role;
cache.update(role_id, role_from_cache);
collectRolesInfo();
}
void RoleCache::roleRemoved(const UUID & role_id)
{
std::lock_guard lock{mutex};
cache.remove(role_id);
collectRolesInfo();
}
}

View File

@ -0,0 +1,36 @@
#pragma once
#include <Access/EnabledRoles.h>
#include <Poco/ExpireCache.h>
#include <map>
#include <mutex>
namespace DB
{
class AccessControlManager;
struct Role;
using RolePtr = std::shared_ptr<const Role>;
class RoleCache
{
public:
RoleCache(const AccessControlManager & manager_);
~RoleCache();
std::shared_ptr<const EnabledRoles> getEnabledRoles(const std::vector<UUID> & current_roles, const std::vector<UUID> & current_roles_with_admin_option);
private:
void collectRolesInfo();
void collectRolesInfoFor(EnabledRoles & enabled);
RolePtr getRole(const UUID & role_id);
void roleChanged(const UUID & role_id, const RolePtr & changed_role);
void roleRemoved(const UUID & role_id);
const AccessControlManager & manager;
Poco::ExpireCache<UUID, std::pair<RolePtr, ext::scope_guard>> cache;
std::map<EnabledRoles::Params, std::weak_ptr<EnabledRoles>> enabled_roles;
mutable std::mutex mutex;
};
}

View File

@ -1,200 +0,0 @@
#include <Access/RoleContext.h>
#include <Access/Role.h>
#include <Access/CurrentRolesInfo.h>
#include <Access/AccessControlManager.h>
#include <boost/range/algorithm/copy.hpp>
#include <boost/range/algorithm/find.hpp>
#include <boost/range/algorithm/sort.hpp>
namespace DB
{
namespace
{
void makeUnique(std::vector<UUID> & vec)
{
boost::range::sort(vec);
vec.erase(std::unique(vec.begin(), vec.end()), vec.end());
}
}
RoleContext::RoleContext(const AccessControlManager & manager_, const UUID & current_role_, bool with_admin_option_)
: manager(&manager_), current_role(current_role_), with_admin_option(with_admin_option_)
{
update();
}
RoleContext::RoleContext(std::vector<RoleContextPtr> && children_)
: children(std::move(children_))
{
update();
}
RoleContext::~RoleContext() = default;
void RoleContext::update()
{
std::vector<OnChangeHandler> handlers_to_notify;
CurrentRolesInfoPtr info_to_notify;
{
std::lock_guard lock{mutex};
auto old_info = info;
updateImpl();
if (!handlers.empty() && (!old_info || (*old_info != *info)))
{
boost::range::copy(handlers, std::back_inserter(handlers_to_notify));
info_to_notify = info;
}
}
for (const auto & handler : handlers_to_notify)
handler(info_to_notify);
}
void RoleContext::updateImpl()
{
if (!current_role && children.empty())
{
info = std::make_shared<CurrentRolesInfo>();
return;
}
if (!children.empty())
{
if (subscriptions_for_change_children.empty())
{
for (const auto & child : children)
subscriptions_for_change_children.emplace_back(
child->subscribeForChanges([this](const CurrentRolesInfoPtr &) { update(); }));
}
auto new_info = std::make_shared<CurrentRolesInfo>();
auto & new_info_ref = *new_info;
for (const auto & child : children)
{
auto child_info = child->getInfo();
new_info_ref.access.merge(child_info->access);
new_info_ref.access_with_grant_option.merge(child_info->access_with_grant_option);
boost::range::copy(child_info->current_roles, std::back_inserter(new_info_ref.current_roles));
boost::range::copy(child_info->enabled_roles, std::back_inserter(new_info_ref.enabled_roles));
boost::range::copy(child_info->enabled_roles_with_admin_option, std::back_inserter(new_info_ref.enabled_roles_with_admin_option));
boost::range::copy(child_info->names_of_roles, std::inserter(new_info_ref.names_of_roles, new_info_ref.names_of_roles.end()));
}
makeUnique(new_info_ref.current_roles);
makeUnique(new_info_ref.enabled_roles);
makeUnique(new_info_ref.enabled_roles_with_admin_option);
info = new_info;
return;
}
assert(current_role);
traverseRoles(*current_role, with_admin_option);
auto new_info = std::make_shared<CurrentRolesInfo>();
auto & new_info_ref = *new_info;
for (auto it = roles_map.begin(); it != roles_map.end();)
{
const auto & id = it->first;
auto & entry = it->second;
if (!entry.in_use)
{
it = roles_map.erase(it);
continue;
}
if (id == *current_role)
new_info_ref.current_roles.push_back(id);
new_info_ref.enabled_roles.push_back(id);
if (entry.with_admin_option)
new_info_ref.enabled_roles_with_admin_option.push_back(id);
new_info_ref.access.merge(entry.role->access);
new_info_ref.access_with_grant_option.merge(entry.role->access_with_grant_option);
new_info_ref.names_of_roles[id] = entry.role->getName();
entry.in_use = false;
entry.with_admin_option = false;
++it;
}
info = new_info;
}
void RoleContext::traverseRoles(const UUID & id_, bool with_admin_option_)
{
auto it = roles_map.find(id_);
if (it == roles_map.end())
{
assert(manager);
auto subscription = manager->subscribeForChanges(id_, [this, id_](const UUID &, const AccessEntityPtr & entity)
{
{
std::lock_guard lock{mutex};
auto it2 = roles_map.find(id_);
if (it2 == roles_map.end())
return;
if (entity)
it2->second.role = typeid_cast<RolePtr>(entity);
else
roles_map.erase(it2);
}
update();
});
auto role = manager->tryRead<Role>(id_);
if (!role)
return;
RoleEntry new_entry;
new_entry.role = role;
new_entry.subscription_for_change_role = std::move(subscription);
it = roles_map.emplace(id_, std::move(new_entry)).first;
}
RoleEntry & entry = it->second;
entry.with_admin_option |= with_admin_option_;
if (entry.in_use)
return;
entry.in_use = true;
for (const auto & granted_role : entry.role->granted_roles)
traverseRoles(granted_role, false);
for (const auto & granted_role : entry.role->granted_roles_with_admin_option)
traverseRoles(granted_role, true);
}
CurrentRolesInfoPtr RoleContext::getInfo() const
{
std::lock_guard lock{mutex};
return info;
}
ext::scope_guard RoleContext::subscribeForChanges(const OnChangeHandler & handler) const
{
std::lock_guard lock{mutex};
handlers.push_back(handler);
auto it = std::prev(handlers.end());
return [this, it]
{
std::lock_guard lock2{mutex};
handlers.erase(it);
};
}
}

View File

@ -1,64 +0,0 @@
#pragma once
#include <Core/UUID.h>
#include <ext/scope_guard.h>
#include <ext/shared_ptr_helper.h>
#include <list>
#include <mutex>
#include <unordered_map>
#include <vector>
namespace DB
{
struct Role;
using RolePtr = std::shared_ptr<const Role>;
struct CurrentRolesInfo;
using CurrentRolesInfoPtr = std::shared_ptr<const CurrentRolesInfo>;
class AccessControlManager;
class RoleContext
{
public:
~RoleContext();
/// Returns all the roles specified in the constructor.
CurrentRolesInfoPtr getInfo() const;
using OnChangeHandler = std::function<void(const CurrentRolesInfoPtr & info)>;
/// Called when either the specified roles or the roles granted to the specified roles are changed.
ext::scope_guard subscribeForChanges(const OnChangeHandler & handler) const;
private:
friend struct ext::shared_ptr_helper<RoleContext>;
RoleContext(const AccessControlManager & manager_, const UUID & current_role_, bool with_admin_option_);
RoleContext(std::vector<std::shared_ptr<const RoleContext>> && children_);
void update();
void updateImpl();
void traverseRoles(const UUID & id_, bool with_admin_option_);
const AccessControlManager * manager = nullptr;
std::optional<UUID> current_role;
bool with_admin_option = false;
std::vector<std::shared_ptr<const RoleContext>> children;
std::vector<ext::scope_guard> subscriptions_for_change_children;
struct RoleEntry
{
RolePtr role;
ext::scope_guard subscription_for_change_role;
bool with_admin_option = false;
bool in_use = false;
};
mutable std::unordered_map<UUID, RoleEntry> roles_map;
mutable CurrentRolesInfoPtr info;
mutable std::list<OnChangeHandler> handlers;
mutable std::mutex mutex;
};
using RoleContextPtr = std::shared_ptr<const RoleContext>;
}

View File

@ -1,52 +0,0 @@
#include <Access/RoleContextFactory.h>
#include <boost/container/flat_set.hpp>
namespace DB
{
RoleContextFactory::RoleContextFactory(const AccessControlManager & manager_)
: manager(manager_), cache(600000 /* 10 minutes */) {}
RoleContextFactory::~RoleContextFactory() = default;
RoleContextPtr RoleContextFactory::createContext(
const std::vector<UUID> & roles, const std::vector<UUID> & roles_with_admin_option)
{
if (roles.size() == 1 && roles_with_admin_option.empty())
return createContextImpl(roles[0], false);
if (roles.size() == 1 && roles_with_admin_option == roles)
return createContextImpl(roles[0], true);
std::vector<RoleContextPtr> children;
children.reserve(roles.size());
for (const auto & role : roles_with_admin_option)
children.push_back(createContextImpl(role, true));
boost::container::flat_set<UUID> roles_with_admin_option_set{roles_with_admin_option.begin(), roles_with_admin_option.end()};
for (const auto & role : roles)
{
if (!roles_with_admin_option_set.contains(role))
children.push_back(createContextImpl(role, false));
}
return ext::shared_ptr_helper<RoleContext>::create(std::move(children));
}
RoleContextPtr RoleContextFactory::createContextImpl(const UUID & id, bool with_admin_option)
{
std::lock_guard lock{mutex};
auto key = std::make_pair(id, with_admin_option);
auto x = cache.get(key);
if (x)
return *x;
auto res = ext::shared_ptr_helper<RoleContext>::create(manager, id, with_admin_option);
cache.add(key, res);
return res;
}
}

View File

@ -1,29 +0,0 @@
#pragma once
#include <Access/RoleContext.h>
#include <Poco/ExpireCache.h>
#include <mutex>
namespace DB
{
class AccessControlManager;
class RoleContextFactory
{
public:
RoleContextFactory(const AccessControlManager & manager_);
~RoleContextFactory();
RoleContextPtr createContext(const std::vector<UUID> & roles, const std::vector<UUID> & roles_with_admin_option);
private:
RoleContextPtr createContextImpl(const UUID & id, bool with_admin_option);
const AccessControlManager & manager;
Poco::ExpireCache<std::pair<UUID, bool>, RoleContextPtr> cache;
std::mutex mutex;
};
}

View File

@ -77,11 +77,11 @@ bool RowPolicy::equal(const IAccessEntity & other) const
const auto & other_policy = typeid_cast<const RowPolicy &>(other);
return (database == other_policy.database) && (table_name == other_policy.table_name) && (policy_name == other_policy.policy_name)
&& boost::range::equal(conditions, other_policy.conditions) && restrictive == other_policy.restrictive
&& (roles == other_policy.roles);
&& (to_roles == other_policy.to_roles);
}
const char * RowPolicy::conditionIndexToString(ConditionIndex index)
const char * RowPolicy::conditionTypeToString(ConditionType index)
{
switch (index)
{
@ -95,7 +95,7 @@ const char * RowPolicy::conditionIndexToString(ConditionIndex index)
}
const char * RowPolicy::conditionIndexToColumnName(ConditionIndex index)
const char * RowPolicy::conditionTypeToColumnName(ConditionType index)
{
switch (index)
{

View File

@ -1,7 +1,7 @@
#pragma once
#include <Access/IAccessEntity.h>
#include <Access/GenericRoleSet.h>
#include <Access/ExtendedRoleSet.h>
namespace DB
@ -37,7 +37,7 @@ struct RowPolicy : public IAccessEntity
/// Check is a SQL condition expression used to check whether a row can be written into
/// the table. If the expression returns NULL or false an exception is thrown.
/// If a conditional expression here is empty it means no filtering is applied.
enum ConditionIndex
enum ConditionType
{
SELECT_FILTER,
INSERT_CHECK,
@ -45,11 +45,11 @@ struct RowPolicy : public IAccessEntity
UPDATE_CHECK,
DELETE_FILTER,
};
static constexpr size_t MAX_CONDITION_INDEX = 5;
static const char * conditionIndexToString(ConditionIndex index);
static const char * conditionIndexToColumnName(ConditionIndex index);
static constexpr size_t MAX_CONDITION_TYPE = 5;
static const char * conditionTypeToString(ConditionType index);
static const char * conditionTypeToColumnName(ConditionType index);
String conditions[MAX_CONDITION_INDEX];
String conditions[MAX_CONDITION_TYPE];
/// Sets that the policy is permissive.
/// A row is only accessible if at least one of the permissive policies passes,
@ -67,7 +67,7 @@ struct RowPolicy : public IAccessEntity
std::shared_ptr<IAccessEntity> clone() const override { return cloneImpl<RowPolicy>(); }
/// Which roles or users should use this row policy.
GenericRoleSet roles;
ExtendedRoleSet to_roles;
private:
String database;

View File

@ -1,5 +1,5 @@
#include <Access/RowPolicyContextFactory.h>
#include <Access/RowPolicyContext.h>
#include <Access/RowPolicyCache.h>
#include <Access/EnabledRowPolicies.h>
#include <Access/AccessControlManager.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
@ -92,8 +92,8 @@ namespace
}
using ConditionIndex = RowPolicy::ConditionIndex;
constexpr size_t MAX_CONDITION_INDEX = RowPolicy::MAX_CONDITION_INDEX;
using ConditionType = RowPolicy::ConditionType;
constexpr size_t MAX_CONDITION_TYPE = RowPolicy::MAX_CONDITION_TYPE;
/// Accumulates conditions from multiple row policies and joins them using the AND logical operation.
@ -124,24 +124,24 @@ namespace
}
void RowPolicyContextFactory::PolicyInfo::setPolicy(const RowPolicyPtr & policy_)
void RowPolicyCache::PolicyInfo::setPolicy(const RowPolicyPtr & policy_)
{
policy = policy_;
roles = &policy->roles;
roles = &policy->to_roles;
for (auto index : ext::range_with_static_cast<ConditionIndex>(0, MAX_CONDITION_INDEX))
for (auto type : ext::range_with_static_cast<ConditionType>(0, MAX_CONDITION_TYPE))
{
parsed_conditions[index] = nullptr;
const String & condition = policy->conditions[index];
parsed_conditions[type] = nullptr;
const String & condition = policy->conditions[type];
if (condition.empty())
continue;
auto previous_range = std::pair(std::begin(policy->conditions), std::begin(policy->conditions) + index);
auto previous_range = std::pair(std::begin(policy->conditions), std::begin(policy->conditions) + type);
auto previous_it = std::find(previous_range.first, previous_range.second, condition);
if (previous_it != previous_range.second)
{
/// The condition is already parsed before.
parsed_conditions[index] = parsed_conditions[previous_it - previous_range.first];
parsed_conditions[type] = parsed_conditions[previous_it - previous_range.first];
continue;
}
@ -149,45 +149,52 @@ void RowPolicyContextFactory::PolicyInfo::setPolicy(const RowPolicyPtr & policy_
try
{
ParserExpression parser;
parsed_conditions[index] = parseQuery(parser, condition, 0);
parsed_conditions[type] = parseQuery(parser, condition, 0);
}
catch (...)
{
tryLogCurrentException(
&Poco::Logger::get("RowPolicy"),
String("Could not parse the condition ") + RowPolicy::conditionIndexToString(index) + " of row policy "
String("Could not parse the condition ") + RowPolicy::conditionTypeToString(type) + " of row policy "
+ backQuote(policy->getFullName()));
}
}
}
bool RowPolicyContextFactory::PolicyInfo::canUseWithContext(const RowPolicyContext & context) const
{
return roles->match(context.user_id, context.enabled_roles);
}
RowPolicyContextFactory::RowPolicyContextFactory(const AccessControlManager & access_control_manager_)
RowPolicyCache::RowPolicyCache(const AccessControlManager & access_control_manager_)
: access_control_manager(access_control_manager_)
{
}
RowPolicyContextFactory::~RowPolicyContextFactory() = default;
RowPolicyCache::~RowPolicyCache() = default;
RowPolicyContextPtr RowPolicyContextFactory::createContext(const UUID & user_id, const std::vector<UUID> & enabled_roles)
std::shared_ptr<const EnabledRowPolicies> RowPolicyCache::getEnabledRowPolicies(const UUID & user_id, const std::vector<UUID> & enabled_roles)
{
std::lock_guard lock{mutex};
ensureAllRowPoliciesRead();
auto context = ext::shared_ptr_helper<RowPolicyContext>::create(user_id, enabled_roles);
contexts.push_back(context);
mixConditionsForContext(*context);
return context;
EnabledRowPolicies::Params params;
params.user_id = user_id;
params.enabled_roles = enabled_roles;
auto it = enabled_row_policies.find(params);
if (it != enabled_row_policies.end())
{
auto from_cache = it->second.lock();
if (from_cache)
return from_cache;
enabled_row_policies.erase(it);
}
auto res = std::shared_ptr<EnabledRowPolicies>(new EnabledRowPolicies(params));
enabled_row_policies.emplace(std::move(params), res);
mixConditionsFor(*res);
return res;
}
void RowPolicyContextFactory::ensureAllRowPoliciesRead()
void RowPolicyCache::ensureAllRowPoliciesRead()
{
/// `mutex` is already locked.
if (all_policies_read)
@ -212,7 +219,7 @@ void RowPolicyContextFactory::ensureAllRowPoliciesRead()
}
void RowPolicyContextFactory::rowPolicyAddedOrChanged(const UUID & policy_id, const RowPolicyPtr & new_policy)
void RowPolicyCache::rowPolicyAddedOrChanged(const UUID & policy_id, const RowPolicyPtr & new_policy)
{
std::lock_guard lock{mutex};
auto it = all_policies.find(policy_id);
@ -228,46 +235,46 @@ void RowPolicyContextFactory::rowPolicyAddedOrChanged(const UUID & policy_id, co
auto & info = it->second;
info.setPolicy(new_policy);
mixConditionsForAllContexts();
mixConditions();
}
void RowPolicyContextFactory::rowPolicyRemoved(const UUID & policy_id)
void RowPolicyCache::rowPolicyRemoved(const UUID & policy_id)
{
std::lock_guard lock{mutex};
all_policies.erase(policy_id);
mixConditionsForAllContexts();
mixConditions();
}
void RowPolicyContextFactory::mixConditionsForAllContexts()
void RowPolicyCache::mixConditions()
{
/// `mutex` is already locked.
boost::range::remove_erase_if(
contexts,
[&](const std::weak_ptr<RowPolicyContext> & weak)
std::erase_if(
enabled_row_policies,
[&](const std::pair<EnabledRowPolicies::Params, std::weak_ptr<EnabledRowPolicies>> & pr)
{
auto context = weak.lock();
if (!context)
return true; // remove from the `contexts` list.
mixConditionsForContext(*context);
return false; // keep in the `contexts` list.
auto elem = pr.second.lock();
if (!elem)
return true; // remove from the `enabled_row_policies` map.
mixConditionsFor(*elem);
return false; // keep in the `enabled_row_policies` map.
});
}
void RowPolicyContextFactory::mixConditionsForContext(RowPolicyContext & context)
void RowPolicyCache::mixConditionsFor(EnabledRowPolicies & enabled)
{
/// `mutex` is already locked.
struct Mixers
{
ConditionsMixer mixers[MAX_CONDITION_INDEX];
ConditionsMixer mixers[MAX_CONDITION_TYPE];
std::vector<UUID> policy_ids;
};
using MapOfMixedConditions = RowPolicyContext::MapOfMixedConditions;
using DatabaseAndTableName = RowPolicyContext::DatabaseAndTableName;
using DatabaseAndTableNameRef = RowPolicyContext::DatabaseAndTableNameRef;
using Hash = RowPolicyContext::Hash;
using MapOfMixedConditions = EnabledRowPolicies::MapOfMixedConditions;
using DatabaseAndTableName = EnabledRowPolicies::DatabaseAndTableName;
using DatabaseAndTableNameRef = EnabledRowPolicies::DatabaseAndTableNameRef;
using Hash = EnabledRowPolicies::Hash;
std::unordered_map<DatabaseAndTableName, Mixers, Hash> map_of_mixers;
@ -275,12 +282,12 @@ void RowPolicyContextFactory::mixConditionsForContext(RowPolicyContext & context
{
const auto & policy = *info.policy;
auto & mixers = map_of_mixers[std::pair{policy.getDatabase(), policy.getTableName()}];
if (info.canUseWithContext(context))
if (info.roles->match(enabled.params.user_id, enabled.params.enabled_roles))
{
mixers.policy_ids.push_back(policy_id);
for (auto index : ext::range(0, MAX_CONDITION_INDEX))
if (info.parsed_conditions[index])
mixers.mixers[index].add(info.parsed_conditions[index], policy.isRestrictive());
for (auto type : ext::range(0, MAX_CONDITION_TYPE))
if (info.parsed_conditions[type])
mixers.mixers[type].add(info.parsed_conditions[type], policy.isRestrictive());
}
}
@ -294,11 +301,11 @@ void RowPolicyContextFactory::mixConditionsForContext(RowPolicyContext & context
database_and_table_name_keeper->second}];
mixed_conditions.database_and_table_name_keeper = std::move(database_and_table_name_keeper);
mixed_conditions.policy_ids = std::move(mixers.policy_ids);
for (auto index : ext::range(0, MAX_CONDITION_INDEX))
mixed_conditions.mixed_conditions[index] = std::move(mixers.mixers[index]).getResult();
for (auto type : ext::range(0, MAX_CONDITION_TYPE))
mixed_conditions.mixed_conditions[type] = std::move(mixers.mixers[type]).getResult();
}
context.map_of_mixed_conditions.store(map_of_mixed_conditions);
enabled.map_of_mixed_conditions.store(map_of_mixed_conditions);
}
}

View File

@ -1,8 +1,9 @@
#pragma once
#include <Access/RowPolicyContext.h>
#include <Access/EnabledRowPolicies.h>
#include <ext/scope_guard.h>
#include <mutex>
#include <map>
#include <unordered_map>
@ -11,39 +12,38 @@ namespace DB
class AccessControlManager;
/// Stores read and parsed row policies.
class RowPolicyContextFactory
class RowPolicyCache
{
public:
RowPolicyContextFactory(const AccessControlManager & access_control_manager_);
~RowPolicyContextFactory();
RowPolicyCache(const AccessControlManager & access_control_manager_);
~RowPolicyCache();
RowPolicyContextPtr createContext(const UUID & user_id, const std::vector<UUID> & enabled_roles);
std::shared_ptr<const EnabledRowPolicies> getEnabledRowPolicies(const UUID & user_id, const std::vector<UUID> & enabled_roles);
private:
using ParsedConditions = RowPolicyContext::ParsedConditions;
using ParsedConditions = EnabledRowPolicies::ParsedConditions;
struct PolicyInfo
{
PolicyInfo(const RowPolicyPtr & policy_) { setPolicy(policy_); }
void setPolicy(const RowPolicyPtr & policy_);
bool canUseWithContext(const RowPolicyContext & context) const;
RowPolicyPtr policy;
const GenericRoleSet * roles = nullptr;
const ExtendedRoleSet * roles = nullptr;
ParsedConditions parsed_conditions;
};
void ensureAllRowPoliciesRead();
void rowPolicyAddedOrChanged(const UUID & policy_id, const RowPolicyPtr & new_policy);
void rowPolicyRemoved(const UUID & policy_id);
void mixConditionsForAllContexts();
void mixConditionsForContext(RowPolicyContext & context);
void mixConditions();
void mixConditionsFor(EnabledRowPolicies & enabled);
const AccessControlManager & access_control_manager;
std::unordered_map<UUID, PolicyInfo> all_policies;
bool all_policies_read = false;
ext::scope_guard subscription;
std::vector<std::weak_ptr<RowPolicyContext>> contexts;
std::map<EnabledRowPolicies::Params, std::weak_ptr<EnabledRowPolicies>> enabled_row_policies;
std::mutex mutex;
};

View File

@ -4,7 +4,7 @@
#include <Access/Authentication.h>
#include <Access/AllowedClientHosts.h>
#include <Access/AccessRights.h>
#include <Access/GenericRoleSet.h>
#include <Access/ExtendedRoleSet.h>
#include <Core/UUID.h>
#include <boost/container/flat_set.hpp>
@ -21,7 +21,7 @@ struct User : public IAccessEntity
AccessRights access_with_grant_option;
boost::container::flat_set<UUID> granted_roles;
boost::container::flat_set<UUID> granted_roles_with_admin_option;
GenericRoleSet default_roles = GenericRoleSet::AllTag{};
ExtendedRoleSet default_roles = ExtendedRoleSet::AllTag{};
String profile;
bool equal(const IAccessEntity & other) const override;

View File

@ -224,7 +224,7 @@ namespace
limits.max[ResourceType::EXECUTION_TIME] = Quota::secondsToExecutionTime(config.getUInt64(interval_config + ".execution_time", Quota::UNLIMITED));
}
quota->roles.add(user_ids);
quota->to_roles.add(user_ids);
return quota;
}
@ -324,7 +324,7 @@ namespace
auto policy = std::make_shared<RowPolicy>();
policy->setFullName(database, table_name, user_name);
policy->conditions[RowPolicy::SELECT_FILTER] = filter;
policy->roles.add(generateID(typeid(User), user_name));
policy->to_roles.add(generateID(typeid(User), user_name));
policies.push_back(policy);
}
}

View File

@ -2,7 +2,7 @@
#include <Core/Field.h>
#include <Interpreters/ProcessList.h>
#include <Access/QuotaContext.h>
#include <Access/EnabledQuota.h>
#include <Common/CurrentThread.h>
#include <common/sleep.h>

View File

@ -21,8 +21,7 @@ namespace ErrorCodes
}
class ProcessListElement;
class QuotaContext;
using QuotaContextPtr = std::shared_ptr<const QuotaContext>;
class EnabledQuota;
class QueryStatus;
struct SortColumnDescription;
using SortDescription = std::vector<SortColumnDescription>;
@ -219,7 +218,7 @@ public:
/** Set the quota. If you set a quota on the amount of raw data,
* then you should also set mode = LIMITS_TOTAL to LocalLimits with setLimits.
*/
virtual void setQuota(const QuotaContextPtr & quota_)
virtual void setQuota(const std::shared_ptr<const EnabledQuota> & quota_)
{
quota = quota_;
}
@ -277,7 +276,7 @@ private:
LocalLimits limits;
QuotaContextPtr quota; /// If nullptr - the quota is not used.
std::shared_ptr<const EnabledQuota> quota; /// If nullptr - the quota is not used.
UInt64 prev_elapsed = 0;
/// The approximate total number of rows to read. For progress bar.

View File

@ -3,7 +3,8 @@
#include <Interpreters/Context.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeUUID.h>
#include <Access/QuotaContext.h>
#include <Access/EnabledQuota.h>
#include <Access/QuotaUsageInfo.h>
#include <Core/Field.h>

View File

@ -9,7 +9,7 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnTuple.h>
#include <Interpreters/Context.h>
#include <Access/RowPolicyContext.h>
#include <Access/EnabledRowPolicies.h>
#include <Access/AccessControlManager.h>
#include <ext/range.h>
#include <IO/WriteHelpers.h>
@ -65,17 +65,20 @@ public:
auto database_column = ColumnString::create();
auto table_name_column = ColumnString::create();
auto policy_name_column = ColumnString::create();
for (const auto & policy_id : context.getRowPolicy()->getCurrentPolicyIDs())
if (auto policies = context.getRowPolicies())
{
const auto policy = context.getAccessControlManager().tryRead<RowPolicy>(policy_id);
if (policy)
for (const auto & policy_id : policies->getCurrentPolicyIDs())
{
const String database = policy->getDatabase();
const String table_name = policy->getTableName();
const String policy_name = policy->getName();
database_column->insertData(database.data(), database.length());
table_name_column->insertData(table_name.data(), table_name.length());
policy_name_column->insertData(policy_name.data(), policy_name.length());
const auto policy = context.getAccessControlManager().tryRead<RowPolicy>(policy_id);
if (policy)
{
const String database = policy->getDatabase();
const String table_name = policy->getTableName();
const String policy_name = policy->getName();
database_column->insertData(database.data(), database.length());
table_name_column->insertData(table_name.data(), table_name.length());
policy_name_column->insertData(policy_name.data(), policy_name.length());
}
}
}
auto offset_column = ColumnArray::ColumnOffsets::create();
@ -113,13 +116,16 @@ public:
{
String database = database_column ? database_column->getDataAt(i).toString() : context.getCurrentDatabase();
String table_name = table_name_column->getDataAt(i).toString();
for (const auto & policy_id : context.getRowPolicy()->getCurrentPolicyIDs(database, table_name))
if (auto policies = context.getRowPolicies())
{
const auto policy = context.getAccessControlManager().tryRead<RowPolicy>(policy_id);
if (policy)
for (const auto & policy_id : policies->getCurrentPolicyIDs(database, table_name))
{
const String policy_name = policy->getName();
policy_name_column->insertData(policy_name.data(), policy_name.length());
const auto policy = context.getAccessControlManager().tryRead<RowPolicy>(policy_id);
if (policy)
{
const String policy_name = policy->getName();
policy_name_column->insertData(policy_name.data(), policy_name.length());
}
}
}
offset_column->insertValue(policy_name_column->size());
@ -169,8 +175,11 @@ public:
if (arguments.empty())
{
auto policy_id_column = ColumnVector<UInt128>::create();
for (const auto & policy_id : context.getRowPolicy()->getCurrentPolicyIDs())
policy_id_column->insertValue(policy_id);
if (auto policies = context.getRowPolicies())
{
for (const auto & policy_id : policies->getCurrentPolicyIDs())
policy_id_column->insertValue(policy_id);
}
auto offset_column = ColumnArray::ColumnOffsets::create();
offset_column->insertValue(policy_id_column->size());
block.getByPosition(result_pos).column
@ -203,8 +212,11 @@ public:
{
String database = database_column ? database_column->getDataAt(i).toString() : context.getCurrentDatabase();
String table_name = table_name_column->getDataAt(i).toString();
for (const auto & policy_id : context.getRowPolicy()->getCurrentPolicyIDs(database, table_name))
policy_id_column->insertValue(policy_id);
if (auto policies = context.getRowPolicies())
{
for (const auto & policy_id : policies->getCurrentPolicyIDs(database, table_name))
policy_id_column->insertValue(policy_id);
}
offset_column->insertValue(policy_id_column->size());
}

View File

@ -27,8 +27,8 @@
#include <Interpreters/ActionLocksManager.h>
#include <Core/Settings.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessRightsContext.h>
#include <Access/RowPolicyContext.h>
#include <Access/ContextAccess.h>
#include <Access/EnabledRowPolicies.h>
#include <Access/User.h>
#include <Access/SettingsConstraints.h>
#include <Interpreters/ExpressionJIT.h>
@ -444,8 +444,6 @@ Context & Context::operator=(const Context &) = default;
Context Context::createGlobal()
{
Context res;
res.access_rights = std::make_shared<AccessRightsContext>();
res.initial_row_policy = std::make_shared<RowPolicyContext>();
res.shared = std::make_shared<ContextShared>();
return res;
}
@ -632,22 +630,22 @@ void Context::setUser(const String & name, const String & password, const Poco::
client_info.quota_key = quota_key;
auto new_user_id = getAccessControlManager().find<User>(name);
AccessRightsContextPtr new_access_rights;
std::shared_ptr<const ContextAccess> new_access;
if (new_user_id)
{
new_access_rights = getAccessControlManager().getAccessRightsContext(*new_user_id, {}, true, settings, current_database, client_info);
if (!new_access_rights->isClientHostAllowed() || !new_access_rights->isCorrectPassword(password))
new_access = getAccessControlManager().getContextAccess(*new_user_id, {}, true, settings, current_database, client_info);
if (!new_access->isClientHostAllowed() || !new_access->isCorrectPassword(password))
{
new_user_id = {};
new_access_rights = nullptr;
new_access = nullptr;
}
}
if (!new_user_id || !new_access_rights)
if (!new_user_id || !new_access)
throw Exception(name + ": Authentication failed: password is incorrect or there is no user with such name", ErrorCodes::AUTHENTICATION_FAILED);
user_id = new_user_id;
access_rights = std::move(new_access_rights);
access = std::move(new_access);
current_roles.clear();
use_default_roles = true;
@ -657,13 +655,13 @@ void Context::setUser(const String & name, const String & password, const Poco::
std::shared_ptr<const User> Context::getUser() const
{
auto lock = getLock();
return access_rights->getUser();
return access->getUser();
}
String Context::getUserName() const
{
auto lock = getLock();
return access_rights->getUserName();
return access->getUserName();
}
UUID Context::getUserID() const
@ -697,22 +695,22 @@ void Context::setCurrentRolesDefault()
std::vector<UUID> Context::getCurrentRoles() const
{
return getAccessRights()->getCurrentRoles();
return getAccess()->getCurrentRoles();
}
Strings Context::getCurrentRolesNames() const
{
return getAccessRights()->getCurrentRolesNames();
return getAccess()->getCurrentRolesNames();
}
std::vector<UUID> Context::getEnabledRoles() const
{
return getAccessRights()->getEnabledRoles();
return getAccess()->getEnabledRoles();
}
Strings Context::getEnabledRolesNames() const
{
return getAccessRights()->getEnabledRolesNames();
return getAccess()->getEnabledRolesNames();
}
@ -720,59 +718,61 @@ void Context::calculateAccessRights()
{
auto lock = getLock();
if (user_id)
access_rights = getAccessControlManager().getAccessRightsContext(*user_id, current_roles, use_default_roles, settings, current_database, client_info);
access = getAccessControlManager().getContextAccess(*user_id, current_roles, use_default_roles, settings, current_database, client_info);
}
template <typename... Args>
void Context::checkAccessImpl(const Args &... args) const
{
getAccessRights()->checkAccess(args...);
return getAccess()->checkAccess(args...);
}
void Context::checkAccess(const AccessFlags & access) const { return checkAccessImpl(access); }
void Context::checkAccess(const AccessFlags & access, const std::string_view & database) const { return checkAccessImpl(access, database); }
void Context::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl(access, database, table); }
void Context::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl(access, database, table, column); }
void Context::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const { return checkAccessImpl(access, database, table, columns); }
void Context::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl(access, database, table, columns); }
void Context::checkAccess(const AccessRightsElement & access) const { return checkAccessImpl(access); }
void Context::checkAccess(const AccessRightsElements & access) const { return checkAccessImpl(access); }
void Context::checkAccess(const AccessFlags & flags) const { return checkAccessImpl(flags); }
void Context::checkAccess(const AccessFlags & flags, const std::string_view & database) const { return checkAccessImpl(flags, database); }
void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl(flags, database, table); }
void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl(flags, database, table, column); }
void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const { return checkAccessImpl(flags, database, table, columns); }
void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); }
void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName()); }
void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::string_view & column) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), column); }
void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::vector<std::string_view> & columns) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), columns); }
void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const Strings & columns) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), columns); }
void Context::checkAccess(const AccessRightsElement & element) const { return checkAccessImpl(element); }
void Context::checkAccess(const AccessRightsElements & elements) const { return checkAccessImpl(elements); }
void Context::checkAccess(const AccessFlags & access, const StorageID & table_id) const { checkAccessImpl(access, table_id.getDatabaseName(), table_id.getTableName()); }
void Context::checkAccess(const AccessFlags & access, const StorageID & table_id, const std::string_view & column) const { checkAccessImpl(access, table_id.getDatabaseName(), table_id.getTableName(), column); }
void Context::checkAccess(const AccessFlags & access, const StorageID & table_id, const std::vector<std::string_view> & columns) const { checkAccessImpl(access, table_id.getDatabaseName(), table_id.getTableName(), columns); }
void Context::checkAccess(const AccessFlags & access, const StorageID & table_id, const Strings & columns) const { checkAccessImpl(access, table_id.getDatabaseName(), table_id.getTableName(), columns); }
AccessRightsContextPtr Context::getAccessRights() const
std::shared_ptr<const ContextAccess> Context::getAccess() const
{
auto lock = getLock();
return access_rights;
return access ? access : ContextAccess::getFullAccess();
}
RowPolicyContextPtr Context::getRowPolicy() const
ASTPtr Context::getRowPolicyCondition(const String & database, const String & table_name, RowPolicy::ConditionType type) const
{
return getAccessRights()->getRowPolicy();
auto lock = getLock();
auto initial_condition = initial_row_policy ? initial_row_policy->getCondition(database, table_name, type) : nullptr;
return getAccess()->getRowPolicyCondition(database, table_name, type, initial_condition);
}
std::shared_ptr<const EnabledRowPolicies> Context::getRowPolicies() const
{
return getAccess()->getRowPolicies();
}
void Context::setInitialRowPolicy()
{
auto lock = getLock();
auto initial_user_id = getAccessControlManager().find<User>(client_info.initial_user);
initial_row_policy = nullptr;
if (initial_user_id)
initial_row_policy = getAccessControlManager().getRowPolicyContext(*initial_user_id, {});
}
RowPolicyContextPtr Context::getInitialRowPolicy() const
{
auto lock = getLock();
return initial_row_policy;
initial_row_policy = getAccessControlManager().getEnabledRowPolicies(*initial_user_id, {});
}
QuotaContextPtr Context::getQuota() const
std::shared_ptr<const EnabledQuota> Context::getQuota() const
{
return getAccessRights()->getQuota();
return getAccess()->getQuota();
}

View File

@ -9,6 +9,7 @@
#include <Interpreters/ClientInfo.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Parsers/IAST_fwd.h>
#include <Access/RowPolicy.h>
#include <Common/LRUCache.h>
#include <Common/MultiVersion.h>
#include <Common/ThreadPool.h>
@ -44,14 +45,11 @@ namespace DB
struct ContextShared;
class Context;
class AccessRightsContext;
using AccessRightsContextPtr = std::shared_ptr<const AccessRightsContext>;
class ContextAccess;
struct User;
using UserPtr = std::shared_ptr<const User>;
class RowPolicyContext;
using RowPolicyContextPtr = std::shared_ptr<const RowPolicyContext>;
class QuotaContext;
using QuotaContextPtr = std::shared_ptr<const QuotaContext>;
class EnabledRowPolicies;
class EnabledQuota;
class AccessFlags;
struct AccessRightsElement;
class AccessRightsElements;
@ -151,8 +149,8 @@ private:
std::optional<UUID> user_id;
std::vector<UUID> current_roles;
bool use_default_roles = false;
AccessRightsContextPtr access_rights;
RowPolicyContextPtr initial_row_policy;
std::shared_ptr<const ContextAccess> access;
std::shared_ptr<const EnabledRowPolicies> initial_row_policy;
String current_database;
Settings settings; /// Setting for query execution.
std::shared_ptr<const SettingsConstraints> settings_constraints;
@ -246,31 +244,30 @@ public:
/// Checks access rights.
/// Empty database means the current database.
void checkAccess(const AccessFlags & access) const;
void checkAccess(const AccessFlags & access, const std::string_view & database) const;
void checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const;
void checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const;
void checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const;
void checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const;
void checkAccess(const AccessRightsElement & access) const;
void checkAccess(const AccessRightsElements & access) const;
void checkAccess(const AccessFlags & flags) const;
void checkAccess(const AccessFlags & flags, const std::string_view & database) const;
void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const;
void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const;
void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const;
void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const;
void checkAccess(const AccessFlags & flags, const StorageID & table_id) const;
void checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::string_view & column) const;
void checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::vector<std::string_view> & columns) const;
void checkAccess(const AccessFlags & flags, const StorageID & table_id, const Strings & columns) const;
void checkAccess(const AccessRightsElement & element) const;
void checkAccess(const AccessRightsElements & elements) const;
void checkAccess(const AccessFlags & access, const StorageID & table_id) const;
void checkAccess(const AccessFlags & access, const StorageID & table_id, const std::string_view & column) const;
void checkAccess(const AccessFlags & access, const StorageID & table_id, const std::vector<std::string_view> & columns) const;
void checkAccess(const AccessFlags & access, const StorageID & table_id, const Strings & columns) const;
std::shared_ptr<const ContextAccess> getAccess() const;
AccessRightsContextPtr getAccessRights() const;
RowPolicyContextPtr getRowPolicy() const;
std::shared_ptr<const EnabledRowPolicies> getRowPolicies() const;
ASTPtr getRowPolicyCondition(const String & database, const String & table_name, RowPolicy::ConditionType type) const;
/// Sets an extra row policy based on `client_info.initial_user`, if it exists.
/// TODO: we need a better solution here. It seems we should pass the initial row policy
/// because a shard is allowed to don't have the initial user or it may be another user with the same name.
void setInitialRowPolicy();
RowPolicyContextPtr getInitialRowPolicy() const;
QuotaContextPtr getQuota() const;
std::shared_ptr<const EnabledQuota> getQuota() const;
/// We have to copy external tables inside executeQuery() to track limits. Therefore, set callback for it. Must set once.
void setExternalTablesInitializer(ExternalTablesInitializer && initializer);

View File

@ -1,6 +1,6 @@
#include <Interpreters/InterpreterCreateQuotaQuery.h>
#include <Parsers/ASTCreateQuotaQuery.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Interpreters/Context.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessFlags.h>
@ -14,7 +14,7 @@ namespace DB
{
namespace
{
void updateQuotaFromQueryImpl(Quota & quota, const ASTCreateQuotaQuery & query, const std::optional<GenericRoleSet> & roles_from_query = {})
void updateQuotaFromQueryImpl(Quota & quota, const ASTCreateQuotaQuery & query, const std::optional<ExtendedRoleSet> & roles_from_query = {})
{
if (query.alter)
{
@ -61,15 +61,15 @@ void updateQuotaFromQueryImpl(Quota & quota, const ASTCreateQuotaQuery & query,
}
}
const GenericRoleSet * roles = nullptr;
std::optional<GenericRoleSet> temp_role_set;
const ExtendedRoleSet * roles = nullptr;
std::optional<ExtendedRoleSet> temp_role_set;
if (roles_from_query)
roles = &*roles_from_query;
else if (query.roles)
roles = &temp_role_set.emplace(*query.roles);
if (roles)
quota.roles = *roles;
quota.to_roles = *roles;
}
}
@ -80,9 +80,9 @@ BlockIO InterpreterCreateQuotaQuery::execute()
auto & access_control = context.getAccessControlManager();
context.checkAccess(query.alter ? AccessType::ALTER_QUOTA : AccessType::CREATE_QUOTA);
std::optional<GenericRoleSet> roles_from_query;
std::optional<ExtendedRoleSet> roles_from_query;
if (query.roles)
roles_from_query = GenericRoleSet{*query.roles, access_control, context.getUserID()};
roles_from_query = ExtendedRoleSet{*query.roles, access_control, context.getUserID()};
if (query.alter)
{

View File

@ -1,6 +1,6 @@
#include <Interpreters/InterpreterCreateRowPolicyQuery.h>
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Parsers/formatAST.h>
#include <Interpreters/Context.h>
#include <Access/AccessControlManager.h>
@ -27,7 +27,7 @@ namespace
void updateRowPolicyFromQueryImpl(
RowPolicy & policy,
const ASTCreateRowPolicyQuery & query,
const std::optional<GenericRoleSet> & roles_from_query = {},
const std::optional<ExtendedRoleSet> & roles_from_query = {},
const String & current_database = {})
{
if (query.alter)
@ -48,15 +48,15 @@ namespace
for (const auto & [index, condition] : query.conditions)
policy.conditions[index] = condition ? serializeAST(*condition) : String{};
const GenericRoleSet * roles = nullptr;
std::optional<GenericRoleSet> temp_role_set;
const ExtendedRoleSet * roles = nullptr;
std::optional<ExtendedRoleSet> temp_role_set;
if (roles_from_query)
roles = &*roles_from_query;
else if (query.roles)
roles = &temp_role_set.emplace(*query.roles);
if (roles)
policy.roles = *roles;
policy.to_roles = *roles;
}
}
@ -67,9 +67,9 @@ BlockIO InterpreterCreateRowPolicyQuery::execute()
auto & access_control = context.getAccessControlManager();
context.checkAccess(query.alter ? AccessType::ALTER_POLICY : AccessType::CREATE_POLICY);
std::optional<GenericRoleSet> roles_from_query;
std::optional<ExtendedRoleSet> roles_from_query;
if (query.roles)
roles_from_query = GenericRoleSet{*query.roles, access_control, context.getUserID()};
roles_from_query = ExtendedRoleSet{*query.roles, access_control, context.getUserID()};
const String current_database = context.getCurrentDatabase();

View File

@ -4,8 +4,8 @@
#include <Parsers/ASTCreateUserQuery.h>
#include <Access/AccessControlManager.h>
#include <Access/User.h>
#include <Access/GenericRoleSet.h>
#include <Access/AccessRightsContext.h>
#include <Access/ExtendedRoleSet.h>
#include <Access/ContextAccess.h>
#include <boost/range/algorithm/copy.hpp>
@ -13,7 +13,7 @@ namespace DB
{
namespace
{
void updateUserFromQueryImpl(User & user, const ASTCreateUserQuery & query, const std::optional<GenericRoleSet> & default_roles_from_query = {})
void updateUserFromQueryImpl(User & user, const ASTCreateUserQuery & query, const std::optional<ExtendedRoleSet> & default_roles_from_query = {})
{
if (query.alter)
{
@ -33,8 +33,8 @@ namespace
if (query.add_hosts)
user.allowed_client_hosts.add(*query.add_hosts);
const GenericRoleSet * default_roles = nullptr;
std::optional<GenericRoleSet> temp_role_set;
const ExtendedRoleSet * default_roles = nullptr;
std::optional<ExtendedRoleSet> temp_role_set;
if (default_roles_from_query)
default_roles = &*default_roles_from_query;
else if (query.default_roles)
@ -58,16 +58,17 @@ BlockIO InterpreterCreateUserQuery::execute()
{
const auto & query = query_ptr->as<const ASTCreateUserQuery &>();
auto & access_control = context.getAccessControlManager();
context.checkAccess(query.alter ? AccessType::ALTER_USER : AccessType::CREATE_USER);
auto access = context.getAccess();
access->checkAccess(query.alter ? AccessType::ALTER_USER : AccessType::CREATE_USER);
std::optional<GenericRoleSet> default_roles_from_query;
std::optional<ExtendedRoleSet> default_roles_from_query;
if (query.default_roles)
{
default_roles_from_query = GenericRoleSet{*query.default_roles, access_control};
default_roles_from_query = ExtendedRoleSet{*query.default_roles, access_control};
if (!query.alter && !default_roles_from_query->all)
{
for (const UUID & role : default_roles_from_query->getMatchingIDs())
context.getAccessRights()->checkAdminOption(role);
access->checkAdminOption(role);
}
}

View File

@ -2,8 +2,8 @@
#include <Parsers/ASTGrantQuery.h>
#include <Interpreters/Context.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessRightsContext.h>
#include <Access/GenericRoleSet.h>
#include <Access/ContextAccess.h>
#include <Access/ExtendedRoleSet.h>
#include <Access/User.h>
#include <Access/Role.h>
#include <boost/range/algorithm/copy.hpp>
@ -61,17 +61,18 @@ BlockIO InterpreterGrantQuery::execute()
{
const auto & query = query_ptr->as<const ASTGrantQuery &>();
auto & access_control = context.getAccessControlManager();
context.getAccessRights()->checkGrantOption(query.access_rights_elements);
auto access = context.getAccess();
access->checkGrantOption(query.access_rights_elements);
std::vector<UUID> roles_from_query;
if (query.roles)
{
roles_from_query = GenericRoleSet{*query.roles, access_control}.getMatchingRoles(access_control);
roles_from_query = ExtendedRoleSet{*query.roles, access_control}.getMatchingIDs(access_control);
for (const UUID & role_from_query : roles_from_query)
context.getAccessRights()->checkAdminOption(role_from_query);
access->checkAdminOption(role_from_query);
}
std::vector<UUID> to_roles = GenericRoleSet{*query.to_roles, access_control, context.getUserID()}.getMatchingUsersAndRoles(access_control);
std::vector<UUID> to_roles = ExtendedRoleSet{*query.to_roles, access_control, context.getUserID()}.getMatchingIDs(access_control);
String current_database = context.getCurrentDatabase();
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
@ -101,7 +102,7 @@ void InterpreterGrantQuery::updateUserFromQuery(User & user, const ASTGrantQuery
{
std::vector<UUID> roles_from_query;
if (query.roles)
roles_from_query = GenericRoleSet{*query.roles}.getMatchingIDs();
roles_from_query = ExtendedRoleSet{*query.roles}.getMatchingIDs();
updateFromQueryImpl(user, query, roles_from_query, {});
}
@ -110,7 +111,7 @@ void InterpreterGrantQuery::updateRoleFromQuery(Role & role, const ASTGrantQuery
{
std::vector<UUID> roles_from_query;
if (query.roles)
roles_from_query = GenericRoleSet{*query.roles}.getMatchingIDs();
roles_from_query = ExtendedRoleSet{*query.roles}.getMatchingIDs();
updateFromQueryImpl(role, query, roles_from_query, {});
}

View File

@ -10,7 +10,7 @@
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ParserAlterQuery.h>
#include <Parsers/parseQuery.h>
#include <Access/AccessRightsContext.h>
#include <Access/ContextAccess.h>
#include <Columns/ColumnString.h>
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeString.h>
@ -83,7 +83,18 @@ static QueryDescriptors extractQueriesExceptMeAndCheckAccess(const Block & proce
const ColumnString & query_id_col = typeid_cast<const ColumnString &>(*processes_block.getByName("query_id").column);
const ColumnString & user_col = typeid_cast<const ColumnString &>(*processes_block.getByName("user").column);
const ClientInfo & my_client = context.getProcessListElement()->getClientInfo();
std::optional<bool> can_kill_query_started_by_another_user;
std::optional<bool> can_kill_query_started_by_another_user_cached;
auto can_kill_query_started_by_another_user = [&]() -> bool
{
if (!can_kill_query_started_by_another_user_cached)
{
can_kill_query_started_by_another_user_cached
= context.getAccess()->isGranted(&Poco::Logger::get("InterpreterKillQueryQuery"), AccessType::KILL_QUERY);
}
return *can_kill_query_started_by_another_user_cached;
};
String query_user;
bool access_denied = false;
@ -96,15 +107,10 @@ static QueryDescriptors extractQueriesExceptMeAndCheckAccess(const Block & proce
auto query_id = query_id_col.getDataAt(i).toString();
query_user = user_col.getDataAt(i).toString();
if (my_client.current_user != query_user)
if ((my_client.current_user != query_user) && !can_kill_query_started_by_another_user())
{
if (!can_kill_query_started_by_another_user)
can_kill_query_started_by_another_user = context.getAccessRights()->isGranted(&Poco::Logger::get("InterpreterKillQueryQuery"), AccessType::KILL_QUERY);
if (!*can_kill_query_started_by_another_user)
{
access_denied = true;
continue;
}
access_denied = true;
continue;
}
res.emplace_back(std::move(query_id), query_user, i, false);
@ -244,6 +250,7 @@ BlockIO InterpreterKillQueryQuery::execute()
MutableColumns res_columns = header.cloneEmptyColumns();
auto table_id = StorageID::createEmpty();
AccessRightsElements required_access_rights;
auto access = context.getAccess();
bool access_denied = false;
for (size_t i = 0; i < mutations_block.rows(); ++i)
@ -262,7 +269,7 @@ BlockIO InterpreterKillQueryQuery::execute()
ParserAlterCommand parser;
auto command_ast = parseQuery(parser, command_col.getDataAt(i).toString(), 0);
required_access_rights = InterpreterAlterQuery::getRequiredAccessForCommand(command_ast->as<const ASTAlterCommand &>(), table_id.database_name, table_id.table_name);
if (!context.getAccessRights()->isGranted(&Poco::Logger::get("InterpreterKillQueryQuery"), required_access_rights))
if (!access->isGranted(&Poco::Logger::get("InterpreterKillQueryQuery"), required_access_rights))
{
access_denied = true;
continue;

View File

@ -38,7 +38,6 @@
#include <Parsers/parseQuery.h>
#include <Access/AccessFlags.h>
#include <Access/RowPolicyContext.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
@ -348,8 +347,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
source_header = storage->getSampleBlockForColumns(required_columns);
/// Fix source_header for filter actions.
auto row_policy_filter = context->getRowPolicy()->getCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER);
row_policy_filter = RowPolicyContext::combineConditionsUsingAnd(row_policy_filter, context->getInitialRowPolicy()->getCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER));
auto row_policy_filter = context->getRowPolicyCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER);
if (row_policy_filter)
{
filter_info = std::make_shared<FilterInfo>();
@ -477,8 +475,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl(bool try_move_to_prewhere)
/// PREWHERE optimization.
/// Turn off, if the table filter (row-level security) is applied.
if (!context->getRowPolicy()->getCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER)
&& !context->getInitialRowPolicy()->getCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER))
if (!context->getRowPolicyCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER))
{
auto optimize_prewhere = [&](auto & merge_tree)
{
@ -1115,7 +1112,7 @@ void InterpreterSelectQuery::executeFetchColumns(
if (storage)
{
/// Append columns from the table filter to required
auto row_policy_filter = context->getRowPolicy()->getCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER);
auto row_policy_filter = context->getRowPolicyCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER);
if (row_policy_filter)
{
auto initial_required_columns = required_columns;

View File

@ -1,8 +1,8 @@
#include <Interpreters/InterpreterSetRoleQuery.h>
#include <Parsers/ASTSetRoleQuery.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Interpreters/Context.h>
#include <Access/GenericRoleSet.h>
#include <Access/ExtendedRoleSet.h>
#include <Access/AccessControlManager.h>
#include <Access/User.h>
@ -38,7 +38,7 @@ void InterpreterSetRoleQuery::setRole(const ASTSetRoleQuery & query)
}
else
{
GenericRoleSet roles_from_query{*query.roles, access_control};
ExtendedRoleSet roles_from_query{*query.roles, access_control};
std::vector<UUID> new_current_roles;
if (roles_from_query.all)
{
@ -65,8 +65,8 @@ void InterpreterSetRoleQuery::setDefaultRole(const ASTSetRoleQuery & query)
context.checkAccess(AccessType::CREATE_USER | AccessType::DROP_USER);
auto & access_control = context.getAccessControlManager();
std::vector<UUID> to_users = GenericRoleSet{*query.to_users, access_control, context.getUserID()}.getMatchingUsers(access_control);
GenericRoleSet roles_from_query{*query.roles, access_control};
std::vector<UUID> to_users = ExtendedRoleSet{*query.to_users, access_control, context.getUserID()}.getMatchingIDs(access_control);
ExtendedRoleSet roles_from_query{*query.roles, access_control};
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
{
@ -79,7 +79,7 @@ void InterpreterSetRoleQuery::setDefaultRole(const ASTSetRoleQuery & query)
}
void InterpreterSetRoleQuery::updateUserSetDefaultRoles(User & user, const GenericRoleSet & roles_from_query)
void InterpreterSetRoleQuery::updateUserSetDefaultRoles(User & user, const ExtendedRoleSet & roles_from_query)
{
if (!roles_from_query.all)
{

View File

@ -7,7 +7,7 @@
namespace DB
{
class ASTSetRoleQuery;
struct GenericRoleSet;
struct ExtendedRoleSet;
struct User;
@ -18,7 +18,7 @@ public:
BlockIO execute() override;
static void updateUserSetDefaultRoles(User & user, const GenericRoleSet & roles_from_query);
static void updateUserSetDefaultRoles(User & user, const ExtendedRoleSet & roles_from_query);
private:
void setRole(const ASTSetRoleQuery & query);

View File

@ -5,12 +5,13 @@
#include <Parsers/ASTCreateQuotaQuery.h>
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Parsers/ASTShowCreateAccessEntityQuery.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Access/AccessControlManager.h>
#include <Access/QuotaContext.h>
#include <Access/EnabledQuota.h>
#include <Access/QuotaUsageInfo.h>
#include <Access/User.h>
#include <Access/Role.h>
#include <Columns/ColumnString.h>
@ -45,12 +46,12 @@ namespace
if (!user.profile.empty())
query->profile = user.profile;
if (user.default_roles != GenericRoleSet::AllTag{})
if (user.default_roles != ExtendedRoleSet::AllTag{})
{
if (attach_mode)
query->default_roles = GenericRoleSet{user.default_roles}.toAST();
query->default_roles = ExtendedRoleSet{user.default_roles}.toAST();
else
query->default_roles = GenericRoleSet{user.default_roles}.toASTWithNames(*manager);
query->default_roles = ExtendedRoleSet{user.default_roles}.toASTWithNames(*manager);
}
if (attach_mode && (user.authentication.getType() != Authentication::NO_PASSWORD))
@ -94,12 +95,12 @@ namespace
query->all_limits.push_back(create_query_limits);
}
if (!quota.roles.empty())
if (!quota.to_roles.empty())
{
if (attach_mode)
query->roles = quota.roles.toAST();
query->roles = quota.to_roles.toAST();
else
query->roles = quota.roles.toASTWithNames(*manager);
query->roles = quota.to_roles.toASTWithNames(*manager);
}
return query;
@ -118,7 +119,7 @@ namespace
if (policy.isRestrictive())
query->is_restrictive = policy.isRestrictive();
for (auto index : ext::range_with_static_cast<RowPolicy::ConditionIndex>(RowPolicy::MAX_CONDITION_INDEX))
for (auto index : ext::range_with_static_cast<RowPolicy::ConditionType>(RowPolicy::MAX_CONDITION_TYPE))
{
const auto & condition = policy.conditions[index];
if (!condition.empty())
@ -129,12 +130,12 @@ namespace
}
}
if (!policy.roles.empty())
if (!policy.to_roles.empty())
{
if (attach_mode)
query->roles = policy.roles.toAST();
query->roles = policy.to_roles.toAST();
else
query->roles = policy.roles.toASTWithNames(*manager);
query->roles = policy.to_roles.toASTWithNames(*manager);
}
return query;

View File

@ -1,7 +1,7 @@
#include <Interpreters/InterpreterShowGrantsQuery.h>
#include <Parsers/ASTShowGrantsQuery.h>
#include <Parsers/ASTGrantQuery.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Parsers/formatAST.h>
#include <Interpreters/Context.h>
#include <Columns/ColumnString.h>
@ -62,7 +62,7 @@ namespace
{
ASTs res;
std::shared_ptr<ASTGenericRoleSet> to_roles = std::make_shared<ASTGenericRoleSet>();
std::shared_ptr<ASTExtendedRoleSet> to_roles = std::make_shared<ASTExtendedRoleSet>();
to_roles->names.push_back(grantee.getName());
for (bool grant_option : {true, false})
@ -104,9 +104,9 @@ namespace
grant_query->admin_option = admin_option;
grant_query->to_roles = to_roles;
if (attach_mode)
grant_query->roles = GenericRoleSet{roles}.toAST();
grant_query->roles = ExtendedRoleSet{roles}.toAST();
else
grant_query->roles = GenericRoleSet{roles}.toASTWithNames(*manager);
grant_query->roles = ExtendedRoleSet{roles}.toASTWithNames(*manager);
res.push_back(std::move(grant_query));
}

View File

@ -20,7 +20,7 @@
#include <Interpreters/TraceLog.h>
#include <Interpreters/TextLog.h>
#include <Interpreters/MetricLog.h>
#include <Access/AccessRightsContext.h>
#include <Access/ContextAccess.h>
#include <Databases/IDatabase.h>
#include <Storages/StorageDistributed.h>
#include <Storages/StorageReplicatedMergeTree.h>
@ -137,17 +137,17 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type,
}
else
{
auto access = context.getAccess();
for (auto & elem : DatabaseCatalog::instance().getDatabases())
{
for (auto iterator = elem.second->getTablesIterator(context); iterator->isValid(); iterator->next())
{
if (context.getAccessRights()->isGranted(log, getRequiredAccessType(action_type), elem.first, iterator->name()))
{
if (start)
manager->remove(iterator->table(), action_type);
else
manager->add(iterator->table(), action_type);
}
if (!access->isGranted(log, getRequiredAccessType(action_type), elem.first, iterator->name()))
continue;
if (start)
manager->remove(iterator->table(), action_type);
else
manager->add(iterator->table(), action_type);
}
}
}

View File

@ -24,7 +24,7 @@
#include <Storages/StorageInput.h>
#include <Access/QuotaContext.h>
#include <Access/EnabledQuota.h>
#include <Interpreters/InterpreterFactory.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/QueryLog.h>
@ -148,7 +148,8 @@ static void logException(Context & context, QueryLogElement & elem)
static void onExceptionBeforeStart(const String & query_for_logging, Context & context, time_t current_time)
{
/// Exception before the query execution.
context.getQuota()->used(Quota::ERRORS, 1, /* check_exceeded = */ false);
if (auto quota = context.getQuota())
quota->used(Quota::ERRORS, 1, /* check_exceeded = */ false);
const Settings & settings = context.getSettingsRef();
@ -307,12 +308,15 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
auto interpreter = InterpreterFactory::get(ast, context, stage);
bool use_processors = settings.experimental_use_processors && allow_processors && interpreter->canExecuteWithProcessors();
QuotaContextPtr quota;
std::shared_ptr<const EnabledQuota> quota;
if (!interpreter->ignoreQuota())
{
quota = context.getQuota();
quota->used(Quota::QUERIES, 1);
quota->checkExceeded(Quota::ERRORS);
if (quota)
{
quota->used(Quota::QUERIES, 1);
quota->checkExceeded(Quota::ERRORS);
}
}
IBlockInputStream::LocalLimits limits;
@ -486,9 +490,10 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
}
};
auto exception_callback = [elem, &context, log_queries] () mutable
auto exception_callback = [elem, &context, log_queries, quota(quota)] () mutable
{
context.getQuota()->used(Quota::ERRORS, 1, /* check_exceeded = */ false);
if (quota)
quota->used(Quota::ERRORS, 1, /* check_exceeded = */ false);
elem.type = QueryLogElement::EXCEPTION_WHILE_PROCESSING;

View File

@ -1,5 +1,5 @@
#include <Parsers/ASTCreateQuotaQuery.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Common/quoteString.h>
#include <Common/IntervalKind.h>
#include <ext/range.h>
@ -94,7 +94,7 @@ namespace
}
}
void formatToRoles(const ASTGenericRoleSet & roles, const IAST::FormatSettings & settings)
void formatToRoles(const ASTExtendedRoleSet & roles, const IAST::FormatSettings & settings)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " TO " << (settings.hilite ? IAST::hilite_none : "");
roles.format(settings);

View File

@ -6,7 +6,7 @@
namespace DB
{
class ASTGenericRoleSet;
class ASTExtendedRoleSet;
/** CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name
@ -53,7 +53,7 @@ public:
};
std::vector<Limits> all_limits;
std::shared_ptr<ASTGenericRoleSet> roles;
std::shared_ptr<ASTExtendedRoleSet> roles;
String getID(char) const override;
ASTPtr clone() const override;

View File

@ -1,5 +1,5 @@
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Parsers/formatAST.h>
#include <Common/quoteString.h>
#include <boost/range/algorithm/transform.hpp>
@ -10,7 +10,7 @@ namespace DB
{
namespace
{
using ConditionIndex = RowPolicy::ConditionIndex;
using ConditionType = RowPolicy::ConditionType;
void formatRenameTo(const String & new_policy_name, const IAST::FormatSettings & settings)
{
@ -37,13 +37,13 @@ namespace
}
std::vector<std::pair<ConditionIndex, String>>
conditionalExpressionsToStrings(const std::vector<std::pair<ConditionIndex, ASTPtr>> & exprs, const IAST::FormatSettings & settings)
std::vector<std::pair<ConditionType, String>>
conditionalExpressionsToStrings(const std::vector<std::pair<ConditionType, ASTPtr>> & exprs, const IAST::FormatSettings & settings)
{
std::vector<std::pair<ConditionIndex, String>> result;
std::vector<std::pair<ConditionType, String>> result;
std::stringstream ss;
IAST::FormatSettings temp_settings(ss, settings);
boost::range::transform(exprs, std::back_inserter(result), [&](const std::pair<ConditionIndex, ASTPtr> & in)
boost::range::transform(exprs, std::back_inserter(result), [&](const std::pair<ConditionType, ASTPtr> & in)
{
formatConditionalExpression(in.second, temp_settings);
auto out = std::pair{in.first, ss.str()};
@ -70,9 +70,9 @@ namespace
}
void formatMultipleConditions(const std::vector<std::pair<ConditionIndex, ASTPtr>> & conditions, bool alter, const IAST::FormatSettings & settings)
void formatMultipleConditions(const std::vector<std::pair<ConditionType, ASTPtr>> & conditions, bool alter, const IAST::FormatSettings & settings)
{
std::optional<String> scond[RowPolicy::MAX_CONDITION_INDEX];
std::optional<String> scond[RowPolicy::MAX_CONDITION_TYPE];
for (const auto & [index, scondition] : conditionalExpressionsToStrings(conditions, settings))
scond[index] = scondition;
@ -112,7 +112,7 @@ namespace
}
}
void formatToRoles(const ASTGenericRoleSet & roles, const IAST::FormatSettings & settings)
void formatToRoles(const ASTExtendedRoleSet & roles, const IAST::FormatSettings & settings)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " TO " << (settings.hilite ? IAST::hilite_none : "");
roles.format(settings);

View File

@ -8,7 +8,7 @@
namespace DB
{
class ASTGenericRoleSet;
class ASTExtendedRoleSet;
/** CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] name ON [database.]table
* [AS {PERMISSIVE | RESTRICTIVE}]
@ -39,10 +39,10 @@ public:
String new_policy_name;
std::optional<bool> is_restrictive;
using ConditionIndex = RowPolicy::ConditionIndex;
std::vector<std::pair<ConditionIndex, ASTPtr>> conditions;
using ConditionType = RowPolicy::ConditionType;
std::vector<std::pair<ConditionType, ASTPtr>> conditions;
std::shared_ptr<ASTGenericRoleSet> roles;
std::shared_ptr<ASTExtendedRoleSet> roles;
String getID(char) const override;
ASTPtr clone() const override;

View File

@ -1,5 +1,5 @@
#include <Parsers/ASTCreateUserQuery.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Common/quoteString.h>
@ -135,7 +135,7 @@ namespace
}
void formatDefaultRoles(const ASTGenericRoleSet & default_roles, const IAST::FormatSettings & settings)
void formatDefaultRoles(const ASTExtendedRoleSet & default_roles, const IAST::FormatSettings & settings)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " DEFAULT ROLE " << (settings.hilite ? IAST::hilite_none : "");
default_roles.format(settings);

View File

@ -7,7 +7,7 @@
namespace DB
{
class ASTGenericRoleSet;
class ASTExtendedRoleSet;
/** CREATE USER [IF NOT EXISTS | OR REPLACE] name
* [IDENTIFIED [WITH {NO_PASSWORD|PLAINTEXT_PASSWORD|SHA256_PASSWORD|SHA256_HASH|DOUBLE_SHA1_PASSWORD|DOUBLE_SHA1_HASH}] BY {'password'|'hash'}]
@ -41,7 +41,7 @@ public:
std::optional<AllowedClientHosts> add_hosts;
std::optional<AllowedClientHosts> remove_hosts;
std::shared_ptr<ASTGenericRoleSet> default_roles;
std::shared_ptr<ASTExtendedRoleSet> default_roles;
std::optional<String> profile;

View File

@ -1,4 +1,4 @@
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Common/quoteString.h>
@ -20,7 +20,7 @@ namespace
}
}
void ASTGenericRoleSet::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
void ASTExtendedRoleSet::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
if (empty())
{

View File

@ -8,7 +8,7 @@ namespace DB
{
/// Represents a set of users/roles like
/// {user_name | role_name | CURRENT_USER} [,...] | NONE | ALL | ALL EXCEPT {user_name | role_name | CURRENT_USER} [,...]
class ASTGenericRoleSet : public IAST
class ASTExtendedRoleSet : public IAST
{
public:
Strings names;
@ -20,8 +20,8 @@ public:
bool empty() const { return names.empty() && !current_user && !all; }
String getID(char) const override { return "GenericRoleSet"; }
ASTPtr clone() const override { return std::make_shared<ASTGenericRoleSet>(*this); }
String getID(char) const override { return "ExtendedRoleSet"; }
ASTPtr clone() const override { return std::make_shared<ASTExtendedRoleSet>(*this); }
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
}

View File

@ -1,5 +1,5 @@
#include <Parsers/ASTGrantQuery.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Common/quoteString.h>
#include <boost/range/adaptor/map.hpp>
#include <boost/range/algorithm/sort.hpp>
@ -97,7 +97,7 @@ namespace
}
void formatToRoles(const ASTGenericRoleSet & to_roles, ASTGrantQuery::Kind kind, const IAST::FormatSettings & settings)
void formatToRoles(const ASTExtendedRoleSet & to_roles, ASTGrantQuery::Kind kind, const IAST::FormatSettings & settings)
{
using Kind = ASTGrantQuery::Kind;
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << ((kind == Kind::GRANT) ? " TO " : " FROM ")

View File

@ -6,7 +6,7 @@
namespace DB
{
class ASTGenericRoleSet;
class ASTExtendedRoleSet;
/** GRANT access_type[(column_name [,...])] [,...] ON {db.table|db.*|*.*|table|*} TO {user_name | CURRENT_USER} [,...] [WITH GRANT OPTION]
@ -26,8 +26,8 @@ public:
Kind kind = Kind::GRANT;
bool attach = false;
AccessRightsElements access_rights_elements;
std::shared_ptr<ASTGenericRoleSet> roles;
std::shared_ptr<ASTGenericRoleSet> to_roles;
std::shared_ptr<ASTExtendedRoleSet> roles;
std::shared_ptr<ASTExtendedRoleSet> to_roles;
bool grant_option = false;
bool admin_option = false;

View File

@ -1,5 +1,5 @@
#include <Parsers/ASTSetRoleQuery.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Common/quoteString.h>

View File

@ -5,7 +5,7 @@
namespace DB
{
class ASTGenericRoleSet;
class ASTExtendedRoleSet;
/** SET ROLE {DEFAULT | NONE | role [,...] | ALL | ALL EXCEPT role [,...]}
* SET DEFAULT ROLE {NONE | role [,...] | ALL | ALL EXCEPT role [,...]} TO {user|CURRENT_USER} [,...]
@ -21,8 +21,8 @@ public:
};
Kind kind = Kind::SET_ROLE;
std::shared_ptr<ASTGenericRoleSet> roles;
std::shared_ptr<ASTGenericRoleSet> to_users;
std::shared_ptr<ASTExtendedRoleSet> roles;
std::shared_ptr<ASTExtendedRoleSet> to_users;
String getID(char) const override;
ASTPtr clone() const override;

View File

@ -3,10 +3,10 @@
#include <Parsers/CommonParsers.h>
#include <Parsers/parseIntervalKind.h>
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Parsers/ParserGenericRoleSet.h>
#include <Parsers/ParserExtendedRoleSet.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <ext/range.h>
#include <boost/algorithm/string/predicate.hpp>
@ -187,15 +187,15 @@ namespace
});
}
bool parseToRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr<ASTGenericRoleSet> & roles)
bool parseToRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr<ASTExtendedRoleSet> & roles)
{
return IParserBase::wrapParseImpl(pos, [&]
{
ASTPtr node;
if (roles || !ParserKeyword{"TO"}.ignore(pos, expected) || !ParserGenericRoleSet{}.enableIDMode(id_mode).parse(pos, node, expected))
if (roles || !ParserKeyword{"TO"}.ignore(pos, expected) || !ParserExtendedRoleSet{}.useIDMode(id_mode).parse(pos, node, expected))
return false;
roles = std::static_pointer_cast<ASTGenericRoleSet>(node);
roles = std::static_pointer_cast<ASTExtendedRoleSet>(node);
return true;
});
}
@ -243,7 +243,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
String new_name;
std::optional<KeyType> key_type;
std::vector<ASTCreateQuotaQuery::Limits> all_limits;
std::shared_ptr<ASTGenericRoleSet> roles;
std::shared_ptr<ASTExtendedRoleSet> roles;
while (true)
{

View File

@ -1,8 +1,8 @@
#include <Parsers/ParserCreateRowPolicyQuery.h>
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Access/RowPolicy.h>
#include <Parsers/ParserGenericRoleSet.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/ParserExtendedRoleSet.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Parsers/parseDatabaseAndTableName.h>
#include <Parsers/ExpressionListParsers.h>
@ -19,7 +19,7 @@ namespace ErrorCodes
namespace
{
using ConditionIndex = RowPolicy::ConditionIndex;
using ConditionType = RowPolicy::ConditionType;
bool parseRenameTo(IParserBase::Pos & pos, Expected & expected, String & new_policy_name)
{
@ -73,7 +73,7 @@ namespace
});
}
bool parseConditions(IParserBase::Pos & pos, Expected & expected, bool alter, std::vector<std::pair<ConditionIndex, ASTPtr>> & conditions)
bool parseConditions(IParserBase::Pos & pos, Expected & expected, bool alter, std::vector<std::pair<ConditionType, ASTPtr>> & conditions)
{
return IParserBase::wrapParseImpl(pos, [&]
{
@ -136,14 +136,14 @@ namespace
if (filter && !check && !alter)
check = filter;
auto set_condition = [&](ConditionIndex index, const ASTPtr & condition)
auto set_condition = [&](ConditionType index, const ASTPtr & condition)
{
auto it = std::find_if(conditions.begin(), conditions.end(), [index](const std::pair<ConditionIndex, ASTPtr> & element)
auto it = std::find_if(conditions.begin(), conditions.end(), [index](const std::pair<ConditionType, ASTPtr> & element)
{
return element.first == index;
});
if (it == conditions.end())
it = conditions.insert(conditions.end(), std::pair<ConditionIndex, ASTPtr>{index, nullptr});
it = conditions.insert(conditions.end(), std::pair<ConditionType, ASTPtr>{index, nullptr});
it->second = condition;
};
@ -170,11 +170,11 @@ namespace
});
}
bool parseMultipleConditions(IParserBase::Pos & pos, Expected & expected, bool alter, std::vector<std::pair<ConditionIndex, ASTPtr>> & conditions)
bool parseMultipleConditions(IParserBase::Pos & pos, Expected & expected, bool alter, std::vector<std::pair<ConditionType, ASTPtr>> & conditions)
{
return IParserBase::wrapParseImpl(pos, [&]
{
std::vector<std::pair<ConditionIndex, ASTPtr>> res_conditions;
std::vector<std::pair<ConditionType, ASTPtr>> res_conditions;
do
{
if (!parseConditions(pos, expected, alter, res_conditions))
@ -187,16 +187,16 @@ namespace
});
}
bool parseToRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr<ASTGenericRoleSet> & roles)
bool parseToRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr<ASTExtendedRoleSet> & roles)
{
return IParserBase::wrapParseImpl(pos, [&]
{
ASTPtr ast;
if (roles || !ParserKeyword{"TO"}.ignore(pos, expected)
|| !ParserGenericRoleSet{}.enableIDMode(id_mode).parse(pos, ast, expected))
|| !ParserExtendedRoleSet{}.useIDMode(id_mode).parse(pos, ast, expected))
return false;
roles = std::static_pointer_cast<ASTGenericRoleSet>(ast);
roles = std::static_pointer_cast<ASTExtendedRoleSet>(ast);
return true;
});
}
@ -247,8 +247,8 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
String new_policy_name;
std::optional<bool> is_restrictive;
std::vector<std::pair<ConditionIndex, ASTPtr>> conditions;
std::shared_ptr<ASTGenericRoleSet> roles;
std::vector<std::pair<ConditionType, ASTPtr>> conditions;
std::shared_ptr<ASTExtendedRoleSet> roles;
while (true)
{

View File

@ -5,8 +5,8 @@
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/ParserGenericRoleSet.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Parsers/ParserExtendedRoleSet.h>
#include <ext/range.h>
#include <boost/algorithm/string/predicate.hpp>
@ -208,7 +208,7 @@ namespace
}
bool parseDefaultRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr<ASTGenericRoleSet> & default_roles)
bool parseDefaultRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr<ASTExtendedRoleSet> & default_roles)
{
return IParserBase::wrapParseImpl(pos, [&]
{
@ -216,10 +216,10 @@ namespace
return false;
ASTPtr ast;
if (!ParserGenericRoleSet{}.enableCurrentUserKeyword(false).enableIDMode(id_mode).parse(pos, ast, expected))
if (!ParserExtendedRoleSet{}.enableCurrentUserKeyword(false).useIDMode(id_mode).parse(pos, ast, expected))
return false;
default_roles = typeid_cast<std::shared_ptr<ASTGenericRoleSet>>(ast);
default_roles = typeid_cast<std::shared_ptr<ASTExtendedRoleSet>>(ast);
return true;
});
}
@ -289,7 +289,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
std::optional<AllowedClientHosts> hosts;
std::optional<AllowedClientHosts> add_hosts;
std::optional<AllowedClientHosts> remove_hosts;
std::shared_ptr<ASTGenericRoleSet> default_roles;
std::shared_ptr<ASTExtendedRoleSet> default_roles;
std::optional<String> profile;
while (true)

View File

@ -1,8 +1,8 @@
#include <Parsers/ParserGenericRoleSet.h>
#include <Parsers/ParserExtendedRoleSet.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Parsers/parseUserName.h>
#include <boost/range/algorithm/find.hpp>
@ -109,7 +109,7 @@ namespace
}
bool ParserGenericRoleSet::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserExtendedRoleSet::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Strings names;
bool current_user = false;
@ -125,7 +125,7 @@ bool ParserGenericRoleSet::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
if (all)
names.clear();
auto result = std::make_shared<ASTGenericRoleSet>();
auto result = std::make_shared<ASTExtendedRoleSet>();
result->names = std::move(names);
result->current_user = current_user;
result->all = all;

View File

@ -0,0 +1,28 @@
#pragma once
#include <Parsers/IParserBase.h>
namespace DB
{
/** Parses a string like this:
* {role|CURRENT_USER} [,...] | NONE | ALL | ALL EXCEPT {role|CURRENT_USER} [,...]
*/
class ParserExtendedRoleSet : public IParserBase
{
public:
ParserExtendedRoleSet & enableAllKeyword(bool enable_) { all_keyword = enable_; return *this; }
ParserExtendedRoleSet & enableCurrentUserKeyword(bool enable_) { current_user_keyword = enable_; return *this; }
ParserExtendedRoleSet & useIDMode(bool enable_) { id_mode = enable_; return *this; }
protected:
const char * getName() const override { return "ExtendedRoleSet"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
private:
bool all_keyword = true;
bool current_user_keyword = true;
bool id_mode = false;
};
}

View File

@ -1,28 +0,0 @@
#pragma once
#include <Parsers/IParserBase.h>
namespace DB
{
/** Parses a string like this:
* {role|CURRENT_USER} [,...] | NONE | ALL | ALL EXCEPT {role|CURRENT_USER} [,...]
*/
class ParserGenericRoleSet : public IParserBase
{
public:
ParserGenericRoleSet & enableAllKeyword(bool enable_) { all_keyword = enable_; return *this; }
ParserGenericRoleSet & enableCurrentUserKeyword(bool enable_) { current_user_keyword = enable_; return *this; }
ParserGenericRoleSet & enableIDMode(bool enable_) { id_mode = enable_; return *this; }
protected:
const char * getName() const override { return "GenericRoleSet"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
private:
bool all_keyword = true;
bool current_user_keyword = true;
bool id_mode = false;
};
}

View File

@ -1,10 +1,10 @@
#include <Parsers/ParserGrantQuery.h>
#include <Parsers/ASTGrantQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ParserGenericRoleSet.h>
#include <Parsers/ParserExtendedRoleSet.h>
#include <boost/algorithm/string/predicate.hpp>
@ -209,21 +209,21 @@ namespace
}
bool parseRoles(IParser::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr<ASTGenericRoleSet> & roles)
bool parseRoles(IParser::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr<ASTExtendedRoleSet> & roles)
{
return IParserBase::wrapParseImpl(pos, [&]
{
ASTPtr ast;
if (!ParserGenericRoleSet{}.enableAllKeyword(false).enableCurrentUserKeyword(false).enableIDMode(id_mode).parse(pos, ast, expected))
if (!ParserExtendedRoleSet{}.enableAllKeyword(false).enableCurrentUserKeyword(false).useIDMode(id_mode).parse(pos, ast, expected))
return false;
roles = typeid_cast<std::shared_ptr<ASTGenericRoleSet>>(ast);
roles = typeid_cast<std::shared_ptr<ASTExtendedRoleSet>>(ast);
return true;
});
}
bool parseToRoles(IParser::Pos & pos, Expected & expected, ASTGrantQuery::Kind kind, std::shared_ptr<ASTGenericRoleSet> & to_roles)
bool parseToRoles(IParser::Pos & pos, Expected & expected, ASTGrantQuery::Kind kind, std::shared_ptr<ASTExtendedRoleSet> & to_roles)
{
return IParserBase::wrapParseImpl(pos, [&]
{
@ -240,10 +240,10 @@ namespace
}
ASTPtr ast;
if (!ParserGenericRoleSet{}.enableAllKeyword(kind == Kind::REVOKE).parse(pos, ast, expected))
if (!ParserExtendedRoleSet{}.enableAllKeyword(kind == Kind::REVOKE).parse(pos, ast, expected))
return false;
to_roles = typeid_cast<std::shared_ptr<ASTGenericRoleSet>>(ast);
to_roles = typeid_cast<std::shared_ptr<ASTExtendedRoleSet>>(ast);
return true;
});
}
@ -280,11 +280,11 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
AccessRightsElements elements;
std::shared_ptr<ASTGenericRoleSet> roles;
std::shared_ptr<ASTExtendedRoleSet> roles;
if (!parseAccessRightsElements(pos, expected, elements) && !parseRoles(pos, expected, attach, roles))
return false;
std::shared_ptr<ASTGenericRoleSet> to_roles;
std::shared_ptr<ASTExtendedRoleSet> to_roles;
if (!parseToRoles(pos, expected, kind, to_roles))
return false;

View File

@ -1,28 +1,28 @@
#include <Parsers/ParserSetRoleQuery.h>
#include <Parsers/ASTSetRoleQuery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/ParserGenericRoleSet.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Parsers/ParserExtendedRoleSet.h>
namespace DB
{
namespace
{
bool parseRoles(IParserBase::Pos & pos, Expected & expected, std::shared_ptr<ASTGenericRoleSet> & roles)
bool parseRoles(IParserBase::Pos & pos, Expected & expected, std::shared_ptr<ASTExtendedRoleSet> & roles)
{
return IParserBase::wrapParseImpl(pos, [&]
{
ASTPtr ast;
if (!ParserGenericRoleSet{}.enableCurrentUserKeyword(false).parse(pos, ast, expected))
if (!ParserExtendedRoleSet{}.enableCurrentUserKeyword(false).parse(pos, ast, expected))
return false;
roles = typeid_cast<std::shared_ptr<ASTGenericRoleSet>>(ast);
roles = typeid_cast<std::shared_ptr<ASTExtendedRoleSet>>(ast);
return true;
});
}
bool parseToUsers(IParserBase::Pos & pos, Expected & expected, std::shared_ptr<ASTGenericRoleSet> & to_users)
bool parseToUsers(IParserBase::Pos & pos, Expected & expected, std::shared_ptr<ASTExtendedRoleSet> & to_users)
{
return IParserBase::wrapParseImpl(pos, [&]
{
@ -30,10 +30,10 @@ namespace
return false;
ASTPtr ast;
if (!ParserGenericRoleSet{}.enableAllKeyword(false).parse(pos, ast, expected))
if (!ParserExtendedRoleSet{}.enableAllKeyword(false).parse(pos, ast, expected))
return false;
to_users = typeid_cast<std::shared_ptr<ASTGenericRoleSet>>(ast);
to_users = typeid_cast<std::shared_ptr<ASTExtendedRoleSet>>(ast);
return true;
});
}
@ -53,8 +53,8 @@ bool ParserSetRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
else
return false;
std::shared_ptr<ASTGenericRoleSet> roles;
std::shared_ptr<ASTGenericRoleSet> to_users;
std::shared_ptr<ASTExtendedRoleSet> roles;
std::shared_ptr<ASTExtendedRoleSet> to_users;
if ((kind == Kind::SET_ROLE) || (kind == Kind::SET_DEFAULT_ROLE))
{

View File

@ -335,7 +335,7 @@ void TreeExecutorBlockInputStream::setLimits(const IBlockInputStream::LocalLimit
source->setLimits(limits_);
}
void TreeExecutorBlockInputStream::setQuota(const QuotaContextPtr & quota_)
void TreeExecutorBlockInputStream::setQuota(const std::shared_ptr<const EnabledQuota> & quota_)
{
for (auto & source : sources_with_progress)
source->setQuota(quota_);

View File

@ -43,7 +43,7 @@ public:
void setProgressCallback(const ProgressCallback & callback) final;
void setProcessListElement(QueryStatus * elem) final;
void setLimits(const LocalLimits & limits_) final;
void setQuota(const QuotaContextPtr & quota_) final;
void setQuota(const std::shared_ptr<const EnabledQuota> & quota_) final;
void addTotalRowsApprox(size_t value) final;
protected:

View File

@ -106,7 +106,7 @@ void Pipe::setLimits(const ISourceWithProgress::LocalLimits & limits)
}
}
void Pipe::setQuota(const QuotaContextPtr & quota)
void Pipe::setQuota(const std::shared_ptr<const EnabledQuota> & quota)
{
for (auto & processor : processors)
{

View File

@ -40,7 +40,7 @@ public:
/// Specify quotas and limits for every ISourceWithProgress.
void setLimits(const SourceWithProgress::LocalLimits & limits);
void setQuota(const QuotaContextPtr & quota);
void setQuota(const std::shared_ptr<const EnabledQuota> & quota);
/// Set information about preferred executor number for sources.
void pinSources(size_t executor_number);

View File

@ -28,7 +28,7 @@ public:
/// Implementation for methods from ISourceWithProgress.
void setLimits(const LocalLimits & limits_) final { stream->setLimits(limits_); }
void setQuota(const QuotaContextPtr & quota_) final { stream->setQuota(quota_); }
void setQuota(const std::shared_ptr<const EnabledQuota> & quota_) final { stream->setQuota(quota_); }
void setProcessListElement(QueryStatus * elem) final { stream->setProcessListElement(elem); }
void setProgressCallback(const ProgressCallback & callback) final { stream->setProgressCallback(callback); }
void addTotalRowsApprox(size_t value) final { stream->addTotalRowsApprox(value); }

View File

@ -1,7 +1,7 @@
#include <Processors/Sources/SourceWithProgress.h>
#include <Interpreters/ProcessList.h>
#include <Access/QuotaContext.h>
#include <Access/EnabledQuota.h>
namespace DB
{

View File

@ -21,7 +21,7 @@ public:
/// Set the quota. If you set a quota on the amount of raw data,
/// then you should also set mode = LIMITS_TOTAL to LocalLimits with setLimits.
virtual void setQuota(const QuotaContextPtr & quota_) = 0;
virtual void setQuota(const std::shared_ptr<const EnabledQuota> & quota_) = 0;
/// Set the pointer to the process list item.
/// General information about the resources spent on the request will be written into it.
@ -49,7 +49,7 @@ public:
using LimitsMode = IBlockInputStream::LimitsMode;
void setLimits(const LocalLimits & limits_) final { limits = limits_; }
void setQuota(const QuotaContextPtr & quota_) final { quota = quota_; }
void setQuota(const std::shared_ptr<const EnabledQuota> & quota_) final { quota = quota_; }
void setProcessListElement(QueryStatus * elem) final { process_list_elem = elem; }
void setProgressCallback(const ProgressCallback & callback) final { progress_callback = callback; }
void addTotalRowsApprox(size_t value) final { total_rows_approx += value; }
@ -62,7 +62,7 @@ protected:
private:
LocalLimits limits;
QuotaContextPtr quota;
std::shared_ptr<const EnabledQuota> quota;
ProgressCallback progress_callback;
QueryStatus * process_list_elem = nullptr;

View File

@ -1,5 +1,5 @@
#include <Processors/Transforms/LimitsCheckingTransform.h>
#include <Access/QuotaContext.h>
#include <Access/EnabledQuota.h>
namespace DB
{

View File

@ -33,7 +33,7 @@ public:
String getName() const override { return "LimitsCheckingTransform"; }
void setQuota(const QuotaContextPtr & quota_) { quota = quota_; }
void setQuota(const std::shared_ptr<const EnabledQuota> & quota_) { quota = quota_; }
protected:
void transform(Chunk & chunk) override;
@ -41,7 +41,7 @@ protected:
private:
LocalLimits limits;
QuotaContextPtr quota;
std::shared_ptr<const EnabledQuota> quota;
UInt64 prev_elapsed = 0;
ProcessorProfileInfo info;

View File

@ -9,7 +9,7 @@
#include <Storages/VirtualColumnUtils.h>
#include <Parsers/queryToString.h>
#include <Parsers/ASTSelectQuery.h>
#include <Access/AccessRightsContext.h>
#include <Access/ContextAccess.h>
#include <Databases/IDatabase.h>
#include <Processors/Sources/NullSource.h>
@ -62,12 +62,12 @@ public:
ColumnPtr databases_,
ColumnPtr tables_,
Storages storages_,
const AccessRightsContextPtr & access_rights_,
const std::shared_ptr<const ContextAccess> & access_,
String query_id_)
: SourceWithProgress(header_)
, columns_mask(std::move(columns_mask_)), max_block_size(max_block_size_)
, databases(std::move(databases_)), tables(std::move(tables_)), storages(std::move(storages_))
, query_id(std::move(query_id_)), total_tables(tables->size()), access_rights(access_rights_)
, query_id(std::move(query_id_)), total_tables(tables->size()), access(access_)
{
}
@ -82,7 +82,7 @@ protected:
MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns();
size_t rows_count = 0;
const bool check_access_for_tables = !access_rights->isGranted(AccessType::SHOW_COLUMNS);
const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_COLUMNS);
while (rows_count < max_block_size && db_table_num < total_tables)
{
@ -128,14 +128,14 @@ protected:
column_sizes = storage->getColumnSizes();
}
bool check_access_for_columns = check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name);
bool check_access_for_columns = check_access_for_tables && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name);
for (const auto & column : columns)
{
if (column.is_virtual)
continue;
if (check_access_for_columns && !access_rights->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name, column.name))
if (check_access_for_columns && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name, column.name))
continue;
size_t src_index = 0;
@ -230,7 +230,7 @@ private:
String query_id;
size_t db_table_num = 0;
size_t total_tables;
AccessRightsContextPtr access_rights;
std::shared_ptr<const ContextAccess> access;
};
@ -332,7 +332,7 @@ Pipes StorageSystemColumns::read(
pipes.emplace_back(std::make_shared<ColumnsSource>(
std::move(columns_mask), std::move(header), max_block_size,
std::move(filtered_database_column), std::move(filtered_table_column), std::move(storages),
context.getAccessRights(), context.getCurrentQueryId()));
context.getAccess(), context.getCurrentQueryId()));
return pipes;
}

View File

@ -1,7 +1,7 @@
#include <Databases/IDatabase.h>
#include <DataTypes/DataTypeString.h>
#include <Interpreters/Context.h>
#include <Access/AccessRightsContext.h>
#include <Access/ContextAccess.h>
#include <Storages/System/StorageSystemDatabases.h>
@ -20,13 +20,13 @@ NamesAndTypesList StorageSystemDatabases::getNamesAndTypes()
void StorageSystemDatabases::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const
{
const auto access_rights = context.getAccessRights();
const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW_DATABASES);
const auto access = context.getAccess();
const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_DATABASES);
auto databases = DatabaseCatalog::instance().getDatabases();
for (const auto & database : databases)
{
if (check_access_for_databases && !access_rights->isGranted(AccessType::SHOW_DATABASES, database.first))
if (check_access_for_databases && !access->isGranted(AccessType::SHOW_DATABASES, database.first))
continue;
res_columns[0]->insert(database.first);

View File

@ -8,7 +8,7 @@
#include <Dictionaries/DictionaryStructure.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Access/AccessRightsContext.h>
#include <Access/ContextAccess.h>
#include <Storages/System/StorageSystemDictionaries.h>
#include <Storages/VirtualColumnUtils.h>
#include <Columns/ColumnString.h>
@ -49,8 +49,8 @@ NamesAndTypesList StorageSystemDictionaries::getNamesAndTypes()
void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & /*query_info*/) const
{
const auto access_rights = context.getAccessRights();
const bool check_access_for_dictionaries = !access_rights->isGranted(AccessType::SHOW_DICTIONARIES);
const auto access = context.getAccess();
const bool check_access_for_dictionaries = !access->isGranted(AccessType::SHOW_DICTIONARIES);
const auto & external_dictionaries = context.getExternalDictionariesLoader();
for (const auto & load_result : external_dictionaries.getCurrentLoadResults())
@ -74,7 +74,7 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Con
}
if (check_access_for_dictionaries
&& !access_rights->isGranted(AccessType::SHOW_DICTIONARIES, database.empty() ? IDictionary::NO_DATABASE_TAG : database, short_name))
&& !access->isGranted(AccessType::SHOW_DICTIONARIES, database.empty() ? IDictionary::NO_DATABASE_TAG : database, short_name))
continue;
size_t i = 0;

View File

@ -1,7 +1,7 @@
#include <Interpreters/Context.h>
#include <Storages/MergeTree/MergeList.h>
#include <Storages/System/StorageSystemMerges.h>
#include <Access/AccessRightsContext.h>
#include <Access/ContextAccess.h>
namespace DB
@ -36,12 +36,12 @@ NamesAndTypesList StorageSystemMerges::getNamesAndTypes()
void StorageSystemMerges::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const
{
const auto access_rights = context.getAccessRights();
const bool check_access_for_tables = !access_rights->isGranted(AccessType::SHOW_TABLES);
const auto access = context.getAccess();
const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_TABLES);
for (const auto & merge : context.getMergeList().get())
{
if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_TABLES, merge.database, merge.table))
if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, merge.database, merge.table))
continue;
size_t i = 0;

View File

@ -7,7 +7,7 @@
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeMutationStatus.h>
#include <Storages/VirtualColumnUtils.h>
#include <Access/AccessRightsContext.h>
#include <Access/ContextAccess.h>
#include <Databases/IDatabase.h>
@ -37,8 +37,8 @@ NamesAndTypesList StorageSystemMutations::getNamesAndTypes()
void StorageSystemMutations::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const
{
const auto access_rights = context.getAccessRights();
const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW_TABLES);
const auto access = context.getAccess();
const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES);
/// Collect a set of *MergeTree tables.
std::map<String, std::map<String, StoragePtr>> merge_tree_tables;
@ -48,14 +48,14 @@ void StorageSystemMutations::fillData(MutableColumns & res_columns, const Contex
if (db.second->getEngineName() == "Lazy")
continue;
const bool check_access_for_tables = check_access_for_databases && !access_rights->isGranted(AccessType::SHOW_TABLES, db.first);
const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first);
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
{
if (!dynamic_cast<const MergeTreeData *>(iterator->table().get()))
continue;
if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name()))
if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name()))
continue;
merge_tree_tables[db.first][iterator->name()] = iterator->table();

View File

@ -9,7 +9,7 @@
#include <DataStreams/OneBlockInputStream.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/VirtualColumnUtils.h>
#include <Access/AccessRightsContext.h>
#include <Access/ContextAccess.h>
#include <Databases/IDatabase.h>
#include <Parsers/queryToString.h>
#include <Parsers/ASTIdentifier.h>
@ -73,8 +73,8 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const
MutableColumnPtr engine_column_mut = ColumnString::create();
MutableColumnPtr active_column_mut = ColumnUInt8::create();
const auto access_rights = context.getAccessRights();
const bool check_access_for_tables = !access_rights->isGranted(AccessType::SHOW_TABLES);
const auto access = context.getAccess();
const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_TABLES);
{
Databases databases = DatabaseCatalog::instance().getDatabases();
@ -119,7 +119,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const
if (!dynamic_cast<MergeTreeData *>(storage.get()))
continue;
if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_TABLES, database_name, table_name))
if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name))
continue;
storages[std::make_pair(database_name, iterator->name())] = storage;

View File

@ -6,7 +6,8 @@
#include <DataTypes/DataTypeNullable.h>
#include <Interpreters/Context.h>
#include <Access/AccessControlManager.h>
#include <Access/QuotaContext.h>
#include <Access/EnabledQuota.h>
#include <Access/QuotaUsageInfo.h>
#include <ext/range.h>

View File

@ -87,7 +87,7 @@ void StorageSystemQuotas::fillData(MutableColumns & res_columns, const Context &
storage_name_column.insert(storage_name);
key_type_column.insert(static_cast<UInt8>(quota->key_type));
for (const String & role : quota->roles.toStringsWithNames(access_control))
for (const String & role : quota->to_roles.toStringsWithNames(access_control))
roles_data.insert(role);
roles_offsets.push_back(roles_data.size());

View File

@ -6,7 +6,7 @@
#include <Storages/System/StorageSystemReplicas.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/VirtualColumnUtils.h>
#include <Access/AccessRightsContext.h>
#include <Access/ContextAccess.h>
#include <Common/typeid_cast.h>
#include <Databases/IDatabase.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
@ -65,8 +65,8 @@ Pipes StorageSystemReplicas::read(
{
check(column_names);
const auto access_rights = context.getAccessRights();
const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW_TABLES);
const auto access = context.getAccess();
const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES);
/// We collect a set of replicated tables.
std::map<String, std::map<String, StoragePtr>> replicated_tables;
@ -75,12 +75,12 @@ Pipes StorageSystemReplicas::read(
/// Lazy database can not contain replicated tables
if (db.second->getEngineName() == "Lazy")
continue;
const bool check_access_for_tables = check_access_for_databases && !access_rights->isGranted(AccessType::SHOW_TABLES, db.first);
const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first);
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
{
if (!dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
continue;
if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name()))
if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name()))
continue;
replicated_tables[db.first][iterator->name()] = iterator->table();
}

View File

@ -8,7 +8,7 @@
#include <Storages/System/StorageSystemReplicationQueue.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/VirtualColumnUtils.h>
#include <Access/AccessRightsContext.h>
#include <Access/ContextAccess.h>
#include <Common/typeid_cast.h>
#include <Databases/IDatabase.h>
@ -48,8 +48,8 @@ NamesAndTypesList StorageSystemReplicationQueue::getNamesAndTypes()
void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const
{
const auto access_rights = context.getAccessRights();
const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW_TABLES);
const auto access = context.getAccess();
const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES);
std::map<String, std::map<String, StoragePtr>> replicated_tables;
for (const auto & db : DatabaseCatalog::instance().getDatabases())
@ -58,13 +58,13 @@ void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, const
if (db.second->getEngineName() == "Lazy")
continue;
const bool check_access_for_tables = check_access_for_databases && !access_rights->isGranted(AccessType::SHOW_TABLES, db.first);
const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first);
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
{
if (!dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
continue;
if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name()))
if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name()))
continue;
replicated_tables[db.first][iterator->name()] = iterator->table();
}

View File

@ -24,8 +24,8 @@ NamesAndTypesList StorageSystemRowPolicies::getNamesAndTypes()
{"restrictive", std::make_shared<DataTypeUInt8>()},
};
for (auto index : ext::range_with_static_cast<RowPolicy::ConditionIndex>(RowPolicy::MAX_CONDITION_INDEX))
names_and_types.push_back({RowPolicy::conditionIndexToColumnName(index), std::make_shared<DataTypeString>()});
for (auto index : ext::range_with_static_cast<RowPolicy::ConditionType>(RowPolicy::MAX_CONDITION_TYPE))
names_and_types.push_back({RowPolicy::conditionTypeToColumnName(index), std::make_shared<DataTypeString>()});
return names_and_types;
}
@ -52,7 +52,7 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, const Cont
res_columns[i++]->insert(storage ? storage->getStorageName() : "");
res_columns[i++]->insert(policy->isRestrictive());
for (auto index : ext::range(RowPolicy::MAX_CONDITION_INDEX))
for (auto index : ext::range(RowPolicy::MAX_CONDITION_TYPE))
res_columns[i++]->insert(policy->conditions[index]);
}
}

View File

@ -6,7 +6,7 @@
#include <Storages/System/StorageSystemTables.h>
#include <Storages/VirtualColumnUtils.h>
#include <Databases/IDatabase.h>
#include <Access/AccessRightsContext.h>
#include <Access/ContextAccess.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/queryToString.h>
@ -105,8 +105,8 @@ protected:
MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns();
const auto access_rights = context.getAccessRights();
const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW_TABLES);
const auto access = context.getAccess();
const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES);
size_t rows_count = 0;
while (rows_count < max_block_size)
@ -196,7 +196,7 @@ protected:
return Chunk(std::move(res_columns), num_rows);
}
const bool check_access_for_tables = check_access_for_databases && !access_rights->isGranted(AccessType::SHOW_TABLES, database_name);
const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name);
if (!tables_it || !tables_it->isValid())
tables_it = database->getTablesWithDictionaryTablesIterator(context);
@ -206,7 +206,7 @@ protected:
for (; rows_count < max_block_size && tables_it->isValid(); tables_it->next())
{
auto table_name = tables_it->name();
if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_TABLES, database_name, table_name))
if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name))
continue;
StoragePtr table = nullptr;