Merge pull request #26714 from vitlibar/new-function-current-profiles

New functions currentProfiles(), enabledProfiles(), defaultProfiles().
This commit is contained in:
Vitaly Baranov 2021-07-23 09:10:29 +03:00 committed by GitHub
commit 19d5a6ab2f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
24 changed files with 471 additions and 138 deletions

View File

@ -0,0 +1,24 @@
#pragma once
#include <vector>
/// Removes duplicates from a container without changing the order of its elements.
/// Keeps the last occurrence of each element.
/// Should NOT be used for containers with a lot of elements because it has O(N^2) complexity.
template <typename T>
void removeDuplicatesKeepLast(std::vector<T> & vec)
{
auto begin = vec.begin();
auto end = vec.end();
auto new_begin = end;
for (auto current = end; current != begin;)
{
--current;
if (std::find(new_begin, end, *current) == end)
{
--new_begin;
if (new_begin != current)
*new_begin = *current;
}
}
vec.erase(begin, new_begin);
}

View File

@ -489,11 +489,12 @@ std::shared_ptr<const EnabledSettings> AccessControlManager::getEnabledSettings(
return settings_profiles_cache->getEnabledSettings(user_id, settings_from_user, enabled_roles, settings_from_enabled_roles);
}
std::shared_ptr<const SettingsChanges> AccessControlManager::getProfileSettings(const String & profile_name) const
std::shared_ptr<const SettingsProfilesInfo> AccessControlManager::getSettingsProfileInfo(const UUID & profile_id)
{
return settings_profiles_cache->getProfileSettings(profile_name);
return settings_profiles_cache->getSettingsProfileInfo(profile_id);
}
const ExternalAuthenticators & AccessControlManager::getExternalAuthenticators() const
{
return *external_authenticators;

View File

@ -32,8 +32,7 @@ class RowPolicyCache;
class EnabledQuota;
class QuotaCache;
struct QuotaUsage;
struct SettingsProfile;
using SettingsProfilePtr = std::shared_ptr<const SettingsProfile>;
struct SettingsProfilesInfo;
class EnabledSettings;
class SettingsProfilesCache;
class SettingsProfileElements;
@ -145,7 +144,7 @@ public:
const boost::container::flat_set<UUID> & enabled_roles,
const SettingsProfileElements & settings_from_enabled_roles) const;
std::shared_ptr<const SettingsChanges> getProfileSettings(const String & profile_name) const;
std::shared_ptr<const SettingsProfilesInfo> getSettingsProfileInfo(const UUID & profile_id);
const ExternalAuthenticators & getExternalAuthenticators() const;

View File

@ -7,6 +7,7 @@
#include <Access/User.h>
#include <Access/EnabledRolesInfo.h>
#include <Access/EnabledSettings.h>
#include <Access/SettingsProfilesInfo.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Common/Exception.h>
#include <Common/quoteString.h>
@ -307,23 +308,25 @@ std::shared_ptr<const ContextAccess> ContextAccess::getFullAccess()
}
std::shared_ptr<const Settings> ContextAccess::getDefaultSettings() const
SettingsChanges ContextAccess::getDefaultSettings() const
{
std::lock_guard lock{mutex};
if (enabled_settings)
return enabled_settings->getSettings();
static const auto everything_by_default = std::make_shared<Settings>();
return everything_by_default;
{
if (auto info = enabled_settings->getInfo())
return info->settings;
}
return {};
}
std::shared_ptr<const SettingsConstraints> ContextAccess::getSettingsConstraints() const
std::shared_ptr<const SettingsProfilesInfo> ContextAccess::getDefaultProfileInfo() const
{
std::lock_guard lock{mutex};
if (enabled_settings)
return enabled_settings->getConstraints();
static const auto no_constraints = std::make_shared<SettingsConstraints>();
return no_constraints;
return enabled_settings->getInfo();
static const auto everything_by_default = std::make_shared<SettingsProfilesInfo>(*manager);
return everything_by_default;
}

View File

@ -23,7 +23,8 @@ class EnabledQuota;
class EnabledSettings;
struct QuotaUsage;
struct Settings;
class SettingsConstraints;
struct SettingsProfilesInfo;
class SettingsChanges;
class AccessControlManager;
class IAST;
using ASTPtr = std::shared_ptr<IAST>;
@ -84,11 +85,9 @@ public:
std::shared_ptr<const EnabledQuota> getQuota() const;
std::optional<QuotaUsage> getQuotaUsage() const;
/// Returns the default settings, i.e. the settings to apply on user's login.
std::shared_ptr<const Settings> getDefaultSettings() const;
/// Returns the settings' constraints.
std::shared_ptr<const SettingsConstraints> getSettingsConstraints() const;
/// Returns the default settings, i.e. the settings which should be applied on user's login.
SettingsChanges getDefaultSettings() const;
std::shared_ptr<const SettingsProfilesInfo> getDefaultProfileInfo() const;
/// Returns the current access rights.
std::shared_ptr<const AccessRights> getAccessRights() const;

