Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1355,6 +1355,7 @@ atanh
atomicity
auth
authType
authenticatedUser
authenticator
authenticators
autocompletion
Expand Down
42 changes: 42 additions & 0 deletions docs/en/sql-reference/statements/execute_as.md
Original file line number Diff line number Diff line change
@@ -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"
```
1 change: 1 addition & 0 deletions src/Access/Common/AccessType.h
Original file line number Diff line number Diff line change
Expand Up @@ -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) \
Expand Down
4 changes: 2 additions & 2 deletions src/Core/ServerSettings.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1019,8 +1019,8 @@ namespace DB
<wait_dictionaries_load_at_startup>true</wait_dictionaries_load_at_startup>
```
)", 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

Expand Down
82 changes: 82 additions & 0 deletions src/Functions/authenticatedUser.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,82 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Interpreters/Context.h>
#include <DataTypes/DataTypeString.h>
#include <Core/Field.h>


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<FunctionAuthenticatedUser>(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<DataTypeString>();
}

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<FunctionAuthenticatedUser>(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");
}

}
118 changes: 118 additions & 0 deletions src/Interpreters/Access/InterpreterExecuteAsQuery.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,118 @@
#include <Interpreters/Access/InterpreterExecuteAsQuery.h>

#include <Access/AccessControl.h>
#include <Access/User.h>
#include <Core/Settings.h>
#include <Core/ServerSettings.h>
#include <Parsers/Access/ASTExecuteAsQuery.h>
#include <Parsers/Access/ASTUserNameWithHost.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterFactory.h>
#include <Interpreters/QueryFlags.h>
#include <Interpreters/executeQuery.h>


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<User>(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<User>(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<const ASTExecuteAsQuery &>();
String target_user_name = query.target_user->toString();
getContext()->checkAccess(AccessType::IMPERSONATE, target_user_name);

if (query.subquery)
{
/// EXECUTE AS <user> <subquery>
auto subquery_context = impersonateQueryContext(getContext(), target_user_name);
return executeQuery(query.subquery->formatWithSecretsOneLine(), subquery_context, QueryFlags{ .internal = true }).second;
}
else
{
/// EXECUTE AS <user>
impersonateSessionContext(getContext()->getSessionContext(), target_user_name);
return {};
}
}


void registerInterpreterExecuteAsQuery(InterpreterFactory & factory)
{
auto create_fn = [] (const InterpreterFactory::Arguments & args)
{
return std::make_unique<InterpreterExecuteAsQuery>(args.query, args.context);
};
factory.registerInterpreter("InterpreterExecuteAsQuery", create_fn);
}

}
20 changes: 20 additions & 0 deletions src/Interpreters/Access/InterpreterExecuteAsQuery.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
#pragma once

#include <Interpreters/IInterpreter.h>
#include <Parsers/IAST_fwd.h>


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

}
3 changes: 3 additions & 0 deletions src/Interpreters/ClientInfo.h
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,9 @@ class ClientInfo
String current_query_id;
std::shared_ptr<Poco::Net::SocketAddress> 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;
Expand Down
5 changes: 5 additions & 0 deletions src/Interpreters/InterpreterFactory.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@
#include <Parsers/Access/ASTShowCreateAccessEntityQuery.h>
#include <Parsers/Access/ASTShowGrantsQuery.h>
#include <Parsers/Access/ASTShowPrivilegesQuery.h>
#include <Parsers/Access/ASTExecuteAsQuery.h>
#include <Parsers/ASTDescribeCacheQuery.h>

#include <Interpreters/InterpreterFactory.h>
Expand Down Expand Up @@ -382,6 +383,10 @@ InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, Conte
{
interpreter_name = "InterpreterParallelWithQuery";
}
else if (query->as<ASTExecuteAsQuery>())
{
interpreter_name = "InterpreterExecuteAsQuery";
}

if (!interpreters.contains(interpreter_name))
throw Exception(ErrorCodes::UNKNOWN_TYPE_OF_QUERY, "Unknown type of query: {}", query->getID());
Expand Down
3 changes: 3 additions & 0 deletions src/Interpreters/QueryLog.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -103,6 +103,7 @@ ColumnsDescription QueryLogElement::getColumnsDescription()
{"initial_port", std::make_shared<DataTypeUInt16>(), "The client port that was used to make the parent query."},
{"initial_query_start_time", std::make_shared<DataTypeDateTime>(), "Initial query starting time (for distributed query execution)."},
{"initial_query_start_time_microseconds", std::make_shared<DataTypeDateTime64>(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<DataTypeUInt8>(), "Interface that the query was initiated from. Possible values: 1 — TCP, 2 — HTTP."},
{"is_secure", std::make_shared<DataTypeUInt8>(), "The flag whether a query was executed over a secure interface"},
{"os_user", low_cardinality_string, "Operating system username who runs clickhouse-client."},
Expand Down Expand Up @@ -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<UInt64>(client_info.interface));
columns[i++]->insert(static_cast<UInt64>(client_info.is_secure));

Expand Down
1 change: 1 addition & 0 deletions src/Interpreters/QueryThreadLog.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ ColumnsDescription QueryThreadLogElement::getColumnsDescription()
{"initial_port", std::make_shared<DataTypeUInt16>(), "The client port that was used to make the parent query."},
{"initial_query_start_time", std::make_shared<DataTypeDateTime>(), "Start time of the initial query execution."},
{"initial_query_start_time_microseconds", std::make_shared<DataTypeDateTime64>(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<DataTypeUInt8>(), "Interface that the query was initiated from. Possible values: 1 — TCP, 2 — HTTP."},
{"is_secure", std::make_shared<DataTypeUInt8>(), "The flag which shows whether the connection was secure."},
{"os_user", low_cardinality_string, "OSs username who runs clickhouse-client."},
Expand Down
1 change: 1 addition & 0 deletions src/Interpreters/Session.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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<Poco::Net::SocketAddress>(address);
}

Expand Down
2 changes: 2 additions & 0 deletions src/Interpreters/registerInterpreters.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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()
{
Expand Down Expand Up @@ -128,5 +129,6 @@ void registerInterpreters()
registerInterpreterBackupQuery(factory);
registerInterpreterDeleteQuery(factory);
registerInterpreterParallelWithQuery(factory);
registerInterpreterExecuteAsQuery(factory);
}
}
Loading