diff --git a/contrib/jwt-cpp-cmake/CMakeLists.txt b/contrib/jwt-cpp-cmake/CMakeLists.txt
index 4cb8716bc68f..606c13d29de2 100644
--- a/contrib/jwt-cpp-cmake/CMakeLists.txt
+++ b/contrib/jwt-cpp-cmake/CMakeLists.txt
@@ -1,7 +1,4 @@
-set(ENABLE_JWT_CPP_DEFAULT OFF)
-if(ENABLE_LIBRARIES AND CLICKHOUSE_CLOUD)
- set(ENABLE_JWT_CPP_DEFAULT ON)
-endif()
+set(ENABLE_JWT_CPP_DEFAULT ON)
option(ENABLE_JWT_CPP "Enable jwt-cpp library" ${ENABLE_JWT_CPP_DEFAULT})
@@ -20,4 +17,4 @@ set (JWT_CPP_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/jwt-cpp/include")
add_library (_jwt-cpp INTERFACE)
target_include_directories(_jwt-cpp SYSTEM BEFORE INTERFACE ${JWT_CPP_INCLUDE_DIR})
-add_library(ch_contrib::jwt-cpp ALIAS _jwt-cpp)
+add_library(ch_contrib::jwt-cpp ALIAS _jwt-cpp)
\ No newline at end of file
diff --git a/docs/en/operations/external-authenticators/index.md b/docs/en/operations/external-authenticators/index.md
index 568ecf0383fd..346eadd9a323 100644
--- a/docs/en/operations/external-authenticators/index.md
+++ b/docs/en/operations/external-authenticators/index.md
@@ -16,4 +16,5 @@ The following external authenticators and directories are supported:
- [LDAP](./ldap.md#external-authenticators-ldap) [Authenticator](./ldap.md#ldap-external-authenticator) and [Directory](./ldap.md#ldap-external-user-directory)
- Kerberos [Authenticator](./kerberos.md#external-authenticators-kerberos)
- [SSL X.509 authentication](./ssl-x509.md#ssl-external-authentication)
-- HTTP [Authenticator](./http.md)
\ No newline at end of file
+- HTTP [Authenticator](./http.md)
+- JWT [Authenticator](./jwt.md)
diff --git a/docs/en/operations/external-authenticators/jwt.md b/docs/en/operations/external-authenticators/jwt.md
new file mode 100644
index 000000000000..fbc36f8399f7
--- /dev/null
+++ b/docs/en/operations/external-authenticators/jwt.md
@@ -0,0 +1,219 @@
+---
+slug: /en/operations/external-authenticators/jwt
+---
+# JWT
+import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_no_roadmap.md';
+
+
+
+Existing and properly configured ClickHouse users can be authenticated via JWT.
+
+Currently, JWT can only be used as an external authenticator for existing users, which are defined in `users.xml` or in local access control paths.
+The username will be extracted from the JWT after validating the token expiration and against the signature. Signature can be validated by:
+- static public key
+- static JWKS
+- received from the JWKS servers
+
+It is mandatory for a JWT tot indicate the name of the ClickHouse user under `"sub"` claim, otherwise it will not be accepted.
+
+A JWT may additionally be verified by checking the JWT payload.
+In this case, the occurrence of specified claims from the user settings in the JWT payload is checked.
+See [Enabling JWT authentication in `users.xml`](#enabling-jwt-auth-in-users-xml)
+
+To use JWT authentication, JWT validators must be configured in ClickHouse config.
+
+
+## Enabling JWT validators in ClickHouse {#enabling-jwt-validators-in-clickhouse}
+
+To enable JWT validators, add `token_validators` section in `config.xml`. This section may contain several JWT verifiers, minimum is 1.
+
+### Verifying JWT signature using static key {$verifying-jwt-signature-using-static-key}
+
+**Example**
+```xml
+
+
+
+
+ HS256
+ my_static_secret
+
+
+
+```
+
+#### Parameters:
+
+- `algo` - Algorithm for validate signature. Supported:
+
+ | HMAC | RSA | ECDSA | PSS | EdDSA |
+ |-------| ----- | ------ | ----- | ------- |
+ | HS256 | RS256 | ES256 | PS256 | Ed25519 |
+ | HS384 | RS384 | ES384 | PS384 | Ed448 |
+ | HS512 | RS512 | ES512 | PS512 | |
+ | | | ES256K | | |
+ Also support None.
+- `static_key` - key for symmetric algorithms. Mandatory for `HS*` family algorithms.
+- `static_key_in_base64` - indicates if the `static_key` key is base64-encoded. Optional, default: `False`.
+- `public_key` - public key for asymmetric algorithms. Mandatory except for `HS*` family algorithms and `None`.
+- `private_key` - private key for asymmetric algorithms. Optional.
+- `public_key_password` - public key password. Optional.
+- `private_key_password` - private key password. Optional.
+
+### Verifying JWT signature using static JWKS {$verifying-jwt-signature-using-static-jwks}
+
+:::note
+Only RS* family algorithms are supported!
+:::
+
+**Example**
+```xml
+
+
+
+
+ {"keys": [{"kty": "RSA", "alg": "RS256", "kid": "mykid", "n": "_public_key_mod_", "e": "AQAB"}]}
+
+
+
+```
+
+#### Parameters:
+- `static_jwks` - content of JWKS in json
+- `static_jwks_file` - path to file with JWKS
+
+:::note
+Only one of `static_jwks` or `static_jwks_file` keys must be present in one verifier
+:::
+
+### Verifying JWT signature using JWKS servers {$verifying-jwt-signature-using-static-jwks}
+
+**Example**
+```xml
+
+
+
+
+ http://localhost:8000/.well-known/jwks.json
+ 1000
+ 1000
+ 1000
+ 3
+ 50
+ 1000
+ 300000
+
+
+
+```
+
+#### Parameters:
+
+- `uri` - JWKS endpoint. Mandatory.
+- `refresh_ms` - Period for resend request for refreshing JWKS. Optional, default: 300000.
+
+Timeouts in milliseconds on the socket used for communicating with the server (optional):
+- `connection_timeout_ms` - Default: 1000.
+- `receive_timeout_ms` - Default: 1000.
+- `send_timeout_ms` - Default: 1000.
+
+Retry parameters (optional):
+- `max_tries` - The maximum number of attempts to make an authentication request. Default: 3.
+- `retry_initial_backoff_ms` - The backoff initial interval on retry. Default: 50.
+- `retry_max_backoff_ms` - The maximum backoff interval. Default: 1000.
+
+### Verifying access tokens {$verifying-access-tokens}
+
+Access tokens that are not JWT (and thus no data can be extracted from the token directly) need to be resolved by external providers.
+
+**Example**
+```xml
+
+
+
+
+ google
+
+
+
+```
+
+#### Parameters:
+
+- `provider` - name of provider that will be used for token processing. Mandatory parameter. Possible options: `google`.
+
+
+### Enabling JWT authentication in `users.xml` {#enabling-jwt-auth-in-users-xml}
+
+In order to enable JWT authentication for the user, specify `jwt` section instead of `password` or other similar sections in the user definition.
+
+Parameters:
+- `claims` - An optional string containing a json object that should be contained in the token payload.
+
+Example (goes into `users.xml`):
+```xml
+
+
+
+
+
+ {"resource_access":{"account": {"roles": ["view-profile"]}}}
+
+
+
+```
+
+Here, the JWT payload must contain `["view-profile"]` on path `resource_access.account.roles`, otherwise authentication will not succeed even with a valid JWT.
+
+```
+{
+...
+ "resource_access": {
+ "account": {
+ "roles": ["view-profile"]
+ }
+ },
+...
+}
+```
+
+:::note
+JWT authentication cannot be used together with any other authentication method. The presence of any other sections like `password` alongside `jwt` will force ClickHouse to shut down.
+:::
+
+### Enabling JWT authentication using SQL {#enabling-jwt-auth-using-sql}
+
+When [SQL-driven Access Control and Account Management](/docs/en/guides/sre/user-management/index.md#access-control) is enabled in ClickHouse, users identified by JWT authentication can also be created using SQL statements.
+
+```sql
+CREATE USER my_user IDENTIFIED WITH jwt CLAIMS '{"resource_access":{"account": {"roles": ["view-profile"]}}}'
+```
+
+Or without additional JWT payload checks:
+
+```sql
+CREATE USER my_user IDENTIFIED WITH jwt
+```
+
+## JWT authentication examples {#jwt-authentication-examples}
+
+#### Console client
+
+```
+clickhouse-client -jwt
+```
+
+#### HTTP requests
+
+```
+curl 'http://localhost:8080/?' \
+ -H 'Authorization: Bearer ' \
+ -H 'Content type: text/plain;charset=UTF-8' \
+ --data-raw 'SELECT current_user()'
+```
+:::note
+ClickHouse will look for a JWT token in (by priority):
+1. `X-ClickHouse-JWT-Token` header.
+2. `Authorization` header.
+3. `token` request parameter. In this case, the "Bearer" prefix should not exist.
+:::
diff --git a/docs/en/operations/external-authenticators/tokens.md b/docs/en/operations/external-authenticators/tokens.md
new file mode 100644
index 000000000000..6340e318237e
--- /dev/null
+++ b/docs/en/operations/external-authenticators/tokens.md
@@ -0,0 +1,108 @@
+---
+slug: /en/operations/external-authenticators/oauth
+title: "OAuth 2.0"
+---
+import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_no_roadmap.md';
+
+
+
+OAuth 2.0 access tokens can be used to authenticate ClickHouse users. This works in two ways:
+
+- Existing users (defined in `users.xml` or in local access control paths) can be authenticated with access token if this user can be `IDENTIFIED WITH jwt`.
+- Use Identity Provider (IdP) as an external user directory and allow locally undefined users to be authenticated with a token if it is valid and recognized by the provider.
+
+Though this authentication method is different from JWT authentication, it works under the same authentication method to maintain better compatibility.
+
+For both of these approaches a definition of `access_token_processors` is mandatory.
+
+## Access Token Processors
+
+To define an access token processor, add `access_token_processors` section to `config.xml`. Example:
+```xml
+
+
+
+ Google
+ ^[A-Za-z0-9._%+-]+@example\.com$
+ 600
+
+
+ azure
+ CLIENT_ID
+ TENANT_ID
+
+
+
+```
+
+:::note
+Different providers have different sets of parameters.
+:::
+
+**Parameters**
+
+- `provider` -- name of identity provider. Mandatory, case-insensitive. Supported options: "Google", "Azure".
+- `cache_lifetime` -- maximum lifetime of cached token (in seconds). Optional, default: 3600.
+- `email_filter` -- Regex for validation of user emails. Optional parameter, only for Google IdP.
+- `client_id` -- Azure AD (Entra ID) client ID. Optional parameter, only for Azure IdP.
+- `tenant_id` -- Azure AD (Entra ID) tenant ID. Optional parameter, only for Azure IdP.
+
+### Tokens cache
+To reduce number of requests to IdP, tokens are cached internally for no longer then `cache_lifetime` seconds.
+If token expires sooner than `cache_lifetime`, then cache entry for this token will only be valid while token is valid.
+If token lifetime is longer than `cache_lifetime`, cache entry for this token will be valid for `cache_lifetime`.
+
+## IdP as External Authenticator {#idp-external-authenticator}
+
+Locally defined users can be authenticated with an access token. To allow this, `jwt` must be specified as user's authentication method. Example:
+
+```xml
+
+
+
+
+
+
+
+
+
+
+```
+
+At each login attempt, ClickHouse will attempt to validate token and get user info against every defined access token provider.
+
+When SQL-driven [Access Control and Account Management](/docs/en/guides/sre/user-management/index.md#access-control) is enabled, users that are authenticated with tokens can also be created using the [CREATE USER](/docs/en/sql-reference/statements/create/user.md#create-user-statement) statement.
+
+Query:
+
+```sql
+CREATE USER my_user IDENTIFIED WITH jwt;
+```
+
+## Identity Provider as an External User Directory {#idp-external-user-directory}
+
+If there is no suitable user pre-defined in ClickHouse, authentication is still possible: Identity Provider can be used as source of user information.
+To allow this, add `token` section to the `users_directories` section of the `config.xml` file.
+
+At each login attempt, ClickHouse tries to find the user definition locally and authenticate it as usual.
+If the user is not defined, ClickHouse will treat user as externally defined, and will try to validate the token and get user information from the specified processor.
+If validated successfully, the user will be considered existing and authenticated. The user will be assigned roles from the list specified in the `roles` section.
+All this implies that the SQL-driven [Access Control and Account Management](/docs/en/guides/sre/user-management/index.md#access-control) is enabled and roles are created using the [CREATE ROLE](/docs/en/sql-reference/statements/create/role.md#create-role-statement) statement.
+
+**Example**
+
+```xml
+
+
+ gogoogle
+
+
+
+
+
+```
+
+**Parameters**
+
+- `server` — Name of one of processors defined in `access_token_processors` config section described above. This parameter is mandatory and cannot be empty.
+- `roles` — Section with a list of locally defined roles that will be assigned to each user retrieved from the IdP.
diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp
index 432fe19634e6..d40ba7eee961 100644
--- a/src/Access/AccessControl.cpp
+++ b/src/Access/AccessControl.cpp
@@ -5,6 +5,7 @@
#include
#include
#include
+#include
#include
#include
#include
@@ -419,6 +420,12 @@ void AccessControl::addLDAPStorage(const String & storage_name_, const Poco::Uti
LOG_DEBUG(getLogger(), "Added {} access storage '{}', LDAP server name: {}", String(new_storage->getStorageType()), new_storage->getStorageName(), new_storage->getLDAPServerName());
}
+void AccessControl::addTokenStorage(const String & storage_name_, const Poco::Util::AbstractConfiguration & config_, const String & prefix_)
+{
+ auto new_storage = std::make_shared(storage_name_, *this, config_, prefix_);
+ addStorage(new_storage);
+ LOG_DEBUG(getLogger(), "Added {} access storage '{}'", String(new_storage->getStorageType()), new_storage->getStorageName());
+}
void AccessControl::addStoragesFromUserDirectoriesConfig(
const Poco::Util::AbstractConfiguration & config,
@@ -444,6 +451,8 @@ void AccessControl::addStoragesFromUserDirectoriesConfig(
type = DiskAccessStorage::STORAGE_TYPE;
else if (type == "ldap")
type = LDAPAccessStorage::STORAGE_TYPE;
+ else if (type == "token")
+ type = TokenAccessStorage::STORAGE_TYPE;
String name = config.getString(prefix + ".name", type);
@@ -477,6 +486,10 @@ void AccessControl::addStoragesFromUserDirectoriesConfig(
bool allow_backup = config.getBool(prefix + ".allow_backup", true);
addReplicatedStorage(name, zookeeper_path, get_zookeeper_function, allow_backup);
}
+ else if (type == TokenAccessStorage::STORAGE_TYPE)
+ {
+ addTokenStorage(name, config, prefix);
+ }
else
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Unknown storage type '{}' at {} in config", type, prefix);
}
@@ -704,6 +717,11 @@ bool AccessControl::isNoPasswordAllowed() const
return allow_no_password;
}
+bool AccessControl::isJWTEnabled() const
+{
+ return external_authenticators->isJWTAllowed();
+}
+
void AccessControl::setPlaintextPasswordAllowed(bool allow_plaintext_password_)
{
allow_plaintext_password = allow_plaintext_password_;
diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h
index ac64f89a7080..20a9e5818d66 100644
--- a/src/Access/AccessControl.h
+++ b/src/Access/AccessControl.h
@@ -93,6 +93,8 @@ class AccessControl : public MultipleAccessStorage
/// Adds LDAPAccessStorage which allows querying remote LDAP server for user info.
void addLDAPStorage(const String & storage_name_, const Poco::Util::AbstractConfiguration & config_, const String & prefix_);
+ void addTokenStorage(const String & storage_name_, const Poco::Util::AbstractConfiguration & config_, const String & prefix_);
+
void addReplicatedStorage(const String & storage_name,
const String & zookeeper_path,
const zkutil::GetZooKeeper & get_zookeeper_function,
@@ -155,6 +157,8 @@ class AccessControl : public MultipleAccessStorage
void setNoPasswordAllowed(bool allow_no_password_);
bool isNoPasswordAllowed() const;
+ bool isJWTEnabled() const;
+
/// Allows users with plaintext password (by default it's allowed).
void setPlaintextPasswordAllowed(bool allow_plaintext_password_);
bool isPlaintextPasswordAllowed() const;
diff --git a/src/Access/AccessTokenProcessor.cpp b/src/Access/AccessTokenProcessor.cpp
new file mode 100644
index 000000000000..4398fe896527
--- /dev/null
+++ b/src/Access/AccessTokenProcessor.cpp
@@ -0,0 +1,319 @@
+#include
+#include
+#include
+#include
+
+
+namespace DB
+{
+
+namespace
+{
+ /// The JSON reply from provider has only a few key-value pairs, so no need for any advanced parsing.
+ /// Reduce complexity by using picojson.
+ picojson::object parseJSON(const String & json_string) {
+ picojson::value jsonValue;
+ std::string err = picojson::parse(jsonValue, json_string);
+
+ if (!err.empty()) {
+ throw std::runtime_error("JSON parsing error: " + err);
+ }
+
+ if (!jsonValue.is()) {
+ throw std::runtime_error("JSON is not an object");
+ }
+
+ return jsonValue.get();
+ }
+
+ template
+ ValueType getValueByKey(const picojson::object & jsonObject, const std::string & key) {
+ auto it = jsonObject.find(key); // Find the key in the object
+ if (it == jsonObject.end())
+ {
+ throw std::runtime_error("Key not found: " + key);
+ }
+
+ const picojson::value & value = it->second;
+ if (!value.is()) {
+ throw std::runtime_error("Value for key '" + key + "' has incorrect type.");
+ }
+
+ return value.get();
+ }
+
+ picojson::object getObjectFromURI(const Poco::URI & uri, const String & token = "")
+ {
+ Poco::Net::HTTPResponse response;
+ std::ostringstream responseString;
+
+ Poco::Net::HTTPRequest request{Poco::Net::HTTPRequest::HTTP_GET, uri.getPathAndQuery()};
+ if (!token.empty())
+ request.add("Authorization", "Bearer " + token);
+
+ if (uri.getScheme() == "https") {
+ Poco::Net::HTTPSClientSession session(uri.getHost(), uri.getPort());
+ session.sendRequest(request);
+ Poco::StreamCopier::copyStream(session.receiveResponse(response), responseString);
+ }
+ else
+ {
+ Poco::Net::HTTPClientSession session(uri.getHost(), uri.getPort());
+ session.sendRequest(request);
+ Poco::StreamCopier::copyStream(session.receiveResponse(response), responseString);
+ }
+
+ if (response.getStatus() != Poco::Net::HTTPResponse::HTTP_OK)
+ throw Exception(ErrorCodes::AUTHENTICATION_FAILED,
+ "Failed to get user info by access token, code: {}, reason: {}", response.getStatus(),
+ response.getReason());
+
+ try
+ {
+ return parseJSON(responseString.str());
+ }
+ catch (const std::runtime_error & e)
+ {
+ throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Failed to parse server response: {}", e.what());
+ }
+ }
+}
+
+
+[[maybe_unused]] const Poco::URI GoogleAccessTokenProcessor::token_info_uri = Poco::URI("https://www.googleapis.com/oauth2/v3/tokeninfo");
+const Poco::URI GoogleAccessTokenProcessor::user_info_uri = Poco::URI("https://www.googleapis.com/oauth2/v3/userinfo");
+
+const Poco::URI AzureAccessTokenProcessor::user_info_uri = Poco::URI("https://graph.microsoft.com/oidc/userinfo");
+
+
+std::unique_ptr IAccessTokenProcessor::parseTokenProcessor(
+ const Poco::Util::AbstractConfiguration & config,
+ const String & prefix,
+ const String & name)
+{
+ if (config.hasProperty(prefix + ".provider"))
+ {
+ String provider = Poco::toLower(config.getString(prefix + ".provider"));
+
+ String email_regex_str = config.hasProperty(prefix + ".email_filter") ? config.getString(
+ prefix + ".email_filter") : "";
+
+ UInt64 cache_lifetime = config.hasProperty(prefix + ".cache_lifetime") ? config.getUInt64(
+ prefix + ".cache_lifetime") : 3600;
+
+ if (provider == "google")
+ {
+ return std::make_unique(name, cache_lifetime, email_regex_str);
+ }
+ else if (provider == "azure")
+ {
+ if (!config.hasProperty(prefix + ".client_id"))
+ throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER,
+ "Could not parse access token processor {}: client_id must be specified", name);
+
+ if (!config.hasProperty(prefix + ".tenant_id"))
+ throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER,
+ "Could not parse access token processor {}: tenant_id must be specified", name);
+
+ String tenant_id_str = config.getString(prefix + ".tenant_id");
+
+ return std::make_unique(name, cache_lifetime, email_regex_str, tenant_id_str);
+ }
+ else
+ throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER,
+ "Could not parse access token processor {}: unknown provider {}", name, provider);
+ }
+
+ throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER,
+ "Could not parse access token processor {}: provider name must be specified", name);
+}
+
+
+bool GoogleAccessTokenProcessor::resolveAndValidate(const TokenCredentials & credentials)
+{
+ const String & token = credentials.getToken();
+
+ auto user_info = getUserInfo(token);
+ String user_name = user_info["sub"];
+ bool has_email = user_info.contains("email");
+
+ if (email_regex.ok())
+ {
+ if (!has_email)
+ {
+ LOG_TRACE(getLogger("AccessTokenProcessor"), "{}: Failed to validate {} by e-mail", name, user_name);
+ return false;
+ }
+
+ /// Additionally validate user email to match regex from config.
+ if (!RE2::FullMatch(user_info["email"], email_regex))
+ {
+ LOG_TRACE(getLogger("AccessTokenProcessor"), "{}: Failed to authenticate user {}: e-mail address is not permitted.", name, user_name);
+ return false;
+ }
+
+ }
+
+ /// Credentials are passed as const everywhere up the flow, so we have to comply,
+ /// in this case const_cast looks acceptable.
+ const_cast(credentials).setUserName(user_name);
+
+ auto token_info = getObjectFromURI(Poco::URI(token_info_uri), token);
+ if (token_info.contains("exp"))
+ const_cast(credentials).setExpiresAt(std::chrono::system_clock::from_time_t((getValueByKey(token_info, "exp"))));
+
+ /// Groups info can only be retrieved if user email is known.
+ /// If no email found in user info, we skip this step and there are no external groups for the user.
+ if (has_email)
+ {
+ std::set external_groups_names;
+ const Poco::URI get_groups_uri = Poco::URI("https://cloudidentity.googleapis.com/v1/groups/-/memberships:searchDirectGroups?query=member_key_id==" + user_info["email"] + "'");
+
+ try
+ {
+ auto groups_response = getObjectFromURI(get_groups_uri, token);
+
+ if (!groups_response.contains("memberships") || !groups_response["memberships"].is())
+ {
+ LOG_TRACE(getLogger("AccessTokenProcessor"),
+ "{}: Failed to get Google groups: invalid content in response from server", name);
+ return true;
+ }
+
+ for (const auto & group: groups_response["memberships"].get())
+ {
+ if (!group.is())
+ {
+ LOG_TRACE(getLogger("AccessTokenProcessor"),
+ "{}: Failed to get Google groups: invalid content in response from server", name);
+ continue;
+ }
+
+ auto group_data = group.get();
+ String group_name = getValueByKey(group_data["groupKey"].get(), "id");
+ external_groups_names.insert(group_name);
+ LOG_TRACE(getLogger("AccessTokenProcessor"),
+ "{}: User {}: new external group {}", name, user_name, group_name);
+ }
+
+ const_cast(credentials).setGroups(external_groups_names);
+ }
+ catch (const Exception & e)
+ {
+ /// Could not get groups info. Log it and skip it.
+ LOG_TRACE(getLogger("AccessTokenProcessor"),
+ "{}: Failed to get Google groups, no external roles will be mapped. reason: {}", name, e.what());
+ return true;
+ }
+ }
+
+ return true;
+}
+
+std::unordered_map GoogleAccessTokenProcessor::getUserInfo(const String & token) const
+{
+ std::unordered_map user_info_map;
+ picojson::object user_info_json = getObjectFromURI(user_info_uri, token);
+
+ try
+ {
+ user_info_map["email"] = getValueByKey(user_info_json, "email");
+ user_info_map["sub"] = getValueByKey(user_info_json, "sub");
+ return user_info_map;
+ }
+ catch (std::runtime_error & e)
+ {
+ throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "{}: Failed to get user info with token: {}", name, e.what());
+ }
+}
+
+
+bool AzureAccessTokenProcessor::resolveAndValidate(const TokenCredentials & credentials)
+{
+ /// Token is a JWT in this case, but we cannot directly verify it against Azure AD JWKS.
+ /// We will not trust user data in this token except for 'exp' value to determine caching duration.
+ /// Explanation here: https://stackoverflow.com/questions/60778634/failing-signature-validation-of-jwt-tokens-from-azure-ad
+ /// Let Azure validate it: only valid tokens will be accepted.
+ /// Use GET https://graph.microsoft.com/oidc/userinfo to verify token and get sub at the same time
+
+ const String & token = credentials.getToken();
+
+ try
+ {
+ String username = validateTokenAndGetUsername(token);
+ if (!username.empty())
+ {
+ /// Credentials are passed as const everywhere up the flow, so we have to comply,
+ /// in this case const_cast looks acceptable.
+ const_cast(credentials).setUserName(username);
+ }
+ else
+ LOG_TRACE(getLogger("AccessTokenProcessor"), "{}: Failed to get username with token", name);
+
+ }
+ catch (...)
+ {
+ return false;
+ }
+
+ try
+ {
+ const_cast(credentials).setExpiresAt(jwt::decode(token).get_expires_at());
+ }
+ catch (...) {
+ LOG_TRACE(getLogger("AccessTokenProcessor"),
+ "{}: No expiration data found in a valid token, will use default cache lifetime", name);
+ }
+
+ std::set external_groups_names;
+ const Poco::URI get_groups_uri = Poco::URI("https://graph.microsoft.com/v1.0/me/memberOf");
+
+ try
+ {
+ auto groups_response = getObjectFromURI(get_groups_uri, token);
+
+ if (!groups_response.contains("value") || !groups_response["value"].is())
+ {
+ LOG_TRACE(getLogger("AccessTokenProcessor"),
+ "{}: Failed to get Azure groups: invalid content in response from server", name);
+ return true;
+ }
+
+ picojson::array groups_array = groups_response["value"].get();
+
+ for (const auto & group: groups_array)
+ {
+ /// Got some invalid response. Ignore this, log this.
+ if (!group.is())
+ {
+ LOG_TRACE(getLogger("AccessTokenProcessor"),
+ "{}: Failed to get Azure groups: invalid content in response from server", name);
+ continue;
+ }
+
+ auto group_data = group.get();
+ String group_name = getValueByKey(group_data, "id");
+ external_groups_names.insert(group_name);
+ LOG_TRACE(getLogger("AccessTokenProcessor"), "{}: User {}: new external group {}", name, credentials.getUserName(), group_name);
+ }
+ }
+ catch (const Exception & e)
+ {
+ /// Could not get groups info. Log it and skip it.
+ LOG_TRACE(getLogger("AccessTokenProcessor"),
+ "{}: Failed to get Azure groups, no external roles will be mapped. reason: {}", name, e.what());
+ return true;
+ }
+
+ const_cast(credentials).setGroups(external_groups_names);
+
+ return true;
+}
+
+String AzureAccessTokenProcessor::validateTokenAndGetUsername(const String & token) const
+{
+ picojson::object user_info_json = getObjectFromURI(user_info_uri, token);
+ return getValueByKey(user_info_json, "sub");
+}
+
+}
diff --git a/src/Access/AccessTokenProcessor.h b/src/Access/AccessTokenProcessor.h
new file mode 100644
index 000000000000..75005873084d
--- /dev/null
+++ b/src/Access/AccessTokenProcessor.h
@@ -0,0 +1,100 @@
+#include
+
+#include
+#include
+#include
+#include
+
+#include
+#include
+#include
+#include
+#include
+
+
+namespace DB
+{
+
+namespace ErrorCodes
+{
+ extern const int AUTHENTICATION_FAILED;
+ extern const int INVALID_CONFIG_PARAMETER;
+}
+
+class GoogleAccessTokenProcessor;
+
+class IAccessTokenProcessor
+{
+public:
+ IAccessTokenProcessor(const String & name_,
+ const UInt64 cache_invalidation_interval_,
+ const String & email_regex_str)
+ : name(name_),
+ cache_invalidation_interval(cache_invalidation_interval_),
+ email_regex(email_regex_str)
+ {
+ if (!email_regex_str.empty())
+ {
+ /// Later, we will use .ok() to determine whether there was a regex specified in config or not.
+ if (!email_regex.ok())
+ throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Invalid regex in definition of access token processor {}", name);
+ }
+ }
+
+ virtual ~IAccessTokenProcessor() = default;
+
+ String getName() { return name; }
+ UInt64 getCacheInvalidationInterval() { return cache_invalidation_interval; }
+
+ virtual bool resolveAndValidate(const TokenCredentials & credentials) = 0;
+
+ static std::unique_ptr parseTokenProcessor(
+ const Poco::Util::AbstractConfiguration & config,
+ const String & prefix,
+ const String & name);
+
+protected:
+ const String name;
+ const UInt64 cache_invalidation_interval;
+ re2::RE2 email_regex;
+};
+
+
+class GoogleAccessTokenProcessor : public IAccessTokenProcessor
+{
+public:
+ GoogleAccessTokenProcessor(const String & name_,
+ const UInt64 cache_invalidation_interval_,
+ const String & email_regex_str)
+ : IAccessTokenProcessor(name_, cache_invalidation_interval_, email_regex_str) {}
+
+ bool resolveAndValidate(const TokenCredentials & credentials) override;
+
+private:
+ [[maybe_unused]] static const Poco::URI token_info_uri;
+ static const Poco::URI user_info_uri;
+
+ std::unordered_map getUserInfo(const String & token) const;
+};
+
+
+class AzureAccessTokenProcessor : public IAccessTokenProcessor
+{
+public:
+ AzureAccessTokenProcessor(const String & name_,
+ const UInt64 cache_invalidation_interval_,
+ const String & email_regex_str,
+ const String & tenant_id_)
+ : IAccessTokenProcessor(name_, cache_invalidation_interval_, email_regex_str),
+ jwks_uri_str("https://login.microsoftonline.com/" + tenant_id_ + "/discovery/v2.0/keys") {}
+
+ bool resolveAndValidate(const TokenCredentials & credentials) override;
+private:
+ static const Poco::URI user_info_uri;
+
+ const String jwks_uri_str;
+
+ String validateTokenAndGetUsername(const String & token) const;
+};
+
+}
diff --git a/src/Access/Authentication.cpp b/src/Access/Authentication.cpp
index 6aa5f3fdfff6..bbc90c4eee0e 100644
--- a/src/Access/Authentication.cpp
+++ b/src/Access/Authentication.cpp
@@ -235,7 +235,7 @@ bool Authentication::areCredentialsValid(
const ExternalAuthenticators & external_authenticators,
SettingsChanges & settings)
{
- if (!credentials.isReady())
+ if (!typeid_cast(&credentials) && !credentials.isReady())
return false;
if (const auto * gss_acceptor_context = typeid_cast(&credentials))
@@ -270,6 +270,17 @@ bool Authentication::areCredentialsValid(
}
#endif
+ if (const auto * token_credentials = typeid_cast(&credentials))
+ {
+ if (authentication_method.getType() != AuthenticationType::JWT)
+ return false;
+
+ if (external_authenticators.checkJWTClaims(authentication_method.getJWTClaims(), *token_credentials))
+ return true;
+
+ return external_authenticators.checkAccessTokenCredentials(*token_credentials);
+ }
+
if ([[maybe_unused]] const auto * always_allow_credentials = typeid_cast(&credentials))
return true;
diff --git a/src/Access/AuthenticationData.cpp b/src/Access/AuthenticationData.cpp
index 99eed230db73..7bafbf65b267 100644
--- a/src/Access/AuthenticationData.cpp
+++ b/src/Access/AuthenticationData.cpp
@@ -15,7 +15,10 @@
#include
#include
#include
+#include
+#include
+#include "Access/Common/AuthenticationType.h"
#include
#include "config.h"
@@ -337,7 +340,10 @@ std::shared_ptr AuthenticationData::toAST() const
}
case AuthenticationType::JWT:
{
- throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "JWT is available only in ClickHouse Cloud");
+ const auto & claims = getJWTClaims();
+ if (!claims.empty())
+ node->children.push_back(std::make_shared(claims));
+ break;
}
case AuthenticationType::KERBEROS:
{
@@ -570,6 +576,20 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que
auth_data.setHTTPAuthenticationServerName(server);
auth_data.setHTTPAuthenticationScheme(scheme);
}
+ else if (query.type == AuthenticationType::JWT)
+ {
+ if (!args.empty())
+ {
+ String value = checkAndGetLiteralArgument(args[0], "claims");
+ picojson::value json_obj;
+ auto error = picojson::parse(json_obj, value);
+ if (!error.empty())
+ throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad JWT claims: {}", error);
+ if (!json_obj.is())
+ throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad JWT claims: is not an object");
+ auth_data.setJWTClaims(value);
+ }
+ }
else
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected ASTAuthenticationData structure");
diff --git a/src/Access/AuthenticationData.h b/src/Access/AuthenticationData.h
index 239a802edddc..7c0f432f6190 100644
--- a/src/Access/AuthenticationData.h
+++ b/src/Access/AuthenticationData.h
@@ -77,6 +77,9 @@ class AuthenticationData
time_t getValidUntil() const { return valid_until; }
void setValidUntil(time_t valid_until_) { valid_until = valid_until_; }
+ const String & getJWTClaims() const { return jwt_claims; }
+ void setJWTClaims(const String & jwt_claims_) { jwt_claims = jwt_claims_; }
+
friend bool operator ==(const AuthenticationData & lhs, const AuthenticationData & rhs);
friend bool operator !=(const AuthenticationData & lhs, const AuthenticationData & rhs) { return !(lhs == rhs); }
@@ -110,6 +113,7 @@ class AuthenticationData
String http_auth_server_name;
HTTPAuthenticationScheme http_auth_scheme = HTTPAuthenticationScheme::BASIC;
time_t valid_until = 0;
+ String jwt_claims;
};
}
diff --git a/src/Access/Common/JWKSProvider.cpp b/src/Access/Common/JWKSProvider.cpp
new file mode 100644
index 000000000000..94ee5e04cafe
--- /dev/null
+++ b/src/Access/Common/JWKSProvider.cpp
@@ -0,0 +1,89 @@
+#include
+
+#include
+#include
+#include
+
+
+namespace DB
+{
+
+namespace ErrorCodes
+{
+ extern const int AUTHENTICATION_FAILED;
+ extern const int INVALID_CONFIG_PARAMETER;
+}
+
+jwt::jwks JWKSClient::getJWKS()
+{
+ std::shared_lock lock(mutex);
+
+ auto now = std::chrono::high_resolution_clock::now();
+ auto diff = std::chrono::duration(now - last_request_send).count();
+
+ if (diff < refresh_ms) {
+ jwt::jwks result(cached_jwks);
+ return result;
+ }
+
+ Poco::Net::HTTPResponse response;
+ std::ostringstream responseString;
+
+ Poco::Net::HTTPRequest request{Poco::Net::HTTPRequest::HTTP_GET, jwks_uri.getPathAndQuery()};
+
+ if (jwks_uri.getScheme() == "https") {
+ Poco::Net::HTTPSClientSession session = Poco::Net::HTTPSClientSession(jwks_uri.getHost(), jwks_uri.getPort());
+ session.sendRequest(request);
+ std::istream & responseStream = session.receiveResponse(response);
+ if (response.getStatus() != Poco::Net::HTTPResponse::HTTP_OK || !responseStream)
+ throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Failed to get user info by access token, code: {}, reason: {}", response.getStatus(), response.getReason());
+ Poco::StreamCopier::copyStream(responseStream, responseString);
+ } else {
+ Poco::Net::HTTPClientSession session = Poco::Net::HTTPClientSession(jwks_uri.getHost(), jwks_uri.getPort());
+ session.sendRequest(request);
+ std::istream & responseStream = session.receiveResponse(response);
+ if (response.getStatus() != Poco::Net::HTTPResponse::HTTP_OK || !responseStream)
+ throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Failed to get user info by access token, code: {}, reason: {}", response.getStatus(), response.getReason());
+ Poco::StreamCopier::copyStream(responseStream, responseString);
+ }
+
+ last_request_send = std::chrono::high_resolution_clock::now();
+
+ jwt::jwks parsed_jwks;
+
+ try {
+ parsed_jwks = jwt::parse_jwks(responseString.str());
+ }
+ catch (const Exception & e) {
+ throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Failed to parse JWKS: {}", e.what());
+ }
+
+ cached_jwks = std::move(parsed_jwks);
+ return cached_jwks;
+}
+
+StaticJWKSParams::StaticJWKSParams(const std::string &static_jwks_, const std::string &static_jwks_file_)
+{
+ if (static_jwks_.empty() && static_jwks_file_.empty())
+ throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER,
+ "JWT validator misconfigured: `static_jwks` or `static_jwks_file` keys must be present in static JWKS validator configuration");
+ if (!static_jwks_.empty() && !static_jwks_file_.empty())
+ throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER,
+ "JWT validator misconfigured: `static_jwks` and `static_jwks_file` keys cannot both be present in static JWKS validator configuration");
+
+ static_jwks = static_jwks_;
+ static_jwks_file = static_jwks_file_;
+}
+
+StaticJWKS::StaticJWKS(const StaticJWKSParams ¶ms)
+{
+ String content = String(params.static_jwks);
+ if (!params.static_jwks_file.empty()) {
+ std::ifstream ifs(params.static_jwks_file);
+ content = String((std::istreambuf_iterator(ifs)), (std::istreambuf_iterator()));
+ }
+ auto keys = jwt::parse_jwks(content);
+ jwks = std::move(keys);
+}
+
+}
diff --git a/src/Access/Common/JWKSProvider.h b/src/Access/Common/JWKSProvider.h
new file mode 100644
index 000000000000..773208a138aa
--- /dev/null
+++ b/src/Access/Common/JWKSProvider.h
@@ -0,0 +1,67 @@
+#include
+#include
+#include
+
+#include
+#include
+#include
+#include
+#include
+
+
+namespace DB
+{
+
+class IJWKSProvider
+{
+public:
+ virtual ~IJWKSProvider() = default;
+
+ virtual jwt::jwks getJWKS() = 0;
+};
+
+class JWKSClient : public IJWKSProvider
+{
+public:
+ explicit JWKSClient(const String & uri, const size_t refresh_ms_): refresh_ms(refresh_ms_), jwks_uri(uri) {}
+
+ ~JWKSClient() override = default;
+ JWKSClient(const JWKSClient &) = delete;
+ JWKSClient(JWKSClient &&) = delete;
+ JWKSClient &operator=(const JWKSClient &) = delete;
+ JWKSClient &operator=(JWKSClient &&) = delete;
+
+ jwt::jwks getJWKS() override;
+
+private:
+ size_t refresh_ms;
+ Poco::URI jwks_uri;
+
+ std::shared_mutex mutex;
+ jwt::jwks cached_jwks;
+ std::chrono::time_point last_request_send;
+};
+
+struct StaticJWKSParams
+{
+ StaticJWKSParams(const std::string &static_jwks_, const std::string &static_jwks_file_);
+
+ String static_jwks;
+ String static_jwks_file;
+};
+
+class StaticJWKS : public IJWKSProvider
+{
+public:
+ explicit StaticJWKS(const StaticJWKSParams ¶ms);
+
+private:
+ jwt::jwks getJWKS() override
+ {
+ return jwks;
+ }
+
+ jwt::jwks jwks;
+};
+
+}
diff --git a/src/Access/Credentials.cpp b/src/Access/Credentials.cpp
index f01700b6e461..da3adcfdab92 100644
--- a/src/Access/Credentials.cpp
+++ b/src/Access/Credentials.cpp
@@ -1,6 +1,8 @@
#include
-#include
#include
+#include
+
+#include
namespace DB
{
@@ -8,6 +10,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
+ extern const int AUTHENTICATION_FAILED;
}
Credentials::Credentials(const String & user_name_)
@@ -97,4 +100,7 @@ const String & BasicCredentials::getPassword() const
return password;
}
+/// Unless the token is validated, we will not use any data from it, including username.
+TokenCredentials::TokenCredentials(const String & token_) : Credentials(""), token(token_), expires_at(std::chrono::system_clock::now() + std::chrono::hours(1)) {}
+
}
diff --git a/src/Access/Credentials.h b/src/Access/Credentials.h
index 52f33385a0e8..c825c722f0b4 100644
--- a/src/Access/Credentials.h
+++ b/src/Access/Credentials.h
@@ -8,6 +8,8 @@
#include "config.h"
+#include
+
namespace DB
{
@@ -151,4 +153,47 @@ class SSHPTYCredentials : public Credentials
#endif
+class TokenCredentials : public Credentials
+{
+public:
+ explicit TokenCredentials(const String & token_);
+
+ const String & getToken() const
+ {
+ if (token.empty())
+ {
+ throwNotReady();
+ }
+ return token;
+ }
+ void setUserName(const String & user_name_)
+ {
+ user_name = user_name_;
+ if (!user_name.empty())
+ {
+ is_ready = true;
+ }
+ }
+ std::set getGroups() const
+ {
+ return groups;
+ }
+ void setGroups(const std::set & groups_)
+ {
+ groups = groups_;
+ }
+ std::optional getExpiresAt() const
+ {
+ return expires_at;
+ }
+ void setExpiresAt(std::chrono::system_clock::time_point expires_at_)
+ {
+ expires_at = expires_at_;
+ }
+private:
+ String token;
+ std::set groups;
+ std::optional expires_at;
+};
+
}
diff --git a/src/Access/ExternalAuthenticators.cpp b/src/Access/ExternalAuthenticators.cpp
index 91d0ff7ff0f9..680032a6b0de 100644
--- a/src/Access/ExternalAuthenticators.cpp
+++ b/src/Access/ExternalAuthenticators.cpp
@@ -2,14 +2,21 @@
#include
#include
#include
+#include "Common/Logger.h"
+#include "Common/logger_useful.h"
#include
#include
#include
#include
+#include "Access/AccessControl.h"
+#include "Access/Credentials.h"
+#include
#include
#include
+#include