View File

@ -11,27 +11,16 @@ EnabledSettings::EnabledSettings(const Params & params_) : params(params_)
EnabledSettings::~EnabledSettings() = default;
std::shared_ptr<const Settings> EnabledSettings::getSettings() const
std::shared_ptr<const SettingsProfilesInfo> EnabledSettings::getInfo() const
{
std::lock_guard lock{mutex};
return settings;
return info;
}
std::shared_ptr<const SettingsConstraints> EnabledSettings::getConstraints() const
void EnabledSettings::setInfo(const std::shared_ptr<const SettingsProfilesInfo> & info_)
{
std::lock_guard lock{mutex};
return constraints;
}
void EnabledSettings::setSettingsAndConstraints(
const std::shared_ptr<const Settings> & settings_, const std::shared_ptr<const SettingsConstraints> & constraints_)
{
std::lock_guard lock{mutex};
settings = settings_;
constraints = constraints_;
info = info_;
}
}

View File

@ -1,15 +1,15 @@
#pragma once
#include <common/types.h>
#include <Core/UUID.h>
#include <Access/SettingsConstraints.h>
#include <Access/SettingsProfileElement.h>
#include <Core/UUID.h>
#include <boost/container/flat_set.hpp>
#include <mutex>
namespace DB
{
struct SettingsProfilesInfo;
/// Watches settings profiles for a specific user and roles.
class EnabledSettings
{
@ -30,27 +30,19 @@ public:
friend bool operator >=(const Params & lhs, const Params & rhs) { return !(lhs < rhs); }
};
~EnabledSettings();
/// Returns the default settings come from settings profiles defined for the user
/// and the roles passed in the constructor.
std::shared_ptr<const Settings> getSettings() const;
std::shared_ptr<const SettingsProfilesInfo> getInfo() const;
/// Returns the constraints come from settings profiles defined for the user
/// and the roles passed in the constructor.
std::shared_ptr<const SettingsConstraints> getConstraints() const;
~EnabledSettings();
private:
friend class SettingsProfilesCache;
EnabledSettings(const Params & params_);
void setSettingsAndConstraints(
const std::shared_ptr<const Settings> & settings_, const std::shared_ptr<const SettingsConstraints> & constraints_);
void setInfo(const std::shared_ptr<const SettingsProfilesInfo> & info_);
const Params params;
SettingsProfileElements settings_from_enabled;
std::shared_ptr<const Settings> settings;
std::shared_ptr<const SettingsConstraints> constraints;
std::shared_ptr<const SettingsProfilesInfo> info;
mutable std::mutex mutex;
};
}

View File

@ -197,6 +197,16 @@ String IAccessStorage::readName(const UUID & id) const
}
Strings IAccessStorage::readNames(const std::vector<UUID> & ids) const
{
Strings res;
res.reserve(ids.size());
for (const auto & id : ids)
res.emplace_back(readName(id));
return res;
}
std::optional<String> IAccessStorage::tryReadName(const UUID & id) const
{
String name;
@ -207,6 +217,19 @@ std::optional<String> IAccessStorage::tryReadName(const UUID & id) const
}
Strings IAccessStorage::tryReadNames(const std::vector<UUID> & ids) const
{
Strings res;
res.reserve(ids.size());
for (const auto & id : ids)
{
if (auto name = tryReadName(id))
res.emplace_back(std::move(name).value());
}
return res;
}
UUID IAccessStorage::insert(const AccessEntityPtr & entity)
{
return insertImpl(entity, false);

View File

@ -84,7 +84,9 @@ public:
/// Reads only name of an entity.
String readName(const UUID & id) const;
Strings readNames(const std::vector<UUID> & ids) const;
std::optional<String> tryReadName(const UUID & id) const;
Strings tryReadNames(const std::vector<UUID> & ids) const;
/// Returns true if a specified entity can be inserted into this storage.
/// This function doesn't check whether there are no entities with such name in the storage.

View File

@ -18,8 +18,6 @@ namespace ErrorCodes
}
SettingsConstraints::SettingsConstraints() = default;
SettingsConstraints::SettingsConstraints(const AccessControlManager & manager_) : manager(&manager_)
{
}
@ -201,13 +199,10 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh
}
};
if (manager)
{
if (reaction == THROW_ON_VIOLATION)
manager->checkSettingNameIsAllowed(setting_name);
else if (!manager->isSettingNameAllowed(setting_name))
return false;
}
if (reaction == THROW_ON_VIOLATION)
manager->checkSettingNameIsAllowed(setting_name);
else if (!manager->isSettingNameAllowed(setting_name))
return false;
Field current_value, new_value;
if (current_settings.tryGet(setting_name, current_value))

