From c3161b7610775f3f3edf134e995e0980a72f8fe3 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 29 Nov 2019 20:22:56 +0300 Subject: [PATCH] Add DCL to manage row policies. --- dbms/src/Interpreters/Context.cpp | 8 + dbms/src/Interpreters/Context.h | 2 + .../InterpreterCreateRowPolicyQuery.cpp | 93 +++++++ .../InterpreterCreateRowPolicyQuery.h | 26 ++ .../InterpreterDropAccessEntityQuery.cpp | 15 + dbms/src/Interpreters/InterpreterFactory.cpp | 12 + ...InterpreterShowCreateAccessEntityQuery.cpp | 51 +++- .../InterpreterShowCreateAccessEntityQuery.h | 2 + .../InterpreterShowRowPoliciesQuery.cpp | 68 +++++ .../InterpreterShowRowPoliciesQuery.h | 25 ++ dbms/src/Interpreters/Users.cpp | 2 + dbms/src/Interpreters/Users.h | 1 + dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp | 164 +++++++++++ dbms/src/Parsers/ASTCreateRowPolicyQuery.h | 50 ++++ dbms/src/Parsers/ASTDropAccessEntityQuery.cpp | 32 ++- dbms/src/Parsers/ASTDropAccessEntityQuery.h | 4 + .../ASTShowCreateAccessEntityQuery.cpp | 12 +- .../Parsers/ASTShowCreateAccessEntityQuery.h | 4 + dbms/src/Parsers/ASTShowRowPoliciesQuery.cpp | 22 ++ dbms/src/Parsers/ASTShowRowPoliciesQuery.h | 23 ++ .../Parsers/ParserCreateRowPolicyQuery.cpp | 261 ++++++++++++++++++ dbms/src/Parsers/ParserCreateRowPolicyQuery.h | 30 ++ .../Parsers/ParserDropAccessEntityQuery.cpp | 49 +++- dbms/src/Parsers/ParserQuery.cpp | 3 + dbms/src/Parsers/ParserQueryWithOutput.cpp | 5 +- .../ParserShowCreateAccessEntityQuery.cpp | 37 ++- .../Parsers/ParserShowRowPoliciesQuery.cpp | 40 +++ dbms/src/Parsers/ParserShowRowPoliciesQuery.h | 17 ++ .../tests/integration/test_row_policy/test.py | 54 ++++ .../01039_row_policy_dcl.reference | 0 .../0_stateless/01039_row_policy_dcl.sql | 2 + 31 files changed, 1089 insertions(+), 25 deletions(-) create mode 100644 dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.cpp create mode 100644 dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.h create mode 100644 dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.cpp create mode 100644 dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.h create mode 100644 dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp create mode 100644 dbms/src/Parsers/ASTCreateRowPolicyQuery.h create mode 100644 dbms/src/Parsers/ASTShowRowPoliciesQuery.cpp create mode 100644 dbms/src/Parsers/ASTShowRowPoliciesQuery.h create mode 100644 dbms/src/Parsers/ParserCreateRowPolicyQuery.cpp create mode 100644 dbms/src/Parsers/ParserCreateRowPolicyQuery.h create mode 100644 dbms/src/Parsers/ParserShowRowPoliciesQuery.cpp create mode 100644 dbms/src/Parsers/ParserShowRowPoliciesQuery.h create mode 100644 dbms/tests/queries/0_stateless/01039_row_policy_dcl.reference create mode 100644 dbms/tests/queries/0_stateless/01039_row_policy_dcl.sql diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index e6b54f3dbc4..e5aab7075ca 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -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; } diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index f434df05238..79e2936d7d2 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -142,6 +142,7 @@ private: std::shared_ptr 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 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 settings_constraints; @@ -212,6 +213,7 @@ public: std::shared_ptr getQuota() const { return quota; } void checkQuotaManagementIsAllowed(); std::shared_ptr 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. diff --git a/dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.cpp b/dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.cpp new file mode 100644 index 00000000000..daab0762826 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.cpp @@ -0,0 +1,93 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +BlockIO InterpreterCreateRowPolicyQuery::execute() +{ + context.checkRowPolicyManagementIsAllowed(); + const auto & query = query_ptr->as(); + auto & access_control = context.getAccessControlManager(); + + if (query.alter) + { + auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr + { + auto updated_policy = typeid_cast>(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(full_name)) + access_control.tryUpdate(*id, update_func); + } + else + access_control.update(access_control.getID(full_name), update_func); + } + else + { + auto new_policy = std::make_shared(); + 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()); + } +} +} diff --git a/dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.h b/dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.h new file mode 100644 index 00000000000..e7ee47dbe81 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.h @@ -0,0 +1,26 @@ +#pragma once + +#include +#include + + +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; +}; +} diff --git a/dbms/src/Interpreters/InterpreterDropAccessEntityQuery.cpp b/dbms/src/Interpreters/InterpreterDropAccessEntityQuery.cpp index 7f18084038c..c8e7ab43a64 100644 --- a/dbms/src/Interpreters/InterpreterDropAccessEntityQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropAccessEntityQuery.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include namespace DB @@ -24,6 +26,19 @@ BlockIO InterpreterDropAccessEntityQuery::execute() access_control.remove(access_control.getIDs(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(full_names)); + else + access_control.remove(access_control.getIDs(full_names)); + return {}; + } } __builtin_unreachable(); diff --git a/dbms/src/Interpreters/InterpreterFactory.cpp b/dbms/src/Interpreters/InterpreterFactory.cpp index 33e9da95dfc..7d1eccd8581 100644 --- a/dbms/src/Interpreters/InterpreterFactory.cpp +++ b/dbms/src/Interpreters/InterpreterFactory.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -14,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -24,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -41,6 +44,7 @@ #include #include #include +#include #include #include #include @@ -199,6 +203,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else if (query->as()) { return std::make_unique(query, context); @@ -211,6 +219,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else throw Exception("Unknown type of query: " + query->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY); } diff --git a/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp b/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp index d0ef8992691..cf2a07425ba 100644 --- a/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp +++ b/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp @@ -1,9 +1,12 @@ #include #include #include +#include #include #include +#include #include +#include #include #include #include @@ -28,7 +31,7 @@ BlockInputStreamPtr InterpreterShowCreateAccessEntityQuery::executeImpl() const auto & show_query = query_ptr->as(); /// 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(show_query.row_policy_name.getFullName(context)); + + auto create_query = std::make_shared(); + 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::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(); + 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; +} } diff --git a/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.h b/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.h index 94b06dadb19..0a6d94b85d0 100644 --- a/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.h +++ b/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.h @@ -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; }; diff --git a/dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.cpp b/dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.cpp new file mode 100644 index 00000000000..4b9b80d91a8 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.cpp @@ -0,0 +1,68 @@ +#include +#include +#include +#include +#include +#include +#include + + +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(); + + 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; +} +} diff --git a/dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.h b/dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.h new file mode 100644 index 00000000000..84cf6299b84 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.h @@ -0,0 +1,25 @@ +#pragma once + +#include +#include + + +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; +}; + +} diff --git a/dbms/src/Interpreters/Users.cpp b/dbms/src/Interpreters/Users.cpp index 81898ba5135..2b48ce56e3f 100644 --- a/dbms/src/Interpreters/Users.cpp +++ b/dbms/src/Interpreters/Users.cpp @@ -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"); } } diff --git a/dbms/src/Interpreters/Users.h b/dbms/src/Interpreters/Users.h index 189bfb991d5..f151770cefd 100644 --- a/dbms/src/Interpreters/Users.h +++ b/dbms/src/Interpreters/Users.h @@ -43,6 +43,7 @@ struct User std::optional 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); }; diff --git a/dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp b/dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp new file mode 100644 index 00000000000..50e1645f14b --- /dev/null +++ b/dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp @@ -0,0 +1,164 @@ +#include +#include +#include +#include +#include +#include + + +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> + conditionalExpressionsToStrings(const std::vector> & exprs, const IAST::FormatSettings & settings) + { + std::vector> result; + std::stringstream ss; + IAST::FormatSettings temp_settings(ss, settings); + boost::range::transform(exprs, std::back_inserter(result), [&](const std::pair & 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 & filter, const std::optional & 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> & conditions, bool alter, const IAST::FormatSettings & settings) + { + std::optional 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(*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); +} +} diff --git a/dbms/src/Parsers/ASTCreateRowPolicyQuery.h b/dbms/src/Parsers/ASTCreateRowPolicyQuery.h new file mode 100644 index 00000000000..a4caf1aeb85 --- /dev/null +++ b/dbms/src/Parsers/ASTCreateRowPolicyQuery.h @@ -0,0 +1,50 @@ +#pragma once + +#include +#include +#include +#include + + +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 is_restrictive; + using ConditionIndex = RowPolicy::ConditionIndex; + std::vector> conditions; + + std::shared_ptr roles; + + String getID(char) const override; + ASTPtr clone() const override; + void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; +}; +} diff --git a/dbms/src/Parsers/ASTDropAccessEntityQuery.cpp b/dbms/src/Parsers/ASTDropAccessEntityQuery.cpp index 80d69ed5316..b80dc47a9f7 100644 --- a/dbms/src/Parsers/ASTDropAccessEntityQuery.cpp +++ b/dbms/src/Parsers/ASTDropAccessEntityQuery.cpp @@ -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); + } } } } diff --git a/dbms/src/Parsers/ASTDropAccessEntityQuery.h b/dbms/src/Parsers/ASTDropAccessEntityQuery.h index 91b76253db4..690a89995cf 100644 --- a/dbms/src/Parsers/ASTDropAccessEntityQuery.h +++ b/dbms/src/Parsers/ASTDropAccessEntityQuery.h @@ -1,12 +1,14 @@ #pragma once #include +#include 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 row_policies_names; ASTDropAccessEntityQuery(Kind kind_); String getID(char) const override; diff --git a/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.cpp b/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.cpp index 8509a902014..caa9fb620e9 100644 --- a/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.cpp +++ b/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.cpp @@ -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); diff --git a/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.h b/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.h index 32c0ace101b..c19f00b1d6e 100644 --- a/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.h +++ b/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.h @@ -1,11 +1,13 @@ #pragma once #include +#include 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; diff --git a/dbms/src/Parsers/ASTShowRowPoliciesQuery.cpp b/dbms/src/Parsers/ASTShowRowPoliciesQuery.cpp new file mode 100644 index 00000000000..15e0e81f218 --- /dev/null +++ b/dbms/src/Parsers/ASTShowRowPoliciesQuery.cpp @@ -0,0 +1,22 @@ +#include +#include + + +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); + } +} +} diff --git a/dbms/src/Parsers/ASTShowRowPoliciesQuery.h b/dbms/src/Parsers/ASTShowRowPoliciesQuery.h new file mode 100644 index 00000000000..ce82902e96d --- /dev/null +++ b/dbms/src/Parsers/ASTShowRowPoliciesQuery.h @@ -0,0 +1,23 @@ +#pragma once + +#include + + +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(*this); } + +protected: + void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; +}; + +} diff --git a/dbms/src/Parsers/ParserCreateRowPolicyQuery.cpp b/dbms/src/Parsers/ParserCreateRowPolicyQuery.cpp new file mode 100644 index 00000000000..2778ddea93f --- /dev/null +++ b/dbms/src/Parsers/ParserCreateRowPolicyQuery.cpp @@ -0,0 +1,261 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +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 & 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 & 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> & 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 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 filter; + std::optional 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 & element) + { + return element.first == index; + }); + if (it == conditions.end()) + it = conditions.insert(conditions.end(), std::pair{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> & 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 & 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(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 is_restrictive; + std::vector> conditions; + std::shared_ptr 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(); + 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; +} +} diff --git a/dbms/src/Parsers/ParserCreateRowPolicyQuery.h b/dbms/src/Parsers/ParserCreateRowPolicyQuery.h new file mode 100644 index 00000000000..2ec7ff37c94 --- /dev/null +++ b/dbms/src/Parsers/ParserCreateRowPolicyQuery.h @@ -0,0 +1,30 @@ +#pragma once + +#include + + +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; +}; +} diff --git a/dbms/src/Parsers/ParserDropAccessEntityQuery.cpp b/dbms/src/Parsers/ParserDropAccessEntityQuery.cpp index c6d5ff889fc..1c34fd6edb6 100644 --- a/dbms/src/Parsers/ParserDropAccessEntityQuery.cpp +++ b/dbms/src/Parsers/ParserDropAccessEntityQuery.cpp @@ -2,11 +2,30 @@ #include #include #include +#include #include 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 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(kind); node = query; query->if_exists = if_exists; query->names = std::move(names); + query->row_policies_names = std::move(row_policies_names); return true; } diff --git a/dbms/src/Parsers/ParserQuery.cpp b/dbms/src/Parsers/ParserQuery.cpp index a3bb652032e..d5002794734 100644 --- a/dbms/src/Parsers/ParserQuery.cpp +++ b/dbms/src/Parsers/ParserQuery.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include @@ -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; diff --git a/dbms/src/Parsers/ParserQueryWithOutput.cpp b/dbms/src/Parsers/ParserQueryWithOutput.cpp index d08ae984c90..31948194976 100644 --- a/dbms/src/Parsers/ParserQueryWithOutput.cpp +++ b/dbms/src/Parsers/ParserQueryWithOutput.cpp @@ -16,6 +16,7 @@ #include #include #include +#include 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; diff --git a/dbms/src/Parsers/ParserShowCreateAccessEntityQuery.cpp b/dbms/src/Parsers/ParserShowCreateAccessEntityQuery.cpp index 661330ffd0b..0caba5e0495 100644 --- a/dbms/src/Parsers/ParserShowCreateAccessEntityQuery.cpp +++ b/dbms/src/Parsers/ParserShowCreateAccessEntityQuery.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include 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(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; } diff --git a/dbms/src/Parsers/ParserShowRowPoliciesQuery.cpp b/dbms/src/Parsers/ParserShowRowPoliciesQuery.cpp new file mode 100644 index 00000000000..b07e7a386ba --- /dev/null +++ b/dbms/src/Parsers/ParserShowRowPoliciesQuery.cpp @@ -0,0 +1,40 @@ +#include +#include +#include +#include + + +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(); + query->current = current; + query->database = std::move(database); + query->table_name = std::move(table_name); + node = query; + return true; +} +} diff --git a/dbms/src/Parsers/ParserShowRowPoliciesQuery.h b/dbms/src/Parsers/ParserShowRowPoliciesQuery.h new file mode 100644 index 00000000000..df7413fb604 --- /dev/null +++ b/dbms/src/Parsers/ParserShowRowPoliciesQuery.h @@ -0,0 +1,17 @@ +#pragma once + +#include + + +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; +}; +} diff --git a/dbms/tests/integration/test_row_policy/test.py b/dbms/tests/integration/test_row_policy/test.py index 3de63d56fa0..421a4b0510c 100644 --- a/dbms/tests/integration/test_row_policy/test.py +++ b/dbms/tests/integration/test_row_policy/test.py @@ -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 ab") + 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")) diff --git a/dbms/tests/queries/0_stateless/01039_row_policy_dcl.reference b/dbms/tests/queries/0_stateless/01039_row_policy_dcl.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/01039_row_policy_dcl.sql b/dbms/tests/queries/0_stateless/01039_row_policy_dcl.sql new file mode 100644 index 00000000000..dc558a55b26 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01039_row_policy_dcl.sql @@ -0,0 +1,2 @@ +SHOW POLICIES; +CREATE POLICY p1 ON dummytable; -- { serverError 497 }