diff --git a/src/Access/SettingsProfilesCache.cpp b/src/Access/SettingsProfilesCache.cpp index 3f9399ca484..95074ff7d0b 100644 --- a/src/Access/SettingsProfilesCache.cpp +++ b/src/Access/SettingsProfilesCache.cpp @@ -150,32 +150,34 @@ void SettingsProfilesCache::mergeSettingsAndConstraintsFor(EnabledSettings & ena void SettingsProfilesCache::substituteProfiles(SettingsProfileElements & elements) const { - bool stop_substituting = false; boost::container::flat_set 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()); } } diff --git a/src/Interpreters/InterpreterShowAccessEntitiesQuery.cpp b/src/Interpreters/InterpreterShowAccessEntitiesQuery.cpp index f6527d0ba8e..a7fb335aacb 100644 --- a/src/Interpreters/InterpreterShowAccessEntitiesQuery.cpp +++ b/src/Interpreters/InterpreterShowAccessEntitiesQuery.cpp @@ -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); diff --git a/src/Parsers/ASTShowAccessEntitiesQuery.cpp b/src/Parsers/ASTShowAccessEntitiesQuery.cpp index a1da1ac04e2..3fe88a2321c 100644 --- a/src/Parsers/ASTShowAccessEntitiesQuery.cpp +++ b/src/Parsers/ASTShowAccessEntitiesQuery.cpp @@ -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()) { diff --git a/src/Parsers/ASTShowAccessEntitiesQuery.h b/src/Parsers/ASTShowAccessEntitiesQuery.h index 09ccbe9cf91..e41553d2e71 100644 --- a/src/Parsers/ASTShowAccessEntitiesQuery.h +++ b/src/Parsers/ASTShowAccessEntitiesQuery.h @@ -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; }; } diff --git a/src/Parsers/ParserShowAccessEntitiesQuery.cpp b/src/Parsers/ParserShowAccessEntitiesQuery.cpp index 7ca9782dd2c..f70a3ea189c 100644 --- a/src/Parsers/ParserShowAccessEntitiesQuery.cpp +++ b/src/Parsers/ParserShowAccessEntitiesQuery.cpp @@ -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; diff --git a/src/Parsers/ParserShowAccessEntitiesQuery.h b/src/Parsers/ParserShowAccessEntitiesQuery.h index 77838e726e7..bb8b37f40e8 100644 --- a/src/Parsers/ParserShowAccessEntitiesQuery.h +++ b/src/Parsers/ParserShowAccessEntitiesQuery.h @@ -9,6 +9,7 @@ namespace DB * SHOW [ROW] POLICIES [ON [database.]table] SHOW QUOTAS SHOW [CURRENT] QUOTA + SHOW [SETTINGS] PROFILES */ class ParserShowAccessEntitiesQuery : public IParserBase { diff --git a/src/Storages/System/StorageSystemSettingsProfileElements.cpp b/src/Storages/System/StorageSystemSettingsProfileElements.cpp new file mode 100644 index 00000000000..2e4d1ad1e05 --- /dev/null +++ b/src/Storages/System/StorageSystemSettingsProfileElements.cpp @@ -0,0 +1,216 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +using EntityType = IAccessEntity::Type; + + +NamesAndTypesList StorageSystemSettingsProfileElements::getNamesAndTypes() +{ + NamesAndTypesList names_and_types{ + {"profile_name", std::make_shared(std::make_shared())}, + {"user_name", std::make_shared(std::make_shared())}, + {"role_name", std::make_shared(std::make_shared())}, + {"index", std::make_shared()}, + {"setting_name", std::make_shared(std::make_shared())}, + {"value", std::make_shared(std::make_shared())}, + {"min", std::make_shared(std::make_shared())}, + {"max", std::make_shared(std::make_shared())}, + {"readonly", std::make_shared(std::make_shared())}, + {"inherit_profile", std::make_shared(std::make_shared())}, + }; + 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 ids = access_control.findAll(); + boost::range::push_back(ids, access_control.findAll()); + boost::range::push_back(ids, access_control.findAll()); + + size_t i = 0; + auto & column_profile_name = assert_cast(assert_cast(*res_columns[i]).getNestedColumn()); + auto & column_profile_name_null_map = assert_cast(*res_columns[i++]).getNullMapData(); + auto & column_user_name = assert_cast(assert_cast(*res_columns[i]).getNestedColumn()); + auto & column_user_name_null_map = assert_cast(*res_columns[i++]).getNullMapData(); + auto & column_role_name = assert_cast(assert_cast(*res_columns[i]).getNestedColumn()); + auto & column_role_name_null_map = assert_cast(*res_columns[i++]).getNullMapData(); + auto & column_index = assert_cast(*res_columns[i++]).getData(); + auto & column_setting_name = assert_cast(assert_cast(*res_columns[i]).getNestedColumn()); + auto & column_setting_name_null_map = assert_cast(*res_columns[i++]).getNullMapData(); + auto & column_value = assert_cast(assert_cast(*res_columns[i]).getNestedColumn()); + auto & column_value_null_map = assert_cast(*res_columns[i++]).getNullMapData(); + auto & column_min = assert_cast(assert_cast(*res_columns[i]).getNestedColumn()); + auto & column_min_null_map = assert_cast(*res_columns[i++]).getNullMapData(); + auto & column_max = assert_cast(assert_cast(*res_columns[i]).getNestedColumn()); + auto & column_max_null_map = assert_cast(*res_columns[i++]).getNullMapData(); + auto & column_readonly = assert_cast(assert_cast(*res_columns[i]).getNestedColumn()).getData(); + auto & column_readonly_null_map = assert_cast(*res_columns[i++]).getNullMapData(); + auto & column_inherit_profile = assert_cast(assert_cast(*res_columns[i]).getNestedColumn()); + auto & column_inherit_profile_null_map = assert_cast(*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(-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(entity)) + settings = &role->settings; + else if (auto user = typeid_cast(entity)) + settings = &user->settings; + else if (auto profile = typeid_cast(entity)) + settings = &profile->elements; + else + continue; + + add_rows(entity->getName(), entity->getType(), *settings); + } +} + +} diff --git a/src/Storages/System/StorageSystemSettingsProfileElements.h b/src/Storages/System/StorageSystemSettingsProfileElements.h new file mode 100644 index 00000000000..2dc79fed0e7 --- /dev/null +++ b/src/Storages/System/StorageSystemSettingsProfileElements.h @@ -0,0 +1,24 @@ +#pragma once + +#include +#include + + +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, public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemSettingsProfileElements"; } + static NamesAndTypesList getNamesAndTypes(); + +protected: + friend struct ext::shared_ptr_helper; + using IStorageSystemOneBlock::IStorageSystemOneBlock; + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const override; +}; + +} diff --git a/src/Storages/System/StorageSystemSettingsProfiles.cpp b/src/Storages/System/StorageSystemSettingsProfiles.cpp new file mode 100644 index 00000000000..d02c5910608 --- /dev/null +++ b/src/Storages/System/StorageSystemSettingsProfiles.cpp @@ -0,0 +1,87 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +NamesAndTypesList StorageSystemSettingsProfiles::getNamesAndTypes() +{ + NamesAndTypesList names_and_types{ + {"name", std::make_shared()}, + {"id", std::make_shared()}, + {"storage", std::make_shared()}, + {"num_elements", std::make_shared()}, + {"apply_to_all", std::make_shared()}, + {"apply_to_list", std::make_shared(std::make_shared())}, + {"apply_to_except", std::make_shared(std::make_shared())}, + }; + 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 ids = access_control.findAll(); + + size_t column_index = 0; + auto & column_name = assert_cast(*res_columns[column_index++]); + auto & column_id = assert_cast(*res_columns[column_index++]).getData(); + auto & column_storage = assert_cast(*res_columns[column_index++]); + auto & column_num_elements = assert_cast(*res_columns[column_index++]).getData(); + auto & column_apply_to_all = assert_cast(*res_columns[column_index++]).getData(); + auto & column_apply_to_list = assert_cast(assert_cast(*res_columns[column_index]).getData()); + auto & column_apply_to_list_offsets = assert_cast(*res_columns[column_index++]).getOffsets(); + auto & column_apply_to_except = assert_cast(assert_cast(*res_columns[column_index]).getData()); + auto & column_apply_to_except_offsets = assert_cast(*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(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); + } +} + +} diff --git a/src/Storages/System/StorageSystemSettingsProfiles.h b/src/Storages/System/StorageSystemSettingsProfiles.h new file mode 100644 index 00000000000..c6b887c99df --- /dev/null +++ b/src/Storages/System/StorageSystemSettingsProfiles.h @@ -0,0 +1,24 @@ +#pragma once + +#include +#include + + +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, public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemSettingsProfiles"; } + static NamesAndTypesList getNamesAndTypes(); + +protected: + friend struct ext::shared_ptr_helper; + using IStorageSystemOneBlock::IStorageSystemOneBlock; + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 69ad147664f..d30f4f077c8 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -15,6 +15,7 @@ #include #include #include + #include #include #include @@ -25,13 +26,8 @@ #include #include #include -#include -#include -#include -#include #include #include -#include #include #include #include @@ -46,6 +42,14 @@ #include #include +#include +#include +#include +#include +#include +#include +#include + #ifdef OS_LINUX #include #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 diff --git a/src/Storages/ya.make b/src/Storages/ya.make index 2e00dbaec97..85b61aca5a0 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -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 diff --git a/tests/integration/test_settings_profile/test.py b/tests/integration/test_settings_profile/test.py index 8b9d023d56f..9c8b116e6d2 100644 --- a/tests/integration/test_settings_profile/test.py +++ b/tests/integration/test_settings_profile/test.py @@ -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")