View File

@ -51,7 +51,6 @@ class AccessControlManager;
class SettingsConstraints
{
public:
SettingsConstraints();
SettingsConstraints(const AccessControlManager & manager_);
SettingsConstraints(const SettingsConstraints & src);
SettingsConstraints & operator =(const SettingsConstraints & src);

View File

@ -0,0 +1,21 @@
#pragma once
#include <Access/SettingsConstraints.h>
#include <Core/UUID.h>
#include <vector>
namespace DB
{
/// Information about currently applied constraints and profiles.
struct SettingsConstraintsAndProfileIDs
{
SettingsConstraints constraints;
std::vector<UUID> current_profiles;
std::vector<UUID> enabled_profiles;
SettingsConstraintsAndProfileIDs(const AccessControlManager & manager_) : constraints(manager_) {}
};
}

View File

@ -7,6 +7,7 @@
#include <Common/SettingsChanges.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <common/removeDuplicates.h>
namespace DB
@ -172,4 +173,21 @@ SettingsConstraints SettingsProfileElements::toSettingsConstraints(const AccessC
return res;
}
std::vector<UUID> SettingsProfileElements::toProfileIDs() const
{
std::vector<UUID> res;
for (const auto & elem : *this)
{
if (elem.parent_profile)
res.push_back(*elem.parent_profile);
}
/// If some profile occurs multiple times (with some other settings in between),
/// the latest occurrence overrides all the previous ones.
removeDuplicatesKeepLast(res);
return res;
}
}

View File

@ -62,6 +62,7 @@ public:
Settings toSettings() const;
SettingsChanges toSettingsChanges() const;
SettingsConstraints toSettingsConstraints(const AccessControlManager & manager) const;
std::vector<UUID> toProfileIDs() const;
};
}

View File

