Add DCL to manage row policies.

This commit is contained in:
Vitaly Baranov 2019-11-29 20:22:56 +03:00
parent 6baccb963d
commit c3161b7610
31 changed files with 1089 additions and 25 deletions

View File

@ -627,6 +627,13 @@ void Context::checkQuotaManagementIsAllowed()
"User " + client_info.current_user + " doesn't have enough privileges to manage quotas", ErrorCodes::NOT_ENOUGH_PRIVILEGES);
}
void Context::checkRowPolicyManagementIsAllowed()
{
if (!is_row_policy_management_allowed)
throw Exception(
"User " + client_info.current_user + " doesn't have enough privileges to manage row policies", ErrorCodes::NOT_ENOUGH_PRIVILEGES);
}
void Context::setUsersConfig(const ConfigurationPtr & config)
{
auto lock = getLock();
@ -666,6 +673,7 @@ void Context::calculateUserSettings()
client_info.current_user, client_info.current_address.host(), client_info.quota_key);
is_quota_management_allowed = user->is_quota_management_allowed;
row_policy = getAccessControlManager().getRowPolicyContext(client_info.current_user);
is_row_policy_management_allowed = user->is_row_policy_management_allowed;
}

View File

@ -142,6 +142,7 @@ private:
std::shared_ptr<QuotaContext> quota; /// Current quota. By default - empty quota, that have no limits.
bool is_quota_management_allowed = false; /// Whether the current user is allowed to manage quotas via SQL commands.
std::shared_ptr<RowPolicyContext> row_policy;
bool is_row_policy_management_allowed = false; /// Whether the current user is allowed to manage row policies via SQL commands.
String current_database;
Settings settings; /// Setting for query execution.
std::shared_ptr<const SettingsConstraints> settings_constraints;
@ -212,6 +213,7 @@ public:
std::shared_ptr<QuotaContext> getQuota() const { return quota; }
void checkQuotaManagementIsAllowed();
std::shared_ptr<RowPolicyContext> getRowPolicy() const { return row_policy; }
void checkRowPolicyManagementIsAllowed();
/** Take the list of users, quotas and configuration profiles from this config.
* The list of users is completely replaced.

View File

@ -0,0 +1,93 @@
#include <Interpreters/InterpreterCreateRowPolicyQuery.h>
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Parsers/ASTRoleList.h>
#include <Parsers/formatAST.h>
#include <Interpreters/Context.h>
#include <Access/AccessControlManager.h>
#include <boost/range/algorithm/sort.hpp>
namespace DB
{
BlockIO InterpreterCreateRowPolicyQuery::execute()
{
context.checkRowPolicyManagementIsAllowed();
const auto & query = query_ptr->as<const ASTCreateRowPolicyQuery &>();
auto & access_control = context.getAccessControlManager();
if (query.alter)
{
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
{
auto updated_policy = typeid_cast<std::shared_ptr<RowPolicy>>(entity->clone());
updateRowPolicyFromQuery(*updated_policy, query);
return updated_policy;
};
String full_name = query.name_parts.getFullName(context);
if (query.if_exists)
{
if (auto id = access_control.find<RowPolicy>(full_name))
access_control.tryUpdate(*id, update_func);
}
else
access_control.update(access_control.getID<RowPolicy>(full_name), update_func);
}
else
{
auto new_policy = std::make_shared<RowPolicy>();
updateRowPolicyFromQuery(*new_policy, query);
if (query.if_not_exists)
access_control.tryInsert(new_policy);
else if (query.or_replace)
access_control.insertOrReplace(new_policy);
else
access_control.insert(new_policy);
}
return {};
}
void InterpreterCreateRowPolicyQuery::updateRowPolicyFromQuery(RowPolicy & policy, const ASTCreateRowPolicyQuery & query)
{
if (query.alter)
{
if (!query.new_policy_name.empty())
policy.setName(query.new_policy_name);
}
else
{
policy.setDatabase(query.name_parts.database.empty() ? context.getCurrentDatabase() : query.name_parts.database);
policy.setTableName(query.name_parts.table_name);
policy.setName(query.name_parts.policy_name);
}
if (query.is_restrictive)
policy.setRestrictive(*query.is_restrictive);
for (const auto & [index, condition] : query.conditions)
policy.conditions[index] = condition ? serializeAST(*condition) : String{};
if (query.roles)
{
const auto & query_roles = *query.roles;
/// We keep `roles` sorted.
policy.roles = query_roles.roles;
if (query_roles.current_user)
policy.roles.push_back(context.getClientInfo().current_user);
boost::range::sort(policy.roles);
policy.roles.erase(std::unique(policy.roles.begin(), policy.roles.end()), policy.roles.end());
policy.all_roles = query_roles.all_roles;
/// We keep `except_roles` sorted.
policy.except_roles = query_roles.except_roles;
if (query_roles.except_current_user)
policy.except_roles.push_back(context.getClientInfo().current_user);
boost::range::sort(policy.except_roles);
policy.except_roles.erase(std::unique(policy.except_roles.begin(), policy.except_roles.end()), policy.except_roles.end());
}
}
}

View File

@ -0,0 +1,26 @@
#pragma once
#include <Interpreters/IInterpreter.h>
#include <Parsers/IAST_fwd.h>
namespace DB
{
class ASTCreateRowPolicyQuery;
struct RowPolicy;
class InterpreterCreateRowPolicyQuery : public IInterpreter
{
public:
InterpreterCreateRowPolicyQuery(const ASTPtr & query_ptr_, Context & context_) : query_ptr(query_ptr_), context(context_) {}
BlockIO execute() override;
private:
void updateRowPolicyFromQuery(RowPolicy & policy, const ASTCreateRowPolicyQuery & query);
ASTPtr query_ptr;
Context & context;
};
}

View File

@ -3,6 +3,8 @@
#include <Interpreters/Context.h>
#include <Access/AccessControlManager.h>
#include <Access/Quota.h>
#include <Access/RowPolicy.h>
#include <boost/range/algorithm/transform.hpp>
namespace DB
@ -24,6 +26,19 @@ BlockIO InterpreterDropAccessEntityQuery::execute()
access_control.remove(access_control.getIDs<Quota>(query.names));
return {};
}
case Kind::ROW_POLICY:
{
context.checkRowPolicyManagementIsAllowed();
Strings full_names;
boost::range::transform(
query.row_policies_names, std::back_inserter(full_names),
[this](const RowPolicy::FullNameParts & row_policy_name) { return row_policy_name.getFullName(context); });
if (query.if_exists)
access_control.tryRemove(access_control.find<RowPolicy>(full_names));
else
access_control.remove(access_control.getIDs<RowPolicy>(full_names));
return {};
}
}
__builtin_unreachable();

View File

@ -2,6 +2,7 @@
#include <Parsers/ASTCheckQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTCreateQuotaQuery.h>
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Parsers/ASTDropAccessEntityQuery.h>
#include <Parsers/ASTDropQuery.h>
#include <Parsers/ASTInsertQuery.h>
@ -14,6 +15,7 @@
#include <Parsers/ASTShowCreateAccessEntityQuery.h>
#include <Parsers/ASTShowProcesslistQuery.h>
#include <Parsers/ASTShowQuotasQuery.h>
#include <Parsers/ASTShowRowPoliciesQuery.h>
#include <Parsers/ASTShowTablesQuery.h>
#include <Parsers/ASTUseQuery.h>
#include <Parsers/ASTExplainQuery.h>
@ -24,6 +26,7 @@
#include <Interpreters/InterpreterCheckQuery.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/InterpreterCreateQuotaQuery.h>
#include <Interpreters/InterpreterCreateRowPolicyQuery.h>
#include <Interpreters/InterpreterDescribeQuery.h>
#include <Interpreters/InterpreterExplainQuery.h>
#include <Interpreters/InterpreterDropAccessEntityQuery.h>
@ -41,6 +44,7 @@
#include <Interpreters/InterpreterShowCreateQuery.h>
#include <Interpreters/InterpreterShowProcesslistQuery.h>
#include <Interpreters/InterpreterShowQuotasQuery.h>
#include <Interpreters/InterpreterShowRowPoliciesQuery.h>
#include <Interpreters/InterpreterShowTablesQuery.h>
#include <Interpreters/InterpreterSystemQuery.h>
#include <Interpreters/InterpreterUseQuery.h>
@ -199,6 +203,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context &
{
return std::make_unique<InterpreterCreateQuotaQuery>(query, context);
}
else if (query->as<ASTCreateRowPolicyQuery>())
{
return std::make_unique<InterpreterCreateRowPolicyQuery>(query, context);
}
else if (query->as<ASTDropAccessEntityQuery>())
{
return std::make_unique<InterpreterDropAccessEntityQuery>(query, context);
@ -211,6 +219,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context &
{
return std::make_unique<InterpreterShowQuotasQuery>(query, context);
}
else if (query->as<ASTShowRowPoliciesQuery>())
{
return std::make_unique<InterpreterShowRowPoliciesQuery>(query, context);
}
else
throw Exception("Unknown type of query: " + query->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY);
}

View File

@ -1,9 +1,12 @@
#include <Interpreters/InterpreterShowCreateAccessEntityQuery.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTCreateQuotaQuery.h>
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Parsers/ASTShowCreateAccessEntityQuery.h>
#include <Parsers/ASTRoleList.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Access/AccessControlManager.h>
#include <Access/QuotaContext.h>
#include <Columns/ColumnString.h>
@ -28,7 +31,7 @@ BlockInputStreamPtr InterpreterShowCreateAccessEntityQuery::executeImpl()
const auto & show_query = query_ptr->as<ASTShowCreateAccessEntityQuery &>();
/// Build a create query.
ASTPtr create_query = getCreateQuotaQuery(show_query);
ASTPtr create_query = getCreateQuery(show_query);
/// Build the result column.
std::stringstream create_query_ss;
@ -49,6 +52,18 @@ BlockInputStreamPtr InterpreterShowCreateAccessEntityQuery::executeImpl()
}
ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuery(const ASTShowCreateAccessEntityQuery & show_query) const
{
using Kind = ASTShowCreateAccessEntityQuery::Kind;
switch (show_query.kind)
{
case Kind::QUOTA: return getCreateQuotaQuery(show_query);
case Kind::ROW_POLICY: return getCreateRowPolicyQuery(show_query);
}
__builtin_unreachable();
}
ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuotaQuery(const ASTShowCreateAccessEntityQuery & show_query) const
{
auto & access_control = context.getAccessControlManager();
@ -86,4 +101,38 @@ ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuotaQuery(const ASTShow
return create_query;
}
ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateRowPolicyQuery(const ASTShowCreateAccessEntityQuery & show_query) const
{
auto & access_control = context.getAccessControlManager();
RowPolicyPtr policy = access_control.read<RowPolicy>(show_query.row_policy_name.getFullName(context));
auto create_query = std::make_shared<ASTCreateRowPolicyQuery>();
create_query->name_parts = RowPolicy::FullNameParts{policy->getDatabase(), policy->getTableName(), policy->getName()};
if (policy->isRestrictive())
create_query->is_restrictive = policy->isRestrictive();
for (auto index : ext::range_with_static_cast<RowPolicy::ConditionIndex>(RowPolicy::MAX_CONDITION_INDEX))
{
const auto & condition = policy->conditions[index];
if (!condition.empty())
{
ParserExpression parser;
ASTPtr expr = parseQuery(parser, condition, 0);
create_query->conditions.push_back(std::pair{index, expr});
}
}
if (!policy->roles.empty() || policy->all_roles)
{
auto create_query_roles = std::make_shared<ASTRoleList>();
create_query_roles->roles = policy->roles;
create_query_roles->all_roles = policy->all_roles;
create_query_roles->except_roles = policy->except_roles;
create_query->roles = std::move(create_query_roles);
}
return create_query;
}
}

View File

@ -28,7 +28,9 @@ private:
const Context & context;
BlockInputStreamPtr executeImpl();
ASTPtr getCreateQuery(const ASTShowCreateAccessEntityQuery & show_query) const;
ASTPtr getCreateQuotaQuery(const ASTShowCreateAccessEntityQuery & show_query) const;
ASTPtr getCreateRowPolicyQuery(const ASTShowCreateAccessEntityQuery & show_query) const;
};

View File

@ -0,0 +1,68 @@
#include <Interpreters/InterpreterShowRowPoliciesQuery.h>
#include <Parsers/ASTShowRowPoliciesQuery.h>
#include <Parsers/formatAST.h>
#include <Interpreters/executeQuery.h>
#include <Common/quoteString.h>
#include <Interpreters/Context.h>
#include <ext/range.h>
namespace DB
{
InterpreterShowRowPoliciesQuery::InterpreterShowRowPoliciesQuery(const ASTPtr & query_ptr_, Context & context_)
: query_ptr(query_ptr_), context(context_)
{
}
BlockIO InterpreterShowRowPoliciesQuery::execute()
{
return executeQuery(getRewrittenQuery(), context, true);
}
String InterpreterShowRowPoliciesQuery::getRewrittenQuery() const
{
const auto & query = query_ptr->as<ASTShowRowPoliciesQuery &>();
const String & table_name = query.table_name;
String database;
if (!table_name.empty())
{
database = query.database;
if (database.empty())
database = context.getCurrentDatabase();
}
String filter;
if (query.current)
{
if (table_name.empty())
filter = "has(currentRowPolicyIDs(), id)";
else
filter = "has(currentRowPolicyIDs(" + quoteString(database) + ", " + quoteString(table_name) + "), id)";
}
else
{
if (!table_name.empty())
filter = "database = " + quoteString(database) + " AND table = " + quoteString(table_name);
}
String expr = table_name.empty() ? "full_name" : "name";
return "SELECT " + expr + " AS " + backQuote(getResultDescription()) + " from system.row_policies"
+ (filter.empty() ? "" : " WHERE " + filter) + " ORDER BY " + expr;
}
String InterpreterShowRowPoliciesQuery::getResultDescription() const
{
std::stringstream ss;
formatAST(*query_ptr, ss, false, true);
String desc = ss.str();
String prefix = "SHOW ";
if (startsWith(desc, prefix))
desc = desc.substr(prefix.length()); /// `desc` always starts with "SHOW ", so we can trim this prefix.
return desc;
}
}

View File

@ -0,0 +1,25 @@
#pragma once
#include <Interpreters/IInterpreter.h>
#include <Parsers/IAST_fwd.h>
namespace DB
{
class Context;
class InterpreterShowRowPoliciesQuery : public IInterpreter
{
public:
InterpreterShowRowPoliciesQuery(const ASTPtr & query_ptr_, Context & context_);
BlockIO execute() override;
private:
String getRewrittenQuery() const;
String getResultDescription() const;
ASTPtr query_ptr;
Context & context;
};
}

View File

@ -105,6 +105,8 @@ User::User(const String & name_, const String & config_elem, const Poco::Util::A
if (config.has(config_elem + ".allow_quota_management"))
is_quota_management_allowed = config.getBool(config_elem + ".allow_quota_management");
if (config.has(config_elem + ".allow_row_policy_management"))
is_row_policy_management_allowed = config.getBool(config_elem + ".allow_row_policy_management");
}
}

View File

@ -43,6 +43,7 @@ struct User
std::optional<DictionarySet> dictionaries;
bool is_quota_management_allowed = false;
bool is_row_policy_management_allowed = false;
User(const String & name_, const String & config_elem, const Poco::Util::AbstractConfiguration & config);
};

View File

@ -0,0 +1,164 @@
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Parsers/ASTRoleList.h>
#include <Parsers/formatAST.h>
#include <Common/quoteString.h>
#include <boost/range/algorithm/transform.hpp>
#include <sstream>
namespace DB
{
namespace
{
using ConditionIndex = RowPolicy::ConditionIndex;
void formatRenameTo(const String & new_policy_name, const IAST::FormatSettings & settings)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " RENAME TO " << (settings.hilite ? IAST::hilite_none : "")
<< backQuote(new_policy_name);
}
void formatIsRestrictive(bool is_restrictive, const IAST::FormatSettings & settings)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " AS " << (is_restrictive ? "RESTRICTIVE" : "PERMISSIVE")
<< (settings.hilite ? IAST::hilite_none : "");
}
void formatConditionalExpression(const ASTPtr & expr, const IAST::FormatSettings & settings)
{
if (!expr)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " NONE" << (settings.hilite ? IAST::hilite_none : "");
return;
}
expr->format(settings);
}
std::vector<std::pair<ConditionIndex, String>>
conditionalExpressionsToStrings(const std::vector<std::pair<ConditionIndex, ASTPtr>> & exprs, const IAST::FormatSettings & settings)
{
std::vector<std::pair<ConditionIndex, String>> result;
std::stringstream ss;
IAST::FormatSettings temp_settings(ss, settings);
boost::range::transform(exprs, std::back_inserter(result), [&](const std::pair<ConditionIndex, ASTPtr> & in)
{
formatConditionalExpression(in.second, temp_settings);
auto out = std::pair{in.first, ss.str()};
ss.str("");
return out;
});
return result;
}
void formatConditions(const char * op, const std::optional<String> & filter, const std::optional<String> & check, bool alter, const IAST::FormatSettings & settings)
{
if (op)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " FOR" << (settings.hilite ? IAST::hilite_none : "");
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << ' ' << op << (settings.hilite ? IAST::hilite_none : "");
}
if (filter)
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " USING " << (settings.hilite ? IAST::hilite_none : "") << *filter;
if (check && (alter || (check != filter)))
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " WITH CHECK " << (settings.hilite ? IAST::hilite_none : "") << *check;
}
void formatMultipleConditions(const std::vector<std::pair<ConditionIndex, ASTPtr>> & conditions, bool alter, const IAST::FormatSettings & settings)
{
std::optional<String> scond[RowPolicy::MAX_CONDITION_INDEX];
for (const auto & [index, scondition] : conditionalExpressionsToStrings(conditions, settings))
scond[index] = scondition;
if ((scond[RowPolicy::SELECT_FILTER] == scond[RowPolicy::UPDATE_FILTER])
&& (scond[RowPolicy::UPDATE_FILTER] == scond[RowPolicy::DELETE_FILTER])
&& (scond[RowPolicy::INSERT_CHECK] == scond[RowPolicy::UPDATE_CHECK])
&& (scond[RowPolicy::SELECT_FILTER] || scond[RowPolicy::INSERT_CHECK]))
{
formatConditions(nullptr, scond[RowPolicy::SELECT_FILTER], scond[RowPolicy::INSERT_CHECK], alter, settings);
return;
}
bool need_comma = false;
if (scond[RowPolicy::SELECT_FILTER])
{
if (std::exchange(need_comma, true))
settings.ostr << ',';
formatConditions("SELECT", scond[RowPolicy::SELECT_FILTER], {}, alter, settings);
}
if (scond[RowPolicy::INSERT_CHECK])
{
if (std::exchange(need_comma, true))
settings.ostr << ',';
formatConditions("INSERT", {}, scond[RowPolicy::INSERT_CHECK], alter, settings);
}
if (scond[RowPolicy::UPDATE_FILTER] || scond[RowPolicy::UPDATE_CHECK])
{
if (std::exchange(need_comma, true))
settings.ostr << ',';
formatConditions("UPDATE", scond[RowPolicy::UPDATE_FILTER], scond[RowPolicy::UPDATE_CHECK], alter, settings);
}
if (scond[RowPolicy::DELETE_FILTER])
{
if (std::exchange(need_comma, true))
settings.ostr << ',';
formatConditions("DELETE", scond[RowPolicy::DELETE_FILTER], {}, alter, settings);
}
}
void formatRoles(const ASTRoleList & roles, const IAST::FormatSettings & settings)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " TO " << (settings.hilite ? IAST::hilite_none : "");
roles.format(settings);
}
}
String ASTCreateRowPolicyQuery::getID(char) const
{
return "CREATE POLICY or ALTER POLICY query";
}
ASTPtr ASTCreateRowPolicyQuery::clone() const
{
return std::make_shared<ASTCreateRowPolicyQuery>(*this);
}
void ASTCreateRowPolicyQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER POLICY" : "CREATE POLICY")
<< (settings.hilite ? hilite_none : "");
if (if_exists)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF EXISTS" << (settings.hilite ? hilite_none : "");
else if (if_not_exists)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF NOT EXISTS" << (settings.hilite ? hilite_none : "");
else if (or_replace)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " OR REPLACE" << (settings.hilite ? hilite_none : "");
const String & database = name_parts.database;
const String & table_name = name_parts.table_name;
const String & policy_name = name_parts.policy_name;
settings.ostr << " " << backQuoteIfNeed(policy_name) << (settings.hilite ? hilite_keyword : "") << " ON "
<< (settings.hilite ? hilite_none : "") << (database.empty() ? String{} : backQuoteIfNeed(database) + ".") << table_name;
if (!new_policy_name.empty())
formatRenameTo(new_policy_name, settings);
if (is_restrictive)
formatIsRestrictive(*is_restrictive, settings);
formatMultipleConditions(conditions, alter, settings);
if (roles)
formatRoles(*roles, settings);
}
}

View File

@ -0,0 +1,50 @@
#pragma once
#include <Parsers/IAST.h>
#include <Access/RowPolicy.h>
#include <utility>
#include <vector>
namespace DB
{
class ASTRoleList;
/** CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] name ON [database.]table
* [AS {PERMISSIVE | RESTRICTIVE}]
* [FOR {SELECT | INSERT | UPDATE | DELETE | ALL}]
* [USING condition]
* [WITH CHECK condition] [,...]
* [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
*
* ALTER [ROW] POLICY [IF EXISTS] name ON [database.]table
* [RENAME TO new_name]
* [AS {PERMISSIVE | RESTRICTIVE}]
* [FOR {SELECT | INSERT | UPDATE | DELETE | ALL}]
* [USING {condition | NONE}]
* [WITH CHECK {condition | NONE}] [,...]
* [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
*/
class ASTCreateRowPolicyQuery : public IAST
{
public:
bool alter = false;
bool if_exists = false;
bool if_not_exists = false;
bool or_replace = false;
RowPolicy::FullNameParts name_parts;
String new_policy_name;
std::optional<bool> is_restrictive;
using ConditionIndex = RowPolicy::ConditionIndex;
std::vector<std::pair<ConditionIndex, ASTPtr>> conditions;
std::shared_ptr<ASTRoleList> roles;
String getID(char) const override;
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
}

