From 470582e262c936cf1503548407b2be8e9fddab40 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 12 Feb 2022 00:11:43 +0700 Subject: [PATCH] Change type of RowPolicyKind: bool -> enum. --- src/Access/Common/RowPolicyDefs.cpp | 31 +++++++++++++++++++ src/Access/Common/RowPolicyDefs.h | 21 +++++++++++++ src/Access/RowPolicy.cpp | 2 +- src/Access/RowPolicy.h | 16 +++------- src/Access/RowPolicyCache.cpp | 6 ++-- .../InterpreterCreateRowPolicyQuery.cpp | 4 +-- ...InterpreterShowCreateAccessEntityQuery.cpp | 4 +-- .../Access/ASTCreateRowPolicyQuery.cpp | 8 ++--- src/Parsers/Access/ASTCreateRowPolicyQuery.h | 2 +- .../Access/ParserCreateRowPolicyQuery.cpp | 30 +++++++++--------- .../System/StorageSystemRowPolicies.cpp | 31 ++++++++++++++++--- tests/integration/test_row_policy/test.py | 16 +++++----- .../01295_create_row_policy.reference | 6 ++-- .../0_stateless/01295_create_row_policy.sql | 2 +- .../02117_show_create_table_system.reference | 2 +- 15 files changed, 123 insertions(+), 58 deletions(-) diff --git a/src/Access/Common/RowPolicyDefs.cpp b/src/Access/Common/RowPolicyDefs.cpp index 953709d9519..e1722501eaa 100644 --- a/src/Access/Common/RowPolicyDefs.cpp +++ b/src/Access/Common/RowPolicyDefs.cpp @@ -78,4 +78,35 @@ const RowPolicyFilterTypeInfo & RowPolicyFilterTypeInfo::get(RowPolicyFilterType throw Exception("Unknown type: " + std::to_string(static_cast(type_)), ErrorCodes::LOGICAL_ERROR); } +String toString(RowPolicyKind type) +{ + return RowPolicyKindInfo::get(type).raw_name; +} + +const RowPolicyKindInfo & RowPolicyKindInfo::get(RowPolicyKind kind_) +{ + static constexpr auto make_info = [](const char * raw_name_) + { + String init_name = raw_name_; + boost::to_lower(init_name); + return RowPolicyKindInfo{raw_name_, std::move(init_name)}; + }; + + switch (kind_) + { + case RowPolicyKind::PERMISSIVE: + { + static const auto info = make_info("PERMISSIVE"); + return info; + } + case RowPolicyKind::RESTRICTIVE: + { + static const auto info = make_info("RESTRICTIVE"); + return info; + } + case RowPolicyKind::MAX: break; + } + throw Exception("Unknown kind: " + std::to_string(static_cast(kind_)), ErrorCodes::LOGICAL_ERROR); +} + } diff --git a/src/Access/Common/RowPolicyDefs.h b/src/Access/Common/RowPolicyDefs.h index 792884c56df..3fa684bcc36 100644 --- a/src/Access/Common/RowPolicyDefs.h +++ b/src/Access/Common/RowPolicyDefs.h @@ -54,4 +54,25 @@ struct RowPolicyFilterTypeInfo static const RowPolicyFilterTypeInfo & get(RowPolicyFilterType type); }; + +/// Kinds of row policies. It affects how row policies are applied. +/// A row is only accessible if at least one of the permissive policies passes, +/// in addition to all the restrictive policies. +enum class RowPolicyKind +{ + PERMISSIVE, + RESTRICTIVE, + + MAX, +}; + +String toString(RowPolicyKind kind); + +struct RowPolicyKindInfo +{ + const char * const raw_name; + const String name; /// Lowercased with underscores, e.g. "permissive". + static const RowPolicyKindInfo & get(RowPolicyKind kind); +}; + } diff --git a/src/Access/RowPolicy.cpp b/src/Access/RowPolicy.cpp index c09675e0e34..ac532efe3e9 100644 --- a/src/Access/RowPolicy.cpp +++ b/src/Access/RowPolicy.cpp @@ -55,7 +55,7 @@ bool RowPolicy::equal(const IAccessEntity & other) const return false; const auto & other_policy = typeid_cast(other); return (full_name == other_policy.full_name) && boost::range::equal(filters, other_policy.filters) - && restrictive == other_policy.restrictive && (to_roles == other_policy.to_roles); + && (kind == other_policy.kind) && (to_roles == other_policy.to_roles); } } diff --git a/src/Access/RowPolicy.h b/src/Access/RowPolicy.h index 9c143aff725..2e853590493 100644 --- a/src/Access/RowPolicy.h +++ b/src/Access/RowPolicy.h @@ -29,17 +29,9 @@ struct RowPolicy : public IAccessEntity /// for user or available for modification. std::array(RowPolicyFilterType::MAX)> filters; - /// Sets that the policy is permissive. - /// A row is only accessible if at least one of the permissive policies passes, - /// in addition to all the restrictive policies. - void setPermissive(bool permissive_ = true) { setRestrictive(!permissive_); } - bool isPermissive() const { return !isRestrictive(); } - - /// Sets that the policy is restrictive. - /// A row is only accessible if at least one of the permissive policies passes, - /// in addition to all the restrictive policies. - void setRestrictive(bool restrictive_ = true) { restrictive = restrictive_; } - bool isRestrictive() const { return restrictive; } + /// Sets the kind of the policy, it affects how row policies are applied. + void setKind(RowPolicyKind kind_) { kind = kind_; } + RowPolicyKind getKind() const { return kind; } bool equal(const IAccessEntity & other) const override; std::shared_ptr clone() const override { return cloneImpl(); } @@ -53,7 +45,7 @@ private: void setName(const String &) override; RowPolicyName full_name; - bool restrictive = false; + RowPolicyKind kind = RowPolicyKind::PERMISSIVE; }; using RowPolicyPtr = std::shared_ptr; diff --git a/src/Access/RowPolicyCache.cpp b/src/Access/RowPolicyCache.cpp index 55bec427158..ceeb3dbb072 100644 --- a/src/Access/RowPolicyCache.cpp +++ b/src/Access/RowPolicyCache.cpp @@ -20,9 +20,9 @@ namespace class FiltersMixer { public: - void add(const ASTPtr & filter, bool is_restrictive) + void add(const ASTPtr & filter, RowPolicyKind kind) { - if (is_restrictive) + if (kind == RowPolicyKind::RESTRICTIVE) restrictions.push_back(filter); else permissions.push_back(filter); @@ -224,7 +224,7 @@ void RowPolicyCache::mixFiltersFor(EnabledRowPolicies & enabled) auto & mixer = mixers[key]; mixer.database_and_table_name = info.database_and_table_name; if (match) - mixer.mixer.add(info.parsed_filters[filter_type_i], policy.isRestrictive()); + mixer.mixer.add(info.parsed_filters[filter_type_i], policy.getKind()); } } } diff --git a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp index 72b4b149bd7..af7ec68ef28 100644 --- a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp @@ -29,8 +29,8 @@ namespace else if (query.names->full_names.size() == 1) policy.setFullName(query.names->full_names.front()); - if (query.is_restrictive) - policy.setRestrictive(*query.is_restrictive); + if (query.kind) + policy.setKind(*query.kind); for (const auto & [filter_type, filter] : query.filters) policy.filters[static_cast(filter_type)] = filter ? serializeAST(*filter) : String{}; diff --git a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp index 27345218e07..7923c348cd2 100644 --- a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp @@ -190,8 +190,8 @@ namespace query->names->full_names.emplace_back(policy.getFullName()); query->attach = attach_mode; - if (policy.isRestrictive()) - query->is_restrictive = policy.isRestrictive(); + if (policy.getKind() != RowPolicyKind::PERMISSIVE) + query->kind = policy.getKind(); for (auto type : collections::range(RowPolicyFilterType::MAX)) { diff --git a/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp b/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp index d968fdd3250..cb07f0ed0cb 100644 --- a/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp @@ -20,10 +20,10 @@ namespace } - void formatAsRestrictiveOrPermissive(bool is_restrictive, const IAST::FormatSettings & settings) + void formatAsKind(RowPolicyKind kind, const IAST::FormatSettings & settings) { settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " AS " << (settings.hilite ? IAST::hilite_none : "") - << (is_restrictive ? "restrictive" : "permissive"); + << RowPolicyKindInfo::get(kind).name; } @@ -156,8 +156,8 @@ void ASTCreateRowPolicyQuery::formatImpl(const FormatSettings & settings, Format if (!new_short_name.empty()) formatRenameTo(new_short_name, settings); - if (is_restrictive) - formatAsRestrictiveOrPermissive(*is_restrictive, settings); + if (kind) + formatAsKind(*kind, settings); formatForClauses(filters, alter, settings); diff --git a/src/Parsers/Access/ASTCreateRowPolicyQuery.h b/src/Parsers/Access/ASTCreateRowPolicyQuery.h index dc698c25c6d..a98a676e477 100644 --- a/src/Parsers/Access/ASTCreateRowPolicyQuery.h +++ b/src/Parsers/Access/ASTCreateRowPolicyQuery.h @@ -39,7 +39,7 @@ public: std::shared_ptr names; String new_short_name; - std::optional is_restrictive; + std::optional kind; std::vector> filters; /// `nullptr` means set to NONE. std::shared_ptr roles; diff --git a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp index 83156c6a8e1..668caf54f7f 100644 --- a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp @@ -29,24 +29,24 @@ namespace }); } - bool parseAsRestrictiveOrPermissive(IParserBase::Pos & pos, Expected & expected, bool & is_restrictive) + bool parseAsKind(IParserBase::Pos & pos, Expected & expected, RowPolicyKind & kind) { return IParserBase::wrapParseImpl(pos, [&] { if (!ParserKeyword{"AS"}.ignore(pos, expected)) return false; - if (ParserKeyword{"RESTRICTIVE"}.ignore(pos, expected)) + for (auto current_kind : collections::range(RowPolicyKind::MAX)) { - is_restrictive = true; - return true; + const std::string_view & kind_name = RowPolicyKindInfo::get(current_kind).name; + if (ParserKeyword{kind_name.data()}.ignore(pos, expected)) + { + kind = current_kind; + return true; + } } - if (!ParserKeyword{"PERMISSIVE"}.ignore(pos, expected)) - return false; - - is_restrictive = false; - return true; + return false; }); } @@ -243,7 +243,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & String cluster = std::exchange(names->cluster, ""); String new_short_name; - std::optional is_restrictive; + std::optional kind; std::vector> filters; while (true) @@ -251,12 +251,12 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & if (alter && (names->full_names.size() == 1) && new_short_name.empty() && parseRenameTo(pos, expected, new_short_name)) continue; - if (!is_restrictive) + if (!kind) { - bool new_is_restrictive; - if (parseAsRestrictiveOrPermissive(pos, expected, new_is_restrictive)) + RowPolicyKind new_kind; + if (parseAsKind(pos, expected, new_kind)) { - is_restrictive = new_is_restrictive; + kind = new_kind; continue; } } @@ -291,7 +291,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & query->cluster = std::move(cluster); query->names = std::move(names); query->new_short_name = std::move(new_short_name); - query->is_restrictive = is_restrictive; + query->kind = kind; query->filters = std::move(filters); query->roles = std::move(roles); diff --git a/src/Storages/System/StorageSystemRowPolicies.cpp b/src/Storages/System/StorageSystemRowPolicies.cpp index cd4f3dab109..4afe97359b1 100644 --- a/src/Storages/System/StorageSystemRowPolicies.cpp +++ b/src/Storages/System/StorageSystemRowPolicies.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -19,6 +20,26 @@ namespace DB { + +namespace +{ + const std::vector> & getRowPolicyKindEnumValues() + { + static const std::vector> values = [] + { + std::vector> res; + for (auto kind : collections::range(RowPolicyKind::MAX)) + { + const std::string_view & kind_name = RowPolicyKindInfo::get(kind).name; + res.emplace_back(kind_name, static_cast(kind)); + } + return res; + }(); + return values; + } +} + + NamesAndTypesList StorageSystemRowPolicies::getNamesAndTypes() { NamesAndTypesList names_and_types{ @@ -37,7 +58,7 @@ NamesAndTypesList StorageSystemRowPolicies::getNamesAndTypes() } NamesAndTypesList extra_names_and_types{ - {"is_restrictive", std::make_shared()}, + {"kind", std::make_shared(getRowPolicyKindEnumValues())}, {"apply_to_all", std::make_shared()}, {"apply_to_list", std::make_shared(std::make_shared())}, {"apply_to_except", std::make_shared(std::make_shared())} @@ -72,7 +93,7 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr column_filter_null_map[filter_type_i] = &assert_cast(*res_columns[column_index++]).getNullMapData(); } - auto & column_is_restrictive = assert_cast(*res_columns[column_index++]).getData(); + auto & column_kind = 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(); @@ -84,7 +105,7 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr const UUID & id, const String & storage_name, const std::array(RowPolicyFilterType::MAX)> & filters, - bool is_restrictive, + RowPolicyKind kind, const RolesOrUsersSet & apply_to) { column_name.insertData(name.data(), name.length()); @@ -110,7 +131,7 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr } } - column_is_restrictive.push_back(is_restrictive); + column_kind.push_back(static_cast(kind)); auto apply_to_ast = apply_to.toASTWithNames(access_control); column_apply_to_all.push_back(apply_to_ast->all); @@ -133,7 +154,7 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr if (!storage) continue; - add_row(policy->getName(), policy->getFullName(), id, storage->getStorageName(), policy->filters, policy->isRestrictive(), policy->to_roles); + add_row(policy->getName(), policy->getFullName(), id, storage->getStorageName(), policy->filters, policy->getKind(), policy->to_roles); } } } diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index 0a7f6958b4a..c00c559c3ce 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -256,21 +256,21 @@ def test_reload_users_xml_by_timer(): def test_introspection(): policies = [ ["another ON mydb.filtered_table1", "another", "mydb", "filtered_table1", - "6068883a-0e9d-f802-7e22-0144f8e66d3c", "users.xml", "1", 0, 0, "['another']", "[]"], + "6068883a-0e9d-f802-7e22-0144f8e66d3c", "users.xml", "1", "permissive", 0, "['another']", "[]"], ["another ON mydb.filtered_table2", "another", "mydb", "filtered_table2", - "c019e957-c60b-d54e-cc52-7c90dac5fb01", "users.xml", "1", 0, 0, "['another']", "[]"], + "c019e957-c60b-d54e-cc52-7c90dac5fb01", "users.xml", "1", "permissive", 0, "['another']", "[]"], ["another ON mydb.filtered_table3", "another", "mydb", "filtered_table3", - "4cb080d0-44e8-dbef-6026-346655143628", "users.xml", "1", 0, 0, "['another']", "[]"], + "4cb080d0-44e8-dbef-6026-346655143628", "users.xml", "1", "permissive", 0, "['another']", "[]"], ["another ON mydb.local", "another", "mydb", "local", "5b23c389-7e18-06bf-a6bc-dd1afbbc0a97", "users.xml", - "a = 1", 0, 0, "['another']", "[]"], + "a = 1", "permissive", 0, "['another']", "[]"], ["default ON mydb.filtered_table1", "default", "mydb", "filtered_table1", - "9e8a8f62-4965-2b5e-8599-57c7b99b3549", "users.xml", "a = 1", 0, 0, "['default']", "[]"], + "9e8a8f62-4965-2b5e-8599-57c7b99b3549", "users.xml", "a = 1", "permissive", 0, "['default']", "[]"], ["default ON mydb.filtered_table2", "default", "mydb", "filtered_table2", - "cffae79d-b9bf-a2ef-b798-019c18470b25", "users.xml", "a + b < 1 or c - d > 5", 0, 0, "['default']", "[]"], + "cffae79d-b9bf-a2ef-b798-019c18470b25", "users.xml", "a + b < 1 or c - d > 5", "permissive", 0, "['default']", "[]"], ["default ON mydb.filtered_table3", "default", "mydb", "filtered_table3", - "12fc5cef-e3da-3940-ec79-d8be3911f42b", "users.xml", "c = 1", 0, 0, "['default']", "[]"], + "12fc5cef-e3da-3940-ec79-d8be3911f42b", "users.xml", "c = 1", "permissive", 0, "['default']", "[]"], ["default ON mydb.local", "default", "mydb", "local", "cdacaeb5-1d97-f99d-2bb0-4574f290629c", "users.xml", "1", - 0, 0, "['default']", "[]"] + "permissive", 0, "['default']", "[]"] ] assert node.query("SELECT * from system.row_policies ORDER BY short_name, database, table") == TSV(policies) diff --git a/tests/queries/0_stateless/01295_create_row_policy.reference b/tests/queries/0_stateless/01295_create_row_policy.reference index 6e3169b7fec..ea7041a9865 100644 --- a/tests/queries/0_stateless/01295_create_row_policy.reference +++ b/tests/queries/0_stateless/01295_create_row_policy.reference @@ -30,6 +30,6 @@ CREATE ROW POLICY p5_01295 ON db2.table2 FOR SELECT USING a = b CREATE ROW POLICY p1_01295 ON db.table FOR SELECT USING 1 TO ALL CREATE ROW POLICY p2_01295 ON db.table FOR SELECT USING 1 TO ALL -- system.row_policies -p1_01295 ON db.table p1_01295 db table local directory (a < b) AND (c > d) 0 0 [] [] -p2_01295 ON db.table p2_01295 db table local directory id = currentUser() 1 0 ['u1_01295'] [] -p3_01295 ON db.table p3_01295 db table local directory 1 0 1 [] ['r1_01295'] +p1_01295 ON db.table p1_01295 db table local directory (a < b) AND (c > d) permissive 0 [] [] +p2_01295 ON db.table p2_01295 db table local directory id = currentUser() restrictive 0 ['u1_01295'] [] +p3_01295 ON db.table p3_01295 db table local directory 1 permissive 1 [] ['r1_01295'] diff --git a/tests/queries/0_stateless/01295_create_row_policy.sql b/tests/queries/0_stateless/01295_create_row_policy.sql index b484d0ea0f3..28fbf106f07 100644 --- a/tests/queries/0_stateless/01295_create_row_policy.sql +++ b/tests/queries/0_stateless/01295_create_row_policy.sql @@ -72,7 +72,7 @@ SELECT '-- system.row_policies'; CREATE ROW POLICY p1_01295 ON db.table USING ad; CREATE ROW POLICY p2_01295 ON db.table USING id=currentUser() AS RESTRICTIVE TO u1_01295; CREATE ROW POLICY p3_01295 ON db.table USING 1 AS PERMISSIVE TO ALL EXCEPT r1_01295; -SELECT name, short_name, database, table, storage, select_filter, is_restrictive, apply_to_all, apply_to_list, apply_to_except from system.row_policies WHERE short_name LIKE 'p%\_01295' ORDER BY name; +SELECT name, short_name, database, table, storage, select_filter, kind, apply_to_all, apply_to_list, apply_to_except from system.row_policies WHERE short_name LIKE 'p%\_01295' ORDER BY name; DROP ROW POLICY p1_01295, p2_01295, p3_01295 ON db.table; DROP ROLE r1_01295; diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index cecdd0498b1..b1328309ae2 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -49,7 +49,7 @@ CREATE TABLE system.replicated_merge_tree_settings\n(\n `name` String,\n ` CREATE TABLE system.replication_queue\n(\n `database` String,\n `table` String,\n `replica_name` String,\n `position` UInt32,\n `node_name` String,\n `type` String,\n `create_time` DateTime,\n `required_quorum` UInt32,\n `source_replica` String,\n `new_part_name` String,\n `parts_to_merge` Array(String),\n `is_detach` UInt8,\n `is_currently_executing` UInt8,\n `num_tries` UInt32,\n `last_exception` String,\n `last_attempt_time` DateTime,\n `num_postponed` UInt32,\n `postpone_reason` String,\n `last_postpone_time` DateTime,\n `merge_type` String\n)\nENGINE = SystemReplicationQueue()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.role_grants\n(\n `user_name` Nullable(String),\n `role_name` Nullable(String),\n `granted_role_name` String,\n `granted_role_is_default` UInt8,\n `with_admin_option` UInt8\n)\nENGINE = SystemRoleGrants()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.roles\n(\n `name` String,\n `id` UUID,\n `storage` String\n)\nENGINE = SystemRoles()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.row_policies\n(\n `name` String,\n `short_name` String,\n `database` String,\n `table` String,\n `id` UUID,\n `storage` String,\n `select_filter` Nullable(String),\n `is_restrictive` UInt8,\n `apply_to_all` UInt8,\n `apply_to_list` Array(String),\n `apply_to_except` Array(String)\n)\nENGINE = SystemRowPolicies()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.row_policies\n(\n `name` String,\n `short_name` String,\n `database` String,\n `table` String,\n `id` UUID,\n `storage` String,\n `select_filter` Nullable(String),\n `kind` Enum8(\'permissive\' = 0, \'restrictive\' = 1),\n `apply_to_all` UInt8,\n `apply_to_list` Array(String),\n `apply_to_except` Array(String)\n)\nENGINE = SystemRowPolicies()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.settings\n(\n `name` String,\n `value` String,\n `changed` UInt8,\n `description` String,\n `min` Nullable(String),\n `max` Nullable(String),\n `readonly` UInt8,\n `type` String\n)\nENGINE = SystemSettings()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.settings_profile_elements\n(\n `profile_name` Nullable(String),\n `user_name` Nullable(String),\n `role_name` Nullable(String),\n `index` UInt64,\n `setting_name` Nullable(String),\n `value` Nullable(String),\n `min` Nullable(String),\n `max` Nullable(String),\n `readonly` Nullable(UInt8),\n `inherit_profile` Nullable(String)\n)\nENGINE = SystemSettingsProfileElements()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.settings_profiles\n(\n `name` String,\n `id` UUID,\n `storage` String,\n `num_elements` UInt64,\n `apply_to_all` UInt8,\n `apply_to_list` Array(String),\n `apply_to_except` Array(String)\n)\nENGINE = SystemSettingsProfiles()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'