@ -1,11 +1,8 @@
#include <Access/SettingsProfilesCache.h>
#include <Access/AccessControlManager.h>
#include <Access/SettingsProfile.h>
#include <Core/Settings.h>
#include <Common/SettingsChanges.h>
#include <Access/SettingsProfilesInfo.h>
#include <Common/quoteString.h>
#include <boost/range/adaptor/map.hpp>
#include <boost/range/algorithm_ext/push_back.hpp>
namespace DB
@ -15,7 +12,6 @@ namespace ErrorCodes
extern const int THERE_IS_NO_PROFILE;
}
SettingsProfilesCache::SettingsProfilesCache(const AccessControlManager & manager_)
: manager(manager_) {}
@ -67,7 +63,7 @@ void SettingsProfilesCache::profileAddedOrChanged(const UUID & profile_id, const
profiles_by_name.erase(old_profile->getName());
profiles_by_name[new_profile->getName()] = profile_id;
}
settings_for_profiles.clear();
profile_infos_cache.clear();
mergeSettingsAndConstraints();
}
@ -80,7 +76,7 @@ void SettingsProfilesCache::profileRemoved(const UUID & profile_id)
return;
profiles_by_name.erase(it->second->getName());
all_profiles.erase(it);
settings_for_profiles.clear();
profile_infos_cache.clear();
mergeSettingsAndConstraints();
}
@ -142,49 +138,52 @@ void SettingsProfilesCache::mergeSettingsAndConstraintsFor(EnabledSettings & ena
merged_settings.merge(enabled.params.settings_from_enabled_roles);
merged_settings.merge(enabled.params.settings_from_user);
substituteProfiles(merged_settings);
auto info = std::make_shared<SettingsProfilesInfo>(manager);
info->profiles = enabled.params.settings_from_user.toProfileIDs();
substituteProfiles(merged_settings, info->profiles_with_implicit, info->names_of_profiles);
info->settings = merged_settings.toSettingsChanges();
info->constraints = merged_settings.toSettingsConstraints(manager);
auto settings = merged_settings.toSettings();
auto constraints = merged_settings.toSettingsConstraints(manager);
enabled.setSettingsAndConstraints(
std::make_shared<Settings>(std::move(settings)), std::make_shared<SettingsConstraints>(std::move(constraints)));
enabled.setInfo(std::move(info));
}
void SettingsProfilesCache::substituteProfiles(SettingsProfileElements & elements) const
void SettingsProfilesCache::substituteProfiles(
SettingsProfileElements & elements,
std::vector<UUID> & substituted_profiles,
std::unordered_map<UUID, String> & names_of_substituted_profiles) const
{
boost::container::flat_set<UUID> already_substituted;
for (size_t i = 0; i != elements.size();)
/// We should substitute profiles in reversive order because the same profile can occur
/// in `elements` multiple times (with some other settings in between) and in this case
/// the last occurrence should override all the previous ones.
boost::container::flat_set<UUID> substituted_profiles_set;
size_t i = elements.size();
while (i != 0)
{
auto & element = elements[i];
auto & element = elements[--i];
if (!element.parent_profile)
{
++i;
continue;
}
auto parent_profile_id = *element.parent_profile;
auto profile_id = *element.parent_profile;
element.parent_profile.reset();
if (already_substituted.count(parent_profile_id))
{
++i;
if (substituted_profiles_set.count(profile_id))
continue;
}
already_substituted.insert(parent_profile_id);
auto parent_profile = all_profiles.find(parent_profile_id);
if (parent_profile == all_profiles.end())
{
++i;
auto profile_it = all_profiles.find(profile_id);
if (profile_it == all_profiles.end())
continue;
}
const auto & parent_profile_elements = parent_profile->second->elements;
elements.insert(elements.begin() + i, parent_profile_elements.begin(), parent_profile_elements.end());
const auto & profile = profile_it->second;
const auto & profile_elements = profile->elements;
elements.insert(elements.begin() + i, profile_elements.begin(), profile_elements.end());
i += profile_elements.size();
substituted_profiles.push_back(profile_id);
substituted_profiles_set.insert(profile_id);
names_of_substituted_profiles.emplace(profile_id, profile->getName());
}
std::reverse(substituted_profiles.begin(), substituted_profiles.end());
}
std::shared_ptr<const EnabledSettings> SettingsProfilesCache::getEnabledSettings(
const UUID & user_id,
const SettingsProfileElements & settings_from_user,
@ -216,26 +215,26 @@ std::shared_ptr<const EnabledSettings> SettingsProfilesCache::getEnabledSettings
}
std::shared_ptr<const SettingsChanges> SettingsProfilesCache::getProfileSettings(const String & profile_name)
std::shared_ptr<const SettingsProfilesInfo> SettingsProfilesCache::getSettingsProfileInfo(const UUID & profile_id)
{
std::lock_guard lock{mutex};
ensureAllProfilesRead();
auto it = profiles_by_name.find(profile_name);
if (it == profiles_by_name.end())
throw Exception("Settings profile " + backQuote(profile_name) + " not found", ErrorCodes::THERE_IS_NO_PROFILE);
const UUID profile_id = it->second;
auto it2 = settings_for_profiles.find(profile_id);
if (it2 != settings_for_profiles.end())
return it2->second;
if (auto pos = this->profile_infos_cache.get(profile_id))
return *pos;
SettingsProfileElements elements = all_profiles[profile_id]->elements;
substituteProfiles(elements);
auto res = std::make_shared<const SettingsChanges>(elements.toSettingsChanges());
settings_for_profiles.emplace(profile_id, res);
return res;
auto info = std::make_shared<SettingsProfilesInfo>(manager);
info->profiles.push_back(profile_id);
info->profiles_with_implicit.push_back(profile_id);
substituteProfiles(elements, info->profiles_with_implicit, info->names_of_profiles);
info->settings = elements.toSettingsChanges();
info->constraints.merge(elements.toSettingsConstraints(manager));
profile_infos_cache.add(profile_id, info);
return info;
}
}

View File

@ -1,8 +1,7 @@
#pragma once
#include <Access/EnabledSettings.h>
#include <Core/UUID.h>
#include <common/types.h>
#include <Poco/LRUCache.h>
#include <common/scope_guard.h>
#include <map>
#include <unordered_map>
@ -13,9 +12,7 @@ namespace DB
class AccessControlManager;
struct SettingsProfile;
using SettingsProfilePtr = std::shared_ptr<const SettingsProfile>;
class SettingsProfileElements;
class EnabledSettings;
struct SettingsProfilesInfo;
/// Reads and caches all the settings profiles.
class SettingsProfilesCache
@ -32,7 +29,7 @@ public:
const boost::container::flat_set<UUID> & enabled_roles,
const SettingsProfileElements & settings_from_enabled_roles_);
std::shared_ptr<const SettingsChanges> getProfileSettings(const String & profile_name);
std::shared_ptr<const SettingsProfilesInfo> getSettingsProfileInfo(const UUID & profile_id);
private:
void ensureAllProfilesRead();
@ -40,7 +37,7 @@ private:
void profileRemoved(const UUID & profile_id);
void mergeSettingsAndConstraints();
void mergeSettingsAndConstraintsFor(EnabledSettings & enabled) const;
void substituteProfiles(SettingsProfileElements & elements) const;
void substituteProfiles(SettingsProfileElements & elements, std::vector<UUID> & substituted_profiles, std::unordered_map<UUID, String> & names_of_substituted_profiles) const;
const AccessControlManager & manager;
std::unordered_map<UUID, SettingsProfilePtr> all_profiles;
@ -49,7 +46,7 @@ private:
scope_guard subscription;
std::map<EnabledSettings::Params, std::weak_ptr<EnabledSettings>> enabled_settings;
std::optional<UUID> default_profile_id;
std::unordered_map<UUID, std::shared_ptr<const SettingsChanges>> settings_for_profiles;
Poco::LRUCache<UUID, std::shared_ptr<const SettingsProfilesInfo>> profile_infos_cache;
mutable std::mutex mutex;
};
}

