mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Add DCL to manage row policies.
This commit is contained in:
parent
6baccb963d
commit
c3161b7610
@ -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;
|
||||
}
|
||||
|
||||
|
||||
|
@ -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.
|
||||
|
93
dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.cpp
Normal file
93
dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.cpp
Normal 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());
|
||||
}
|
||||
}
|
||||
}
|
26
dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.h
Normal file
26
dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.h
Normal 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;
|
||||
};
|
||||
}
|
@ -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();
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
||||
|
68
dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.cpp
Normal file
68
dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.cpp
Normal 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;
|
||||
}
|
||||
}
|
25
dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.h
Normal file
25
dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
@ -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");
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
};
|
||||
|
164
dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp
Normal file
164
dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp
Normal 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);
|
||||
}
|
||||
}
|
50
dbms/src/Parsers/ASTCreateRowPolicyQuery.h
Normal file
50
dbms/src/Parsers/ASTCreateRowPolicyQuery.h
Normal 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;
|
||||
};
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
22
dbms/src/Parsers/ASTShowRowPoliciesQuery.cpp
Normal file
22
dbms/src/Parsers/ASTShowRowPoliciesQuery.cpp
Normal 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);
|
||||
}
|
||||
}
|
||||
}
|
23
dbms/src/Parsers/ASTShowRowPoliciesQuery.h
Normal file
23
dbms/src/Parsers/ASTShowRowPoliciesQuery.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
261
dbms/src/Parsers/ParserCreateRowPolicyQuery.cpp
Normal file
261
dbms/src/Parsers/ParserCreateRowPolicyQuery.cpp
Normal 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;
|
||||
}
|
||||
}
|
30
dbms/src/Parsers/ParserCreateRowPolicyQuery.h
Normal file
30
dbms/src/Parsers/ParserCreateRowPolicyQuery.h
Normal 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;
|
||||
};
|
||||
}
|
@ -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;
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
}
|
||||
|
40
dbms/src/Parsers/ParserShowRowPoliciesQuery.cpp
Normal file
40
dbms/src/Parsers/ParserShowRowPoliciesQuery.cpp
Normal 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;
|
||||
}
|
||||
}
|
17
dbms/src/Parsers/ParserShowRowPoliciesQuery.h
Normal file
17
dbms/src/Parsers/ParserShowRowPoliciesQuery.h
Normal 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;
|
||||
};
|
||||
}
|
@ -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"))
|
||||
|
2
dbms/tests/queries/0_stateless/01039_row_policy_dcl.sql
Normal file
2
dbms/tests/queries/0_stateless/01039_row_policy_dcl.sql
Normal file
@ -0,0 +1,2 @@
|
||||
SHOW POLICIES;
|
||||
CREATE POLICY p1 ON dummytable; -- { serverError 497 }
|
Loading…
Reference in New Issue
Block a user