Add user_id to Context.

This commit is contained in:
Vitaly Baranov 2020-02-05 01:22:27 +03:00
parent fb3b755599
commit 08c6061bdf
6 changed files with 68 additions and 43 deletions

View File

@ -35,45 +35,49 @@ AccessControlManager::~AccessControlManager()
}
UserPtr AccessControlManager::getUser(const String & user_name) const
UserPtr AccessControlManager::getUser(
const String & user_name, std::function<void(const UserPtr &)> on_change, ext::scope_guard * subscription) const
{
return getUser(user_name, {}, nullptr);
return getUser(getID<User>(user_name), std::move(on_change), subscription);
}
UserPtr AccessControlManager::getUser(
const String & user_name, const std::function<void(const UserPtr &)> & on_change, ext::scope_guard * subscription) const
const UUID & user_id, std::function<void(const UserPtr &)> on_change, ext::scope_guard * subscription) const
{
UUID id = getID<User>(user_name);
if (on_change && subscription)
{
*subscription = subscribeForChanges(id, [on_change](const UUID &, const AccessEntityPtr & user)
*subscription = subscribeForChanges(user_id, [on_change](const UUID &, const AccessEntityPtr & user)
{
if (user)
on_change(typeid_cast<UserPtr>(user));
});
}
return read<User>(id);
return read<User>(user_id);
}
UserPtr AccessControlManager::authorizeAndGetUser(
const String & user_name,
const String & password,
const Poco::Net::IPAddress & address) const
{
return authorizeAndGetUser(user_name, password, address, {}, nullptr);
}
UserPtr AccessControlManager::authorizeAndGetUser(
const String & user_name,
const String & password,
const Poco::Net::IPAddress & address,
const std::function<void(const UserPtr &)> & on_change,
std::function<void(const UserPtr &)> on_change,
ext::scope_guard * subscription) const
{
auto user = getUser(user_name, on_change, subscription);
user->allowed_client_hosts.checkContains(address, user_name);
user->authentication.checkPassword(password, user_name);
return authorizeAndGetUser(getID<User>(user_name), password, address, std::move(on_change), subscription);
}
UserPtr AccessControlManager::authorizeAndGetUser(
const UUID & user_id,
const String & password,
const Poco::Net::IPAddress & address,
std::function<void(const UserPtr &)> on_change,
ext::scope_guard * subscription) const
{
auto user = getUser(user_id, on_change, subscription);
user->allowed_client_hosts.checkContains(address, user->getName());
user->authentication.checkPassword(password, user->getName());
return user;
}
@ -85,9 +89,9 @@ void AccessControlManager::loadFromConfig(const Poco::Util::AbstractConfiguratio
}
std::shared_ptr<const AccessRightsContext> AccessControlManager::getAccessRightsContext(const ClientInfo & client_info, const AccessRights & granted_to_user, const Settings & settings, const String & current_database)
std::shared_ptr<const AccessRightsContext> AccessControlManager::getAccessRightsContext(const UserPtr & user, const ClientInfo & client_info, const Settings & settings, const String & current_database)
{
return std::make_shared<AccessRightsContext>(client_info, granted_to_user, settings, current_database);
return std::make_shared<AccessRightsContext>(user, client_info, settings, current_database);
}

View File

@ -42,12 +42,12 @@ public:
void loadFromConfig(const Poco::Util::AbstractConfiguration & users_config);
UserPtr getUser(const String & user_name) const;
UserPtr getUser(const String & user_name, const std::function<void(const UserPtr &)> & on_change, ext::scope_guard * subscription) const;
UserPtr authorizeAndGetUser(const String & user_name, const String & password, const Poco::Net::IPAddress & address) const;
UserPtr authorizeAndGetUser(const String & user_name, const String & password, const Poco::Net::IPAddress & address, const std::function<void(const UserPtr &)> & on_change, ext::scope_guard * subscription) const;
UserPtr getUser(const String & user_name, std::function<void(const UserPtr &)> on_change = {}, ext::scope_guard * subscription = nullptr) const;
UserPtr getUser(const UUID & user_id, std::function<void(const UserPtr &)> on_change = {}, ext::scope_guard * subscription = nullptr) const;
UserPtr authorizeAndGetUser(const String & user_name, const String & password, const Poco::Net::IPAddress & address, std::function<void(const UserPtr &)> on_change = {}, ext::scope_guard * subscription = nullptr) const;
UserPtr authorizeAndGetUser(const UUID & user_id, const String & password, const Poco::Net::IPAddress & address, std::function<void(const UserPtr &)> on_change = {}, ext::scope_guard * subscription = nullptr) const;
std::shared_ptr<const AccessRightsContext> getAccessRightsContext(const ClientInfo & client_info, const AccessRights & granted_to_user, const Settings & settings, const String & current_database);
std::shared_ptr<const AccessRightsContext> getAccessRightsContext(const UserPtr & user, const ClientInfo & client_info, const Settings & settings, const String & current_database);
std::shared_ptr<QuotaContext>
createQuotaContext(const String & user_name, const Poco::Net::IPAddress & address, const String & custom_quota_key);

View File

@ -1,4 +1,5 @@
#include <Access/AccessRightsContext.h>
#include <Access/User.h>
#include <Common/Exception.h>
#include <Common/quoteString.h>
#include <Core/Settings.h>
@ -88,16 +89,15 @@ AccessRightsContext::AccessRightsContext()
}
AccessRightsContext::AccessRightsContext(const ClientInfo & client_info_, const AccessRights & granted_to_user_, const Settings & settings, const String & current_database_)
: user_name(client_info_.current_user)
, granted_to_user(granted_to_user_)
AccessRightsContext::AccessRightsContext(const UserPtr & user_, const ClientInfo & client_info_, const Settings & settings, const String & current_database_)
: user(user_)
, 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)
, trace_log(&Poco::Logger::get("AccessRightsContext (" + user_name + ")"))
, trace_log(&Poco::Logger::get("AccessRightsContext (" + user_->getName() + ")"))
{
}
@ -126,9 +126,9 @@ bool AccessRightsContext::checkImpl(Poco::Logger * log_, const AccessFlags & acc
auto show_error = [&](const String & msg, [[maybe_unused]] int error_code)
{
if constexpr (mode == THROW_IF_ACCESS_DENIED)
throw Exception(msg, error_code);
throw Exception(user->getName() + ": " + msg, error_code);
else if constexpr (mode == LOG_WARNING_IF_ACCESS_DENIED)
LOG_WARNING(log_, msg + formatSkippedMessage(args...));
LOG_WARNING(log_, user->getName() + ": " + msg + formatSkippedMessage(args...));
};
if (readonly && calculateResultAccess(false, allow_ddl, allow_introspection)->isGranted(access, args...))
@ -152,7 +152,7 @@ bool AccessRightsContext::checkImpl(Poco::Logger * log_, const AccessFlags & acc
else
{
show_error(
user_name + ": Not enough privileges. To perform this operation you should have grant "
"Not enough privileges. To execute this query it's necessary to have the grant "
+ AccessRightsElement{access, args...}.toString(),
ErrorCodes::ACCESS_DENIED);
}
@ -160,6 +160,7 @@ bool AccessRightsContext::checkImpl(Poco::Logger * log_, const AccessFlags & acc
return false;
}
template <int mode>
bool AccessRightsContext::checkImpl(Poco::Logger * log_, const AccessRightsElement & element) const
{
@ -231,9 +232,6 @@ bool AccessRightsContext::isGranted(Poco::Logger * log_, const AccessRightsEleme
boost::shared_ptr<const AccessRights> AccessRightsContext::calculateResultAccess() const
{
auto res = result_access_cache[0].load();
if (res)
return res;
return calculateResultAccess(readonly, allow_ddl, allow_introspection);
}
@ -256,7 +254,7 @@ boost::shared_ptr<const AccessRights> AccessRightsContext::calculateResultAccess
auto result_ptr = boost::make_shared<AccessRights>();
auto & result = *result_ptr;
result = granted_to_user;
result = user->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
@ -282,7 +280,11 @@ boost::shared_ptr<const AccessRights> AccessRightsContext::calculateResultAccess
result.fullRevoke(AccessType::INTROSPECTION);
result_access_cache[cache_index].store(result_ptr);
return std::move(result_ptr);
if (trace_log && (readonly == readonly_) && (allow_ddl == allow_ddl_) && (allow_introspection == allow_introspection_))
LOG_TRACE(trace_log, "List of all grants: " << result_ptr->toString());
return result_ptr;
}
}

View File

@ -11,6 +11,8 @@ namespace Poco { class Logger; }
namespace DB
{
struct Settings;
struct User;
using UserPtr = std::shared_ptr<const User>;
class AccessRightsContext
@ -19,7 +21,7 @@ public:
/// Default constructor creates access rights' context which allows everything.
AccessRightsContext();
AccessRightsContext(const ClientInfo & client_info_, const AccessRights & granted_to_user, const Settings & settings, const String & current_database_);
AccessRightsContext(const UserPtr & user_, const ClientInfo & client_info_, const Settings & settings, const String & current_database_);
/// Checks if a specified access granted, and throws an exception if not.
/// Empty database means the current database.
@ -65,8 +67,7 @@ private:
boost::shared_ptr<const AccessRights> calculateResultAccess() const;
boost::shared_ptr<const AccessRights> calculateResultAccess(UInt64 readonly_, bool allow_ddl_, bool allow_introspection_) const;
const String user_name;
const AccessRights granted_to_user;
const UserPtr user;
const UInt64 readonly = 0;
const bool allow_ddl = true;
const bool allow_introspection = true;

View File

@ -688,7 +688,7 @@ void Context::calculateAccessRights()
{
auto lock = getLock();
if (user)
std::atomic_store(&access_rights, getAccessControlManager().getAccessRightsContext(client_info, user->access, settings, current_database));
std::atomic_store(&access_rights, getAccessControlManager().getAccessRightsContext(user, client_info, settings, current_database));
}
void Context::setProfile(const String & profile)
@ -706,6 +706,20 @@ std::shared_ptr<const User> Context::getUser(const String & user_name) const
return shared->access_control_manager.getUser(user_name);
}
std::shared_ptr<const User> Context::getUser() const
{
if (!user)
throw Exception("No current user", ErrorCodes::LOGICAL_ERROR);
return user;
}
UUID Context::getUserID() const
{
if (!user)
throw Exception("No current user", ErrorCodes::LOGICAL_ERROR);
return user_id;
}
void Context::setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address, const String & quota_key)
{
auto lock = getLock();
@ -717,8 +731,9 @@ void Context::setUser(const String & name, const String & password, const Poco::
if (!quota_key.empty())
client_info.quota_key = quota_key;
user_id = shared->access_control_manager.getID<User>(name);
user = shared->access_control_manager.authorizeAndGetUser(
name,
user_id,
password,
address.host(),
[this](const UserPtr & changed_user)

View File

@ -4,6 +4,7 @@
#include <Core/NamesAndTypes.h>
#include <Core/Settings.h>
#include <Core/Types.h>
#include <Core/UUID.h>
#include <DataStreams/IBlockStream_fwd.h>
#include <Interpreters/ClientInfo.h>
#include <Parsers/IAST_fwd.h>
@ -161,6 +162,7 @@ private:
InputBlocksReader input_blocks_reader;
std::shared_ptr<const User> user;
UUID user_id;
SubscriptionForUserChange subscription_for_user_change;
std::shared_ptr<const AccessRightsContext> access_rights;
std::shared_ptr<QuotaContext> quota; /// Current quota. By default - empty quota, that have no limits.
@ -260,7 +262,8 @@ public:
/// Must be called before getClientInfo.
void setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address, const String & quota_key);
std::shared_ptr<const User> getUser() const { return user; }
std::shared_ptr<const User> getUser() const;
UUID getUserID() const;
/// Used by MySQL Secure Password Authentication plugin.
std::shared_ptr<const User> getUser(const String & user_name) const;