View File

@ -0,0 +1,52 @@
#include <Access/SettingsProfilesInfo.h>
#include <Access/SettingsConstraintsAndProfileIDs.h>
#include <common/removeDuplicates.h>
namespace DB
{
bool operator==(const SettingsProfilesInfo & lhs, const SettingsProfilesInfo & rhs)
{
if (lhs.settings != rhs.settings)
return false;
if (lhs.constraints != rhs.constraints)
return false;
if (lhs.profiles != rhs.profiles)
return false;
if (lhs.profiles_with_implicit != rhs.profiles_with_implicit)
return false;
if (lhs.names_of_profiles != rhs.names_of_profiles)
return false;
return true;
}
std::shared_ptr<const SettingsConstraintsAndProfileIDs>
SettingsProfilesInfo::getConstraintsAndProfileIDs(const std::shared_ptr<const SettingsConstraintsAndProfileIDs> & previous) const
{
auto res = std::make_shared<SettingsConstraintsAndProfileIDs>(manager);
res->current_profiles = profiles;
res->constraints = previous ? previous->constraints : constraints;
if (previous)
{
res->enabled_profiles.reserve(previous->enabled_profiles.size() + profiles_with_implicit.size());
res->enabled_profiles = previous->enabled_profiles;
}
res->enabled_profiles.insert(res->enabled_profiles.end(), profiles_with_implicit.begin(), profiles_with_implicit.end());
/// If some profile occurs multiple times (with some other settings in between),
/// the latest occurrence overrides all the previous ones.
removeDuplicatesKeepLast(res->current_profiles);
removeDuplicatesKeepLast(res->enabled_profiles);
return res;
}
}

View File

@ -0,0 +1,43 @@
#pragma once
#include <Access/SettingsConstraints.h>
#include <Common/SettingsChanges.h>
#include <Core/UUID.h>
#include <unordered_map>
namespace DB
{
struct SettingsConstraintsAndProfileIDs;
/// Information about the default settings which are applied to an user on login.
struct SettingsProfilesInfo
{
SettingsChanges settings;
SettingsConstraints constraints;
/// Profiles explicitly assigned to the user.
std::vector<UUID> profiles;
/// Profiles assigned to the user both explicitly and implicitly.
/// Implicitly assigned profiles include parent profiles of other assigned profiles,
/// profiles assigned via granted roles, profiles assigned via their own settings,
/// and the main default profile (see the section `default_profile` in the main configuration file).
/// The order of IDs in this vector corresponds the order of applying of these profiles.
std::vector<UUID> profiles_with_implicit;
/// Names of all the profiles in `profiles`.
std::unordered_map<UUID, String> names_of_profiles;
SettingsProfilesInfo(const AccessControlManager & manager_) : constraints(manager_), manager(manager_) {}
std::shared_ptr<const SettingsConstraintsAndProfileIDs> getConstraintsAndProfileIDs(
const std::shared_ptr<const SettingsConstraintsAndProfileIDs> & previous = nullptr) const;
friend bool operator ==(const SettingsProfilesInfo & lhs, const SettingsProfilesInfo & rhs);
friend bool operator !=(const SettingsProfilesInfo & lhs, const SettingsProfilesInfo & rhs) { return !(lhs == rhs); }
private:
const AccessControlManager & manager;
};
}

View File

