diff --git a/ci/jobs/scripts/check_style/aspell-ignore/en/aspell-dict.txt b/ci/jobs/scripts/check_style/aspell-ignore/en/aspell-dict.txt index 78dd223eea76..32293917c69a 100644 --- a/ci/jobs/scripts/check_style/aspell-ignore/en/aspell-dict.txt +++ b/ci/jobs/scripts/check_style/aspell-ignore/en/aspell-dict.txt @@ -1355,6 +1355,7 @@ atanh atomicity auth authType +authenticatedUser authenticator authenticators autocompletion diff --git a/docs/en/sql-reference/statements/execute_as.md b/docs/en/sql-reference/statements/execute_as.md new file mode 100644 index 000000000000..78e4edb12ec0 --- /dev/null +++ b/docs/en/sql-reference/statements/execute_as.md @@ -0,0 +1,42 @@ +--- +description: 'Documentation for EXECUTE AS Statement' +sidebar_label: 'EXECUTE AS' +sidebar_position: 53 +slug: /sql-reference/statements/execute_as +title: 'EXECUTE AS Statement' +doc_type: 'reference' +--- + +# EXECUTE AS Statement + +Allows to execute queries on behalf of a different user. + +## Syntax {#syntax} + +```sql +EXECUTE AS target_user; +EXECUTE AS target_user subquery; +``` + +The first form (without `subquery`) sets that all the following queries in the current session will be executed on behalf of the specified `target_user`. + +The second form (with `subquery`) executes only the specified `subquery` on behalf of the specified `target_user`. + +In order to work both forms require server setting [allow_impersonate_user](/operations/server-configuration-parameters/settings#allow_impersonate_user) +to be set to `1` and the `IMPERSONATE` privilege to be granted. For example, the following commands +```sql +GRANT IMPERSONATE ON user1 TO user2; +GRANT IMPERSONATE ON * TO user3; +``` +allow user `user2` to execute commands `EXECUTE AS user1 ...` and also allow user `user3` to execute commands as any user. + +While impersonating another user function [currentUser()](/sql-reference/functions/other-functions#currentUser) returns the name of that other user, +and function [authenticatedUser()](/sql-reference/functions/other-functions#authenticatedUser) returns the name of the user who has been actually authenticated. + +## Examples {#examples} + +```sql +SELECT currentUser(), authenticatedUser(); -- outputs "default default" +CREATE USER james; +EXECUTE AS james SELECT currentUser(), authenticatedUser(); -- outputs "james default" +``` diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index ec64b2870e24..f50a057b1d1e 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -154,6 +154,7 @@ enum class AccessType : uint8_t M(SHOW_QUOTAS, "SHOW CREATE QUOTA", GLOBAL, SHOW_ACCESS) \ M(SHOW_SETTINGS_PROFILES, "SHOW PROFILES, SHOW CREATE SETTINGS PROFILE, SHOW CREATE PROFILE", GLOBAL, SHOW_ACCESS) \ M(SHOW_ACCESS, "", GROUP, ACCESS_MANAGEMENT) \ + M(IMPERSONATE, "EXECUTE AS", USER_NAME, ACCESS_MANAGEMENT) \ M(ACCESS_MANAGEMENT, "", GROUP, ALL) \ M(SHOW_NAMED_COLLECTIONS, "SHOW NAMED COLLECTIONS", NAMED_COLLECTION, NAMED_COLLECTION_ADMIN) \ M(SHOW_NAMED_COLLECTIONS_SECRETS, "SHOW NAMED COLLECTIONS SECRETS", NAMED_COLLECTION, NAMED_COLLECTION_ADMIN) \ diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 125ab86d2137..a48159707180 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -1019,8 +1019,8 @@ namespace DB true ``` )", 0) \ - DECLARE(Bool, storage_shared_set_join_use_inner_uuid, false, "If enabled, an inner UUID is generated during the creation of SharedSet and SharedJoin. ClickHouse Cloud only", 0) - + DECLARE(Bool, storage_shared_set_join_use_inner_uuid, false, "If enabled, an inner UUID is generated during the creation of SharedSet and SharedJoin. ClickHouse Cloud only", 0) \ + DECLARE(Bool, allow_impersonate_user, false, R"(Enable/disable the IMPERSONATE feature (EXECUTE AS target_user).)", 0) \ // clang-format on diff --git a/src/Functions/authenticatedUser.cpp b/src/Functions/authenticatedUser.cpp new file mode 100644 index 000000000000..c4a0d50784fc --- /dev/null +++ b/src/Functions/authenticatedUser.cpp @@ -0,0 +1,82 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + +class FunctionAuthenticatedUser : public IFunction +{ + const String user_name; + +public: + static constexpr auto name = "authenticatedUser"; + static FunctionPtr create(ContextPtr context) + { + return std::make_shared(context->getClientInfo().authenticated_user); + } + + explicit FunctionAuthenticatedUser(const String & user_name_) : user_name{user_name_} + { + } + + String getName() const override + { + return name; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + bool isDeterministic() const override { return false; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override + { + return DataTypeString().createColumnConst(input_rows_count, user_name); + } +}; + +} + +REGISTER_FUNCTION(AuthenticatedUser) +{ + factory.registerFunction(FunctionDocumentation{ + .description=R"( +If the session user has been switched using the EXECUTE AS command, this function returns the name of the original user that was used for authentication and creating the session. +Alias: authUser() + )", + .syntax=R"(authenticatedUser())", + .arguments={}, + .returned_value="String - The name of the authenticated user.", + .examples{ + {"Usage example", + R"( + EXECUTE as u1; + SELECT currentUser(), authenticatedUser(); + )", + R"( +┌─currentUser()─┬─authenticatedUser()─┐ +│ u1 │ default │ +└───────────────┴─────────────────────┘ + )" + }}, + .category = {"Other"} + }); + + factory.registerAlias("authUser", "authenticatedUser"); +} + +} diff --git a/src/Interpreters/Access/InterpreterExecuteAsQuery.cpp b/src/Interpreters/Access/InterpreterExecuteAsQuery.cpp new file mode 100644 index 000000000000..b4b85c23a84e --- /dev/null +++ b/src/Interpreters/Access/InterpreterExecuteAsQuery.cpp @@ -0,0 +1,118 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SUPPORT_IS_DISABLED; +} + +namespace ServerSetting +{ + extern const ServerSettingsBool allow_impersonate_user; +} + +namespace +{ + /// Creates another query context to execute a query as another user. + ContextMutablePtr impersonateQueryContext(ContextPtr context, const String & target_user_name) + { + auto new_context = Context::createCopy(context->getGlobalContext()); + new_context->setClientInfo(context->getClientInfo()); + new_context->makeQueryContext(); + + const auto & database = context->getCurrentDatabase(); + if (!database.empty() && database != new_context->getCurrentDatabase()) + new_context->setCurrentDatabase(database); + + new_context->setInsertionTable(context->getInsertionTable(), context->getInsertionTableColumnNames()); + new_context->setProgressCallback(context->getProgressCallback()); + new_context->setProcessListElement(context->getProcessListElement()); + + if (context->getCurrentTransaction()) + new_context->setCurrentTransaction(context->getCurrentTransaction()); + + if (context->getZooKeeperMetadataTransaction()) + new_context->initZooKeeperMetadataTransaction(context->getZooKeeperMetadataTransaction()); + + new_context->setUser(context->getAccessControl().getID(target_user_name)); + + /// We need to update the client info to make currentUser() return `target_user_name`. + new_context->setCurrentUserName(target_user_name); + new_context->setInitialUserName(target_user_name); + + auto changed_settings = context->getSettingsRef().changes(); + new_context->clampToSettingsConstraints(changed_settings, SettingSource::QUERY); + new_context->applySettingsChanges(changed_settings); + + return new_context; + } + + /// Changes the session context to execute all following queries in this session as another user. + void impersonateSessionContext(ContextMutablePtr context, const String & target_user_name) + { + auto database = context->getCurrentDatabase(); + auto changed_settings = context->getSettingsRef().changes(); + + context->setUser(context->getAccessControl().getID(target_user_name)); + + /// We need to update the client info to make currentUser() return `target_user_name`. + context->setCurrentUserName(target_user_name); + context->setInitialUserName(target_user_name); + + context->clampToSettingsConstraints(changed_settings, SettingSource::QUERY); + context->applySettingsChanges(changed_settings); + + if (!database.empty() && database != context->getCurrentDatabase()) + context->setCurrentDatabase(database); + } +} + + +BlockIO InterpreterExecuteAsQuery::execute() +{ + if (!getContext()->getGlobalContext()->getServerSettings()[ServerSetting::allow_impersonate_user]) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "IMPERSONATE feature is disabled, set allow_impersonate_user to 1 to enable"); + + const auto & query = query_ptr->as(); + String target_user_name = query.target_user->toString(); + getContext()->checkAccess(AccessType::IMPERSONATE, target_user_name); + + if (query.subquery) + { + /// EXECUTE AS + auto subquery_context = impersonateQueryContext(getContext(), target_user_name); + return executeQuery(query.subquery->formatWithSecretsOneLine(), subquery_context, QueryFlags{ .internal = true }).second; + } + else + { + /// EXECUTE AS + impersonateSessionContext(getContext()->getSessionContext(), target_user_name); + return {}; + } +} + + +void registerInterpreterExecuteAsQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterExecuteAsQuery", create_fn); +} + +} diff --git a/src/Interpreters/Access/InterpreterExecuteAsQuery.h b/src/Interpreters/Access/InterpreterExecuteAsQuery.h new file mode 100644 index 000000000000..24b9e6f0eb3c --- /dev/null +++ b/src/Interpreters/Access/InterpreterExecuteAsQuery.h @@ -0,0 +1,20 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class InterpreterExecuteAsQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterExecuteAsQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} + BlockIO execute() override; + +private: + ASTPtr query_ptr; +}; + +} diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index 647eff601da9..3a946a4f702a 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -63,6 +63,9 @@ class ClientInfo String current_query_id; std::shared_ptr current_address; + /// For IMPERSONATEd session, stores the original authenticated user + String authenticated_user; + /// When query_kind == INITIAL_QUERY, these values are equal to current. String initial_user; String initial_query_id; diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 77a001641312..61b2bcf49069 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -53,6 +53,7 @@ #include #include #include +#include #include #include @@ -382,6 +383,10 @@ InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, Conte { interpreter_name = "InterpreterParallelWithQuery"; } + else if (query->as()) + { + interpreter_name = "InterpreterExecuteAsQuery"; + } if (!interpreters.contains(interpreter_name)) throw Exception(ErrorCodes::UNKNOWN_TYPE_OF_QUERY, "Unknown type of query: {}", query->getID()); diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index 6debee7c494e..73be47e46c15 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -103,6 +103,7 @@ ColumnsDescription QueryLogElement::getColumnsDescription() {"initial_port", std::make_shared(), "The client port that was used to make the parent query."}, {"initial_query_start_time", std::make_shared(), "Initial query starting time (for distributed query execution)."}, {"initial_query_start_time_microseconds", std::make_shared(6), "Initial query starting time with microseconds precision (for distributed query execution)."}, + {"authenticated_user", low_cardinality_string, "Name of the user who was authenticated in the session."}, {"interface", std::make_shared(), "Interface that the query was initiated from. Possible values: 1 — TCP, 2 — HTTP."}, {"is_secure", std::make_shared(), "The flag whether a query was executed over a secure interface"}, {"os_user", low_cardinality_string, "Operating system username who runs clickhouse-client."}, @@ -331,6 +332,8 @@ void QueryLogElement::appendClientInfo(const ClientInfo & client_info, MutableCo columns[i++]->insert(client_info.initial_query_start_time); columns[i++]->insert(client_info.initial_query_start_time_microseconds); + columns[i++]->insertData(client_info.authenticated_user.data(), client_info.authenticated_user.size()); + columns[i++]->insert(static_cast(client_info.interface)); columns[i++]->insert(static_cast(client_info.is_secure)); diff --git a/src/Interpreters/QueryThreadLog.cpp b/src/Interpreters/QueryThreadLog.cpp index c46ee278c6a0..feba2a3e88e3 100644 --- a/src/Interpreters/QueryThreadLog.cpp +++ b/src/Interpreters/QueryThreadLog.cpp @@ -61,6 +61,7 @@ ColumnsDescription QueryThreadLogElement::getColumnsDescription() {"initial_port", std::make_shared(), "The client port that was used to make the parent query."}, {"initial_query_start_time", std::make_shared(), "Start time of the initial query execution."}, {"initial_query_start_time_microseconds", std::make_shared(6), "Start time of the initial query execution "}, + {"authenticated_user", low_cardinality_string, "Name of the user who was authenticated in the session."}, {"interface", std::make_shared(), "Interface that the query was initiated from. Possible values: 1 — TCP, 2 — HTTP."}, {"is_secure", std::make_shared(), "The flag which shows whether the connection was secure."}, {"os_user", low_cardinality_string, "OSs username who runs clickhouse-client."}, diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 15b453137dcf..03a768927c14 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -390,6 +390,7 @@ void Session::authenticate(const Credentials & credentials_, const Poco::Net::So } prepared_client_info->current_user = credentials_.getUserName(); + prepared_client_info->authenticated_user = credentials_.getUserName(); prepared_client_info->current_address = std::make_shared(address); } diff --git a/src/Interpreters/registerInterpreters.cpp b/src/Interpreters/registerInterpreters.cpp index 4beb3d173e73..f6eb1dddeade 100644 --- a/src/Interpreters/registerInterpreters.cpp +++ b/src/Interpreters/registerInterpreters.cpp @@ -63,6 +63,7 @@ void registerInterpreterDropIndexQuery(InterpreterFactory & factory); void registerInterpreterBackupQuery(InterpreterFactory & factory); void registerInterpreterDeleteQuery(InterpreterFactory & factory); void registerInterpreterParallelWithQuery(InterpreterFactory & factory); +void registerInterpreterExecuteAsQuery(InterpreterFactory & factory); void registerInterpreters() { @@ -128,5 +129,6 @@ void registerInterpreters() registerInterpreterBackupQuery(factory); registerInterpreterDeleteQuery(factory); registerInterpreterParallelWithQuery(factory); + registerInterpreterExecuteAsQuery(factory); } } diff --git a/src/Parsers/Access/ASTExecuteAsQuery.cpp b/src/Parsers/Access/ASTExecuteAsQuery.cpp new file mode 100644 index 000000000000..52e463b96d44 --- /dev/null +++ b/src/Parsers/Access/ASTExecuteAsQuery.cpp @@ -0,0 +1,42 @@ +#include + +#include +#include + + +namespace DB +{ + +String ASTExecuteAsQuery::getID(char) const +{ + return "ExecuteAsQuery"; +} + + +ASTPtr ASTExecuteAsQuery::clone() const +{ + auto res = std::make_shared(*this); + + if (target_user) + res->set(res->target_user, target_user->clone()); + if (subquery) + res->set(res->subquery, subquery->clone()); + + return res; +} + + +void ASTExecuteAsQuery::formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + ostr << "EXECUTE AS "; + + target_user->format(ostr, settings); + + if (subquery) + { + ostr << settings.nl_or_ws; + subquery->format(ostr, settings, state, frame); + } +} + +} diff --git a/src/Parsers/Access/ASTExecuteAsQuery.h b/src/Parsers/Access/ASTExecuteAsQuery.h new file mode 100644 index 000000000000..d8bb939a6fff --- /dev/null +++ b/src/Parsers/Access/ASTExecuteAsQuery.h @@ -0,0 +1,27 @@ +#pragma once + +#include + + +namespace DB +{ +class ASTUserNameWithHost; + +/** EXECUTE AS + * or + * EXECUTE AS + */ +class ASTExecuteAsQuery : public ASTQueryWithOutput +{ +public: + ASTUserNameWithHost * target_user; + IAST * subquery = nullptr; + + String getID(char) const override; + ASTPtr clone() const override; +protected: + void formatQueryImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState &, FormatStateStacked) const override; + +}; + +} diff --git a/src/Parsers/Access/ParserExecuteAsQuery.cpp b/src/Parsers/Access/ParserExecuteAsQuery.cpp new file mode 100644 index 000000000000..3689801e30f3 --- /dev/null +++ b/src/Parsers/Access/ParserExecuteAsQuery.cpp @@ -0,0 +1,40 @@ +#include + +#include +#include +#include +#include + + +namespace DB +{ + +ParserExecuteAsQuery::ParserExecuteAsQuery(IParser & subquery_parser_) + : subquery_parser(subquery_parser_) +{ +} + +bool ParserExecuteAsQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + if (!ParserKeyword{Keyword::EXECUTE_AS}.ignore(pos, expected)) + return false; + + ASTPtr target_user; + if (!ParserUserNameWithHost{}.parse(pos, target_user, expected)) + return false; + + auto query = std::make_shared(); + node = query; + + query->set(query->target_user, target_user); + + /// support 1) EXECUTE AS 2) EXECUTE AS SELECT ... + + ASTPtr subquery; + if (subquery_parser.parse(pos, subquery, expected)) + query->set(query->subquery, subquery); + + return true; +} + +} diff --git a/src/Parsers/Access/ParserExecuteAsQuery.h b/src/Parsers/Access/ParserExecuteAsQuery.h new file mode 100644 index 000000000000..f9b74a268793 --- /dev/null +++ b/src/Parsers/Access/ParserExecuteAsQuery.h @@ -0,0 +1,26 @@ +#pragma once + +#include + + +namespace DB +{ +/** Parses queries like : + * EXECUTE AS + * or + * EXECUTE AS + */ +class ParserExecuteAsQuery : public IParserBase +{ +public: + explicit ParserExecuteAsQuery(IParser & subquery_parser_); + const char * getName() const override { return "EXECUTE AS query"; } + +protected: + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + +private: + IParser & subquery_parser; +}; + +} diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 8d69ae5d5fe5..ff4132b49d03 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -184,6 +184,7 @@ namespace DB MR_MACROS(EXCHANGE_DICTIONARIES, "EXCHANGE DICTIONARIES") \ MR_MACROS(EXCHANGE_TABLES, "EXCHANGE TABLES") \ MR_MACROS(EXECUTE, "EXECUTE") \ + MR_MACROS(EXECUTE_AS, "EXECUTE AS") \ MR_MACROS(EXISTS, "EXISTS") \ MR_MACROS(EXPLAIN, "EXPLAIN") \ MR_MACROS(EXPRESSION, "EXPRESSION") \ diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 9c355794b6d8..22866ebb8584 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -35,6 +35,7 @@ #include #include #include +#include namespace DB @@ -104,6 +105,14 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || transaction_control_p.parse(pos, node, expected) || delete_p.parse(pos, node, expected); + if (!res && allow_execute_as) + { + ParserQuery subquery_p{end, allow_settings_after_format_in_insert, implicit_select}; + subquery_p.allow_execute_as = false; + ParserExecuteAsQuery execute_as_p{subquery_p}; + res = execute_as_p.parse(pos, node, expected); + } + if (res && allow_in_parallel_with) { ParserQuery subquery_p{end, allow_settings_after_format_in_insert, implicit_select}; diff --git a/src/Parsers/ParserQuery.h b/src/Parsers/ParserQuery.h index 565fb3d335c8..caa44232965e 100644 --- a/src/Parsers/ParserQuery.h +++ b/src/Parsers/ParserQuery.h @@ -13,6 +13,7 @@ class ParserQuery : public IParserBase bool allow_settings_after_format_in_insert = false; bool implicit_select = false; + bool allow_execute_as = true; bool allow_in_parallel_with = true; const char * getName() const override { return "Query"; } diff --git a/tests/config/config.d/allow_impersonate_user.xml b/tests/config/config.d/allow_impersonate_user.xml new file mode 100644 index 000000000000..903ea479cedb --- /dev/null +++ b/tests/config/config.d/allow_impersonate_user.xml @@ -0,0 +1,3 @@ + + 1 + diff --git a/tests/config/install.sh b/tests/config/install.sh index 53bd3a6cc684..38b64e1cfaff 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -87,6 +87,7 @@ ln -sf $SRC_PATH/config.d/handlers.yaml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/threadpool_writer_pool_size.yaml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/serverwide_trace_collector.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/rocksdb.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/allow_impersonate_user.xml $DEST_SERVER_PATH/config.d/ # Not supported with fasttest. if [ "$FAST_TEST" != "1" ]; then diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 0ac4f68abc21..27bfb4517aef 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -105,6 +105,7 @@ SHOW ROW POLICIES ['SHOW POLICIES','SHOW CREATE ROW POLICY','SHOW CREATE POLICY' SHOW QUOTAS ['SHOW CREATE QUOTA'] GLOBAL SHOW ACCESS SHOW SETTINGS PROFILES ['SHOW PROFILES','SHOW CREATE SETTINGS PROFILE','SHOW CREATE PROFILE'] GLOBAL SHOW ACCESS SHOW ACCESS [] \N ACCESS MANAGEMENT +IMPERSONATE ['EXECUTE AS'] USER_NAME ACCESS MANAGEMENT ACCESS MANAGEMENT [] \N ALL SHOW NAMED COLLECTIONS ['SHOW NAMED COLLECTIONS'] NAMED_COLLECTION NAMED COLLECTION ADMIN SHOW NAMED COLLECTIONS SECRETS ['SHOW NAMED COLLECTIONS SECRETS'] NAMED_COLLECTION NAMED COLLECTION ADMIN diff --git a/tests/queries/0_stateless/03252_execute_as.reference b/tests/queries/0_stateless/03252_execute_as.reference new file mode 100644 index 000000000000..bd68735b916a --- /dev/null +++ b/tests/queries/0_stateless/03252_execute_as.reference @@ -0,0 +1,34 @@ +default default + +--- EXECUTE AS app_user1 --- +appuser1_default testuser_default +0 + +appuser1_default testuser_default +testuser_default testuser_default + +0 +testuser_default testuser_default + +--- EXECUTE AS app_user2 --- +appuser2_default testuser_default + +OK + +appuser2_default testuser_default +testuser_default testuser_default + +OK + +--- EXECUTE AS other_user --- +OK + +OK + +--- Multiple EXECUTE AS --- +appuser1_default testuser_default +appuser2_default testuser_default +appuser1_default testuser_default +testuser_default testuser_default + +OK diff --git a/tests/queries/0_stateless/03252_execute_as.sh b/tests/queries/0_stateless/03252_execute_as.sh new file mode 100755 index 000000000000..1766484c8ec3 --- /dev/null +++ b/tests/queries/0_stateless/03252_execute_as.sh @@ -0,0 +1,52 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +TEST_DATABASE=${CLICKHOUSE_DATABASE} + +TEST_USER="testuser_${CLICKHOUSE_DATABASE}" +SERVICE_USER="serviceuser_${CLICKHOUSE_DATABASE}" +APP_USER1="appuser1_${CLICKHOUSE_DATABASE}" +APP_USER2="appuser2_${CLICKHOUSE_DATABASE}" +OTHER_USER="otheruser_${CLICKHOUSE_DATABASE}" + +$CLICKHOUSE_CLIENT --query "DROP USER IF EXISTS ${TEST_USER}, ${SERVICE_USER}, ${APP_USER1}, ${APP_USER2}, ${OTHER_USER}" +$CLICKHOUSE_CLIENT --query "CREATE USER ${TEST_USER}, ${SERVICE_USER}, ${APP_USER1}, ${APP_USER2}, ${OTHER_USER}" + +$CLICKHOUSE_CLIENT --query "GRANT ALL ON ${TEST_DATABASE}.* TO ${SERVICE_USER} WITH GRANT OPTION" +$CLICKHOUSE_CLIENT --query "GRANT IMPERSONATE ON ${APP_USER1} TO ${TEST_USER}" +$CLICKHOUSE_CLIENT --query "GRANT IMPERSONATE ON ${APP_USER2} TO ${TEST_USER}" + +$CLICKHOUSE_CLIENT --query "CREATE TABLE test1 (id UInt64) Engine=Memory()" +$CLICKHOUSE_CLIENT --query "GRANT SELECT ON test1 TO ${APP_USER1}" +$CLICKHOUSE_CLIENT --query "SELECT currentUser(), authenticatedUser()" + +echo -e "\n--- EXECUTE AS app_user1 ---" +$CLICKHOUSE_CLIENT --user ${TEST_USER} --query "EXECUTE AS ${APP_USER1}; SELECT currentUser(), authenticatedUser(); SELECT count() FROM test1;" +echo +$CLICKHOUSE_CLIENT --user ${TEST_USER} --query "EXECUTE AS ${APP_USER1} SELECT currentUser(), authenticatedUser(); SELECT currentUser(), authenticatedUser();" +echo +$CLICKHOUSE_CLIENT --user ${TEST_USER} --query "EXECUTE AS ${APP_USER1} SELECT count() FROM test1; SELECT currentUser(), authenticatedUser();" + +echo -e "\n--- EXECUTE AS app_user2 ---" +$CLICKHOUSE_CLIENT --user ${TEST_USER} --query "EXECUTE AS ${APP_USER2}; SELECT currentUser(), authenticatedUser();" +echo +$CLICKHOUSE_CLIENT --user ${TEST_USER} --query "EXECUTE AS ${APP_USER2}; SELECT currentUser(), authenticatedUser(); SELECT count() FROM test1;" 2>&1 | grep -q -F "ACCESS_DENIED" && echo "OK" || echo "FAIL" +echo +$CLICKHOUSE_CLIENT --user ${TEST_USER} --query "EXECUTE AS ${APP_USER2} SELECT currentUser(), authenticatedUser(); SELECT currentUser(), authenticatedUser();" +echo +$CLICKHOUSE_CLIENT --user ${TEST_USER} --query "EXECUTE AS ${APP_USER2} SELECT count() FROM test1" 2>&1 | grep -q -F "ACCESS_DENIED" && echo "OK" || echo "FAIL" + +echo -e "\n--- EXECUTE AS other_user ---" +$CLICKHOUSE_CLIENT --user ${TEST_USER} --query "EXECUTE AS ${OTHER_USER}" 2>&1 | grep -q -F "ACCESS_DENIED" && echo "OK" || echo "FAIL" +echo +$CLICKHOUSE_CLIENT --user ${TEST_USER} --query "EXECUTE AS ${OTHER_USER} SELECT 1" 2>&1 | grep -q -F "ACCESS_DENIED" && echo "OK" || echo "FAIL" + +echo -e "\n--- Multiple EXECUTE AS ---" +$CLICKHOUSE_CLIENT --user ${TEST_USER} --query "EXECUTE AS ${APP_USER1} SELECT currentUser(), authenticatedUser(); EXECUTE AS ${APP_USER2} SELECT currentUser(), authenticatedUser(); EXECUTE AS ${APP_USER1} SELECT currentUser(), authenticatedUser(); SELECT currentUser(), authenticatedUser();" +echo +$CLICKHOUSE_CLIENT --user ${TEST_USER} --query "EXECUTE AS ${APP_USER1}; SELECT currentUser(), authenticatedUser(); EXECUTE AS ${APP_USER2};" 2>&1 | grep -q -F "ACCESS_DENIED" && echo "OK" || echo "FAIL" + +$CLICKHOUSE_CLIENT --query "DROP USER ${TEST_USER}, ${SERVICE_USER}, ${APP_USER1}, ${APP_USER2}, ${OTHER_USER}"