提交 42b8ed3e 编写于 作者: V Vitaly Baranov

Implement "ON CLUSTER" clause for access control SQL.

上级 b77e0a5b
......@@ -51,25 +51,25 @@ ExtendedRoleSet::ExtendedRoleSet(const boost::container::flat_set<UUID> & ids_)
ExtendedRoleSet::ExtendedRoleSet(const ASTExtendedRoleSet & ast)
{
init(ast, nullptr, nullptr);
init(ast, nullptr);
}
ExtendedRoleSet::ExtendedRoleSet(const ASTExtendedRoleSet & ast, const UUID & current_user_id)
ExtendedRoleSet::ExtendedRoleSet(const ASTExtendedRoleSet & ast, const std::optional<UUID> & current_user_id)
{
init(ast, nullptr, &current_user_id);
init(ast, nullptr, current_user_id);
}
ExtendedRoleSet::ExtendedRoleSet(const ASTExtendedRoleSet & ast, const AccessControlManager & manager)
{
init(ast, &manager, nullptr);
init(ast, &manager);
}
ExtendedRoleSet::ExtendedRoleSet(const ASTExtendedRoleSet & ast, const AccessControlManager & manager, const UUID & current_user_id)
ExtendedRoleSet::ExtendedRoleSet(const ASTExtendedRoleSet & ast, const AccessControlManager & manager, const std::optional<UUID> & current_user_id)
{
init(ast, &manager, &current_user_id);
init(ast, &manager, current_user_id);
}
void ExtendedRoleSet::init(const ASTExtendedRoleSet & ast, const AccessControlManager * manager, const UUID * current_user_id)
void ExtendedRoleSet::init(const ASTExtendedRoleSet & ast, const AccessControlManager * manager, const std::optional<UUID> & current_user_id)
{
all = ast.all;
......
......@@ -32,9 +32,9 @@ struct ExtendedRoleSet
/// The constructor from AST requires the AccessControlManager if `ast.id_mode == false`.
ExtendedRoleSet(const ASTExtendedRoleSet & ast);
ExtendedRoleSet(const ASTExtendedRoleSet & ast, const UUID & current_user_id);
ExtendedRoleSet(const ASTExtendedRoleSet & ast, const std::optional<UUID> & current_user_id);
ExtendedRoleSet(const ASTExtendedRoleSet & ast, const AccessControlManager & manager);
ExtendedRoleSet(const ASTExtendedRoleSet & ast, const AccessControlManager & manager, const UUID & current_user_id);
ExtendedRoleSet(const ASTExtendedRoleSet & ast, const AccessControlManager & manager, const std::optional<UUID> & current_user_id);
std::shared_ptr<ASTExtendedRoleSet> toAST() const;
String toString() const;
......@@ -69,7 +69,7 @@ struct ExtendedRoleSet
boost::container::flat_set<UUID> except_ids;
private:
void init(const ASTExtendedRoleSet & ast, const AccessControlManager * manager = nullptr, const UUID * current_user_id = nullptr);
void init(const ASTExtendedRoleSet & ast, const AccessControlManager * manager = nullptr, const std::optional<UUID> & current_user_id = {});
};
}
......@@ -665,12 +665,10 @@ String Context::getUserName() const
return access->getUserName();
}
UUID Context::getUserID() const
std::optional<UUID> Context::getUserID() const
{
auto lock = getLock();
if (!user_id)
throw Exception("No current user", ErrorCodes::LOGICAL_ERROR);
return *user_id;
return user_id;
}
......
......@@ -233,7 +233,7 @@ public:
UserPtr getUser() const;
String getUserName() const;
UUID getUserID() const;
std::optional<UUID> getUserID() const;
void setCurrentRoles(const std::vector<UUID> & current_roles_);
void setCurrentRolesDefault();
......
......@@ -1377,4 +1377,9 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & cont
}
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & context)
{
return executeDDLQueryOnCluster(query_ptr_, context, {});
}
}
......@@ -24,6 +24,7 @@ struct DDLTask;
/// Pushes distributed DDL query to the queue
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context, AccessRightsElements && query_required_access);
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context);
class DDLWorker
......
......@@ -2,6 +2,7 @@
#include <Parsers/ASTCreateQuotaQuery.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessFlags.h>
#include <ext/range.h>
......@@ -76,10 +77,16 @@ void updateQuotaFromQueryImpl(Quota & quota, const ASTCreateQuotaQuery & query,
BlockIO InterpreterCreateQuotaQuery::execute()
{
const auto & query = query_ptr->as<const ASTCreateQuotaQuery &>();
auto & query = query_ptr->as<ASTCreateQuotaQuery &>();
auto & access_control = context.getAccessControlManager();
context.checkAccess(query.alter ? AccessType::ALTER_QUOTA : AccessType::CREATE_QUOTA);
if (!query.cluster.empty())
{
query.replaceCurrentUserTagWithName(context.getUserName());
return executeDDLQueryOnCluster(query_ptr, context);
}
std::optional<ExtendedRoleSet> roles_from_query;
if (query.roles)
roles_from_query = ExtendedRoleSet{*query.roles, access_control, context.getUserID()};
......
#include <Interpreters/InterpreterCreateRoleQuery.h>
#include <Parsers/ASTCreateRoleQuery.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Access/AccessControlManager.h>
#include <Access/Role.h>
......@@ -44,6 +45,9 @@ BlockIO InterpreterCreateRoleQuery::execute()
else
context.checkAccess(AccessType::CREATE_ROLE);
if (!query.cluster.empty())
return executeDDLQueryOnCluster(query_ptr, context);
std::optional<SettingsProfileElements> settings_from_query;
if (query.settings)
settings_from_query = SettingsProfileElements{*query.settings, access_control};
......
......@@ -3,6 +3,7 @@
#include <Parsers/ASTExtendedRoleSet.h>
#include <Parsers/formatAST.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessFlags.h>
#include <boost/range/algorithm/sort.hpp>
......@@ -63,10 +64,16 @@ namespace
BlockIO InterpreterCreateRowPolicyQuery::execute()
{
const auto & query = query_ptr->as<const ASTCreateRowPolicyQuery &>();
auto & query = query_ptr->as<ASTCreateRowPolicyQuery &>();
auto & access_control = context.getAccessControlManager();
context.checkAccess(query.alter ? AccessType::ALTER_ROW_POLICY : AccessType::CREATE_ROW_POLICY);
if (!query.cluster.empty())
{
query.replaceCurrentUserTagWithName(context.getUserName());
return executeDDLQueryOnCluster(query_ptr, context);
}
std::optional<ExtendedRoleSet> roles_from_query;
if (query.roles)
roles_from_query = ExtendedRoleSet{*query.roles, access_control, context.getUserID()};
......
#include <Interpreters/InterpreterCreateSettingsProfileQuery.h>
#include <Parsers/ASTCreateSettingsProfileQuery.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Access/AccessControlManager.h>
#include <Access/SettingsProfile.h>
#include <Access/AccessFlags.h>
......@@ -49,13 +51,19 @@ namespace
BlockIO InterpreterCreateSettingsProfileQuery::execute()
{
const auto & query = query_ptr->as<const ASTCreateSettingsProfileQuery &>();
auto & query = query_ptr->as<ASTCreateSettingsProfileQuery &>();
auto & access_control = context.getAccessControlManager();
if (query.alter)
context.checkAccess(AccessType::ALTER_SETTINGS_PROFILE);
else
context.checkAccess(AccessType::CREATE_SETTINGS_PROFILE);
if (!query.cluster.empty())
{
query.replaceCurrentUserTagWithName(context.getUserName());
return executeDDLQueryOnCluster(query_ptr, context);
}
std::optional<SettingsProfileElements> settings_from_query;
if (query.settings)
settings_from_query = SettingsProfileElements{*query.settings, access_control};
......
#include <Interpreters/InterpreterCreateUserQuery.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterSetRoleQuery.h>
#include <Interpreters/DDLWorker.h>
#include <Parsers/ASTCreateUserQuery.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Access/AccessControlManager.h>
#include <Access/User.h>
#include <Access/ExtendedRoleSet.h>
#include <Access/ContextAccess.h>
#include <boost/range/algorithm/copy.hpp>
......@@ -67,7 +68,7 @@ namespace
BlockIO InterpreterCreateUserQuery::execute()
{
const auto & query = query_ptr->as<const ASTCreateUserQuery &>();
auto & query = query_ptr->as<const ASTCreateUserQuery &>();
auto & access_control = context.getAccessControlManager();
auto access = context.getAccess();
access->checkAccess(query.alter ? AccessType::ALTER_USER : AccessType::CREATE_USER);
......@@ -83,6 +84,9 @@ BlockIO InterpreterCreateUserQuery::execute()
}
}
if (!query.cluster.empty())
return executeDDLQueryOnCluster(query_ptr, context);
std::optional<SettingsProfileElements> settings_from_query;
if (query.settings)
settings_from_query = SettingsProfileElements{*query.settings, access_control};
......
#include <Interpreters/InterpreterDropAccessEntityQuery.h>
#include <Parsers/ASTDropAccessEntityQuery.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessFlags.h>
#include <Access/User.h>
......@@ -52,6 +53,9 @@ BlockIO InterpreterDropAccessEntityQuery::execute()
std::type_index type = getType(query.kind);
context.checkAccess(getRequiredAccessType(query.kind));
if (!query.cluster.empty())
return executeDDLQueryOnCluster(query_ptr, context);
if (query.kind == Kind::ROW_POLICY)
{
Strings full_names;
......
#include <Interpreters/InterpreterGrantQuery.h>
#include <Parsers/ASTGrantQuery.h>
#include <Parsers/ASTExtendedRoleSet.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Access/AccessControlManager.h>
#include <Access/ContextAccess.h>
#include <Access/ExtendedRoleSet.h>
......@@ -59,7 +61,7 @@ namespace
BlockIO InterpreterGrantQuery::execute()
{
const auto & query = query_ptr->as<const ASTGrantQuery &>();
auto & query = query_ptr->as<ASTGrantQuery &>();
auto & access_control = context.getAccessControlManager();
auto access = context.getAccess();
access->checkGrantOption(query.access_rights_elements);
......@@ -72,6 +74,12 @@ BlockIO InterpreterGrantQuery::execute()
access->checkAdminOption(role_from_query);
}
if (!query.cluster.empty())
{
query.replaceCurrentUserTagWithName(context.getUserName());
return executeDDLQueryOnCluster(query_ptr, context);
}
std::vector<UUID> to_roles = ExtendedRoleSet{*query.to_roles, access_control, context.getUserID()}.getMatchingIDs(access_control);
String current_database = context.getCurrentDatabase();
......
......@@ -135,6 +135,8 @@ void ASTCreateQuotaQuery::formatImpl(const FormatSettings & settings, FormatStat
settings.ostr << " " << backQuoteIfNeed(name);
formatOnCluster(settings);
if (!new_name.empty())
formatRenameTo(new_name, settings);
......@@ -146,4 +148,12 @@ void ASTCreateQuotaQuery::formatImpl(const FormatSettings & settings, FormatStat
if (roles && (!roles->empty() || alter))
formatToRoles(*roles, settings);
}
void ASTCreateQuotaQuery::replaceCurrentUserTagWithName(const String & current_user_name)
{
if (roles)
roles->replaceCurrentUserTagWithName(current_user_name);
}
}
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Access/Quota.h>
......@@ -25,7 +26,7 @@ class ASTExtendedRoleSet;
* UNSET TRACKING} [,...]]
* [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
*/
class ASTCreateQuotaQuery : public IAST
class ASTCreateQuotaQuery : public IAST, public ASTQueryWithOnCluster
{
public:
bool alter = false;
......@@ -58,5 +59,7 @@ public:
String getID(char) const override;
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void replaceCurrentUserTagWithName(const String & current_user_name);
ASTPtr getRewrittenASTWithoutOnCluster(const std::string &) const override { return removeOnCluster<ASTCreateQuotaQuery>(clone()); }
};
}
......@@ -54,6 +54,8 @@ void ASTCreateRoleQuery::formatImpl(const FormatSettings & format, FormatState &
format.ostr << " " << backQuoteIfNeed(name);
formatOnCluster(format);
if (!new_name.empty())
formatRenameTo(new_name, format);
......
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOnCluster.h>
namespace DB
......@@ -15,7 +16,7 @@ class ASTSettingsProfileElements;
* [RENAME TO new_name]
* [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]
*/
class ASTCreateRoleQuery : public IAST
class ASTCreateRoleQuery : public IAST, public ASTQueryWithOnCluster
{
public:
bool alter = false;
......@@ -33,5 +34,6 @@ public:
String getID(char) const override;
ASTPtr clone() const override;
void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const std::string &) const override { return removeOnCluster<ASTCreateRoleQuery>(clone()); }
};
}
......@@ -157,6 +157,8 @@ void ASTCreateRowPolicyQuery::formatImpl(const FormatSettings & settings, Format
settings.ostr << " " << backQuoteIfNeed(policy_name) << (settings.hilite ? hilite_keyword : "") << " ON "
<< (settings.hilite ? hilite_none : "") << (database.empty() ? String{} : backQuoteIfNeed(database) + ".") << table_name;
formatOnCluster(settings);
if (!new_policy_name.empty())
formatRenameTo(new_policy_name, settings);
......@@ -168,4 +170,11 @@ void ASTCreateRowPolicyQuery::formatImpl(const FormatSettings & settings, Format
if (roles && (!roles->empty() || alter))
formatToRoles(*roles, settings);
}
void ASTCreateRowPolicyQuery::replaceCurrentUserTagWithName(const String & current_user_name)
{
if (roles)
roles->replaceCurrentUserTagWithName(current_user_name);
}
}
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Access/RowPolicy.h>
#include <utility>
#include <vector>
......@@ -25,7 +26,7 @@ class ASTExtendedRoleSet;
* [WITH CHECK {condition | NONE}] [,...]
* [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
*/
class ASTCreateRowPolicyQuery : public IAST
class ASTCreateRowPolicyQuery : public IAST, public ASTQueryWithOnCluster
{
public:
bool alter = false;
......@@ -47,5 +48,7 @@ public:
String getID(char) const override;
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void replaceCurrentUserTagWithName(const String & current_user_name);
ASTPtr getRewrittenASTWithoutOnCluster(const std::string &) const override { return removeOnCluster<ASTCreateRowPolicyQuery>(clone()); }
};
}
......@@ -61,6 +61,8 @@ void ASTCreateSettingsProfileQuery::formatImpl(const FormatSettings & format, Fo
format.ostr << " " << backQuoteIfNeed(name);
formatOnCluster(format);
if (!new_name.empty())
formatRenameTo(new_name, format);
......@@ -71,4 +73,10 @@ void ASTCreateSettingsProfileQuery::formatImpl(const FormatSettings & format, Fo
formatToRoles(*to_roles, format);
}
void ASTCreateSettingsProfileQuery::replaceCurrentUserTagWithName(const String & current_user_name)
{
if (to_roles)
to_roles->replaceCurrentUserTagWithName(current_user_name);
}
}
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOnCluster.h>
namespace DB
......@@ -16,7 +17,7 @@ class ASTExtendedRoleSet;
* [RENAME TO new_name]
* [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]
*/
class ASTCreateSettingsProfileQuery : public IAST
class ASTCreateSettingsProfileQuery : public IAST, public ASTQueryWithOnCluster
{
public:
bool alter = false;
......@@ -36,5 +37,7 @@ public:
String getID(char) const override;
ASTPtr clone() const override;
void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override;
void replaceCurrentUserTagWithName(const String & current_user_name);
ASTPtr getRewrittenASTWithoutOnCluster(const std::string &) const override { return removeOnCluster<ASTCreateSettingsProfileQuery>(clone()); }
};
}
......@@ -184,6 +184,8 @@ void ASTCreateUserQuery::formatImpl(const FormatSettings & format, FormatState &
format.ostr << " " << backQuoteIfNeed(name);
formatOnCluster(format);
if (!new_name.empty())
formatRenameTo(new_name, format);
......
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Access/Authentication.h>
#include <Access/AllowedClientHosts.h>
......@@ -23,7 +24,7 @@ class ASTSettingsProfileElements;
* [DEFAULT ROLE role [,...] | ALL | ALL EXCEPT role [,...] ]
* [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]
*/
class ASTCreateUserQuery : public IAST
class ASTCreateUserQuery : public IAST, public ASTQueryWithOnCluster
{
public:
bool alter = false;
......@@ -49,5 +50,6 @@ public:
String getID(char) const override;
ASTPtr clone() const override;
void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const std::string &) const override { return removeOnCluster<ASTCreateUserQuery>(clone()); }
};
}
......@@ -75,5 +75,7 @@ void ASTDropAccessEntityQuery::formatImpl(const FormatSettings & settings, Forma
settings.ostr << ' ' << backQuoteIfNeed(name);
}
}
formatOnCluster(settings);
}
}
......@@ -2,6 +2,7 @@
#include <Parsers/IAST.h>
#include <Access/RowPolicy.h>
#include <Parsers/ASTQueryWithOnCluster.h>
namespace DB
......@@ -13,7 +14,7 @@ namespace DB
* DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...]
* DROP [SETTINGS] PROFILE [IF EXISTS] name [,...]
*/
class ASTDropAccessEntityQuery : public IAST
class ASTDropAccessEntityQuery : public IAST, public ASTQueryWithOnCluster
{
public:
enum class Kind
......@@ -34,5 +35,6 @@ public:
String getID(char) const override;
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const std::string &) const override { return removeOnCluster<ASTDropAccessEntityQuery>(clone()); }
};
}
......@@ -72,4 +72,21 @@ void ASTExtendedRoleSet::formatImpl(const FormatSettings & settings, FormatState
}
}
}
void ASTExtendedRoleSet::replaceCurrentUserTagWithName(const String & current_user_name)
{
if (current_user)
{
names.push_back(current_user_name);
current_user = false;
}
if (except_current_user)
{
except_names.push_back(current_user_name);
except_current_user = false;
}
}
}
......@@ -18,6 +18,7 @@ public:
bool id_mode = false; /// If true then `names` and `except_names` keeps UUIDs, not names.
bool empty() const { return names.empty() && !current_user && !all; }
void replaceCurrentUserTagWithName(const String & current_user_name);
String getID(char) const override { return "ExtendedRoleSet"; }
ASTPtr clone() const override { return std::make_shared<ASTExtendedRoleSet>(*this); }
......
......@@ -122,19 +122,22 @@ ASTPtr ASTGrantQuery::clone() const
void ASTGrantQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << (attach ? "ATTACH " : "") << ((kind == Kind::GRANT) ? "GRANT" : "REVOKE")
<< (settings.hilite ? IAST::hilite_none : "") << " ";
<< (settings.hilite ? IAST::hilite_none : "");
formatOnCluster(settings);
if (kind == Kind::REVOKE)
{
if (grant_option)
settings.ostr << (settings.hilite ? hilite_keyword : "") << "GRANT OPTION FOR " << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_keyword : "") << " GRANT OPTION FOR" << (settings.hilite ? hilite_none : "");
else if (admin_option)
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADMIN OPTION FOR " << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_keyword : "") << " ADMIN OPTION FOR" << (settings.hilite ? hilite_none : "");
}
if ((!!roles + !access_rights_elements.empty()) != 1)
throw Exception("Either roles or access rights elements should be set", ErrorCodes::LOGICAL_ERROR);
settings.ostr << " ";
if (roles)
roles->format(settings);
else
......@@ -150,4 +153,11 @@ void ASTGrantQuery::formatImpl(const FormatSettings & settings, FormatState &, F
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH ADMIN OPTION" << (settings.hilite ? hilite_none : "");
}
}
void ASTGrantQuery::replaceCurrentUserTagWithName(const String & current_user_name)
{
if (to_roles)
to_roles->replaceCurrentUserTagWithName(current_user_name);
}
}
......@@ -2,6 +2,7 @@
#include <Parsers/IAST.h>
#include <Access/AccessRightsElement.h>
#include <Parsers/ASTQueryWithOnCluster.h>
namespace DB
......@@ -15,7 +16,7 @@ class ASTExtendedRoleSet;
* GRANT role [,...] TO {user_name | role_name | CURRENT_USER} [,...] [WITH ADMIN OPTION]
* REVOKE [ADMIN OPTION FOR] role [,...] FROM {user_name | role_name | CURRENT_USER} [,...] | ALL | ALL EXCEPT {user_name | role_name | CURRENT_USER} [,...]
*/
class ASTGrantQuery : public IAST
class ASTGrantQuery : public IAST, public ASTQueryWithOnCluster
{
public:
enum class Kind
......@@ -34,5 +35,7 @@ public:
String getID(char) const override;
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void replaceCurrentUserTagWithName(const String & current_user_name);
ASTPtr getRewrittenASTWithoutOnCluster(const std::string &) const override { return removeOnCluster<ASTGrantQuery>(clone()); }
};
}
......@@ -238,6 +238,13 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
if (!parseIdentifierOrStringLiteral(pos, expected, name))
return false;
String cluster;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster, expected))
return false;
}
String new_name;
std::optional<KeyType> key_type;
std::vector<ASTCreateQuotaQuery::Limits> all_limits;
......@@ -266,6 +273,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
query->if_exists = if_exists;
query->if_not_exists = if_not_exists;
query->or_replace = or_replace;
query->cluster = std::move(cluster);
query->name = std::move(name);
query->new_name = std::move(new_name);
query->key_type = key_type;
......
......@@ -80,6 +80,13 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (!parseRoleName(pos, expected, name))
return false;
String cluster;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster, expected))
return false;
}
String new_name;
std::shared_ptr<ASTSettingsProfileElements> settings;
while (true)
......@@ -101,6 +108,7 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
query->if_exists = if_exists;
query->if_not_exists = if_not_exists;
query->or_replace = or_replace;
query->cluster = std::move(cluster);
query->name = std::move(name);
query->new_name = std::move(new_name);
query->settings = std::move(settings);
......
......@@ -243,6 +243,13 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|| !parseDatabaseAndTableName(pos, expected, database, table_name))
return false;
String cluster;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster, expected))
return false;
}
String new_policy_name;
std::optional<bool> is_restrictive;
std::vector<std::pair<ConditionType, ASTPtr>> conditions;
......@@ -272,6 +279,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
query->if_exists = if_exists;
query->if_not_exists = if_not_exists;
query->or_replace = or_replace;
query->cluster = std::move(cluster);
query->name_parts = std::move(name_parts);
query->new_policy_name = std::move(new_policy_name);
query->is_restrictive = is_restrictive;
......
......@@ -96,6 +96,13 @@ bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expec
if (!parseIdentifierOrStringLiteral(pos, expected, name))
return false;
String cluster;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster, expected))
return false;
}
String new_name;
std::shared_ptr<ASTSettingsProfileElements> settings;
while (true)
......@@ -120,6 +127,7 @@ bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expec
query->if_exists = if_exists;
query->if_not_exists = if_not_exists;
query->or_replace = or_replace;
query->cluster = std::move(cluster);
query->name = std::move(name);
query->new_name = std::move(new_name);
query->settings = std::move(settings);
......
......@@ -290,6 +290,13 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (!parseUserName(pos, expected, name, host_pattern))
return false;
String cluster;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster, expected))
return false;
}
String new_name;
std::optional<String> new_host_pattern;
std::optional<Authentication> authentication;
......@@ -341,6 +348,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
query->if_exists = if_exists;
query->if_not_exists = if_not_exists;
query->or_replace = or_replace;
query->cluster = std::move(cluster);
query->name = std::move(name);
query->new_name = std::move(new_name);
query->authentication = std::move(authentication);
......
......@@ -117,10 +117,18 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
return false;
}
String cluster;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster, expected))
return false;
}
auto query = std::make_shared<ASTDropAccessEntityQuery>(kind);
node = query;
query->if_exists = if_exists;
query->cluster = std::move(cluster);
query->names = std::move(names);
query->row_policies_names = std::move(row_policies_names);
......
......@@ -259,6 +259,13 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
else
return false;
String cluster;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster, expected))
return false;
}
bool grant_option = false;
bool admin_option = false;
if (kind == Kind::REVOKE)
......@@ -296,6 +303,7 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
query->kind = kind;
query->attach = attach;
query->cluster = std::move(cluster);
query->access_rights_elements = std::move(elements);
query->roles = std::move(roles);
query->to_roles = std::move(to_roles);
......
<yandex>
<remote_servers>
<cluster>
<shard>
<replica>
<host>ch1</host>
<port>9000</port>
</replica>
<replica>
<host>ch2</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>ch3</host>
<port>9000</port>
</replica>
</shard>
</cluster>
</remote_servers>
</yandex>
<yandex>
<users>
<default>
<access_management>1</access_management>
</default>
</users>
</yandex>
import time
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.client import QueryRuntimeException
cluster = ClickHouseCluster(__file__)
ch1 = cluster.add_instance('ch1', config_dir="configs", with_zookeeper=True)
ch2 = cluster.add_instance('ch2', config_dir="configs", with_zookeeper=True)
ch3 = cluster.add_instance('ch3', config_dir="configs", with_zookeeper=True)
@pytest.fixture(scope="module", autouse=True)
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_access_control_on_cluster():
ch1.query("CREATE USER Alex ON CLUSTER 'cluster'")
assert ch1.query("SHOW CREATE USER Alex") == "CREATE USER Alex\n"
assert ch2.query("SHOW CREATE USER Alex") == "CREATE USER Alex\n"
assert ch3.query("SHOW CREATE USER Alex") == "CREATE USER Alex\n"
ch2.query("GRANT ON CLUSTER 'cluster' SELECT ON *.* TO Alex")
assert ch1.query("SHOW GRANTS FOR Alex") == "GRANT SELECT ON *.* TO Alex\n"
assert ch2.query("SHOW GRANTS FOR Alex") == "GRANT SELECT ON *.* TO Alex\n"
assert ch3.query("SHOW GRANTS FOR Alex") == "GRANT SELECT ON *.* TO Alex\n"
ch3.query("REVOKE ON CLUSTER 'cluster' SELECT ON *.* FROM Alex")
assert ch1.query("SHOW GRANTS FOR Alex") == ""
assert ch2.query("SHOW GRANTS FOR Alex") == ""
assert ch3.query("SHOW GRANTS FOR Alex") == ""
ch2.query("DROP USER Alex ON CLUSTER 'cluster'")
assert "User `Alex` not found" in ch1.query_and_get_error("SHOW CREATE USER Alex")
assert "User `Alex` not found" in ch2.query_and_get_error("SHOW CREATE USER Alex")
assert "User `Alex` not found" in ch3.query_and_get_error("SHOW CREATE USER Alex")
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册