@ -0,0 +1,88 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Interpreters/Context.h>
#include <Access/AccessControlManager.h>
#include <Access/User.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeArray.h>
namespace DB
{
namespace
{
enum class Kind
{
CURRENT_PROFILES,
ENABLED_PROFILES,
DEFAULT_PROFILES,
};
template <Kind kind>
class FunctionCurrentProfiles : public IFunction
{
public:
static constexpr auto name = (kind == Kind::CURRENT_PROFILES) ? "currentProfiles" : ((kind == Kind::ENABLED_PROFILES) ? "enabledProfiles" : "defaultProfiles");
static FunctionPtr create(const ContextPtr & context) { return std::make_shared<FunctionCurrentProfiles>(context); }
String getName() const override { return name; }
explicit FunctionCurrentProfiles(const ContextPtr & context)
{
const auto & manager = context->getAccessControlManager();
std::vector<UUID> profile_ids;
if constexpr (kind == Kind::CURRENT_PROFILES)
{
profile_ids = context->getCurrentProfiles();
}
else if constexpr (kind == Kind::ENABLED_PROFILES)
{
profile_ids = context->getEnabledProfiles();
}
else
{
static_assert(kind == Kind::DEFAULT_PROFILES);
if (auto user = context->getUser())
profile_ids = user->settings.toProfileIDs();
}
profile_names = manager.tryReadNames(profile_ids);
}
size_t getNumberOfArguments() const override { return 0; }
bool isDeterministic() const override { return false; }
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override
{
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>());
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override
{
auto col_res = ColumnArray::create(ColumnString::create());
ColumnString & res_strings = typeid_cast<ColumnString &>(col_res->getData());
ColumnArray::Offsets & res_offsets = col_res->getOffsets();
for (const String & profile_name : profile_names)
res_strings.insertData(profile_name.data(), profile_name.length());
res_offsets.push_back(res_strings.size());
return ColumnConst::create(std::move(col_res), input_rows_count);
}
private:
Strings profile_names;
};
}
void registerFunctionCurrentProfiles(FunctionFactory & factory)
{
factory.registerFunction<FunctionCurrentProfiles<Kind::CURRENT_PROFILES>>();
factory.registerFunction<FunctionCurrentProfiles<Kind::ENABLED_PROFILES>>();
factory.registerFunction<FunctionCurrentProfiles<Kind::DEFAULT_PROFILES>>();
}
}

View File

@ -9,6 +9,7 @@ class FunctionFactory;
void registerFunctionCurrentDatabase(FunctionFactory &);
void registerFunctionCurrentUser(FunctionFactory &);
void registerFunctionCurrentProfiles(FunctionFactory &);
void registerFunctionHostName(FunctionFactory &);
void registerFunctionFQDN(FunctionFactory &);
void registerFunctionVisibleWidth(FunctionFactory &);
@ -85,6 +86,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
{
registerFunctionCurrentDatabase(factory);
registerFunctionCurrentUser(factory);
registerFunctionCurrentProfiles(factory);
registerFunctionHostName(factory);
registerFunctionFQDN(factory);
registerFunctionVisibleWidth(factory);

View File

@ -42,7 +42,8 @@
#include <Access/User.h>
#include <Access/Credentials.h>
#include <Access/SettingsProfile.h>
#include <Access/SettingsConstraints.h>
#include <Access/SettingsProfilesInfo.h>
#include <Access/SettingsConstraintsAndProfileIDs.h>
#include <Access/ExternalAuthenticators.h>
#include <Access/GSSAcceptor.h>
#include <Dictionaries/Embedded/GeoDictionariesLoader.h>
@ -801,7 +802,9 @@ void Context::setUser(const Credentials & credentials, const Poco::Net::SocketAd
current_roles.clear();
use_default_roles = true;
setSettings(*access->getDefaultSettings());
auto default_profile_info = access->getDefaultProfileInfo();
settings_constraints_and_current_profiles = default_profile_info->getConstraintsAndProfileIDs();
applySettingsChanges(default_profile_info->settings);
}
void Context::setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address)
@ -936,19 +939,41 @@ std::optional<QuotaUsage> Context::getQuotaUsage() const
}
void Context::setProfile(const String & profile_name)
void Context::setCurrentProfile(const String & profile_name)
{
SettingsChanges profile_settings_changes = *getAccessControlManager().getProfileSettings(profile_name);
auto lock = getLock();
try
{
checkSettingsConstraints(profile_settings_changes);
UUID profile_id = getAccessControlManager().getID<SettingsProfile>(profile_name);
setCurrentProfile(profile_id);
}
catch (Exception & e)
{
e.addMessage(", while trying to set settings profile {}", profile_name);
throw;
}
applySettingsChanges(profile_settings_changes);
}
void Context::setCurrentProfile(const UUID & profile_id)
{
auto lock = getLock();
auto profile_info = getAccessControlManager().getSettingsProfileInfo(profile_id);
checkSettingsConstraints(profile_info->settings);
applySettingsChanges(profile_info->settings);
settings_constraints_and_current_profiles = profile_info->getConstraintsAndProfileIDs(settings_constraints_and_current_profiles);
}
std::vector<UUID> Context::getCurrentProfiles() const
{
auto lock = getLock();
return settings_constraints_and_current_profiles->current_profiles;
}
std::vector<UUID> Context::getEnabledProfiles() const
{
auto lock = getLock();
return settings_constraints_and_current_profiles->enabled_profiles;
}
@ -1147,7 +1172,7 @@ void Context::setSetting(const StringRef & name, const String & value)
auto lock = getLock();
if (name == "profile")
{
setProfile(value);
setCurrentProfile(value);
return;
}
settings.set(std::string_view{name}, value);
@ -1162,7 +1187,7 @@ void Context::setSetting(const StringRef & name, const Field & value)
auto lock = getLock();
if (name == "profile")
{
setProfile(value.safeGet<String>());
setCurrentProfile(value.safeGet<String>());
return;
}
settings.set(std::string_view{name}, value);
@ -1198,27 +1223,31 @@ void Context::applySettingsChanges(const SettingsChanges & changes)
void Context::checkSettingsConstraints(const SettingChange & change) const
{
getSettingsConstraints()->check(settings, change);
getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, change);
}
void Context::checkSettingsConstraints(const SettingsChanges & changes) const
{
getSettingsConstraints()->check(settings, changes);
getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, changes);
}
void Context::checkSettingsConstraints(SettingsChanges & changes) const
{
getSettingsConstraints()->check(settings, changes);
getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, changes);
}
void Context::clampToSettingsConstraints(SettingsChanges & changes) const
{
getSettingsConstraints()->clamp(settings, changes);
getSettingsConstraintsAndCurrentProfiles()->constraints.clamp(settings, changes);
}
std::shared_ptr<const SettingsConstraints> Context::getSettingsConstraints() const
std::shared_ptr<const SettingsConstraintsAndProfileIDs> Context::getSettingsConstraintsAndCurrentProfiles() const
{
return getAccess()->getSettingsConstraints();
auto lock = getLock();
if (settings_constraints_and_current_profiles)
return settings_constraints_and_current_profiles;
static auto no_constraints_or_profiles = std::make_shared<SettingsConstraintsAndProfileIDs>(getAccessControlManager());
return no_constraints_or_profiles;
}
@ -2409,13 +2438,13 @@ void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & confi
getAccessControlManager().setDefaultProfileName(shared->default_profile_name);
shared->system_profile_name = config.getString("system_profile", shared->default_profile_name);
setProfile(shared->system_profile_name);
setCurrentProfile(shared->system_profile_name);
applySettingsQuirks(settings, &Poco::Logger::get("SettingsQuirks"));
shared->buffer_profile_name = config.getString("buffer_profile", shared->system_profile_name);
buffer_context = Context::createCopy(shared_from_this());
buffer_context->setProfile(shared->buffer_profile_name);
buffer_context->setCurrentProfile(shared->buffer_profile_name);
}
String Context::getDefaultProfileName() const

