mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 09:02:00 +00:00
Merge pull request #34596 from vitlibar/allow-restrictive-without-permissive
Allow restrictive row policies without permissive
This commit is contained in:
commit
874b2c8dcb
@ -78,4 +78,35 @@ const RowPolicyFilterTypeInfo & RowPolicyFilterTypeInfo::get(RowPolicyFilterType
|
|||||||
throw Exception("Unknown type: " + std::to_string(static_cast<size_t>(type_)), ErrorCodes::LOGICAL_ERROR);
|
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);
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -54,4 +54,25 @@ struct RowPolicyFilterTypeInfo
|
|||||||
static const RowPolicyFilterTypeInfo & get(RowPolicyFilterType type);
|
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);
|
||||||
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -55,7 +55,7 @@ bool RowPolicy::equal(const IAccessEntity & other) const
|
|||||||
return false;
|
return false;
|
||||||
const auto & other_policy = typeid_cast<const RowPolicy &>(other);
|
const auto & other_policy = typeid_cast<const RowPolicy &>(other);
|
||||||
return (full_name == other_policy.full_name) && boost::range::equal(filters, other_policy.filters)
|
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);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -29,17 +29,9 @@ struct RowPolicy : public IAccessEntity
|
|||||||
/// for user or available for modification.
|
/// for user or available for modification.
|
||||||
std::array<String, static_cast<size_t>(RowPolicyFilterType::MAX)> filters;
|
std::array<String, static_cast<size_t>(RowPolicyFilterType::MAX)> filters;
|
||||||
|
|
||||||
/// Sets that the policy is permissive.
|
/// Sets the kind of the policy, it affects how row policies are applied.
|
||||||
/// A row is only accessible if at least one of the permissive policies passes,
|
void setKind(RowPolicyKind kind_) { kind = kind_; }
|
||||||
/// in addition to all the restrictive policies.
|
RowPolicyKind getKind() const { return kind; }
|
||||||
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; }
|
|
||||||
|
|
||||||
bool equal(const IAccessEntity & other) const override;
|
bool equal(const IAccessEntity & other) const override;
|
||||||
std::shared_ptr<IAccessEntity> clone() const override { return cloneImpl<RowPolicy>(); }
|
std::shared_ptr<IAccessEntity> clone() const override { return cloneImpl<RowPolicy>(); }
|
||||||
@ -53,7 +45,7 @@ private:
|
|||||||
void setName(const String &) override;
|
void setName(const String &) override;
|
||||||
|
|
||||||
RowPolicyName full_name;
|
RowPolicyName full_name;
|
||||||
bool restrictive = false;
|
RowPolicyKind kind = RowPolicyKind::PERMISSIVE;
|
||||||
};
|
};
|
||||||
|
|
||||||
using RowPolicyPtr = std::shared_ptr<const RowPolicy>;
|
using RowPolicyPtr = std::shared_ptr<const RowPolicy>;
|
||||||
|
@ -20,21 +20,34 @@ namespace
|
|||||||
class FiltersMixer
|
class FiltersMixer
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
void add(const ASTPtr & filter, bool is_restrictive)
|
void add(const ASTPtr & filter, RowPolicyKind kind)
|
||||||
{
|
{
|
||||||
if (is_restrictive)
|
if (kind == RowPolicyKind::PERMISSIVE)
|
||||||
restrictions.push_back(filter);
|
{
|
||||||
|
setPermissiveFiltersExist();
|
||||||
|
permissive_filters.push_back(filter);
|
||||||
|
}
|
||||||
else
|
else
|
||||||
permissions.push_back(filter);
|
{
|
||||||
|
restrictive_filters.push_back(filter);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void setPermissiveFiltersExist()
|
||||||
|
{
|
||||||
|
permissive_filters_exist = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
ASTPtr getResult() &&
|
ASTPtr getResult() &&
|
||||||
{
|
{
|
||||||
/// Process permissive filters.
|
if (permissive_filters_exist)
|
||||||
restrictions.push_back(makeASTForLogicalOr(std::move(permissions)));
|
{
|
||||||
|
/// Process permissive filters.
|
||||||
|
restrictive_filters.push_back(makeASTForLogicalOr(std::move(permissive_filters)));
|
||||||
|
}
|
||||||
|
|
||||||
/// Process restrictive filters.
|
/// Process restrictive filters.
|
||||||
auto result = makeASTForLogicalAnd(std::move(restrictions));
|
auto result = makeASTForLogicalAnd(std::move(restrictive_filters));
|
||||||
|
|
||||||
bool value;
|
bool value;
|
||||||
if (tryGetLiteralBool(result.get(), value) && value)
|
if (tryGetLiteralBool(result.get(), value) && value)
|
||||||
@ -44,8 +57,9 @@ namespace
|
|||||||
}
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
ASTs permissions;
|
ASTs permissive_filters;
|
||||||
ASTs restrictions;
|
bool permissive_filters_exist = false;
|
||||||
|
ASTs restrictive_filters;
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -223,8 +237,13 @@ void RowPolicyCache::mixFiltersFor(EnabledRowPolicies & enabled)
|
|||||||
key.filter_type = filter_type;
|
key.filter_type = filter_type;
|
||||||
auto & mixer = mixers[key];
|
auto & mixer = mixers[key];
|
||||||
mixer.database_and_table_name = info.database_and_table_name;
|
mixer.database_and_table_name = info.database_and_table_name;
|
||||||
|
if (policy.getKind() == RowPolicyKind::PERMISSIVE)
|
||||||
|
{
|
||||||
|
/// We call setPermissiveFiltersExist() even if the current user doesn't match to the current policy's TO clause.
|
||||||
|
mixer.mixer.setPermissiveFiltersExist();
|
||||||
|
}
|
||||||
if (match)
|
if (match)
|
||||||
mixer.mixer.add(info.parsed_filters[filter_type_i], policy.isRestrictive());
|
mixer.mixer.add(info.parsed_filters[filter_type_i], policy.getKind());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -29,8 +29,8 @@ namespace
|
|||||||
else if (query.names->full_names.size() == 1)
|
else if (query.names->full_names.size() == 1)
|
||||||
policy.setFullName(query.names->full_names.front());
|
policy.setFullName(query.names->full_names.front());
|
||||||
|
|
||||||
if (query.is_restrictive)
|
if (query.kind)
|
||||||
policy.setRestrictive(*query.is_restrictive);
|
policy.setKind(*query.kind);
|
||||||
|
|
||||||
for (const auto & [filter_type, filter] : query.filters)
|
for (const auto & [filter_type, filter] : query.filters)
|
||||||
policy.filters[static_cast<size_t>(filter_type)] = filter ? serializeAST(*filter) : String{};
|
policy.filters[static_cast<size_t>(filter_type)] = filter ? serializeAST(*filter) : String{};
|
||||||
|
@ -189,9 +189,7 @@ namespace
|
|||||||
query->names = std::make_shared<ASTRowPolicyNames>();
|
query->names = std::make_shared<ASTRowPolicyNames>();
|
||||||
query->names->full_names.emplace_back(policy.getFullName());
|
query->names->full_names.emplace_back(policy.getFullName());
|
||||||
query->attach = attach_mode;
|
query->attach = attach_mode;
|
||||||
|
query->kind = policy.getKind();
|
||||||
if (policy.isRestrictive())
|
|
||||||
query->is_restrictive = policy.isRestrictive();
|
|
||||||
|
|
||||||
for (auto type : collections::range(RowPolicyFilterType::MAX))
|
for (auto type : collections::range(RowPolicyFilterType::MAX))
|
||||||
{
|
{
|
||||||
|
@ -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 : "")
|
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " AS " << (settings.hilite ? IAST::hilite_none : "")
|
||||||
<< (is_restrictive ? "restrictive" : "permissive");
|
<< RowPolicyKindInfo::get(kind).name;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -156,11 +156,11 @@ void ASTCreateRowPolicyQuery::formatImpl(const FormatSettings & settings, Format
|
|||||||
if (!new_short_name.empty())
|
if (!new_short_name.empty())
|
||||||
formatRenameTo(new_short_name, settings);
|
formatRenameTo(new_short_name, settings);
|
||||||
|
|
||||||
if (is_restrictive)
|
|
||||||
formatAsRestrictiveOrPermissive(*is_restrictive, settings);
|
|
||||||
|
|
||||||
formatForClauses(filters, alter, settings);
|
formatForClauses(filters, alter, settings);
|
||||||
|
|
||||||
|
if (kind)
|
||||||
|
formatAsKind(*kind, settings);
|
||||||
|
|
||||||
if (roles && (!roles->empty() || alter))
|
if (roles && (!roles->empty() || alter))
|
||||||
formatToRoles(*roles, settings);
|
formatToRoles(*roles, settings);
|
||||||
}
|
}
|
||||||
|
@ -39,7 +39,7 @@ public:
|
|||||||
std::shared_ptr<ASTRowPolicyNames> names;
|
std::shared_ptr<ASTRowPolicyNames> names;
|
||||||
String new_short_name;
|
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::vector<std::pair<RowPolicyFilterType, ASTPtr>> filters; /// `nullptr` means set to NONE.
|
||||||
|
|
||||||
std::shared_ptr<ASTRolesOrUsersSet> roles;
|
std::shared_ptr<ASTRolesOrUsersSet> roles;
|
||||||
|
@ -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, [&]
|
return IParserBase::wrapParseImpl(pos, [&]
|
||||||
{
|
{
|
||||||
if (!ParserKeyword{"AS"}.ignore(pos, expected))
|
if (!ParserKeyword{"AS"}.ignore(pos, expected))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
if (ParserKeyword{"RESTRICTIVE"}.ignore(pos, expected))
|
for (auto current_kind : collections::range(RowPolicyKind::MAX))
|
||||||
{
|
{
|
||||||
is_restrictive = true;
|
const std::string_view & kind_name = RowPolicyKindInfo::get(current_kind).name;
|
||||||
return true;
|
if (ParserKeyword{kind_name.data()}.ignore(pos, expected))
|
||||||
|
{
|
||||||
|
kind = current_kind;
|
||||||
|
return true;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!ParserKeyword{"PERMISSIVE"}.ignore(pos, expected))
|
return false;
|
||||||
return false;
|
|
||||||
|
|
||||||
is_restrictive = false;
|
|
||||||
return true;
|
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -243,7 +243,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
|||||||
String cluster = std::exchange(names->cluster, "");
|
String cluster = std::exchange(names->cluster, "");
|
||||||
|
|
||||||
String new_short_name;
|
String new_short_name;
|
||||||
std::optional<bool> is_restrictive;
|
std::optional<RowPolicyKind> kind;
|
||||||
std::vector<std::pair<RowPolicyFilterType, ASTPtr>> filters;
|
std::vector<std::pair<RowPolicyFilterType, ASTPtr>> filters;
|
||||||
|
|
||||||
while (true)
|
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))
|
if (alter && (names->full_names.size() == 1) && new_short_name.empty() && parseRenameTo(pos, expected, new_short_name))
|
||||||
continue;
|
continue;
|
||||||
|
|
||||||
if (!is_restrictive)
|
if (!kind)
|
||||||
{
|
{
|
||||||
bool new_is_restrictive;
|
RowPolicyKind new_kind;
|
||||||
if (parseAsRestrictiveOrPermissive(pos, expected, new_is_restrictive))
|
if (parseAsKind(pos, expected, new_kind))
|
||||||
{
|
{
|
||||||
is_restrictive = new_is_restrictive;
|
kind = new_kind;
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -291,7 +291,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
|||||||
query->cluster = std::move(cluster);
|
query->cluster = std::move(cluster);
|
||||||
query->names = std::move(names);
|
query->names = std::move(names);
|
||||||
query->new_short_name = std::move(new_short_name);
|
query->new_short_name = std::move(new_short_name);
|
||||||
query->is_restrictive = is_restrictive;
|
query->kind = kind;
|
||||||
query->filters = std::move(filters);
|
query->filters = std::move(filters);
|
||||||
query->roles = std::move(roles);
|
query->roles = std::move(roles);
|
||||||
|
|
||||||
|
@ -10,6 +10,7 @@
|
|||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
#include <DataTypes/DataTypeUUID.h>
|
#include <DataTypes/DataTypeUUID.h>
|
||||||
#include <DataTypes/DataTypeNullable.h>
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
|
#include <DataTypes/DataTypeEnum.h>
|
||||||
#include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Parsers/Access/ASTRolesOrUsersSet.h>
|
#include <Parsers/Access/ASTRolesOrUsersSet.h>
|
||||||
@ -19,6 +20,26 @@
|
|||||||
|
|
||||||
namespace DB
|
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 StorageSystemRowPolicies::getNamesAndTypes()
|
||||||
{
|
{
|
||||||
NamesAndTypesList names_and_types{
|
NamesAndTypesList names_and_types{
|
||||||
@ -37,7 +58,7 @@ NamesAndTypesList StorageSystemRowPolicies::getNamesAndTypes()
|
|||||||
}
|
}
|
||||||
|
|
||||||
NamesAndTypesList extra_names_and_types{
|
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_all", std::make_shared<DataTypeUInt8>()},
|
||||||
{"apply_to_list", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
{"apply_to_list", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||||
{"apply_to_except", 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();
|
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_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 = 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_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 UUID & id,
|
||||||
const String & storage_name,
|
const String & storage_name,
|
||||||
const std::array<String, static_cast<size_t>(RowPolicyFilterType::MAX)> & filters,
|
const std::array<String, static_cast<size_t>(RowPolicyFilterType::MAX)> & filters,
|
||||||
bool is_restrictive,
|
RowPolicyKind kind,
|
||||||
const RolesOrUsersSet & apply_to)
|
const RolesOrUsersSet & apply_to)
|
||||||
{
|
{
|
||||||
column_name.insertData(name.data(), name.length());
|
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);
|
auto apply_to_ast = apply_to.toASTWithNames(access_control);
|
||||||
column_apply_to_all.push_back(apply_to_ast->all);
|
column_apply_to_all.push_back(apply_to_ast->all);
|
||||||
@ -132,7 +153,7 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr
|
|||||||
if (!storage)
|
if (!storage)
|
||||||
continue;
|
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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -42,7 +42,7 @@ def test_create():
|
|||||||
assert instance.query(
|
assert instance.query(
|
||||||
"SHOW CREATE USER u2") == "CREATE USER u2 IDENTIFIED WITH sha256_password HOST LOCAL DEFAULT ROLE rx\n"
|
"SHOW CREATE USER u2") == "CREATE USER u2 IDENTIFIED WITH sha256_password HOST LOCAL DEFAULT ROLE rx\n"
|
||||||
assert instance.query(
|
assert instance.query(
|
||||||
"SHOW CREATE ROW POLICY p ON mydb.mytable") == "CREATE ROW POLICY p ON mydb.mytable FOR SELECT USING a < 1000 TO u1, u2\n"
|
"SHOW CREATE ROW POLICY p ON mydb.mytable") == "CREATE ROW POLICY p ON mydb.mytable FOR SELECT USING a < 1000 AS permissive TO u1, u2\n"
|
||||||
assert instance.query(
|
assert instance.query(
|
||||||
"SHOW CREATE QUOTA q") == "CREATE QUOTA q FOR INTERVAL 1 hour MAX queries = 100 TO ALL EXCEPT rx\n"
|
"SHOW CREATE QUOTA q") == "CREATE QUOTA q FOR INTERVAL 1 hour MAX queries = 100 TO ALL EXCEPT rx\n"
|
||||||
assert instance.query("SHOW GRANTS FOR u1") == ""
|
assert instance.query("SHOW GRANTS FOR u1") == ""
|
||||||
|
@ -256,21 +256,21 @@ def test_reload_users_xml_by_timer():
|
|||||||
def test_introspection():
|
def test_introspection():
|
||||||
policies = [
|
policies = [
|
||||||
["another ON mydb.filtered_table1", "another", "mydb", "filtered_table1",
|
["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",
|
["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",
|
["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",
|
["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",
|
["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",
|
["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",
|
["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",
|
["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)
|
assert node.query("SELECT * from system.row_policies ORDER BY short_name, database, table") == TSV(policies)
|
||||||
|
|
||||||
@ -292,49 +292,49 @@ def test_dcl_introspection():
|
|||||||
"default ON mydb.local"])
|
"default ON mydb.local"])
|
||||||
|
|
||||||
assert node.query(
|
assert node.query(
|
||||||
"SHOW CREATE POLICY default ON mydb.filtered_table1") == "CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default\n"
|
"SHOW CREATE POLICY default ON mydb.filtered_table1") == "CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 AS permissive TO default\n"
|
||||||
assert node.query(
|
assert node.query(
|
||||||
"SHOW CREATE POLICY default ON mydb.filtered_table2") == "CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default\n"
|
"SHOW CREATE POLICY default ON mydb.filtered_table2") == "CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) AS permissive TO default\n"
|
||||||
assert node.query(
|
assert node.query(
|
||||||
"SHOW CREATE POLICY default ON mydb.filtered_table3") == "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default\n"
|
"SHOW CREATE POLICY default ON mydb.filtered_table3") == "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 AS permissive TO default\n"
|
||||||
assert node.query(
|
assert node.query(
|
||||||
"SHOW CREATE POLICY default ON mydb.local") == "CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default\n"
|
"SHOW CREATE POLICY default ON mydb.local") == "CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 AS permissive TO default\n"
|
||||||
|
|
||||||
assert node.query("SHOW CREATE POLICY default") == TSV(
|
assert node.query("SHOW CREATE POLICY default") == TSV(
|
||||||
["CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default",
|
["CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 AS permissive TO default",
|
||||||
"CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default",
|
"CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) AS permissive TO default",
|
||||||
"CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default",
|
"CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 AS permissive TO default",
|
||||||
"CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default"])
|
"CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 AS permissive TO default"])
|
||||||
assert node.query("SHOW CREATE POLICIES ON mydb.filtered_table1") == TSV(
|
assert node.query("SHOW CREATE POLICIES ON mydb.filtered_table1") == TSV(
|
||||||
["CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 TO another",
|
["CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 AS permissive TO another",
|
||||||
"CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default"])
|
"CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 AS permissive TO default"])
|
||||||
assert node.query("SHOW CREATE POLICIES ON mydb.*") == TSV(
|
assert node.query("SHOW CREATE POLICIES ON mydb.*") == TSV(
|
||||||
["CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 TO another",
|
["CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 AS permissive TO another",
|
||||||
"CREATE ROW POLICY another ON mydb.filtered_table2 FOR SELECT USING 1 TO another",
|
"CREATE ROW POLICY another ON mydb.filtered_table2 FOR SELECT USING 1 AS permissive TO another",
|
||||||
"CREATE ROW POLICY another ON mydb.filtered_table3 FOR SELECT USING 1 TO another",
|
"CREATE ROW POLICY another ON mydb.filtered_table3 FOR SELECT USING 1 AS permissive TO another",
|
||||||
"CREATE ROW POLICY another ON mydb.local FOR SELECT USING a = 1 TO another",
|
"CREATE ROW POLICY another ON mydb.local FOR SELECT USING a = 1 AS permissive TO another",
|
||||||
"CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default",
|
"CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 AS permissive TO default",
|
||||||
"CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default",
|
"CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) AS permissive TO default",
|
||||||
"CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default",
|
"CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 AS permissive TO default",
|
||||||
"CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default"])
|
"CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 AS permissive TO default"])
|
||||||
assert node.query("SHOW CREATE POLICIES") == TSV(
|
assert node.query("SHOW CREATE POLICIES") == TSV(
|
||||||
["CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 TO another",
|
["CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 AS permissive TO another",
|
||||||
"CREATE ROW POLICY another ON mydb.filtered_table2 FOR SELECT USING 1 TO another",
|
"CREATE ROW POLICY another ON mydb.filtered_table2 FOR SELECT USING 1 AS permissive TO another",
|
||||||
"CREATE ROW POLICY another ON mydb.filtered_table3 FOR SELECT USING 1 TO another",
|
"CREATE ROW POLICY another ON mydb.filtered_table3 FOR SELECT USING 1 AS permissive TO another",
|
||||||
"CREATE ROW POLICY another ON mydb.local FOR SELECT USING a = 1 TO another",
|
"CREATE ROW POLICY another ON mydb.local FOR SELECT USING a = 1 AS permissive TO another",
|
||||||
"CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default",
|
"CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 AS permissive TO default",
|
||||||
"CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default",
|
"CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) AS permissive TO default",
|
||||||
"CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default",
|
"CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 AS permissive TO default",
|
||||||
"CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default"])
|
"CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 AS permissive TO default"])
|
||||||
|
|
||||||
expected_access = "CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 TO another\n" \
|
expected_access = "CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 AS permissive TO another\n" \
|
||||||
"CREATE ROW POLICY another ON mydb.filtered_table2 FOR SELECT USING 1 TO another\n" \
|
"CREATE ROW POLICY another ON mydb.filtered_table2 FOR SELECT USING 1 AS permissive TO another\n" \
|
||||||
"CREATE ROW POLICY another ON mydb.filtered_table3 FOR SELECT USING 1 TO another\n" \
|
"CREATE ROW POLICY another ON mydb.filtered_table3 FOR SELECT USING 1 AS permissive TO another\n" \
|
||||||
"CREATE ROW POLICY another ON mydb.local FOR SELECT USING a = 1 TO another\n" \
|
"CREATE ROW POLICY another ON mydb.local FOR SELECT USING a = 1 AS permissive TO another\n" \
|
||||||
"CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default\n" \
|
"CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 AS permissive TO default\n" \
|
||||||
"CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default\n" \
|
"CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) AS permissive TO default\n" \
|
||||||
"CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default\n" \
|
"CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 AS permissive TO default\n" \
|
||||||
"CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default\n"
|
"CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 AS permissive TO default\n"
|
||||||
assert expected_access in node.query("SHOW ACCESS")
|
assert expected_access in node.query("SHOW ACCESS")
|
||||||
|
|
||||||
copy_policy_xml('all_rows.xml')
|
copy_policy_xml('all_rows.xml')
|
||||||
@ -342,22 +342,22 @@ def test_dcl_introspection():
|
|||||||
["another ON mydb.filtered_table1", "another ON mydb.filtered_table2", "another ON mydb.filtered_table3",
|
["another ON mydb.filtered_table1", "another ON mydb.filtered_table2", "another ON mydb.filtered_table3",
|
||||||
"default ON mydb.filtered_table1", "default ON mydb.filtered_table2", "default ON mydb.filtered_table3"])
|
"default ON mydb.filtered_table1", "default ON mydb.filtered_table2", "default ON mydb.filtered_table3"])
|
||||||
assert node.query(
|
assert node.query(
|
||||||
"SHOW CREATE POLICY default ON mydb.filtered_table1") == "CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING 1 TO default\n"
|
"SHOW CREATE POLICY default ON mydb.filtered_table1") == "CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING 1 AS permissive TO default\n"
|
||||||
assert node.query(
|
assert node.query(
|
||||||
"SHOW CREATE POLICY default ON mydb.filtered_table2") == "CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING 1 TO default\n"
|
"SHOW CREATE POLICY default ON mydb.filtered_table2") == "CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING 1 AS permissive TO default\n"
|
||||||
assert node.query(
|
assert node.query(
|
||||||
"SHOW CREATE POLICY default ON mydb.filtered_table3") == "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING 1 TO default\n"
|
"SHOW CREATE POLICY default ON mydb.filtered_table3") == "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING 1 AS permissive TO default\n"
|
||||||
|
|
||||||
copy_policy_xml('no_rows.xml')
|
copy_policy_xml('no_rows.xml')
|
||||||
assert node.query("SHOW POLICIES") == TSV(
|
assert node.query("SHOW POLICIES") == TSV(
|
||||||
["another ON mydb.filtered_table1", "another ON mydb.filtered_table2", "another ON mydb.filtered_table3",
|
["another ON mydb.filtered_table1", "another ON mydb.filtered_table2", "another ON mydb.filtered_table3",
|
||||||
"default ON mydb.filtered_table1", "default ON mydb.filtered_table2", "default ON mydb.filtered_table3"])
|
"default ON mydb.filtered_table1", "default ON mydb.filtered_table2", "default ON mydb.filtered_table3"])
|
||||||
assert node.query(
|
assert node.query(
|
||||||
"SHOW CREATE POLICY default ON mydb.filtered_table1") == "CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING NULL TO default\n"
|
"SHOW CREATE POLICY default ON mydb.filtered_table1") == "CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING NULL AS permissive TO default\n"
|
||||||
assert node.query(
|
assert node.query(
|
||||||
"SHOW CREATE POLICY default ON mydb.filtered_table2") == "CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING NULL TO default\n"
|
"SHOW CREATE POLICY default ON mydb.filtered_table2") == "CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING NULL AS permissive TO default\n"
|
||||||
assert node.query(
|
assert node.query(
|
||||||
"SHOW CREATE POLICY default ON mydb.filtered_table3") == "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING NULL TO default\n"
|
"SHOW CREATE POLICY default ON mydb.filtered_table3") == "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING NULL AS permissive TO default\n"
|
||||||
|
|
||||||
copy_policy_xml('no_filters.xml')
|
copy_policy_xml('no_filters.xml')
|
||||||
assert node.query("SHOW POLICIES") == ""
|
assert node.query("SHOW POLICIES") == ""
|
||||||
@ -382,7 +382,7 @@ def test_dcl_management():
|
|||||||
assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[1, 0]])
|
assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[1, 0]])
|
||||||
assert node.query("SHOW POLICIES ON mydb.filtered_table1") == "pB\n"
|
assert node.query("SHOW POLICIES ON mydb.filtered_table1") == "pB\n"
|
||||||
assert node.query(
|
assert node.query(
|
||||||
"SHOW CREATE POLICY pB ON mydb.filtered_table1") == "CREATE ROW POLICY pB ON mydb.filtered_table1 FOR SELECT USING a > b TO default\n"
|
"SHOW CREATE POLICY pB ON mydb.filtered_table1") == "CREATE ROW POLICY pB ON mydb.filtered_table1 FOR SELECT USING a > b AS permissive TO default\n"
|
||||||
|
|
||||||
node.query("DROP POLICY pB ON mydb.filtered_table1")
|
node.query("DROP POLICY pB ON mydb.filtered_table1")
|
||||||
assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[0, 0], [0, 1], [1, 0], [1, 1]])
|
assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[0, 0], [0, 1], [1, 0], [1, 1]])
|
||||||
@ -435,6 +435,23 @@ def test_grant_create_row_policy():
|
|||||||
node.query("DROP USER X")
|
node.query("DROP USER X")
|
||||||
|
|
||||||
|
|
||||||
|
def test_some_users_without_policies():
|
||||||
|
copy_policy_xml('no_filters.xml')
|
||||||
|
assert node.query("SHOW POLICIES") == ""
|
||||||
|
node.query("CREATE USER X, Y")
|
||||||
|
node.query("GRANT SELECT ON mydb.filtered_table1 TO X, Y")
|
||||||
|
|
||||||
|
node.query("CREATE POLICY pA ON mydb.filtered_table1 FOR SELECT USING a<b AS permissive TO X")
|
||||||
|
assert node.query("SELECT * FROM mydb.filtered_table1", user='X') == TSV([[0, 1]])
|
||||||
|
assert node.query("SELECT * FROM mydb.filtered_table1", user='Y') == ""
|
||||||
|
|
||||||
|
node.query("ALTER POLICY pA ON mydb.filtered_table1 AS restrictive")
|
||||||
|
assert node.query("SELECT * FROM mydb.filtered_table1", user='X') == TSV([[0, 1]])
|
||||||
|
assert node.query("SELECT * FROM mydb.filtered_table1", user='Y') == TSV([[0, 0], [0, 1], [1, 0], [1, 1]])
|
||||||
|
|
||||||
|
node.query("DROP USER X, Y")
|
||||||
|
|
||||||
|
|
||||||
def test_users_xml_is_readonly():
|
def test_users_xml_is_readonly():
|
||||||
assert re.search("storage is readonly", node.query_and_get_error("DROP POLICY default ON mydb.filtered_table1"))
|
assert re.search("storage is readonly", node.query_and_get_error("DROP POLICY default ON mydb.filtered_table1"))
|
||||||
|
|
||||||
@ -448,10 +465,10 @@ def test_tags_with_db_and_table_names():
|
|||||||
assert node.query("SELECT * FROM mydb.`.filtered_table4`") == TSV([[1, 1]])
|
assert node.query("SELECT * FROM mydb.`.filtered_table4`") == TSV([[1, 1]])
|
||||||
|
|
||||||
assert node.query("SHOW CREATE POLICIES default") == TSV(
|
assert node.query("SHOW CREATE POLICIES default") == TSV(
|
||||||
["CREATE ROW POLICY default ON mydb.`.filtered_table4` FOR SELECT USING c = 2 TO default",
|
["CREATE ROW POLICY default ON mydb.`.filtered_table4` FOR SELECT USING c = 2 AS permissive TO default",
|
||||||
"CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING c > (d + 5) TO default",
|
"CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING c > (d + 5) AS permissive TO default",
|
||||||
"CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 0 TO default",
|
"CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 0 AS permissive TO default",
|
||||||
"CREATE ROW POLICY default ON mydb.table FOR SELECT USING a = 0 TO default"])
|
"CREATE ROW POLICY default ON mydb.table FOR SELECT USING a = 0 AS permissive TO default"])
|
||||||
|
|
||||||
|
|
||||||
def test_miscellaneous_engines():
|
def test_miscellaneous_engines():
|
||||||
|
@ -1,35 +1,35 @@
|
|||||||
-- default
|
-- default
|
||||||
CREATE ROW POLICY p1_01295 ON db.table
|
CREATE ROW POLICY p1_01295 ON db.table AS permissive
|
||||||
-- same as default
|
-- same as default
|
||||||
CREATE ROW POLICY p2_01295 ON db.table
|
CREATE ROW POLICY p2_01295 ON db.table AS permissive
|
||||||
CREATE ROW POLICY p3_01295 ON db.table
|
CREATE ROW POLICY p3_01295 ON db.table AS permissive
|
||||||
-- rename
|
-- rename
|
||||||
CREATE ROW POLICY p2_01295_renamed ON db.table
|
CREATE ROW POLICY p2_01295_renamed ON db.table AS permissive
|
||||||
-- filter
|
-- filter
|
||||||
CREATE ROW POLICY p1_01295 ON db.table FOR SELECT USING (a < b) AND (c > d)
|
CREATE ROW POLICY p1_01295 ON db.table FOR SELECT USING (a < b) AND (c > d) AS permissive
|
||||||
CREATE ROW POLICY p2_01295 ON db.table AS restrictive FOR SELECT USING id = currentUser()
|
CREATE ROW POLICY p2_01295 ON db.table FOR SELECT USING id = currentUser() AS restrictive
|
||||||
CREATE ROW POLICY p3_01295 ON db.table FOR SELECT USING 1
|
CREATE ROW POLICY p3_01295 ON db.table FOR SELECT USING 1 AS permissive
|
||||||
CREATE ROW POLICY p1_01295 ON db.table AS restrictive FOR SELECT USING 0
|
CREATE ROW POLICY p1_01295 ON db.table FOR SELECT USING 0 AS restrictive
|
||||||
-- to roles
|
-- to roles
|
||||||
CREATE ROW POLICY p1_01295 ON db.table
|
CREATE ROW POLICY p1_01295 ON db.table AS permissive
|
||||||
CREATE ROW POLICY p2_01295 ON db.table TO ALL
|
CREATE ROW POLICY p2_01295 ON db.table AS permissive TO ALL
|
||||||
CREATE ROW POLICY p3_01295 ON db.table TO r1_01295
|
CREATE ROW POLICY p3_01295 ON db.table AS permissive TO r1_01295
|
||||||
CREATE ROW POLICY p4_01295 ON db.table TO u1_01295
|
CREATE ROW POLICY p4_01295 ON db.table AS permissive TO u1_01295
|
||||||
CREATE ROW POLICY p5_01295 ON db.table TO r1_01295, u1_01295
|
CREATE ROW POLICY p5_01295 ON db.table AS permissive TO r1_01295, u1_01295
|
||||||
CREATE ROW POLICY p6_01295 ON db.table TO ALL EXCEPT r1_01295
|
CREATE ROW POLICY p6_01295 ON db.table AS permissive TO ALL EXCEPT r1_01295
|
||||||
CREATE ROW POLICY p7_01295 ON db.table TO ALL EXCEPT r1_01295, u1_01295
|
CREATE ROW POLICY p7_01295 ON db.table AS permissive TO ALL EXCEPT r1_01295, u1_01295
|
||||||
CREATE ROW POLICY p1_01295 ON db.table TO u1_01295
|
CREATE ROW POLICY p1_01295 ON db.table AS permissive TO u1_01295
|
||||||
CREATE ROW POLICY p2_01295 ON db.table
|
CREATE ROW POLICY p2_01295 ON db.table AS permissive
|
||||||
-- multiple policies in one command
|
-- multiple policies in one command
|
||||||
CREATE ROW POLICY p1_01295 ON db.table FOR SELECT USING 1
|
CREATE ROW POLICY p1_01295 ON db.table FOR SELECT USING 1 AS permissive
|
||||||
CREATE ROW POLICY p2_01295 ON db.table FOR SELECT USING 1
|
CREATE ROW POLICY p2_01295 ON db.table FOR SELECT USING 1 AS permissive
|
||||||
CREATE ROW POLICY p3_01295 ON db.table TO u1_01295
|
CREATE ROW POLICY p3_01295 ON db.table AS permissive TO u1_01295
|
||||||
CREATE ROW POLICY p3_01295 ON db2.table2 TO u1_01295
|
CREATE ROW POLICY p3_01295 ON db2.table2 AS permissive TO u1_01295
|
||||||
CREATE ROW POLICY p4_01295 ON db.table FOR SELECT USING a = b
|
CREATE ROW POLICY p4_01295 ON db.table FOR SELECT USING a = b AS permissive
|
||||||
CREATE ROW POLICY p5_01295 ON db2.table2 FOR SELECT USING a = b
|
CREATE ROW POLICY p5_01295 ON db2.table2 FOR SELECT USING a = b AS permissive
|
||||||
CREATE ROW POLICY p1_01295 ON db.table FOR SELECT USING 1 TO ALL
|
CREATE ROW POLICY p1_01295 ON db.table FOR SELECT USING 1 AS permissive TO ALL
|
||||||
CREATE ROW POLICY p2_01295 ON db.table FOR SELECT USING 1 TO ALL
|
CREATE ROW POLICY p2_01295 ON db.table FOR SELECT USING 1 AS permissive TO ALL
|
||||||
-- system.row_policies
|
-- system.row_policies
|
||||||
p1_01295 ON db.table p1_01295 db table local directory (a < b) AND (c > d) 0 0 [] []
|
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() 1 0 ['u1_01295'] []
|
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 0 1 [] ['r1_01295']
|
p3_01295 ON db.table p3_01295 db table local directory 1 permissive 1 [] ['r1_01295']
|
||||||
|
@ -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 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 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;
|
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 ROW POLICY p1_01295, p2_01295, p3_01295 ON db.table;
|
||||||
|
|
||||||
DROP ROLE r1_01295;
|
DROP ROLE r1_01295;
|
||||||
|
@ -1,20 +1,20 @@
|
|||||||
-- one policy
|
-- one policy
|
||||||
CREATE ROW POLICY p1_01296 ON db_01296.table
|
CREATE ROW POLICY p1_01296 ON db_01296.table AS permissive
|
||||||
CREATE ROW POLICY p1_01296 ON db_01296.table
|
CREATE ROW POLICY p1_01296 ON db_01296.table AS permissive
|
||||||
CREATE ROW POLICY p1_01296 ON db_01296.table FOR SELECT USING 1
|
CREATE ROW POLICY p1_01296 ON db_01296.table FOR SELECT USING 1 AS permissive
|
||||||
CREATE ROW POLICY p1_01296 ON db_01296.table FOR SELECT USING 1
|
CREATE ROW POLICY p1_01296 ON db_01296.table FOR SELECT USING 1 AS permissive
|
||||||
-- multiple policies
|
-- multiple policies
|
||||||
CREATE ROW POLICY p1_01296 ON db_01296.table FOR SELECT USING 1
|
CREATE ROW POLICY p1_01296 ON db_01296.table FOR SELECT USING 1 AS permissive
|
||||||
CREATE ROW POLICY p2_01296 ON db_01296.table FOR SELECT USING 1
|
CREATE ROW POLICY p2_01296 ON db_01296.table FOR SELECT USING 1 AS permissive
|
||||||
CREATE ROW POLICY p3_01296 ON db_01296.table TO u1_01296
|
CREATE ROW POLICY p3_01296 ON db_01296.table AS permissive TO u1_01296
|
||||||
CREATE ROW POLICY p3_01296 ON db_01296.table2 TO u1_01296
|
CREATE ROW POLICY p3_01296 ON db_01296.table2 AS permissive TO u1_01296
|
||||||
CREATE ROW POLICY p4_01296 ON db_01296.table FOR SELECT USING a = b
|
CREATE ROW POLICY p4_01296 ON db_01296.table FOR SELECT USING a = b AS permissive
|
||||||
CREATE ROW POLICY p5_01296 ON db_01296.table2 FOR SELECT USING a = b
|
CREATE ROW POLICY p5_01296 ON db_01296.table2 FOR SELECT USING a = b AS permissive
|
||||||
CREATE ROW POLICY p1_01296 ON db_01296.table FOR SELECT USING 1
|
CREATE ROW POLICY p1_01296 ON db_01296.table FOR SELECT USING 1 AS permissive
|
||||||
CREATE ROW POLICY p2_01296 ON db_01296.table FOR SELECT USING 1
|
CREATE ROW POLICY p2_01296 ON db_01296.table FOR SELECT USING 1 AS permissive
|
||||||
CREATE ROW POLICY p3_01296 ON db_01296.table TO u1_01296
|
CREATE ROW POLICY p3_01296 ON db_01296.table AS permissive TO u1_01296
|
||||||
CREATE ROW POLICY p3_01296 ON db_01296.table2 TO u1_01296
|
CREATE ROW POLICY p3_01296 ON db_01296.table2 AS permissive TO u1_01296
|
||||||
CREATE ROW POLICY p4_01296 ON db_01296.table FOR SELECT USING a = b
|
CREATE ROW POLICY p4_01296 ON db_01296.table FOR SELECT USING a = b AS permissive
|
||||||
CREATE ROW POLICY p5_01296 ON db_01296.table2 FOR SELECT USING a = b
|
CREATE ROW POLICY p5_01296 ON db_01296.table2 FOR SELECT USING a = b AS permissive
|
||||||
CREATE ROW POLICY p1_01296 ON db_01296.table FOR SELECT USING 1 TO ALL
|
CREATE ROW POLICY p1_01296 ON db_01296.table FOR SELECT USING 1 AS permissive TO ALL
|
||||||
CREATE ROW POLICY p2_01296 ON db_01296.table FOR SELECT USING 1 TO ALL
|
CREATE ROW POLICY p2_01296 ON db_01296.table FOR SELECT USING 1 AS permissive TO ALL
|
||||||
|
@ -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.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.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.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\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_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.\'
|
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.\'
|
||||||
|
@ -0,0 +1,33 @@
|
|||||||
|
None
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
R1: x == 1
|
||||||
|
1
|
||||||
|
R1, R2: (x == 1) OR (x == 2)
|
||||||
|
1
|
||||||
|
2
|
||||||
|
R1, R2, R3: (x == 1) OR (x == 2) OR (x == 3)
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
R1, R2, R3, R4: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2)
|
||||||
|
1
|
||||||
|
2
|
||||||
|
R1, R2, R3, R4, R5: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2) AND (x >= 2)
|
||||||
|
2
|
||||||
|
R2, R3, R4, R5: ((x == 2) OR (x == 3)) AND (x <= 2) AND (x >= 2)
|
||||||
|
2
|
||||||
|
R3, R4, R5: (x == 3) AND (x <= 2) AND (x >= 2)
|
||||||
|
R4, R5: (x <= 2) AND (x >= 2)
|
||||||
|
2
|
||||||
|
R5: (x >= 2)
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
None
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
@ -0,0 +1,55 @@
|
|||||||
|
DROP TABLE IF EXISTS 02131_multiple_row_policies_on_same_column;
|
||||||
|
CREATE TABLE 02131_multiple_row_policies_on_same_column (x UInt8) ENGINE = MergeTree ORDER BY x;
|
||||||
|
INSERT INTO 02131_multiple_row_policies_on_same_column VALUES (1), (2), (3), (4);
|
||||||
|
|
||||||
|
|
||||||
|
DROP ROW POLICY IF EXISTS 02131_filter_1 ON 02131_multiple_row_policies_on_same_column;
|
||||||
|
DROP ROW POLICY IF EXISTS 02131_filter_2 ON 02131_multiple_row_policies_on_same_column;
|
||||||
|
DROP ROW POLICY IF EXISTS 02131_filter_3 ON 02131_multiple_row_policies_on_same_column;
|
||||||
|
DROP ROW POLICY IF EXISTS 02131_filter_4 ON 02131_multiple_row_policies_on_same_column;
|
||||||
|
DROP ROW POLICY IF EXISTS 02131_filter_5 ON 02131_multiple_row_policies_on_same_column;
|
||||||
|
|
||||||
|
SELECT 'None';
|
||||||
|
SELECT * FROM 02131_multiple_row_policies_on_same_column;
|
||||||
|
|
||||||
|
CREATE ROW POLICY 02131_filter_1 ON 02131_multiple_row_policies_on_same_column USING x=1 AS permissive TO ALL;
|
||||||
|
SELECT 'R1: x == 1';
|
||||||
|
SELECT * FROM 02131_multiple_row_policies_on_same_column;
|
||||||
|
|
||||||
|
CREATE ROW POLICY 02131_filter_2 ON 02131_multiple_row_policies_on_same_column USING x=2 AS permissive TO ALL;
|
||||||
|
SELECT 'R1, R2: (x == 1) OR (x == 2)';
|
||||||
|
SELECT * FROM 02131_multiple_row_policies_on_same_column;
|
||||||
|
|
||||||
|
CREATE ROW POLICY 02131_filter_3 ON 02131_multiple_row_policies_on_same_column USING x=3 AS permissive TO ALL;
|
||||||
|
SELECT 'R1, R2, R3: (x == 1) OR (x == 2) OR (x == 3)';
|
||||||
|
SELECT * FROM 02131_multiple_row_policies_on_same_column;
|
||||||
|
|
||||||
|
CREATE ROW POLICY 02131_filter_4 ON 02131_multiple_row_policies_on_same_column USING x<=2 AS restrictive TO ALL;
|
||||||
|
SELECT 'R1, R2, R3, R4: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2)';
|
||||||
|
SELECT * FROM 02131_multiple_row_policies_on_same_column;
|
||||||
|
|
||||||
|
CREATE ROW POLICY 02131_filter_5 ON 02131_multiple_row_policies_on_same_column USING x>=2 AS restrictive TO ALL;
|
||||||
|
SELECT 'R1, R2, R3, R4, R5: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2) AND (x >= 2)';
|
||||||
|
SELECT * FROM 02131_multiple_row_policies_on_same_column;
|
||||||
|
|
||||||
|
DROP ROW POLICY 02131_filter_1 ON 02131_multiple_row_policies_on_same_column;
|
||||||
|
SELECT 'R2, R3, R4, R5: ((x == 2) OR (x == 3)) AND (x <= 2) AND (x >= 2)';
|
||||||
|
SELECT * FROM 02131_multiple_row_policies_on_same_column;
|
||||||
|
|
||||||
|
DROP ROW POLICY 02131_filter_2 ON 02131_multiple_row_policies_on_same_column;
|
||||||
|
SELECT 'R3, R4, R5: (x == 3) AND (x <= 2) AND (x >= 2)';
|
||||||
|
SELECT * FROM 02131_multiple_row_policies_on_same_column;
|
||||||
|
|
||||||
|
DROP ROW POLICY 02131_filter_3 ON 02131_multiple_row_policies_on_same_column;
|
||||||
|
SELECT 'R4, R5: (x <= 2) AND (x >= 2)';
|
||||||
|
SELECT * FROM 02131_multiple_row_policies_on_same_column;
|
||||||
|
|
||||||
|
DROP ROW POLICY 02131_filter_4 ON 02131_multiple_row_policies_on_same_column;
|
||||||
|
SELECT 'R5: (x >= 2)';
|
||||||
|
SELECT * FROM 02131_multiple_row_policies_on_same_column;
|
||||||
|
|
||||||
|
DROP ROW POLICY 02131_filter_5 ON 02131_multiple_row_policies_on_same_column;
|
||||||
|
SELECT 'None';
|
||||||
|
SELECT * FROM 02131_multiple_row_policies_on_same_column;
|
||||||
|
|
||||||
|
DROP TABLE 02131_multiple_row_policies_on_same_column;
|
@ -1,8 +0,0 @@
|
|||||||
4
|
|
||||||
1
|
|
||||||
2
|
|
||||||
3
|
|
||||||
3
|
|
||||||
3
|
|
||||||
3
|
|
||||||
4
|
|
@ -1,30 +0,0 @@
|
|||||||
DROP TABLE IF EXISTS 02131_multiply_row_policies_on_same_column;
|
|
||||||
CREATE TABLE 02131_multiply_row_policies_on_same_column (x UInt8) ENGINE = MergeTree ORDER BY x;
|
|
||||||
INSERT INTO 02131_multiply_row_policies_on_same_column VALUES (1), (2), (3), (4);
|
|
||||||
|
|
||||||
|
|
||||||
DROP ROW POLICY IF EXISTS 02131_filter_1 ON 02131_multiply_row_policies_on_same_column;
|
|
||||||
DROP ROW POLICY IF EXISTS 02131_filter_2 ON 02131_multiply_row_policies_on_same_column;
|
|
||||||
DROP ROW POLICY IF EXISTS 02131_filter_3 ON 02131_multiply_row_policies_on_same_column;
|
|
||||||
SELECT count() FROM 02131_multiply_row_policies_on_same_column;
|
|
||||||
|
|
||||||
|
|
||||||
CREATE ROW POLICY 02131_filter_1 ON 02131_multiply_row_policies_on_same_column USING x=1 TO ALL;
|
|
||||||
SELECT count() FROM 02131_multiply_row_policies_on_same_column;
|
|
||||||
CREATE ROW POLICY 02131_filter_2 ON 02131_multiply_row_policies_on_same_column USING x=2 TO ALL;
|
|
||||||
SELECT count() FROM 02131_multiply_row_policies_on_same_column;
|
|
||||||
CREATE ROW POLICY 02131_filter_3 ON 02131_multiply_row_policies_on_same_column USING x=3 TO ALL;
|
|
||||||
SELECT count() FROM 02131_multiply_row_policies_on_same_column;
|
|
||||||
SELECT count() FROM 02131_multiply_row_policies_on_same_column;
|
|
||||||
|
|
||||||
|
|
||||||
CREATE ROW POLICY 02131_filter_4 ON 02131_multiply_row_policies_on_same_column USING x<4 AS RESTRICTIVE TO ALL;
|
|
||||||
SELECT count() FROM 02131_multiply_row_policies_on_same_column;
|
|
||||||
SELECT count() FROM 02131_multiply_row_policies_on_same_column;
|
|
||||||
|
|
||||||
DROP ROW POLICY 02131_filter_1 ON 02131_multiply_row_policies_on_same_column;
|
|
||||||
DROP ROW POLICY 02131_filter_2 ON 02131_multiply_row_policies_on_same_column;
|
|
||||||
DROP ROW POLICY 02131_filter_3 ON 02131_multiply_row_policies_on_same_column;
|
|
||||||
DROP ROW POLICY 02131_filter_4 ON 02131_multiply_row_policies_on_same_column;
|
|
||||||
SELECT count() FROM 02131_multiply_row_policies_on_same_column;
|
|
||||||
DROP TABLE 02131_multiply_row_policies_on_same_column;
|
|
Loading…
Reference in New Issue
Block a user