View File

@ -13,6 +13,7 @@ namespace
switch (kind)
{
case Kind::QUOTA: return "QUOTA";
case Kind::ROW_POLICY: return "POLICY";
}
__builtin_unreachable();
}
@ -44,13 +45,32 @@ void ASTDropAccessEntityQuery::formatImpl(const FormatSettings & settings, Forma
<< (if_exists ? " IF EXISTS" : "")
<< (settings.hilite ? hilite_none : "");
bool need_comma = false;
for (const auto & name : names)
if (kind == Kind::ROW_POLICY)
{
if (need_comma)
settings.ostr << ',';
need_comma = true;
settings.ostr << ' ' << backQuoteIfNeed(name);
bool need_comma = false;
for (const auto & row_policy_name : row_policies_names)
{
if (need_comma)
settings.ostr << ',';
need_comma = true;
const String & database = row_policy_name.database;
const String & table_name = row_policy_name.table_name;
const String & policy_name = row_policy_name.policy_name;
settings.ostr << ' ' << backQuoteIfNeed(policy_name) << (settings.hilite ? hilite_keyword : "") << " ON "
<< (settings.hilite ? hilite_none : "") << (database.empty() ? String{} : backQuoteIfNeed(database) + ".")
<< backQuoteIfNeed(table_name);
}
}
else
{
bool need_comma = false;
for (const auto & name : names)
{
if (need_comma)
settings.ostr << ',';
need_comma = true;
settings.ostr << ' ' << backQuoteIfNeed(name);
}
}
}
}