View File

@ -89,7 +89,7 @@ class ICompressionCodec;
class AccessControlManager;
class Credentials;
class GSSAcceptorContext;
class SettingsConstraints;
struct SettingsConstraintsAndProfileIDs;
class RemoteHostFilter;
struct StorageID;
class IDisk;
@ -177,6 +177,7 @@ private:
std::optional<UUID> user_id;
std::vector<UUID> current_roles;
bool use_default_roles = false;
std::shared_ptr<const SettingsConstraintsAndProfileIDs> settings_constraints_and_current_profiles;
std::shared_ptr<const ContextAccess> access;
std::shared_ptr<const EnabledRowPolicies> initial_row_policy;
String current_database;
@ -378,6 +379,11 @@ public:
boost::container::flat_set<UUID> getEnabledRoles() const;
std::shared_ptr<const EnabledRolesInfo> getRolesInfo() const;
void setCurrentProfile(const String & profile_name);
void setCurrentProfile(const UUID & profile_id);
std::vector<UUID> getCurrentProfiles() const;
std::vector<UUID> getEnabledProfiles() const;
/// Checks access rights.
/// Empty database means the current database.
void checkAccess(const AccessFlags & flags) const;
@ -516,7 +522,7 @@ public:
void clampToSettingsConstraints(SettingsChanges & changes) const;
/// Returns the current constraints (can return null).
std::shared_ptr<const SettingsConstraints> getSettingsConstraints() const;
std::shared_ptr<const SettingsConstraintsAndProfileIDs> getSettingsConstraintsAndCurrentProfiles() const;
const EmbeddedDictionaries & getEmbeddedDictionaries() const;
const ExternalDictionariesLoader & getExternalDictionariesLoader() const;
@ -810,8 +816,6 @@ private:
template <typename... Args>
void checkAccessImpl(const Args &... args) const;
void setProfile(const String & profile);
EmbeddedDictionaries & getEmbeddedDictionariesImpl(bool throw_on_error) const;
void checkCanBeDropped(const String & database, const String & table, const size_t & size, const size_t & max_size_to_drop) const;

View File

