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}"