View File

@ -1,12 +1,14 @@
#pragma once
#include <Parsers/IAST.h>
#include <Access/RowPolicy.h>
namespace DB
{
/** DROP QUOTA [IF EXISTS] name [,...]
* DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...]
*/
class ASTDropAccessEntityQuery : public IAST
{
@ -14,11 +16,13 @@ public:
enum class Kind
{
QUOTA,
ROW_POLICY,
};
const Kind kind;
const char * const keyword;
bool if_exists = false;
Strings names;
std::vector<RowPolicy::FullNameParts> row_policies_names;
ASTDropAccessEntityQuery(Kind kind_);
String getID(char) const override;

View File

@ -13,6 +13,7 @@ namespace
switch (kind)
{
case Kind::QUOTA: return "QUOTA";
case Kind::ROW_POLICY: return "POLICY";
}
__builtin_unreachable();
}
@ -43,7 +44,16 @@ void ASTShowCreateAccessEntityQuery::formatQueryImpl(const FormatSettings & sett
<< "SHOW CREATE " << keyword
<< (settings.hilite ? hilite_none : "");
if (current_quota)
if (kind == Kind::ROW_POLICY)
{
const String & database = row_policy_name.database;
const String & table_name = row_policy_name.table_name;
const String & policy_name = row_policy_name.policy_name;
settings.ostr << ' ' << backQuoteIfNeed(policy_name) << (settings.hilite ? hilite_keyword : "") << " ON "
<< (settings.hilite ? hilite_none : "") << (database.empty() ? String{} : backQuoteIfNeed(database) + ".")
<< backQuoteIfNeed(table_name);
}
else if ((kind == Kind::QUOTA) && current_quota)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " CURRENT" << (settings.hilite ? hilite_none : "");
else
settings.ostr << " " << backQuoteIfNeed(name);

View File

@ -1,11 +1,13 @@
#pragma once
#include <Parsers/ASTQueryWithOutput.h>
#include <Access/RowPolicy.h>
namespace DB
{
/** SHOW CREATE QUOTA [name | CURRENT]
* SHOW CREATE [ROW] POLICY name ON [database.]table
*/
class ASTShowCreateAccessEntityQuery : public ASTQueryWithOutput
{
@ -13,11 +15,13 @@ public:
enum class Kind
{
QUOTA,
ROW_POLICY,
};
const Kind kind;
const char * const keyword;
String name;
bool current_quota = false;
RowPolicy::FullNameParts row_policy_name;
ASTShowCreateAccessEntityQuery(Kind kind_);
String getID(char) const override;

View File

@ -0,0 +1,22 @@
#include <Parsers/ASTShowRowPoliciesQuery.h>
#include <Common/quoteString.h>
namespace DB
{
void ASTShowRowPoliciesQuery::formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW POLICIES" << (settings.hilite ? hilite_none : "");
if (current)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " CURRENT" << (settings.hilite ? hilite_none : "");
if (!table_name.empty())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " ON " << (settings.hilite ? hilite_none : "");
if (!database.empty())
settings.ostr << backQuoteIfNeed(database) << ".";
settings.ostr << backQuoteIfNeed(table_name);
}
}
}

View File

@ -0,0 +1,23 @@
#pragma once
#include <Parsers/ASTQueryWithOutput.h>
namespace DB
{
/// SHOW [ROW] POLICIES [CURRENT] [ON [database.]table]
class ASTShowRowPoliciesQuery : public ASTQueryWithOutput
{
public:
bool current = false;
String database;
String table_name;
String getID(char) const override { return "SHOW POLICIES query"; }
ASTPtr clone() const override { return std::make_shared<ASTShowRowPoliciesQuery>(*this); }
protected:
void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
}

View File

@ -0,0 +1,261 @@
#include <Parsers/ParserCreateRowPolicyQuery.h>
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Access/RowPolicy.h>
#include <Parsers/ParserRoleList.h>
#include <Parsers/ASTRoleList.h>
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Parsers/parseDatabaseAndTableName.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ASTLiteral.h>
namespace DB
{
namespace ErrorCodes
{
extern const int SYNTAX_ERROR;
}
namespace
{
using ConditionIndex = RowPolicy::ConditionIndex;
bool parseRenameTo(IParserBase::Pos & pos, Expected & expected, String & new_policy_name, bool alter)
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!new_policy_name.empty() || !alter)
return false;
if (!ParserKeyword{"RENAME TO"}.ignore(pos, expected))
return false;
return parseIdentifierOrStringLiteral(pos, expected, new_policy_name);
});
}
bool parseIsRestrictive(IParserBase::Pos & pos, Expected & expected, std::optional<bool> & is_restrictive)
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (is_restrictive)
return false;
if (!ParserKeyword{"AS"}.ignore(pos, expected))
return false;
if (ParserKeyword{"RESTRICTIVE"}.ignore(pos, expected))
is_restrictive = true;
else if (ParserKeyword{"PERMISSIVE"}.ignore(pos, expected))
is_restrictive = false;
else
return false;
return true;
});
}
bool parseConditionalExpression(IParserBase::Pos & pos, Expected & expected, std::optional<ASTPtr> & expr)
{
if (ParserKeyword("NONE").ignore(pos, expected))
{
expr = nullptr;
return true;
}
ParserExpression parser;
ASTPtr x;
if (parser.parse(pos, x, expected))
{
expr = x;
return true;
}
expr.reset();
return false;
}
bool parseConditions(IParserBase::Pos & pos, Expected & expected, std::vector<std::pair<ConditionIndex, ASTPtr>> & conditions, bool alter)
{
return IParserBase::wrapParseImpl(pos, [&]
{
static constexpr char select_op[] = "SELECT";
static constexpr char insert_op[] = "INSERT";
static constexpr char update_op[] = "UPDATE";
static constexpr char delete_op[] = "DELETE";
std::vector<const char *> ops;
bool keyword_for = false;
if (ParserKeyword{"FOR"}.ignore(pos, expected))
{
keyword_for = true;
do
{
if (ParserKeyword{"SELECT"}.ignore(pos, expected))
ops.push_back(select_op);
else if (ParserKeyword{"INSERT"}.ignore(pos, expected))
ops.push_back(insert_op);
else if (ParserKeyword{"UPDATE"}.ignore(pos, expected))
ops.push_back(update_op);
else if (ParserKeyword{"DELETE"}.ignore(pos, expected))
ops.push_back(delete_op);
else if (ParserKeyword{"ALL"}.ignore(pos, expected))
{
}
else
return false;
}
while (ParserToken{TokenType::Comma}.ignore(pos, expected));
}
if (ops.empty())
{
ops.push_back(select_op);
ops.push_back(insert_op);
ops.push_back(update_op);
ops.push_back(delete_op);
}
std::optional<ASTPtr> filter;
std::optional<ASTPtr> check;
bool keyword_using = false, keyword_with_check = false;
if (ParserKeyword{"USING"}.ignore(pos, expected))
{
keyword_using = true;
if (!parseConditionalExpression(pos, expected, filter))
return false;
}
if (ParserKeyword{"WITH CHECK"}.ignore(pos, expected))
{
keyword_with_check = true;
if (!parseConditionalExpression(pos, expected, check))
return false;
}
if (!keyword_for && !keyword_using && !keyword_with_check)
return false;
if (filter && !check && !alter)
check = filter;
auto set_condition = [&](ConditionIndex index, const ASTPtr & condition)
{
auto it = std::find_if(conditions.begin(), conditions.end(), [index](const std::pair<ConditionIndex, ASTPtr> & element)
{
return element.first == index;
});
if (it == conditions.end())
it = conditions.insert(conditions.end(), std::pair<ConditionIndex, ASTPtr>{index, nullptr});
it->second = condition;
};
for (const auto & op : ops)
{
if ((op == select_op) && filter)
set_condition(RowPolicy::SELECT_FILTER, *filter);
else if ((op == insert_op) && check)
set_condition(RowPolicy::INSERT_CHECK, *check);
else if (op == update_op)
{
if (filter)
set_condition(RowPolicy::UPDATE_FILTER, *filter);
if (check)
set_condition(RowPolicy::UPDATE_CHECK, *check);
}
else if ((op == delete_op) && filter)
set_condition(RowPolicy::DELETE_FILTER, *filter);
else
__builtin_unreachable();
}
return true;
});
}
bool parseMultipleConditions(IParserBase::Pos & pos, Expected & expected, std::vector<std::pair<ConditionIndex, ASTPtr>> & conditions, bool alter)
{
return IParserBase::wrapParseImpl(pos, [&]
{
do
{
if (!parseConditions(pos, expected, conditions, alter))
return false;
}
while (ParserToken{TokenType::Comma}.ignore(pos, expected));
return true;
});
}
bool parseRoles(IParserBase::Pos & pos, Expected & expected, std::shared_ptr<ASTRoleList> & roles)
{
return IParserBase::wrapParseImpl(pos, [&]
{
ASTPtr node;
if (roles || !ParserKeyword{"TO"}.ignore(pos, expected) || !ParserRoleList{}.parse(pos, node, expected))
return false;
roles = std::static_pointer_cast<ASTRoleList>(node);
return true;
});
}
}
bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
bool alter;
if (ParserKeyword{"CREATE POLICY"}.ignore(pos, expected) || ParserKeyword{"CREATE ROW POLICY"}.ignore(pos, expected))
alter = false;
else if (ParserKeyword{"ALTER POLICY"}.ignore(pos, expected) || ParserKeyword{"ALTER ROW POLICY"}.ignore(pos, expected))
alter = true;
else
return false;
bool if_exists = false;
bool if_not_exists = false;
bool or_replace = false;
if (alter)
{
if (ParserKeyword{"IF EXISTS"}.ignore(pos, expected))
if_exists = true;
}
else
{
if (ParserKeyword{"IF NOT EXISTS"}.ignore(pos, expected))
if_not_exists = true;
else if (ParserKeyword{"OR REPLACE"}.ignore(pos, expected))
or_replace = true;
}
RowPolicy::FullNameParts name_parts;
String & database = name_parts.database;
String & table_name = name_parts.table_name;
String & policy_name = name_parts.policy_name;
if (!parseIdentifierOrStringLiteral(pos, expected, policy_name) || !ParserKeyword{"ON"}.ignore(pos, expected)
|| !parseDatabaseAndTableName(pos, expected, database, table_name))
return false;
String new_policy_name;
std::optional<bool> is_restrictive;
std::vector<std::pair<ConditionIndex, ASTPtr>> conditions;
std::shared_ptr<ASTRoleList> roles;
while (parseRenameTo(pos, expected, new_policy_name, alter) || parseIsRestrictive(pos, expected, is_restrictive)
|| parseMultipleConditions(pos, expected, conditions, alter) || parseRoles(pos, expected, roles))
;
auto query = std::make_shared<ASTCreateRowPolicyQuery>();
node = query;
query->alter = alter;
query->if_exists = if_exists;
query->if_not_exists = if_not_exists;
query->or_replace = or_replace;
query->name_parts = std::move(name_parts);
query->new_policy_name = std::move(new_policy_name);
query->is_restrictive = is_restrictive;
query->conditions = std::move(conditions);
query->roles = std::move(roles);
return true;
}
}

