Avoid keeping lock Context::getLock() while recalculating access rights of a connected user.

This commit is contained in:
Vitaly Baranov 2023-07-01 17:39:50 +02:00
parent db1b172770
commit 018546a57d
8 changed files with 362 additions and 171 deletions

View File

@ -71,19 +71,27 @@ public:
explicit ContextAccessCache(const AccessControl & access_control_) : access_control(access_control_) {} explicit ContextAccessCache(const AccessControl & access_control_) : access_control(access_control_) {}
std::shared_ptr<const ContextAccess> getContextAccess(const ContextAccessParams & params) std::shared_ptr<const ContextAccess> getContextAccess(const ContextAccessParams & params)
{
{ {
std::lock_guard lock{mutex}; std::lock_guard lock{mutex};
auto x = cache.get(params); auto x = cache.get(params);
if (x) if (x)
{ {
if ((*x)->tryGetUser()) if ((*x)->getUserID() && !(*x)->tryGetUser())
cache.remove(params); /// The user has been dropped while it was in the cache.
else
return *x; return *x;
/// No user, probably the user has been dropped while it was in the cache.
cache.remove(params);
} }
}
auto res = std::make_shared<ContextAccess>(access_control, params); auto res = std::make_shared<ContextAccess>(access_control, params);
res->initialize(); res->initialize();
{
std::lock_guard lock{mutex};
cache.add(params, res); cache.add(params, res);
}
return res; return res;
} }
@ -713,35 +721,6 @@ int AccessControl::getBcryptWorkfactor() const
} }
std::shared_ptr<const ContextAccess> AccessControl::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) const
{
ContextAccessParams params;
params.user_id = user_id;
params.current_roles.insert(current_roles.begin(), current_roles.end());
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;
/// Extract the last entry from comma separated list of X-Forwarded-For addresses.
/// Only the last proxy can be trusted (if any).
params.forwarded_address = client_info.getLastForwardedFor();
return getContextAccess(params);
}
std::shared_ptr<const ContextAccess> AccessControl::getContextAccess(const ContextAccessParams & params) const std::shared_ptr<const ContextAccess> AccessControl::getContextAccess(const ContextAccessParams & params) const
{ {
return context_access_cache->getContextAccess(params); return context_access_cache->getContextAccess(params);

View File

@ -25,7 +25,7 @@ namespace Poco
namespace DB namespace DB
{ {
class ContextAccess; class ContextAccess;
struct ContextAccessParams; class ContextAccessParams;
struct User; struct User;
using UserPtr = std::shared_ptr<const User>; using UserPtr = std::shared_ptr<const User>;
class EnabledRoles; class EnabledRoles;
@ -181,14 +181,6 @@ public:
void setSettingsConstraintsReplacePrevious(bool enable) { settings_constraints_replace_previous = enable; } void setSettingsConstraintsReplacePrevious(bool enable) { settings_constraints_replace_previous = enable; }
bool doesSettingsConstraintsReplacePrevious() const { return settings_constraints_replace_previous; } bool doesSettingsConstraintsReplacePrevious() const { return settings_constraints_replace_previous; }
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) const;
std::shared_ptr<const ContextAccess> getContextAccess(const ContextAccessParams & params) const; std::shared_ptr<const ContextAccess> getContextAccess(const ContextAccessParams & params) const;
std::shared_ptr<const EnabledRoles> getEnabledRoles( std::shared_ptr<const EnabledRoles> getEnabledRoles(

View File

@ -10,6 +10,7 @@
#include <Access/EnabledSettings.h> #include <Access/EnabledSettings.h>
#include <Access/SettingsProfilesInfo.h> #include <Access/SettingsProfilesInfo.h>
#include <Interpreters/DatabaseCatalog.h> #include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/Context.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <Core/Settings.h> #include <Core/Settings.h>
@ -221,15 +222,15 @@ namespace
} }
ContextAccess::ContextAccess(const AccessControl & access_control_, const Params & params_) std::shared_ptr<const ContextAccess> ContextAccess::fromContext(const ContextPtr & context)
: access_control(&access_control_)
, params(params_)
{ {
return context->getAccess();
} }
ContextAccess::ContextAccess(FullAccess) ContextAccess::ContextAccess(const AccessControl & access_control_, const Params & params_)
: is_full_access(true), access(std::make_shared<AccessRights>(AccessRights::getFullAccess())), access_with_implicit(access) : access_control(&access_control_)
, params(params_)
{ {
} }
@ -252,8 +253,20 @@ ContextAccess::~ContextAccess()
void ContextAccess::initialize() void ContextAccess::initialize()
{ {
std::lock_guard lock{mutex}; std::lock_guard lock{mutex};
if (params.full_access)
{
access = std::make_shared<AccessRights>(AccessRights::getFullAccess());
access_with_implicit = access;
return;
}
if (!params.user_id)
throw Exception(ErrorCodes::LOGICAL_ERROR, "No user in current context, it's a bug");
subscription_for_user_change = access_control->subscribeForChanges( subscription_for_user_change = access_control->subscribeForChanges(
*params.user_id, [weak_ptr = weak_from_this()](const UUID &, const AccessEntityPtr & entity) *params.user_id,
[weak_ptr = weak_from_this()](const UUID &, const AccessEntityPtr & entity)
{ {
auto ptr = weak_ptr.lock(); auto ptr = weak_ptr.lock();
if (!ptr) if (!ptr)
@ -262,6 +275,7 @@ void ContextAccess::initialize()
std::lock_guard lock2{ptr->mutex}; std::lock_guard lock2{ptr->mutex};
ptr->setUser(changed_user); ptr->setUser(changed_user);
}); });
setUser(access_control->read<User>(*params.user_id)); setUser(access_control->read<User>(*params.user_id));
} }
@ -294,10 +308,10 @@ void ContextAccess::setUser(const UserPtr & user_) const
current_roles = user->granted_roles.findGranted(user->default_roles); current_roles = user->granted_roles.findGranted(user->default_roles);
current_roles_with_admin_option = user->granted_roles.findGrantedWithAdminOption(user->default_roles); current_roles_with_admin_option = user->granted_roles.findGrantedWithAdminOption(user->default_roles);
} }
else else if (params.current_roles)
{ {
current_roles = user->granted_roles.findGranted(params.current_roles); current_roles = user->granted_roles.findGranted(*params.current_roles);
current_roles_with_admin_option = user->granted_roles.findGrantedWithAdminOption(params.current_roles); current_roles_with_admin_option = user->granted_roles.findGrantedWithAdminOption(*params.current_roles);
} }
subscription_for_roles_changes.reset(); subscription_for_roles_changes.reset();
@ -316,12 +330,16 @@ void ContextAccess::setRolesInfo(const std::shared_ptr<const EnabledRolesInfo> &
{ {
assert(roles_info_); assert(roles_info_);
roles_info = roles_info_; roles_info = roles_info_;
enabled_row_policies = access_control->getEnabledRowPolicies( enabled_row_policies = access_control->getEnabledRowPolicies(
*params.user_id, roles_info->enabled_roles); *params.user_id, roles_info->enabled_roles);
enabled_quota = access_control->getEnabledQuota( enabled_quota = access_control->getEnabledQuota(
*params.user_id, user_name, roles_info->enabled_roles, params.address, params.forwarded_address, params.quota_key); *params.user_id, user_name, roles_info->enabled_roles, params.address, params.forwarded_address, params.quota_key);
enabled_settings = access_control->getEnabledSettings( enabled_settings = access_control->getEnabledSettings(
*params.user_id, user->settings, roles_info->enabled_roles, roles_info->settings_from_enabled_roles); *params.user_id, user->settings, roles_info->enabled_roles, roles_info->settings_from_enabled_roles);
calculateAccessRights(); calculateAccessRights();
} }
@ -417,14 +435,6 @@ std::optional<QuotaUsage> ContextAccess::getQuotaUsage() const
} }
std::shared_ptr<const ContextAccess> ContextAccess::getFullAccess()
{
static const std::shared_ptr<const ContextAccess> res =
[] { return std::shared_ptr<ContextAccess>(new ContextAccess{kFullAccess}); }();
return res;
}
SettingsChanges ContextAccess::getDefaultSettings() const SettingsChanges ContextAccess::getDefaultSettings() const
{ {
std::lock_guard lock{mutex}; std::lock_guard lock{mutex};
@ -478,7 +488,7 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg
throw Exception(ErrorCodes::UNKNOWN_USER, "{}: User has been dropped", getUserName()); throw Exception(ErrorCodes::UNKNOWN_USER, "{}: User has been dropped", getUserName());
} }
if (is_full_access) if (params.full_access)
return true; return true;
auto access_granted = [&] auto access_granted = [&]
@ -706,7 +716,7 @@ bool ContextAccess::checkAdminOptionImplHelper(const Container & role_ids, const
return false; return false;
}; };
if (is_full_access) if (params.full_access)
return true; return true;
if (user_was_dropped) if (user_was_dropped)
@ -806,7 +816,7 @@ void ContextAccess::checkAdminOption(const std::vector<UUID> & role_ids, const s
void ContextAccess::checkGranteeIsAllowed(const UUID & grantee_id, const IAccessEntity & grantee) const void ContextAccess::checkGranteeIsAllowed(const UUID & grantee_id, const IAccessEntity & grantee) const
{ {
if (is_full_access) if (params.full_access)
return; return;
auto current_user = getUser(); auto current_user = getUser();
@ -816,7 +826,7 @@ void ContextAccess::checkGranteeIsAllowed(const UUID & grantee_id, const IAccess
void ContextAccess::checkGranteesAreAllowed(const std::vector<UUID> & grantee_ids) const void ContextAccess::checkGranteesAreAllowed(const std::vector<UUID> & grantee_ids) const
{ {
if (is_full_access) if (params.full_access)
return; return;
auto current_user = getUser(); auto current_user = getUser();

View File

@ -1,6 +1,7 @@
#pragma once #pragma once
#include <Access/AccessRights.h> #include <Access/AccessRights.h>
#include <Access/ContextAccessParams.h>
#include <Access/EnabledRowPolicies.h> #include <Access/EnabledRowPolicies.h>
#include <Interpreters/ClientInfo.h> #include <Interpreters/ClientInfo.h>
#include <Core/UUID.h> #include <Core/UUID.h>
@ -30,47 +31,18 @@ class AccessControl;
class IAST; class IAST;
struct IAccessEntity; struct IAccessEntity;
using ASTPtr = std::shared_ptr<IAST>; using ASTPtr = std::shared_ptr<IAST>;
class Context;
using ContextPtr = std::shared_ptr<const Context>;
struct ContextAccessParams
{
std::optional<UUID> user_id;
boost::container::flat_set<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 forwarded_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, forwarded_address, quota_key);
}
friend bool operator ==(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return lhs.toTuple() == rhs.toTuple(); }
friend bool operator !=(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return !(lhs == rhs); }
friend bool operator <(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return lhs.toTuple() < rhs.toTuple(); }
friend bool operator >(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return rhs < lhs; }
friend bool operator <=(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return !(rhs < lhs); }
friend bool operator >=(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return !(lhs < rhs); }
};
class ContextAccess : public std::enable_shared_from_this<ContextAccess> class ContextAccess : public std::enable_shared_from_this<ContextAccess>
{ {
public: public:
static std::shared_ptr<const ContextAccess> fromContext(const ContextPtr & context);
using Params = ContextAccessParams; using Params = ContextAccessParams;
const Params & getParams() const { return params; } const Params & getParams() const { return params; }
ContextAccess(const AccessControl & access_control_, const Params & params_);
/// Returns the current user. Throws if user is nullptr. /// Returns the current user. Throws if user is nullptr.
UserPtr getUser() const; UserPtr getUser() const;
/// Same as above, but can return nullptr. /// Same as above, but can return nullptr.
@ -161,22 +133,12 @@ public:
/// Checks if grantees are allowed for the current user, throws an exception if not. /// Checks if grantees are allowed for the current user, throws an exception if not.
void checkGranteesAreAllowed(const std::vector<UUID> & grantee_ids) const; void checkGranteesAreAllowed(const std::vector<UUID> & grantee_ids) const;
/// Makes an instance of ContextAccess which provides full access to everything ContextAccess(const AccessControl & access_control_, const Params & params_);
/// without any limitations. This is used for the global context.
static std::shared_ptr<const ContextAccess> getFullAccess();
~ContextAccess(); ~ContextAccess();
private: private:
friend class AccessControl; friend class AccessControl;
struct FullAccess {};
static const FullAccess kFullAccess;
/// Makes an instance of ContextAccess which provides full access to everything
/// without any limitations. This is used for the global context.
explicit ContextAccess(FullAccess);
void initialize(); void initialize();
void setUser(const UserPtr & user_) const TSA_REQUIRES(mutex); void setUser(const UserPtr & user_) const TSA_REQUIRES(mutex);
void setRolesInfo(const std::shared_ptr<const EnabledRolesInfo> & roles_info_) const TSA_REQUIRES(mutex); void setRolesInfo(const std::shared_ptr<const EnabledRolesInfo> & roles_info_) const TSA_REQUIRES(mutex);
@ -223,7 +185,6 @@ private:
const AccessControl * access_control = nullptr; const AccessControl * access_control = nullptr;
const Params params; const Params params;
const bool is_full_access = false;
mutable std::atomic<bool> user_was_dropped = false; mutable std::atomic<bool> user_was_dropped = false;
mutable std::atomic<Poco::Logger *> trace_log = nullptr; mutable std::atomic<Poco::Logger *> trace_log = nullptr;

View File

@ -0,0 +1,172 @@
#include <Access/ContextAccessParams.h>
#include <Core/Settings.h>
#include <Common/typeid_cast.h>
namespace DB
{
ContextAccessParams::ContextAccessParams(
const std::optional<UUID> user_id_,
bool full_access_,
bool use_default_roles_,
const std::shared_ptr<const std::vector<UUID>> & current_roles_,
const Settings & settings_,
const String & current_database_,
const ClientInfo & client_info_)
: user_id(user_id_)
, full_access(full_access_)
, use_default_roles(use_default_roles_)
, current_roles(current_roles_)
, readonly(settings_.readonly)
, allow_ddl(settings_.allow_ddl)
, allow_introspection(settings_.allow_introspection_functions)
, current_database(current_database_)
, interface(client_info_.interface)
, http_method(client_info_.http_method)
, address(client_info_.current_address.host())
, forwarded_address(client_info_.getLastForwardedFor())
, quota_key(client_info_.quota_key)
{
}
String ContextAccessParams::toString() const
{
WriteBufferFromOwnString out;
auto separator = [&] { return out.stringView().empty() ? "" : ", "; };
if (user_id)
out << separator() << "user_id = " << *user_id;
if (full_access)
out << separator() << "full_access = " << full_access;
if (use_default_roles)
out << separator() << "use_default_roles = " << use_default_roles;
if (current_roles && !current_roles->empty())
{
out << separator() << "current_roles = [";
for (size_t i = 0; i != current_roles->size(); ++i)
{
if (i)
out << ", ";
out << (*current_roles)[i];
}
out << "]";
}
if (readonly)
out << separator() << "readonly = " << readonly;
if (allow_ddl)
out << separator() << "allow_ddl = " << allow_ddl;
if (allow_introspection)
out << separator() << "allow_introspection = " << allow_introspection;
if (!current_database.empty())
out << separator() << "current_database = " << current_database;
out << separator() << "interface = " << magic_enum::enum_name(interface);
if (http_method != ClientInfo::HTTPMethod::UNKNOWN)
out << separator() << "http_method = " << magic_enum::enum_name(http_method);
if (!address.isWildcard())
out << separator() << "address = " << address.toString();
if (!forwarded_address.empty())
out << separator() << "forwarded_address = " << forwarded_address;
if (!quota_key.empty())
out << separator() << "quota_key = " << quota_key;
return out.str();
}
bool operator ==(const ContextAccessParams & left, const ContextAccessParams & right)
{
auto check_equals = [](const auto & x, const auto & y)
{
if constexpr (::detail::is_shared_ptr_v<std::remove_cvref_t<decltype(x)>>)
{
if (!x)
return !y;
else if (!y)
return false;
else
return *x == *y;
}
else
{
return x == y;
}
};
#define CONTEXT_ACCESS_PARAMS_EQUALS(name) \
if (!check_equals(left.name, right.name)) \
return false;
CONTEXT_ACCESS_PARAMS_EQUALS(user_id)
CONTEXT_ACCESS_PARAMS_EQUALS(full_access)
CONTEXT_ACCESS_PARAMS_EQUALS(use_default_roles)
CONTEXT_ACCESS_PARAMS_EQUALS(current_roles)
CONTEXT_ACCESS_PARAMS_EQUALS(readonly)
CONTEXT_ACCESS_PARAMS_EQUALS(allow_ddl)
CONTEXT_ACCESS_PARAMS_EQUALS(allow_introspection)
CONTEXT_ACCESS_PARAMS_EQUALS(current_database)
CONTEXT_ACCESS_PARAMS_EQUALS(interface)
CONTEXT_ACCESS_PARAMS_EQUALS(http_method)
CONTEXT_ACCESS_PARAMS_EQUALS(address)
CONTEXT_ACCESS_PARAMS_EQUALS(forwarded_address)
CONTEXT_ACCESS_PARAMS_EQUALS(quota_key)
#undef CONTEXT_ACCESS_PARAMS_EQUALS
return true; /// All fields are equal, operator == must return true.
}
bool operator <(const ContextAccessParams & left, const ContextAccessParams & right)
{
auto check_less = [](const auto & x, const auto & y)
{
if constexpr (::detail::is_shared_ptr_v<std::remove_cvref_t<decltype(x)>>)
{
if (!x)
return y ? -1 : 0;
else if (!y)
return 1;
else if (*x == *y)
return 0;
else if (*x < *y)
return -1;
else
return 1;
}
else
{
if (x == y)
return 0;
else if (x < y)
return -1;
else
return 1;
}
};
#define CONTEXT_ACCESS_PARAMS_LESS(name) \
if (auto cmp = check_less(left.name, right.name); cmp != 0) \
return cmp < 0;
CONTEXT_ACCESS_PARAMS_LESS(user_id)
CONTEXT_ACCESS_PARAMS_LESS(full_access)
CONTEXT_ACCESS_PARAMS_LESS(use_default_roles)
CONTEXT_ACCESS_PARAMS_LESS(current_roles)
CONTEXT_ACCESS_PARAMS_LESS(readonly)
CONTEXT_ACCESS_PARAMS_LESS(allow_ddl)
CONTEXT_ACCESS_PARAMS_LESS(allow_introspection)
CONTEXT_ACCESS_PARAMS_LESS(current_database)
CONTEXT_ACCESS_PARAMS_LESS(interface)
CONTEXT_ACCESS_PARAMS_LESS(http_method)
CONTEXT_ACCESS_PARAMS_LESS(address)
CONTEXT_ACCESS_PARAMS_LESS(forwarded_address)
CONTEXT_ACCESS_PARAMS_LESS(quota_key)
#undef CONTEXT_ACCESS_PARAMS_LESS
return false; /// All fields are equal, operator < must return false.
}
bool ContextAccessParams::dependsOnSettingName(std::string_view setting_name)
{
return (setting_name == "readonly") || (setting_name == "allow_ddl") || (setting_name == "allow_introspection_functions");
}
}

View File

@ -0,0 +1,64 @@
#pragma once
#include <Interpreters/ClientInfo.h>
#include <Core/UUID.h>
#include <optional>
#include <vector>
namespace DB
{
struct Settings;
/// Parameters which are used to calculate access rights and some related stuff like roles or constraints.
class ContextAccessParams
{
public:
ContextAccessParams(
const std::optional<UUID> user_id_,
bool full_access_,
bool use_default_roles_,
const std::shared_ptr<const std::vector<UUID>> & current_roles_,
const Settings & settings_,
const String & current_database_,
const ClientInfo & client_info_);
const std::optional<UUID> user_id;
/// Full access to everything without any limitations.
/// This is used for the global context.
const bool full_access;
const bool use_default_roles;
const std::shared_ptr<const std::vector<UUID>> current_roles;
const UInt64 readonly;
const bool allow_ddl;
const bool allow_introspection;
const String current_database;
const ClientInfo::Interface interface;
const ClientInfo::HTTPMethod http_method;
const Poco::Net::IPAddress address;
/// The last entry from comma separated list of X-Forwarded-For addresses.
/// Only the last proxy can be trusted (if any).
const String forwarded_address;
const String quota_key;
/// Outputs `ContextAccessParams` to string for logging.
String toString() const;
friend bool operator <(const ContextAccessParams & left, const ContextAccessParams & right);
friend bool operator ==(const ContextAccessParams & left, const ContextAccessParams & right);
friend bool operator !=(const ContextAccessParams & left, const ContextAccessParams & right) { return !(left == right); }
friend bool operator >(const ContextAccessParams & left, const ContextAccessParams & right) { return right < left; }
friend bool operator <=(const ContextAccessParams & left, const ContextAccessParams & right) { return !(right < left); }
friend bool operator >=(const ContextAccessParams & left, const ContextAccessParams & right) { return !(left < right); }
static bool dependsOnSettingName(std::string_view setting_name);
};
}

View File

@ -1063,8 +1063,16 @@ void Context::setUser(const UUID & user_id_)
user_id = user_id_; user_id = user_id_;
access = getAccessControl().getContextAccess( ContextAccessParams params{
user_id_, /* current_roles = */ {}, /* use_default_roles = */ true, settings, current_database, client_info); user_id,
/* full_access= */ false,
/* use_default_roles = */ true,
/* current_roles = */ nullptr,
settings,
current_database,
client_info};
access = getAccessControl().getContextAccess(params);
auto user = access->getUser(); auto user = access->getUser();
@ -1108,7 +1116,7 @@ void Context::setCurrentRoles(const std::vector<UUID> & current_roles_)
if (current_roles ? (*current_roles == current_roles_) : current_roles_.empty()) if (current_roles ? (*current_roles == current_roles_) : current_roles_.empty())
return; return;
current_roles = std::make_shared<std::vector<UUID>>(current_roles_); current_roles = std::make_shared<std::vector<UUID>>(current_roles_);
calculateAccessRights(); need_recalculate_access = true;
} }
void Context::setCurrentRolesDefault() void Context::setCurrentRolesDefault()
@ -1133,20 +1141,6 @@ std::shared_ptr<const EnabledRolesInfo> Context::getRolesInfo() const
} }
void Context::calculateAccessRights()
{
auto lock = getLock();
if (user_id)
access = getAccessControl().getContextAccess(
*user_id,
current_roles ? *current_roles : std::vector<UUID>{},
/* use_default_roles = */ false,
settings,
current_database,
client_info);
}
template <typename... Args> template <typename... Args>
void Context::checkAccessImpl(const Args &... args) const void Context::checkAccessImpl(const Args &... args) const
{ {
@ -1166,11 +1160,50 @@ void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id,
void Context::checkAccess(const AccessRightsElement & element) const { return checkAccessImpl(element); } void Context::checkAccess(const AccessRightsElement & element) const { return checkAccessImpl(element); }
void Context::checkAccess(const AccessRightsElements & elements) const { return checkAccessImpl(elements); } void Context::checkAccess(const AccessRightsElements & elements) const { return checkAccessImpl(elements); }
std::shared_ptr<const ContextAccess> Context::getAccess() const std::shared_ptr<const ContextAccess> Context::getAccess() const
{
/// A helper function to collect parameters for calculating access rights, called with Context::getLock() acquired.
auto get_params = [this]()
{
/// If setUserID() was never called then this must be the global context with the full access.
bool full_access = !user_id;
return ContextAccessParams{user_id, full_access, /* use_default_roles= */ false, current_roles, settings, current_database, client_info};
};
/// Check if the current access rights are still valid, otherwise get parameters for recalculating access rights.
std::optional<ContextAccessParams> params;
{ {
auto lock = getLock(); auto lock = getLock();
return access ? access : ContextAccess::getFullAccess(); if (access && !need_recalculate_access)
return access; /// No need to recalculate access rights.
params.emplace(get_params());
if (access && (access->getParams() == *params))
{
need_recalculate_access = false;
return access; /// No need to recalculate access rights.
}
}
/// Calculate new access rights according to the collected parameters.
/// NOTE: AccessControl::getContextAccess() may require some IO work, so Context::getLock() must be unlocked while we're doing this.
auto res = getAccessControl().getContextAccess(*params);
{
/// If the parameters of access rights were not changed while we were calculated them
/// then we store the new access rights in the Context to allow reusing it later.
auto lock = getLock();
if (get_params() == *params)
{
access = res;
need_recalculate_access = false;
}
}
return res;
} }
RowPolicyFilterPtr Context::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const RowPolicyFilterPtr Context::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const
@ -1700,27 +1733,8 @@ Settings Context::getSettings() const
void Context::setSettings(const Settings & settings_) void Context::setSettings(const Settings & settings_)
{ {
auto lock = getLock(); auto lock = getLock();
const auto old_readonly = settings.readonly;
const auto old_allow_ddl = settings.allow_ddl;
const auto old_allow_introspection_functions = settings.allow_introspection_functions;
const auto old_display_secrets = settings.format_display_secrets_in_show_and_select;
settings = settings_; settings = settings_;
need_recalculate_access = true;
if ((settings.readonly != old_readonly)
|| (settings.allow_ddl != old_allow_ddl)
|| (settings.allow_introspection_functions != old_allow_introspection_functions)
|| (settings.format_display_secrets_in_show_and_select != old_display_secrets))
calculateAccessRights();
}
void Context::recalculateAccessRightsIfNeeded(std::string_view name)
{
if (name == "readonly"
|| name == "allow_ddl"
|| name == "allow_introspection_functions"
|| name == "format_display_secrets_in_show_and_select")
calculateAccessRights();
} }
void Context::setSetting(std::string_view name, const String & value) void Context::setSetting(std::string_view name, const String & value)
@ -1732,7 +1746,8 @@ void Context::setSetting(std::string_view name, const String & value)
return; return;
} }
settings.set(name, value); settings.set(name, value);
recalculateAccessRightsIfNeeded(name); if (ContextAccessParams::dependsOnSettingName(name))
need_recalculate_access = true;
} }
void Context::setSetting(std::string_view name, const Field & value) void Context::setSetting(std::string_view name, const Field & value)
@ -1744,7 +1759,8 @@ void Context::setSetting(std::string_view name, const Field & value)
return; return;
} }
settings.set(name, value); settings.set(name, value);
recalculateAccessRightsIfNeeded(name); if (ContextAccessParams::dependsOnSettingName(name))
need_recalculate_access = true;
} }
void Context::applySettingChange(const SettingChange & change) void Context::applySettingChange(const SettingChange & change)
@ -1853,7 +1869,7 @@ void Context::setCurrentDatabase(const String & name)
DatabaseCatalog::instance().assertDatabaseExists(name); DatabaseCatalog::instance().assertDatabaseExists(name);
auto lock = getLock(); auto lock = getLock();
current_database = name; current_database = name;
calculateAccessRights(); need_recalculate_access = true;
} }
void Context::setCurrentQueryId(const String & query_id) void Context::setCurrentQueryId(const String & query_id)

View File

@ -248,7 +248,8 @@ private:
std::optional<UUID> user_id; std::optional<UUID> user_id;
std::shared_ptr<std::vector<UUID>> current_roles; std::shared_ptr<std::vector<UUID>> current_roles;
std::shared_ptr<const SettingsConstraintsAndProfileIDs> settings_constraints_and_current_profiles; std::shared_ptr<const SettingsConstraintsAndProfileIDs> settings_constraints_and_current_profiles;
std::shared_ptr<const ContextAccess> access; mutable std::shared_ptr<const ContextAccess> access;
mutable bool need_recalculate_access = true;
std::shared_ptr<const EnabledRowPolicies> row_policies_of_initial_user; std::shared_ptr<const EnabledRowPolicies> row_policies_of_initial_user;
String current_database; String current_database;
Settings settings; /// Setting for query execution. Settings settings; /// Setting for query execution.
@ -1149,10 +1150,6 @@ private:
void initGlobal(); void initGlobal();
/// Compute and set actual user settings, client_info.current_user should be set
void calculateAccessRights();
void recalculateAccessRightsIfNeeded(std::string_view setting_name);
template <typename... Args> template <typename... Args>
void checkAccessImpl(const Args &... args) const; void checkAccessImpl(const Args &... args) const;