mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Add system tables for settings profiles.
This commit is contained in:
parent
5b84121d81
commit
a14f322723
@ -150,32 +150,34 @@ void SettingsProfilesCache::mergeSettingsAndConstraintsFor(EnabledSettings & ena
|
||||
|
||||
void SettingsProfilesCache::substituteProfiles(SettingsProfileElements & elements) const
|
||||
{
|
||||
bool stop_substituting = false;
|
||||
boost::container::flat_set<UUID> already_substituted;
|
||||
while (!stop_substituting)
|
||||
for (size_t i = 0; i != elements.size();)
|
||||
{
|
||||
stop_substituting = true;
|
||||
for (size_t i = 0; i != elements.size(); ++i)
|
||||
auto & element = elements[i];
|
||||
if (!element.parent_profile)
|
||||
{
|
||||
auto & element = elements[i];
|
||||
if (!element.parent_profile)
|
||||
continue;
|
||||
|
||||
auto parent_profile_id = *element.parent_profile;
|
||||
element.parent_profile.reset();
|
||||
if (already_substituted.count(parent_profile_id))
|
||||
continue;
|
||||
|
||||
already_substituted.insert(parent_profile_id);
|
||||
auto parent_profile = all_profiles.find(parent_profile_id);
|
||||
if (parent_profile == all_profiles.end())
|
||||
continue;
|
||||
|
||||
const auto & parent_profile_elements = parent_profile->second->elements;
|
||||
elements.insert(elements.begin() + i + 1, parent_profile_elements.begin(), parent_profile_elements.end());
|
||||
i += parent_profile_elements.size();
|
||||
stop_substituting = false;
|
||||
++i;
|
||||
continue;
|
||||
}
|
||||
|
||||
auto parent_profile_id = *element.parent_profile;
|
||||
element.parent_profile.reset();
|
||||
if (already_substituted.count(parent_profile_id))
|
||||
{
|
||||
++i;
|
||||
continue;
|
||||
}
|
||||
|
||||
already_substituted.insert(parent_profile_id);
|
||||
auto parent_profile = all_profiles.find(parent_profile_id);
|
||||
if (parent_profile == all_profiles.end())
|
||||
{
|
||||
++i;
|
||||
continue;
|
||||
}
|
||||
|
||||
const auto & parent_profile_elements = parent_profile->second->elements;
|
||||
elements.insert(elements.begin() + i, parent_profile_elements.begin(), parent_profile_elements.end());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -70,6 +70,10 @@ String InterpreterShowAccessEntitiesQuery::getRewrittenQuery() const
|
||||
origin = "quotas";
|
||||
}
|
||||
}
|
||||
else if (query.type == EntityType::SETTINGS_PROFILE)
|
||||
{
|
||||
origin = "settings_profiles";
|
||||
}
|
||||
else
|
||||
throw Exception(toString(query.type) + ": type is not supported by SHOW query", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
|
@ -10,24 +10,31 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
const char * ASTShowAccessEntitiesQuery::getKeyword() const
|
||||
{
|
||||
switch (type)
|
||||
{
|
||||
case EntityType::ROW_POLICY:
|
||||
return "SHOW ROW POLICIES";
|
||||
case EntityType::QUOTA:
|
||||
return current_quota ? "SHOW CURRENT QUOTA" : "SHOW QUOTAS";
|
||||
case EntityType::SETTINGS_PROFILE:
|
||||
return "SHOW SETTINGS PROFILES";
|
||||
default:
|
||||
throw Exception(toString(type) + ": type is not supported by SHOW query", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
String ASTShowAccessEntitiesQuery::getID(char) const
|
||||
{
|
||||
if (type == EntityType::ROW_POLICY)
|
||||
return "SHOW ROW POLICIES query";
|
||||
else if (type == EntityType::QUOTA)
|
||||
return current_quota ? "SHOW CURRENT QUOTA query" : "SHOW QUOTAS query";
|
||||
else
|
||||
throw Exception(toString(type) + ": type is not supported by SHOW query", ErrorCodes::NOT_IMPLEMENTED);
|
||||
return String(getKeyword()) + " query";
|
||||
}
|
||||
|
||||
void ASTShowAccessEntitiesQuery::formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
|
||||
{
|
||||
if (type == EntityType::ROW_POLICY)
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW ROW POLICIES" << (settings.hilite ? hilite_none : "");
|
||||
else if (type == EntityType::QUOTA)
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << (current_quota ? "SHOW CURRENT QUOTA" : "SHOW QUOTAS") << (settings.hilite ? hilite_none : "");
|
||||
else
|
||||
throw Exception(toString(type) + ": type is not supported by SHOW query", ErrorCodes::NOT_IMPLEMENTED);
|
||||
const char * keyword = getKeyword();
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << keyword << (settings.hilite ? hilite_none : "");
|
||||
|
||||
if ((type == EntityType::ROW_POLICY) && !table_name.empty())
|
||||
{
|
||||
|
@ -10,6 +10,7 @@ namespace DB
|
||||
/// SHOW [ROW] POLICIES [ON [database.]table]
|
||||
/// SHOW QUOTAS
|
||||
/// SHOW [CURRENT] QUOTA
|
||||
/// SHOW [SETTINGS] PROFILES
|
||||
class ASTShowAccessEntitiesQuery : public ASTQueryWithOutput
|
||||
{
|
||||
public:
|
||||
@ -25,6 +26,9 @@ public:
|
||||
|
||||
protected:
|
||||
void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
|
||||
|
||||
private:
|
||||
const char * getKeyword() const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -43,6 +43,10 @@ bool ParserShowAccessEntitiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected
|
||||
type = EntityType::QUOTA;
|
||||
current_quota = true;
|
||||
}
|
||||
else if (ParserKeyword{"PROFILES"}.ignore(pos, expected) || ParserKeyword{"SETTINGS PROFILES"}.ignore(pos, expected))
|
||||
{
|
||||
type = EntityType::SETTINGS_PROFILE;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
|
||||
|
@ -9,6 +9,7 @@ namespace DB
|
||||
* SHOW [ROW] POLICIES [ON [database.]table]
|
||||
SHOW QUOTAS
|
||||
SHOW [CURRENT] QUOTA
|
||||
SHOW [SETTINGS] PROFILES
|
||||
*/
|
||||
class ParserShowAccessEntitiesQuery : public IParserBase
|
||||
{
|
||||
|
216
src/Storages/System/StorageSystemSettingsProfileElements.cpp
Normal file
216
src/Storages/System/StorageSystemSettingsProfileElements.cpp
Normal file
@ -0,0 +1,216 @@
|
||||
#include <Storages/System/StorageSystemSettingsProfileElements.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Access/AccessControlManager.h>
|
||||
#include <Access/Role.h>
|
||||
#include <Access/User.h>
|
||||
#include <Access/SettingsProfile.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <boost/range/algorithm_ext/push_back.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
using EntityType = IAccessEntity::Type;
|
||||
|
||||
|
||||
NamesAndTypesList StorageSystemSettingsProfileElements::getNamesAndTypes()
|
||||
{
|
||||
NamesAndTypesList names_and_types{
|
||||
{"profile_name", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
|
||||
{"user_name", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
|
||||
{"role_name", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
|
||||
{"index", std::make_shared<DataTypeUInt64>()},
|
||||
{"setting_name", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
|
||||
{"value", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
|
||||
{"min", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
|
||||
{"max", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
|
||||
{"readonly", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>())},
|
||||
{"inherit_profile", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
|
||||
};
|
||||
return names_and_types;
|
||||
}
|
||||
|
||||
|
||||
void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const
|
||||
{
|
||||
context.checkAccess(AccessType::SHOW_SETTINGS_PROFILES);
|
||||
const auto & access_control = context.getAccessControlManager();
|
||||
std::vector<UUID> ids = access_control.findAll<User>();
|
||||
boost::range::push_back(ids, access_control.findAll<Role>());
|
||||
boost::range::push_back(ids, access_control.findAll<SettingsProfile>());
|
||||
|
||||
size_t i = 0;
|
||||
auto & column_profile_name = assert_cast<ColumnString &>(assert_cast<ColumnNullable &>(*res_columns[i]).getNestedColumn());
|
||||
auto & column_profile_name_null_map = assert_cast<ColumnNullable &>(*res_columns[i++]).getNullMapData();
|
||||
auto & column_user_name = assert_cast<ColumnString &>(assert_cast<ColumnNullable &>(*res_columns[i]).getNestedColumn());
|
||||
auto & column_user_name_null_map = assert_cast<ColumnNullable &>(*res_columns[i++]).getNullMapData();
|
||||
auto & column_role_name = assert_cast<ColumnString &>(assert_cast<ColumnNullable &>(*res_columns[i]).getNestedColumn());
|
||||
auto & column_role_name_null_map = assert_cast<ColumnNullable &>(*res_columns[i++]).getNullMapData();
|
||||
auto & column_index = assert_cast<ColumnUInt64 &>(*res_columns[i++]).getData();
|
||||
auto & column_setting_name = assert_cast<ColumnString &>(assert_cast<ColumnNullable &>(*res_columns[i]).getNestedColumn());
|
||||
auto & column_setting_name_null_map = assert_cast<ColumnNullable &>(*res_columns[i++]).getNullMapData();
|
||||
auto & column_value = assert_cast<ColumnString &>(assert_cast<ColumnNullable &>(*res_columns[i]).getNestedColumn());
|
||||
auto & column_value_null_map = assert_cast<ColumnNullable &>(*res_columns[i++]).getNullMapData();
|
||||
auto & column_min = assert_cast<ColumnString &>(assert_cast<ColumnNullable &>(*res_columns[i]).getNestedColumn());
|
||||
auto & column_min_null_map = assert_cast<ColumnNullable &>(*res_columns[i++]).getNullMapData();
|
||||
auto & column_max = assert_cast<ColumnString &>(assert_cast<ColumnNullable &>(*res_columns[i]).getNestedColumn());
|
||||
auto & column_max_null_map = assert_cast<ColumnNullable &>(*res_columns[i++]).getNullMapData();
|
||||
auto & column_readonly = assert_cast<ColumnUInt8 &>(assert_cast<ColumnNullable &>(*res_columns[i]).getNestedColumn()).getData();
|
||||
auto & column_readonly_null_map = assert_cast<ColumnNullable &>(*res_columns[i++]).getNullMapData();
|
||||
auto & column_inherit_profile = assert_cast<ColumnString &>(assert_cast<ColumnNullable &>(*res_columns[i]).getNestedColumn());
|
||||
auto & column_inherit_profile_null_map = assert_cast<ColumnNullable &>(*res_columns[i++]).getNullMapData();
|
||||
|
||||
auto add_rows_for_single_element = [&](const String & owner_name, EntityType owner_type, const SettingsProfileElement & element, size_t & index)
|
||||
{
|
||||
switch (owner_type)
|
||||
{
|
||||
case EntityType::SETTINGS_PROFILE:
|
||||
{
|
||||
column_user_name.insertDefault();
|
||||
column_user_name_null_map.push_back(true);
|
||||
column_role_name.insertDefault();
|
||||
column_role_name_null_map.push_back(true);
|
||||
column_profile_name.insertData(owner_name.data(), owner_name.length());
|
||||
column_profile_name_null_map.push_back(false);
|
||||
break;
|
||||
}
|
||||
case EntityType::USER:
|
||||
{
|
||||
column_user_name.insertData(owner_name.data(), owner_name.length());
|
||||
column_user_name_null_map.push_back(false);
|
||||
column_profile_name.insertDefault();
|
||||
column_profile_name_null_map.push_back(true);
|
||||
column_role_name.insertDefault();
|
||||
column_role_name_null_map.push_back(true);
|
||||
break;
|
||||
}
|
||||
case EntityType::ROLE:
|
||||
{
|
||||
column_user_name.insertDefault();
|
||||
column_user_name_null_map.push_back(true);
|
||||
column_role_name.insertData(owner_name.data(), owner_name.length());
|
||||
column_role_name_null_map.push_back(false);
|
||||
column_profile_name.insertDefault();
|
||||
column_profile_name_null_map.push_back(true);
|
||||
break;
|
||||
}
|
||||
default:
|
||||
assert(false);
|
||||
}
|
||||
|
||||
if (element.parent_profile)
|
||||
{
|
||||
auto parent_profile = access_control.tryReadName(*element.parent_profile);
|
||||
if (parent_profile)
|
||||
{
|
||||
column_index.push_back(index++);
|
||||
column_setting_name.insertDefault();
|
||||
column_setting_name_null_map.push_back(true);
|
||||
column_value.insertDefault();
|
||||
column_value_null_map.push_back(true);
|
||||
column_min.insertDefault();
|
||||
column_min_null_map.push_back(true);
|
||||
column_max.insertDefault();
|
||||
column_max_null_map.push_back(true);
|
||||
column_readonly.push_back(0);
|
||||
column_readonly_null_map.push_back(true);
|
||||
const String & parent_profile_str = *parent_profile;
|
||||
column_inherit_profile.insertData(parent_profile_str.data(), parent_profile_str.length());
|
||||
column_inherit_profile_null_map.push_back(false);
|
||||
}
|
||||
}
|
||||
|
||||
if ((element.setting_index != static_cast<size_t>(-1))
|
||||
&& (!element.value.isNull() || !element.min_value.isNull() || !element.max_value.isNull() || element.readonly))
|
||||
{
|
||||
auto setting_name = Settings::getName(element.setting_index);
|
||||
column_index.push_back(index++);
|
||||
column_setting_name.insertData(setting_name.data, setting_name.size);
|
||||
column_setting_name_null_map.push_back(false);
|
||||
|
||||
if (element.value.isNull())
|
||||
{
|
||||
column_value.insertDefault();
|
||||
column_value_null_map.push_back(true);
|
||||
}
|
||||
else
|
||||
{
|
||||
String str = Settings::valueToString(element.setting_index, element.value);
|
||||
column_value.insertData(str.data(), str.length());
|
||||
column_value_null_map.push_back(false);
|
||||
}
|
||||
|
||||
if (element.min_value.isNull())
|
||||
{
|
||||
column_min.insertDefault();
|
||||
column_min_null_map.push_back(true);
|
||||
}
|
||||
else
|
||||
{
|
||||
String str = Settings::valueToString(element.setting_index, element.min_value);
|
||||
column_min.insertData(str.data(), str.length());
|
||||
column_min_null_map.push_back(false);
|
||||
}
|
||||
|
||||
if (element.max_value.isNull())
|
||||
{
|
||||
column_max.insertDefault();
|
||||
column_max_null_map.push_back(true);
|
||||
}
|
||||
else
|
||||
{
|
||||
String str = Settings::valueToString(element.setting_index, element.max_value);
|
||||
column_max.insertData(str.data(), str.length());
|
||||
column_max_null_map.push_back(false);
|
||||
}
|
||||
|
||||
if (element.readonly)
|
||||
{
|
||||
column_readonly.push_back(*element.readonly);
|
||||
column_readonly_null_map.push_back(false);
|
||||
}
|
||||
else
|
||||
{
|
||||
column_readonly.push_back(0);
|
||||
column_readonly_null_map.push_back(true);
|
||||
}
|
||||
|
||||
column_inherit_profile.insertDefault();
|
||||
column_inherit_profile_null_map.push_back(true);
|
||||
}
|
||||
};
|
||||
|
||||
auto add_rows = [&](const String & owner_name, IAccessEntity::Type owner_type, const SettingsProfileElements & elements)
|
||||
{
|
||||
size_t index = 0;
|
||||
for (const auto & element : elements)
|
||||
add_rows_for_single_element(owner_name, owner_type, element, index);
|
||||
};
|
||||
|
||||
for (const auto & id : ids)
|
||||
{
|
||||
auto entity = access_control.tryRead(id);
|
||||
if (!entity)
|
||||
continue;
|
||||
|
||||
const SettingsProfileElements * settings = nullptr;
|
||||
if (auto role = typeid_cast<RolePtr>(entity))
|
||||
settings = &role->settings;
|
||||
else if (auto user = typeid_cast<UserPtr>(entity))
|
||||
settings = &user->settings;
|
||||
else if (auto profile = typeid_cast<SettingsProfilePtr>(entity))
|
||||
settings = &profile->elements;
|
||||
else
|
||||
continue;
|
||||
|
||||
add_rows(entity->getName(), entity->getType(), *settings);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
24
src/Storages/System/StorageSystemSettingsProfileElements.h
Normal file
24
src/Storages/System/StorageSystemSettingsProfileElements.h
Normal file
@ -0,0 +1,24 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
#include <Storages/System/IStorageSystemOneBlock.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class Context;
|
||||
|
||||
/// Implements `settings_profile_elements` system table, which allows you to get information about elements of settings profiles.
|
||||
class StorageSystemSettingsProfileElements final : public ext::shared_ptr_helper<StorageSystemSettingsProfileElements>, public IStorageSystemOneBlock<StorageSystemSettingsProfileElements>
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "SystemSettingsProfileElements"; }
|
||||
static NamesAndTypesList getNamesAndTypes();
|
||||
|
||||
protected:
|
||||
friend struct ext::shared_ptr_helper<StorageSystemSettingsProfileElements>;
|
||||
using IStorageSystemOneBlock::IStorageSystemOneBlock;
|
||||
void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const override;
|
||||
};
|
||||
|
||||
}
|
87
src/Storages/System/StorageSystemSettingsProfiles.cpp
Normal file
87
src/Storages/System/StorageSystemSettingsProfiles.cpp
Normal file
@ -0,0 +1,87 @@
|
||||
#include <Storages/System/StorageSystemSettingsProfiles.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTExtendedRoleSet.h>
|
||||
#include <Access/AccessControlManager.h>
|
||||
#include <Access/SettingsProfile.h>
|
||||
#include <Access/AccessFlags.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
NamesAndTypesList StorageSystemSettingsProfiles::getNamesAndTypes()
|
||||
{
|
||||
NamesAndTypesList names_and_types{
|
||||
{"name", std::make_shared<DataTypeString>()},
|
||||
{"id", std::make_shared<DataTypeUUID>()},
|
||||
{"storage", std::make_shared<DataTypeString>()},
|
||||
{"num_elements", std::make_shared<DataTypeUInt64>()},
|
||||
{"apply_to_all", std::make_shared<DataTypeUInt8>()},
|
||||
{"apply_to_list", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
{"apply_to_except", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
};
|
||||
return names_and_types;
|
||||
}
|
||||
|
||||
|
||||
void StorageSystemSettingsProfiles::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const
|
||||
{
|
||||
context.checkAccess(AccessType::SHOW_SETTINGS_PROFILES);
|
||||
const auto & access_control = context.getAccessControlManager();
|
||||
std::vector<UUID> ids = access_control.findAll<SettingsProfile>();
|
||||
|
||||
size_t column_index = 0;
|
||||
auto & column_name = assert_cast<ColumnString &>(*res_columns[column_index++]);
|
||||
auto & column_id = assert_cast<ColumnUInt128 &>(*res_columns[column_index++]).getData();
|
||||
auto & column_storage = assert_cast<ColumnString &>(*res_columns[column_index++]);
|
||||
auto & column_num_elements = assert_cast<ColumnUInt64 &>(*res_columns[column_index++]).getData();
|
||||
auto & column_apply_to_all = assert_cast<ColumnUInt8 &>(*res_columns[column_index++]).getData();
|
||||
auto & column_apply_to_list = assert_cast<ColumnString &>(assert_cast<ColumnArray &>(*res_columns[column_index]).getData());
|
||||
auto & column_apply_to_list_offsets = assert_cast<ColumnArray &>(*res_columns[column_index++]).getOffsets();
|
||||
auto & column_apply_to_except = assert_cast<ColumnString &>(assert_cast<ColumnArray &>(*res_columns[column_index]).getData());
|
||||
auto & column_apply_to_except_offsets = assert_cast<ColumnArray &>(*res_columns[column_index++]).getOffsets();
|
||||
|
||||
auto add_row = [&](const String & name,
|
||||
const UUID & id,
|
||||
const String & storage_name,
|
||||
const SettingsProfileElements & elements,
|
||||
const ExtendedRoleSet & apply_to)
|
||||
{
|
||||
column_name.insertData(name.data(), name.length());
|
||||
column_id.push_back(id);
|
||||
column_storage.insertData(storage_name.data(), storage_name.length());
|
||||
column_num_elements.push_back(elements.size());
|
||||
|
||||
auto apply_to_ast = apply_to.toASTWithNames(access_control);
|
||||
column_apply_to_all.push_back(apply_to_ast->all);
|
||||
|
||||
for (const auto & role_name : apply_to_ast->names)
|
||||
column_apply_to_list.insertData(role_name.data(), role_name.length());
|
||||
column_apply_to_list_offsets.push_back(column_apply_to_list.size());
|
||||
|
||||
for (const auto & role_name : apply_to_ast->except_names)
|
||||
column_apply_to_except.insertData(role_name.data(), role_name.length());
|
||||
column_apply_to_except_offsets.push_back(column_apply_to_except.size());
|
||||
};
|
||||
|
||||
for (const auto & id : ids)
|
||||
{
|
||||
auto profile = access_control.tryRead<SettingsProfile>(id);
|
||||
if (!profile)
|
||||
continue;
|
||||
|
||||
const auto * storage = access_control.findStorage(id);
|
||||
if (!storage)
|
||||
continue;
|
||||
|
||||
add_row(profile->getName(), id, storage->getStorageName(), profile->elements, profile->to_roles);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
24
src/Storages/System/StorageSystemSettingsProfiles.h
Normal file
24
src/Storages/System/StorageSystemSettingsProfiles.h
Normal file
@ -0,0 +1,24 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
#include <Storages/System/IStorageSystemOneBlock.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class Context;
|
||||
|
||||
/// Implements `settings_profiles` system table, which allows you to get information about profiles.
|
||||
class StorageSystemSettingsProfiles final : public ext::shared_ptr_helper<StorageSystemSettingsProfiles>, public IStorageSystemOneBlock<StorageSystemSettingsProfiles>
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "SystemSettingsProfiles"; }
|
||||
static NamesAndTypesList getNamesAndTypes();
|
||||
|
||||
protected:
|
||||
friend struct ext::shared_ptr_helper<StorageSystemSettingsProfiles>;
|
||||
using IStorageSystemOneBlock::IStorageSystemOneBlock;
|
||||
void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const override;
|
||||
};
|
||||
|
||||
}
|
@ -15,6 +15,7 @@
|
||||
#include <Storages/System/StorageSystemFormats.h>
|
||||
#include <Storages/System/StorageSystemFunctions.h>
|
||||
#include <Storages/System/StorageSystemGraphite.h>
|
||||
|
||||
#include <Storages/System/StorageSystemMacros.h>
|
||||
#include <Storages/System/StorageSystemMerges.h>
|
||||
#include <Storages/System/StorageSystemMetrics.h>
|
||||
@ -25,13 +26,8 @@
|
||||
#include <Storages/System/StorageSystemParts.h>
|
||||
#include <Storages/System/StorageSystemPartsColumns.h>
|
||||
#include <Storages/System/StorageSystemProcesses.h>
|
||||
#include <Storages/System/StorageSystemQuotas.h>
|
||||
#include <Storages/System/StorageSystemQuotaLimits.h>
|
||||
#include <Storages/System/StorageSystemQuotaUsage.h>
|
||||
#include <Storages/System/StorageSystemQuotasUsage.h>
|
||||
#include <Storages/System/StorageSystemReplicas.h>
|
||||
#include <Storages/System/StorageSystemReplicationQueue.h>
|
||||
#include <Storages/System/StorageSystemRowPolicies.h>
|
||||
#include <Storages/System/StorageSystemSettings.h>
|
||||
#include <Storages/System/StorageSystemMergeTreeSettings.h>
|
||||
#include <Storages/System/StorageSystemTableEngines.h>
|
||||
@ -46,6 +42,14 @@
|
||||
#include <Storages/System/StorageSystemStoragePolicies.h>
|
||||
#include <Storages/System/StorageSystemZeros.h>
|
||||
|
||||
#include <Storages/System/StorageSystemSettingsProfiles.h>
|
||||
#include <Storages/System/StorageSystemSettingsProfileElements.h>
|
||||
#include <Storages/System/StorageSystemRowPolicies.h>
|
||||
#include <Storages/System/StorageSystemQuotas.h>
|
||||
#include <Storages/System/StorageSystemQuotaLimits.h>
|
||||
#include <Storages/System/StorageSystemQuotaUsage.h>
|
||||
#include <Storages/System/StorageSystemQuotasUsage.h>
|
||||
|
||||
#ifdef OS_LINUX
|
||||
#include <Storages/System/StorageSystemStackTrace.h>
|
||||
#endif
|
||||
@ -68,11 +72,6 @@ void attachSystemTablesLocal(IDatabase & system_database)
|
||||
system_database.attachTable("functions", StorageSystemFunctions::create("functions"));
|
||||
system_database.attachTable("events", StorageSystemEvents::create("events"));
|
||||
system_database.attachTable("settings", StorageSystemSettings::create("settings"));
|
||||
system_database.attachTable("quotas", StorageSystemQuotas::create("quotas"));
|
||||
system_database.attachTable("quota_limits", StorageSystemQuotaLimits::create("quota_limits"));
|
||||
system_database.attachTable("quota_usage", StorageSystemQuotaUsage::create("quota_usage"));
|
||||
system_database.attachTable("quotas_usage", StorageSystemQuotasUsage::create("all_quotas_usage"));
|
||||
system_database.attachTable("row_policies", StorageSystemRowPolicies::create("row_policies"));
|
||||
system_database.attachTable("merge_tree_settings", SystemMergeTreeSettings::create("merge_tree_settings"));
|
||||
system_database.attachTable("build_options", StorageSystemBuildOptions::create("build_options"));
|
||||
system_database.attachTable("formats", StorageSystemFormats::create("formats"));
|
||||
@ -82,6 +81,13 @@ void attachSystemTablesLocal(IDatabase & system_database)
|
||||
system_database.attachTable("collations", StorageSystemCollations::create("collations"));
|
||||
system_database.attachTable("table_engines", StorageSystemTableEngines::create("table_engines"));
|
||||
system_database.attachTable("contributors", StorageSystemContributors::create("contributors"));
|
||||
system_database.attachTable("settings_profiles", StorageSystemSettingsProfiles::create("settings_profiles"));
|
||||
system_database.attachTable("settings_profile_elements", StorageSystemSettingsProfileElements::create("settings_profile_elements"));
|
||||
system_database.attachTable("row_policies", StorageSystemRowPolicies::create("row_policies"));
|
||||
system_database.attachTable("quotas", StorageSystemQuotas::create("quotas"));
|
||||
system_database.attachTable("quota_limits", StorageSystemQuotaLimits::create("quota_limits"));
|
||||
system_database.attachTable("quota_usage", StorageSystemQuotaUsage::create("quota_usage"));
|
||||
system_database.attachTable("quotas_usage", StorageSystemQuotasUsage::create("all_quotas_usage"));
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
system_database.attachTable("licenses", StorageSystemLicenses::create("licenses"));
|
||||
#endif
|
||||
|
@ -119,6 +119,8 @@ SRCS(
|
||||
System/StorageSystemReplicationQueue.cpp
|
||||
System/StorageSystemRowPolicies.cpp
|
||||
System/StorageSystemSettings.cpp
|
||||
System/StorageSystemSettingsProfileElements.cpp
|
||||
System/StorageSystemSettingsProfiles.cpp
|
||||
System/StorageSystemStackTrace.cpp
|
||||
System/StorageSystemStoragePolicies.cpp
|
||||
System/StorageSystemTableEngines.cpp
|
||||
|
@ -1,10 +1,25 @@
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.test_tools import TSV
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance('instance')
|
||||
|
||||
|
||||
def system_settings_profile(profile_name):
|
||||
return TSV(instance.query("SELECT name, storage, num_elements, apply_to_all, apply_to_list, apply_to_except FROM system.settings_profiles WHERE name='" + profile_name + "'"))
|
||||
|
||||
def system_settings_profile_elements(profile_name=None, user_name=None, role_name=None):
|
||||
where = ""
|
||||
if profile_name:
|
||||
where = " WHERE profile_name='" + profile_name + "'"
|
||||
elif user_name:
|
||||
where = " WHERE user_name='" + user_name + "'"
|
||||
elif role_name:
|
||||
where = " WHERE role_name='" + role_name + "'"
|
||||
return TSV(instance.query("SELECT * FROM system.settings_profile_elements" + where))
|
||||
|
||||
|
||||
@pytest.fixture(scope="module", autouse=True)
|
||||
def setup_nodes():
|
||||
try:
|
||||
@ -28,19 +43,23 @@ def reset_after_test():
|
||||
instance.query("DROP SETTINGS PROFILE IF EXISTS xyz, alpha")
|
||||
|
||||
|
||||
def test_settings_profile():
|
||||
def test_smoke():
|
||||
# Set settings and constraints via CREATE SETTINGS PROFILE ... TO user
|
||||
instance.query("CREATE SETTINGS PROFILE xyz SETTINGS max_memory_usage = 100000001 MIN 90000000 MAX 110000000 TO robin")
|
||||
assert instance.query("SHOW CREATE SETTINGS PROFILE xyz") == "CREATE SETTINGS PROFILE xyz SETTINGS max_memory_usage = 100000001 MIN 90000000 MAX 110000000 TO robin\n"
|
||||
assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "100000001\n"
|
||||
assert "Setting max_memory_usage shouldn't be less than 90000000" in instance.query_and_get_error("SET max_memory_usage = 80000000", user="robin")
|
||||
assert "Setting max_memory_usage shouldn't be greater than 110000000" in instance.query_and_get_error("SET max_memory_usage = 120000000", user="robin")
|
||||
assert system_settings_profile("xyz") == [[ "xyz", "disk", 1, 0, "['robin']", "[]" ]]
|
||||
assert system_settings_profile_elements(profile_name="xyz") == [[ "xyz", "\N", "\N", 0, "max_memory_usage", 100000001, 90000000, 110000000, "\N", "\N" ]]
|
||||
|
||||
instance.query("ALTER SETTINGS PROFILE xyz TO NONE")
|
||||
assert instance.query("SHOW CREATE SETTINGS PROFILE xyz") == "CREATE SETTINGS PROFILE xyz SETTINGS max_memory_usage = 100000001 MIN 90000000 MAX 110000000\n"
|
||||
assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "10000000000\n"
|
||||
instance.query("SET max_memory_usage = 80000000", user="robin")
|
||||
instance.query("SET max_memory_usage = 120000000", user="robin")
|
||||
assert system_settings_profile("xyz") == [[ "xyz", "disk", 1, 0, "[]", "[]" ]]
|
||||
assert system_settings_profile_elements(user_name="robin") == []
|
||||
|
||||
# Set settings and constraints via CREATE USER ... SETTINGS PROFILE
|
||||
instance.query("ALTER USER robin SETTINGS PROFILE xyz")
|
||||
@ -48,52 +67,57 @@ def test_settings_profile():
|
||||
assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "100000001\n"
|
||||
assert "Setting max_memory_usage shouldn't be less than 90000000" in instance.query_and_get_error("SET max_memory_usage = 80000000", user="robin")
|
||||
assert "Setting max_memory_usage shouldn't be greater than 110000000" in instance.query_and_get_error("SET max_memory_usage = 120000000", user="robin")
|
||||
assert system_settings_profile_elements(user_name="robin") == [[ "\N", "robin", "\N", 0, "\N", "\N", "\N", "\N", "\N", "xyz" ]]
|
||||
|
||||
instance.query("ALTER USER robin SETTINGS NONE")
|
||||
assert instance.query("SHOW CREATE USER robin") == "CREATE USER robin\n"
|
||||
assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "10000000000\n"
|
||||
instance.query("SET max_memory_usage = 80000000", user="robin")
|
||||
instance.query("SET max_memory_usage = 120000000", user="robin")
|
||||
assert system_settings_profile_elements(user_name="robin") == []
|
||||
|
||||
|
||||
def test_settings_profile_from_granted_role():
|
||||
def test_settings_from_granted_role():
|
||||
# Set settings and constraints via granted role
|
||||
instance.query("CREATE SETTINGS PROFILE xyz SETTINGS max_memory_usage = 100000001 MIN 90000000 MAX 110000000")
|
||||
instance.query("CREATE SETTINGS PROFILE xyz SETTINGS max_memory_usage = 100000001 MAX 110000000, max_ast_depth = 2000")
|
||||
instance.query("CREATE ROLE worker SETTINGS PROFILE xyz")
|
||||
instance.query("GRANT worker TO robin")
|
||||
assert instance.query("SHOW CREATE SETTINGS PROFILE xyz") == "CREATE SETTINGS PROFILE xyz SETTINGS max_memory_usage = 100000001 MIN 90000000 MAX 110000000\n"
|
||||
assert instance.query("SHOW CREATE SETTINGS PROFILE xyz") == "CREATE SETTINGS PROFILE xyz SETTINGS max_memory_usage = 100000001 MAX 110000000, max_ast_depth = 2000\n"
|
||||
assert instance.query("SHOW CREATE ROLE worker") == "CREATE ROLE worker SETTINGS PROFILE xyz\n"
|
||||
assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "100000001\n"
|
||||
assert "Setting max_memory_usage shouldn't be less than 90000000" in instance.query_and_get_error("SET max_memory_usage = 80000000", user="robin")
|
||||
assert instance.query("SELECT value FROM system.settings WHERE name = 'max_ast_depth'", user="robin") == "2000\n"
|
||||
assert "Setting max_memory_usage shouldn't be greater than 110000000" in instance.query_and_get_error("SET max_memory_usage = 120000000", user="robin")
|
||||
assert system_settings_profile("xyz") == [[ "xyz", "disk", 2, 0, "[]", "[]" ]]
|
||||
assert system_settings_profile_elements(profile_name="xyz") == [[ "xyz", "\N", "\N", 0, "max_memory_usage", 100000001, "\N", 110000000, "\N", "\N" ],
|
||||
[ "xyz", "\N", "\N", 1, "max_ast_depth", 2000, "\N", "\N", "\N", "\N" ]]
|
||||
assert system_settings_profile_elements(role_name="worker") == [[ "\N", "\N", "worker", 0, "\N", "\N", "\N", "\N", "\N", "xyz" ]]
|
||||
|
||||
instance.query("REVOKE worker FROM robin")
|
||||
assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "10000000000\n"
|
||||
instance.query("SET max_memory_usage = 80000000", user="robin")
|
||||
instance.query("SET max_memory_usage = 120000000", user="robin")
|
||||
|
||||
instance.query("ALTER ROLE worker SETTINGS NONE")
|
||||
instance.query("GRANT worker TO robin")
|
||||
assert instance.query("SHOW CREATE ROLE worker") == "CREATE ROLE worker\n"
|
||||
assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "10000000000\n"
|
||||
instance.query("SET max_memory_usage = 80000000", user="robin")
|
||||
instance.query("SET max_memory_usage = 120000000", user="robin")
|
||||
assert system_settings_profile_elements(role_name="worker") == []
|
||||
|
||||
# Set settings and constraints via CREATE SETTINGS PROFILE ... TO granted role
|
||||
instance.query("ALTER SETTINGS PROFILE xyz TO worker")
|
||||
assert instance.query("SHOW CREATE SETTINGS PROFILE xyz") == "CREATE SETTINGS PROFILE xyz SETTINGS max_memory_usage = 100000001 MIN 90000000 MAX 110000000 TO worker\n"
|
||||
assert instance.query("SHOW CREATE SETTINGS PROFILE xyz") == "CREATE SETTINGS PROFILE xyz SETTINGS max_memory_usage = 100000001 MAX 110000000, max_ast_depth = 2000 TO worker\n"
|
||||
assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "100000001\n"
|
||||
assert "Setting max_memory_usage shouldn't be less than 90000000" in instance.query_and_get_error("SET max_memory_usage = 80000000", user="robin")
|
||||
assert "Setting max_memory_usage shouldn't be greater than 110000000" in instance.query_and_get_error("SET max_memory_usage = 120000000", user="robin")
|
||||
assert system_settings_profile("xyz") == [[ "xyz", "disk", 2, 0, "['worker']", "[]" ]]
|
||||
|
||||
instance.query("ALTER SETTINGS PROFILE xyz TO NONE")
|
||||
assert instance.query("SHOW CREATE SETTINGS PROFILE xyz") == "CREATE SETTINGS PROFILE xyz SETTINGS max_memory_usage = 100000001 MIN 90000000 MAX 110000000\n"
|
||||
assert instance.query("SHOW CREATE SETTINGS PROFILE xyz") == "CREATE SETTINGS PROFILE xyz SETTINGS max_memory_usage = 100000001 MAX 110000000, max_ast_depth = 2000\n"
|
||||
assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "10000000000\n"
|
||||
instance.query("SET max_memory_usage = 80000000", user="robin")
|
||||
instance.query("SET max_memory_usage = 120000000", user="robin")
|
||||
assert system_settings_profile("xyz") == [[ "xyz", "disk", 2, 0, "[]", "[]" ]]
|
||||
|
||||
|
||||
def test_inheritance_of_settings_profile():
|
||||
def test_inheritance():
|
||||
instance.query("CREATE SETTINGS PROFILE xyz SETTINGS max_memory_usage = 100000002 READONLY")
|
||||
instance.query("CREATE SETTINGS PROFILE alpha SETTINGS PROFILE xyz TO robin")
|
||||
assert instance.query("SHOW CREATE SETTINGS PROFILE xyz") == "CREATE SETTINGS PROFILE xyz SETTINGS max_memory_usage = 100000002 READONLY\n"
|
||||
@ -101,6 +125,12 @@ def test_inheritance_of_settings_profile():
|
||||
assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "100000002\n"
|
||||
assert "Setting max_memory_usage should not be changed" in instance.query_and_get_error("SET max_memory_usage = 80000000", user="robin")
|
||||
|
||||
assert system_settings_profile("xyz") == [[ "xyz", "disk", 1, 0, "[]", "[]" ]]
|
||||
assert system_settings_profile_elements(profile_name="xyz") == [[ "xyz", "\N", "\N", 0, "max_memory_usage", 100000002, "\N", "\N", 1, "\N" ]]
|
||||
assert system_settings_profile("alpha") == [[ "alpha", "disk", 1, 0, "['robin']", "[]" ]]
|
||||
assert system_settings_profile_elements(profile_name="alpha") == [[ "alpha", "\N", "\N", 0, "\N", "\N", "\N", "\N", "\N", "xyz" ]]
|
||||
assert system_settings_profile_elements(user_name="robin") == []
|
||||
|
||||
|
||||
def test_alter_and_drop():
|
||||
instance.query("CREATE SETTINGS PROFILE xyz SETTINGS max_memory_usage = 100000003 MIN 90000000 MAX 110000000 TO robin")
|
||||
@ -117,6 +147,12 @@ def test_alter_and_drop():
|
||||
instance.query("SET max_memory_usage = 120000000", user="robin")
|
||||
|
||||
|
||||
def test_show_profiles():
|
||||
instance.query("CREATE SETTINGS PROFILE xyz")
|
||||
assert instance.query("SHOW SETTINGS PROFILES") == "default\nreadonly\nxyz\n"
|
||||
assert instance.query("SHOW PROFILES") == "default\nreadonly\nxyz\n"
|
||||
|
||||
|
||||
def test_allow_introspection():
|
||||
assert "Not enough privileges" in instance.query_and_get_error("SELECT demangle('a')", user="robin")
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user