View File

@ -0,0 +1,30 @@
#pragma once
#include <Parsers/IParserBase.h>
namespace DB
{
/** Parses queries like
* CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] name ON [database.]table
* [AS {PERMISSIVE | RESTRICTIVE}]
* [FOR {SELECT | INSERT | UPDATE | DELETE | ALL}]
* [USING condition]
* [WITH CHECK condition] [,...]
* [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
*
* ALTER [ROW] POLICY [IF EXISTS] name ON [database.]table
* [RENAME TO new_name]
* [AS {PERMISSIVE | RESTRICTIVE}]
* [FOR {SELECT | INSERT | UPDATE | DELETE | ALL}]
* [USING {condition | NONE}]
* [WITH CHECK {condition | NONE}] [,...]
* [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
*/
class ParserCreateRowPolicyQuery : public IParserBase
{
protected:
const char * getName() const override { return "CREATE ROW POLICY or ALTER ROW POLICY query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}

View File

@ -2,11 +2,30 @@
#include <Parsers/ASTDropAccessEntityQuery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Parsers/parseDatabaseAndTableName.h>
#include <Access/Quota.h>
namespace DB
{
namespace
{
bool parseNames(IParserBase::Pos & pos, Expected & expected, Strings & names)
{
do
{
String name;
if (!parseIdentifierOrStringLiteral(pos, expected, name))
return false;
names.push_back(std::move(name));
}
while (ParserToken{TokenType::Comma}.ignore(pos, expected));
return true;
}
}
bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserKeyword{"DROP"}.ignore(pos, expected))
@ -16,6 +35,8 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
Kind kind;
if (ParserKeyword{"QUOTA"}.ignore(pos, expected))
kind = Kind::QUOTA;
else if (ParserKeyword{"POLICY"}.ignore(pos, expected) || ParserKeyword{"ROW POLICY"}.ignore(pos, expected))
kind = Kind::ROW_POLICY;
else
return false;
@ -24,21 +45,35 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
if_exists = true;
Strings names;
do
{
String name;
if (!parseIdentifierOrStringLiteral(pos, expected, name))
return false;
std::vector<RowPolicy::FullNameParts> row_policies_names;
names.push_back(std::move(name));
if (kind == Kind::ROW_POLICY)
{
do
{
Strings policy_names;
if (!parseNames(pos, expected, policy_names))
return false;
String database, table_name;
if (!ParserKeyword{"ON"}.ignore(pos, expected) || !parseDatabaseAndTableName(pos, expected, database, table_name))
return false;
for (const String & policy_name : policy_names)
row_policies_names.push_back({database, table_name, policy_name});
}
while (ParserToken{TokenType::Comma}.ignore(pos, expected));
}
else
{
if (!parseNames(pos, expected, names))
return false;
}
while (ParserToken{TokenType::Comma}.ignore(pos, expected));
auto query = std::make_shared<ASTDropAccessEntityQuery>(kind);
node = query;
query->if_exists = if_exists;
query->names = std::move(names);
query->row_policies_names = std::move(row_policies_names);
return true;
}

View File

@ -10,6 +10,7 @@
#include <Parsers/ParserAlterQuery.h>
#include <Parsers/ParserSystemQuery.h>
#include <Parsers/ParserCreateQuotaQuery.h>
#include <Parsers/ParserCreateRowPolicyQuery.h>
#include <Parsers/ParserDropAccessEntityQuery.h>
@ -25,6 +26,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserSetQuery set_p;
ParserSystemQuery system_p;
ParserCreateQuotaQuery create_quota_p;
ParserCreateRowPolicyQuery create_row_policy_p;
ParserDropAccessEntityQuery drop_access_entity_p;
bool res = query_with_output_p.parse(pos, node, expected)
@ -33,6 +35,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|| set_p.parse(pos, node, expected)
|| system_p.parse(pos, node, expected)
|| create_quota_p.parse(pos, node, expected)
|| create_row_policy_p.parse(pos, node, expected)
|| drop_access_entity_p.parse(pos, node, expected);
return res;

View File

@ -16,6 +16,7 @@
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/ParserShowCreateAccessEntityQuery.h>
#include <Parsers/ParserShowQuotasQuery.h>
#include <Parsers/ParserShowRowPoliciesQuery.h>
namespace DB
@ -38,6 +39,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
ParserWatchQuery watch_p;
ParserShowCreateAccessEntityQuery show_create_access_entity_p;
ParserShowQuotasQuery show_quotas_p;
ParserShowRowPoliciesQuery show_row_policies_p;
ASTPtr query;
@ -66,7 +68,8 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|| kill_query_p.parse(pos, query, expected)
|| optimize_p.parse(pos, query, expected)
|| watch_p.parse(pos, query, expected)
|| show_quotas_p.parse(pos, query, expected);
|| show_quotas_p.parse(pos, query, expected)
|| show_row_policies_p.parse(pos, query, expected);
if (!parsed)
return false;

View File

@ -2,6 +2,8 @@
#include <Parsers/ASTShowCreateAccessEntityQuery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Parsers/parseDatabaseAndTableName.h>
#include <assert.h>
namespace DB
@ -15,25 +17,41 @@ bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expe
Kind kind;
if (ParserKeyword{"QUOTA"}.ignore(pos, expected))
kind = Kind::QUOTA;
else if (ParserKeyword{"POLICY"}.ignore(pos, expected) || ParserKeyword{"ROW POLICY"}.ignore(pos, expected))
kind = Kind::ROW_POLICY;
else
return false;
String name;
bool current_quota = false;
RowPolicy::FullNameParts row_policy_name;
if ((kind == Kind::QUOTA) && ParserKeyword{"CURRENT"}.ignore(pos, expected))
if (kind == Kind::ROW_POLICY)
{
/// SHOW CREATE QUOTA CURRENT
current_quota = true;
}
else if (parseIdentifierOrStringLiteral(pos, expected, name))
{
/// SHOW CREATE QUOTA name
String & database = row_policy_name.database;
String & table_name = row_policy_name.table_name;
String & policy_name = row_policy_name.policy_name;
if (!parseIdentifierOrStringLiteral(pos, expected, policy_name) || !ParserKeyword{"ON"}.ignore(pos, expected)
|| !parseDatabaseAndTableName(pos, expected, database, table_name))
return false;
}
else
{
/// SHOW CREATE QUOTA
current_quota = true;
assert(kind == Kind::QUOTA);
if (ParserKeyword{"CURRENT"}.ignore(pos, expected))
{
/// SHOW CREATE QUOTA CURRENT
current_quota = true;
}
else if (parseIdentifierOrStringLiteral(pos, expected, name))
{
/// SHOW CREATE QUOTA name
}
else
{
/// SHOW CREATE QUOTA
current_quota = true;
}
}
auto query = std::make_shared<ASTShowCreateAccessEntityQuery>(kind);
@ -41,6 +59,7 @@ bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expe
query->name = std::move(name);
query->current_quota = current_quota;
query->row_policy_name = std::move(row_policy_name);
return true;
}

View File

@ -0,0 +1,40 @@
#include <Parsers/ParserShowRowPoliciesQuery.h>
#include <Parsers/ASTShowRowPoliciesQuery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/parseDatabaseAndTableName.h>
namespace DB
{
namespace
{
bool parseONDatabaseAndTableName(IParserBase::Pos & pos, Expected & expected, String & database, String & table_name)
{
return IParserBase::wrapParseImpl(pos, [&]
{
database.clear();
table_name.clear();
return ParserKeyword{"ON"}.ignore(pos, expected) && parseDatabaseAndTableName(pos, expected, database, table_name);
});
}
}
bool ParserShowRowPoliciesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserKeyword{"SHOW POLICIES"}.ignore(pos, expected) && !ParserKeyword{"SHOW ROW POLICIES"}.ignore(pos, expected))
return false;
bool current = ParserKeyword{"CURRENT"}.ignore(pos, expected);
String database, table_name;
parseONDatabaseAndTableName(pos, expected, database, table_name);
auto query = std::make_shared<ASTShowRowPoliciesQuery>();
query->current = current;
query->database = std::move(database);
query->table_name = std::move(table_name);
node = query;
return true;
}
}