@ -3,7 +3,7 @@
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/Context.h>
#include <Access/SettingsConstraints.h>
#include <Access/SettingsConstraintsAndProfileIDs.h>
namespace DB
@ -29,7 +29,8 @@ NamesAndTypesList StorageSystemSettings::getNamesAndTypes()
void StorageSystemSettings::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
const Settings & settings = context->getSettingsRef();
auto settings_constraints = context->getSettingsConstraints();
auto constraints_and_current_profiles = context->getSettingsConstraintsAndCurrentProfiles();
const auto & constraints = constraints_and_current_profiles->constraints;
for (const auto & setting : settings.all())
{
const auto & setting_name = setting.getName();
@ -40,7 +41,7 @@ void StorageSystemSettings::fillData(MutableColumns & res_columns, ContextPtr co
Field min, max;
bool read_only = false;
settings_constraints->get(setting_name, min, max, read_only);
constraints.get(setting_name, min, max, read_only);
/// These two columns can accept strings only.
if (!min.isNull())

View File

@ -22,6 +22,13 @@ def system_settings_profile_elements(profile_name=None, user_name=None, role_nam
return TSV(instance.query("SELECT * FROM system.settings_profile_elements" + where))
session_id_counter = 0
def new_session_id():
global session_id_counter
session_id_counter += 1
return 'session #' + str(session_id_counter)
@pytest.fixture(scope="module", autouse=True)
def setup_nodes():
try:
@ -42,7 +49,7 @@ def reset_after_test():
finally:
instance.query("CREATE USER OR REPLACE robin")
instance.query("DROP ROLE IF EXISTS worker")
instance.query("DROP SETTINGS PROFILE IF EXISTS xyz, alpha")
instance.query("DROP SETTINGS PROFILE IF EXISTS xyz, alpha, P1, P2, P3, P4, P5, P6")
def test_smoke():
@ -206,6 +213,51 @@ def test_show_profiles():
assert expected_access in instance.query("SHOW ACCESS")
def test_set_profile():
instance.query("CREATE SETTINGS PROFILE P1 SETTINGS max_memory_usage=10000000001")
session_id = new_session_id()
instance.http_query("SET profile='P1'", user='robin', params={'session_id':session_id})
assert instance.http_query("SELECT getSetting('max_memory_usage')", user='robin', params={'session_id':session_id}) == "10000000001\n"
def test_changing_default_profiles_affects_new_sessions_only():
instance.query("CREATE SETTINGS PROFILE P1 SETTINGS max_memory_usage=10000000001")
instance.query("CREATE SETTINGS PROFILE P2 SETTINGS max_memory_usage=10000000002")
instance.query("ALTER USER robin SETTINGS PROFILE P1")
session_id = new_session_id()
assert instance.http_query("SELECT getSetting('max_memory_usage')", user='robin', params={'session_id':session_id}) == "10000000001\n"
instance.query("ALTER USER robin SETTINGS PROFILE P2")
assert instance.http_query("SELECT getSetting('max_memory_usage')", user='robin', params={'session_id':session_id}) == "10000000001\n"
other_session_id = new_session_id()
assert instance.http_query("SELECT getSetting('max_memory_usage')", user='robin', params={'session_id':other_session_id}) == "10000000002\n"
def test_function_current_profiles():
instance.query("CREATE SETTINGS PROFILE P1, P2")
instance.query("ALTER USER robin SETTINGS PROFILE P1, P2")
instance.query("CREATE SETTINGS PROFILE P3 TO robin")
instance.query("CREATE SETTINGS PROFILE P4")
instance.query("CREATE SETTINGS PROFILE P5 SETTINGS INHERIT P4")
instance.query("CREATE ROLE worker SETTINGS PROFILE P5")
instance.query("GRANT worker TO robin")
instance.query("CREATE SETTINGS PROFILE P6")
session_id = new_session_id()
assert instance.http_query('SELECT defaultProfiles(), currentProfiles(), enabledProfiles()', user='robin', params={'session_id':session_id}) == "['P1','P2']\t['P1','P2']\t['default','P3','P4','P5','P1','P2']\n"
instance.http_query("SET profile='P6'", user='robin', params={'session_id':session_id})
assert instance.http_query('SELECT defaultProfiles(), currentProfiles(), enabledProfiles()', user='robin', params={'session_id':session_id}) == "['P1','P2']\t['P6']\t['default','P3','P4','P5','P1','P2','P6']\n"
instance.http_query("SET profile='P5'", user='robin', params={'session_id':session_id})
assert instance.http_query('SELECT defaultProfiles(), currentProfiles(), enabledProfiles()', user='robin', params={'session_id':session_id}) == "['P1','P2']\t['P5']\t['default','P3','P1','P2','P6','P4','P5']\n"
instance.query("ALTER USER robin SETTINGS PROFILE P2")
assert instance.http_query('SELECT defaultProfiles(), currentProfiles(), enabledProfiles()', user='robin', params={'session_id':session_id}) == "['P2']\t['P5']\t['default','P3','P1','P2','P6','P4','P5']\n"
def test_allow_ddl():
assert "it's necessary to have grant" in instance.query_and_get_error("CREATE TABLE tbl(a Int32) ENGINE=Log", user="robin")
assert "it's necessary to have grant" in instance.query_and_get_error("GRANT CREATE ON tbl TO robin", user="robin")