Change type of RowPolicyKind: bool -> enum.

This commit is contained in:
Vitaly Baranov 2022-02-12 00:11:43 +07:00
parent bc206ec423
commit 2de6e8e575
14 changed files with 120 additions and 55 deletions

View File

@ -78,4 +78,35 @@ const RowPolicyFilterTypeInfo & RowPolicyFilterTypeInfo::get(RowPolicyFilterType
throw Exception("Unknown type: " + std::to_string(static_cast<size_t>(type_)), ErrorCodes::LOGICAL_ERROR);
}
String toString(RowPolicyKind kind)
{
return RowPolicyKindInfo::get(kind).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<size_t>(kind_)), ErrorCodes::LOGICAL_ERROR);
}
}

View File

@ -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);
};
}

View File

@ -55,7 +55,7 @@ bool RowPolicy::equal(const IAccessEntity & other) const
return false;
const auto & other_policy = typeid_cast<const RowPolicy &>(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);
}
}

View File

@ -29,17 +29,9 @@ struct RowPolicy : public IAccessEntity
/// for user or available for modification.
std::array<String, static_cast<size_t>(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<IAccessEntity> clone() const override { return cloneImpl<RowPolicy>(); }
@ -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<const RowPolicy>;

View File

@ -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());
}
}
}

View File

@ -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<size_t>(filter_type)] = filter ? serializeAST(*filter) : String{};

View File

@ -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))
{

View File

@ -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);

View File

@ -39,7 +39,7 @@ public:
std::shared_ptr<ASTRowPolicyNames> names;
String new_short_name;
std::optional<bool> is_restrictive;
std::optional<RowPolicyKind> kind;
std::vector<std::pair<RowPolicyFilterType, ASTPtr>> filters; /// `nullptr` means set to NONE.
std::shared_ptr<ASTRolesOrUsersSet> roles;

View File

@ -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<bool> is_restrictive;
std::optional<RowPolicyKind> kind;
std::vector<std::pair<RowPolicyFilterType, ASTPtr>> 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);

View File

@ -10,6 +10,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeArray.h>
#include <Interpreters/Context.h>
#include <Parsers/Access/ASTRolesOrUsersSet.h>
@ -19,6 +20,26 @@
namespace DB
{
namespace
{
const std::vector<std::pair<String, Int8>> & getRowPolicyKindEnumValues()
{
static const std::vector<std::pair<String, Int8>> values = []
{
std::vector<std::pair<String, Int8>> 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<Int8>(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<DataTypeUInt8>()},
{"kind", std::make_shared<DataTypeEnum8>(getRowPolicyKindEnumValues())},
{"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>())}
@ -71,7 +92,7 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr
column_filter_null_map[filter_type_i] = &assert_cast<ColumnNullable &>(*res_columns[column_index++]).getNullMapData();
}
auto & column_is_restrictive = assert_cast<ColumnUInt8 &>(*res_columns[column_index++]).getData();
auto & column_kind = assert_cast<ColumnInt8 &>(*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();
@ -83,7 +104,7 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr
const UUID & id,
const String & storage_name,
const std::array<String, static_cast<size_t>(RowPolicyFilterType::MAX)> & filters,
bool is_restrictive,
RowPolicyKind kind,
const RolesOrUsersSet & apply_to)
{
column_name.insertData(name.data(), name.length());
@ -109,7 +130,7 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr
}
}
column_is_restrictive.push_back(is_restrictive);
column_kind.push_back(static_cast<Int8>(kind));
auto apply_to_ast = apply_to.toASTWithNames(access_control);
column_apply_to_all.push_back(apply_to_ast->all);
@ -132,7 +153,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);
}
}
}

View File

@ -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)

View File

@ -72,7 +72,7 @@ SELECT '-- system.row_policies';
CREATE ROW POLICY p1_01295 ON db.table USING a<b AND c>d;
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;

View File

@ -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.\'