View File

@ -0,0 +1,17 @@
#pragma once
#include <Parsers/IParserBase.h>
namespace DB
{
/** Parses queries like
* SHOW [ROW] POLICIES [CURRENT] [ON [database.]table]
*/
class ParserShowRowPoliciesQuery : public IParserBase
{
protected:
const char * getName() const override { return "SHOW POLICIES query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}

View File

@ -153,3 +153,57 @@ def test_introspection():
assert instance.query("SELECT * from system.row_policies WHERE has(currentRowPolicyIDs('mydb', 'filtered_table3'), id) ORDER BY table, name") == policy3
assert instance.query("SELECT * from system.row_policies ORDER BY table, name") == policy1 + policy2 + policy3
assert instance.query("SELECT * from system.row_policies WHERE has(currentRowPolicyIDs(), id) ORDER BY table, name") == policy1 + policy2 + policy3
def test_dcl_introspection():
assert instance.query("SHOW POLICIES ON mydb.filtered_table1") == "default\n"
assert instance.query("SHOW POLICIES CURRENT ON mydb.filtered_table2") == "default\n"
assert instance.query("SHOW POLICIES") == "default ON mydb.filtered_table1\ndefault ON mydb.filtered_table2\ndefault ON mydb.filtered_table3\n"
assert instance.query("SHOW POLICIES CURRENT") == "default ON mydb.filtered_table1\ndefault ON mydb.filtered_table2\ndefault ON mydb.filtered_table3\n"
assert instance.query("SHOW CREATE POLICY default ON mydb.filtered_table1") == "CREATE POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default\n"
assert instance.query("SHOW CREATE POLICY default ON mydb.filtered_table2") == "CREATE POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default\n"
assert instance.query("SHOW CREATE POLICY default ON mydb.filtered_table3") == "CREATE POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default\n"
copy_policy_xml('all_rows.xml')
assert instance.query("SHOW CREATE POLICY default ON mydb.filtered_table1") == "CREATE POLICY default ON mydb.filtered_table1 FOR SELECT USING 1 TO default\n"
assert instance.query("SHOW CREATE POLICY default ON mydb.filtered_table2") == "CREATE POLICY default ON mydb.filtered_table2 FOR SELECT USING 1 TO default\n"
assert instance.query("SHOW CREATE POLICY default ON mydb.filtered_table3") == "CREATE POLICY default ON mydb.filtered_table3 FOR SELECT USING 1 TO default\n"
copy_policy_xml('no_rows.xml')
assert instance.query("SHOW CREATE POLICY default ON mydb.filtered_table1") == "CREATE POLICY default ON mydb.filtered_table1 FOR SELECT USING NULL TO default\n"
assert instance.query("SHOW CREATE POLICY default ON mydb.filtered_table2") == "CREATE POLICY default ON mydb.filtered_table2 FOR SELECT USING NULL TO default\n"
assert instance.query("SHOW CREATE POLICY default ON mydb.filtered_table3") == "CREATE POLICY default ON mydb.filtered_table3 FOR SELECT USING NULL TO default\n"
copy_policy_xml('no_filters.xml')
assert instance.query("SHOW POLICIES") == ""
def test_dcl_management():
copy_policy_xml('no_filters.xml')
assert instance.query("SHOW POLICIES") == ""
instance.query("CREATE POLICY pA ON mydb.filtered_table1 FOR SELECT USING a<b")
assert instance.query("SELECT * FROM mydb.filtered_table1") == "0\t0\n0\t1\n1\t0\n1\t1\n"
assert instance.query("SHOW POLICIES CURRENT ON mydb.filtered_table1") == ""
assert instance.query("SHOW POLICIES ON mydb.filtered_table1") == "pA\n"
instance.query("ALTER POLICY pA ON mydb.filtered_table1 TO default")
assert instance.query("SELECT * FROM mydb.filtered_table1") == "0\t1\n"
assert instance.query("SHOW POLICIES CURRENT ON mydb.filtered_table1") == "pA\n"
instance.query("ALTER POLICY pA ON mydb.filtered_table1 FOR SELECT USING a>b")
assert instance.query("SELECT * FROM mydb.filtered_table1") == "1\t0\n"
instance.query("ALTER POLICY pA ON mydb.filtered_table1 RENAME TO pB")
assert instance.query("SELECT * FROM mydb.filtered_table1") == "1\t0\n"
assert instance.query("SHOW POLICIES CURRENT ON mydb.filtered_table1") == "pB\n"
assert instance.query("SHOW CREATE POLICY pB ON mydb.filtered_table1") == "CREATE POLICY pB ON mydb.filtered_table1 FOR SELECT USING a > b TO default\n"
instance.query("DROP POLICY pB ON mydb.filtered_table1")
assert instance.query("SELECT * FROM mydb.filtered_table1") == "0\t0\n0\t1\n1\t0\n1\t1\n"
assert instance.query("SHOW POLICIES") == ""
def test_users_xml_is_readonly():
assert re.search("storage is readonly", instance.query_and_get_error("DROP POLICY default ON mydb.filtered_table1"))

View File

@ -0,0 +1,2 @@
SHOW POLICIES;
CREATE POLICY p1 ON dummytable; -- { serverError 497 }