Skip to content
Merged
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
63 changes: 55 additions & 8 deletions docs/security/authentication-using-sasl.md
Original file line number Diff line number Diff line change
Expand Up @@ -543,18 +543,64 @@ To configure SASL authentication on the clients:

JAAS configuration for clients may alternatively be specified as a JVM parameter similar to brokers as described here. Clients use the login section named `KafkaClient`. This option allows only one user for all client connections from a JVM.

2. Configure the following properties in producer.properties or consumer.properties. For example, if using the OAuth `client_credentials` grant type to communicate with the OAuth identity provider, the configuration might look like this:

2. Configure the following properties in producer.properties or consumer.properties. The `sasl.oauthbearer.jwt.retriever.class` property defaults to `DefaultJwtRetriever`, which automatically delegates to `ClientCredentialsJwtRetriever` for HTTP/HTTPS token endpoint URLs and `FileJwtRetriever` for `file://` URLs. In most cases, this property does not need to be set explicitly.

For example, if using the OAuth `client_credentials` grant type with a client secret to communicate with the OAuth identity provider, the configuration might look like this:

security.protocol=SASL_SSL
sasl.mechanism=OAUTHBEARER
sasl.oauthbearer.jwt.retriever.class=org.apache.kafka.common.security.oauthbearer.ClientCredentialsJwtRetriever
sasl.oauthbearer.client.credentials.client.id=jdoe
sasl.oauthbearer.client.credentials.client.secret=$3cr3+
sasl.oauthbearer.scope=my-application-scope
sasl.oauthbearer.token.endpoint.url=https://example.com/oauth2/v1/token

Or, if using the OAuth `urn:ietf:params:oauth:grant-type:jwt-bearer` grant type to communicate with the OAuth identity provider, the configuration might look like this:

Alternatively, the `client_credentials` grant type also supports client assertion authentication as defined in [RFC 7523](https://tools.ietf.org/html/rfc7523). Instead of sending a client secret, the client authenticates by presenting a signed JWT assertion to the OAuth identity provider. This provides enhanced security since private keys never leave the client and assertions are short-lived. See [KIP-1258](https://cwiki.apache.org/confluence/display/KAFKA/KIP-1258) for details.

When using client assertion with dynamically-generated JWTs (recommended), the configuration might look like this:

security.protocol=SASL_SSL
sasl.mechanism=OAUTHBEARER
sasl.oauthbearer.token.endpoint.url=https://example.com/oauth2/v1/token
sasl.oauthbearer.assertion.private.key.file=/path/to/private-key.pem
sasl.oauthbearer.assertion.algorithm=RS256
sasl.oauthbearer.assertion.claim.iss=my-kafka-client
sasl.oauthbearer.assertion.claim.sub=my-service-account
sasl.oauthbearer.assertion.claim.aud=https://example.com
sasl.oauthbearer.assertion.claim.exp.seconds=300
sasl.oauthbearer.assertion.claim.jti.include=true
sasl.oauthbearer.scope=my-application-scope

Alternatively, a pre-generated JWT assertion can be read from a file. This is useful when assertions are generated by an external process or secrets manager:

security.protocol=SASL_SSL
sasl.mechanism=OAUTHBEARER
sasl.oauthbearer.token.endpoint.url=https://example.com/oauth2/v1/token
sasl.oauthbearer.assertion.file=/path/to/assertion.jwt
sasl.oauthbearer.scope=my-application-scope

For dynamically-generated assertions, additional static claims can be provided via a JSON template file using `sasl.oauthbearer.assertion.template.file`. The template supports `header` and `payload` sections whose values are merged into the generated JWT:

{
"header": {
"kid": "my-key-id"
},
"payload": {
"iss": "my-kafka-client",
"sub": "my-service-account",
"aud": "https://example.com"
}
}

When both client assertion and client secret configurations are present, the `ClientCredentialsJwtRetriever` uses a three-tier preference order:

1. **File-based assertion** (`sasl.oauthbearer.assertion.file`) — highest priority
2. **Locally-generated assertion** (`sasl.oauthbearer.assertion.claim.iss` with private key) — second priority
3. **Client secret** (`sasl.oauthbearer.client.credentials.client.secret`) — fallback

This selection is made at configuration time. Once a method is selected, it persists for the client's lifetime; runtime failures do not cause fallback to an alternative method.

Or, if using the OAuth `urn:ietf:params:oauth:grant-type:jwt-bearer` grant type to communicate with the OAuth identity provider, the `JwtBearerJwtRetriever` must be configured explicitly since the default retriever delegates to `ClientCredentialsJwtRetriever`:

security.protocol=SASL_SSL
sasl.mechanism=OAUTHBEARER
sasl.oauthbearer.jwt.retriever.class=org.apache.kafka.common.security.oauthbearer.JwtBearerJwtRetriever
Expand Down Expand Up @@ -616,14 +662,15 @@ Producer/Consumer/Broker Configuration Property
</td> </tr> </table>
#### Secure/Production Use of SASL/OAUTHBEARER

Production use cases will require writing an implementation of `org.apache.kafka.common.security.auth.AuthenticateCallbackHandler` that can handle an instance of `org.apache.kafka.common.security.oauthbearer.OAuthBearerTokenCallback` and declaring it via either the `sasl.login.callback.handler.class` configuration option for a non-broker client or via the `listener.name.sasl_ssl.oauthbearer.sasl.login.callback.handler.class` configuration option for brokers (when SASL/OAUTHBEARER is the inter-broker protocol).
Kafka provides built-in JWT retriever implementations for production use: `ClientCredentialsJwtRetriever` for the `client_credentials` grant type (supporting both client secret and client assertion authentication) and `JwtBearerJwtRetriever` for the `jwt-bearer` grant type. These can be configured via `sasl.oauthbearer.jwt.retriever.class` as shown in the examples above. Alternatively, production use cases may provide a custom implementation of `org.apache.kafka.common.security.auth.AuthenticateCallbackHandler` that can handle an instance of `org.apache.kafka.common.security.oauthbearer.OAuthBearerTokenCallback` and declaring it via either the `sasl.login.callback.handler.class` configuration option for a non-broker client or via the `listener.name.sasl_ssl.oauthbearer.sasl.login.callback.handler.class` configuration option for brokers (when SASL/OAUTHBEARER is the inter-broker protocol).

Production use cases will also require writing an implementation of `org.apache.kafka.common.security.auth.AuthenticateCallbackHandler` that can handle an instance of `org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback` and declaring it via the `listener.name.sasl_ssl.oauthbearer.sasl.server.callback.handler.class` broker configuration option.
Production use cases will also require writing an implementation of `org.apache.kafka.common.security.auth.AuthenticateCallbackHandler` that can handle an instance of `org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback` and declaring it via the `listener.name.sasl_ssl.oauthbearer.sasl.server.callback.handler.class` broker configuration option.
#### Security Considerations for SASL/OAUTHBEARER

* The default implementation of SASL/OAUTHBEARER in Kafka creates and validates [Unsecured JSON Web Tokens](https://tools.ietf.org/html/rfc7515#appendix-A.5). This is suitable only for non-production use.
* OAUTHBEARER should be used in production environments only with TLS-encryption to prevent interception of tokens.
* The default unsecured SASL/OAUTHBEARER implementation may be overridden (and must be overridden in production environments) using custom login and SASL Server callback handlers as described above.
* The default unsecured SASL/OAUTHBEARER implementation may be overridden (and must be overridden in production environments) using custom login and SASL Server callback handlers or the built-in JWT retriever implementations as described above.
* When using client assertion authentication, private keys never leave the client and are not transmitted over the network, unlike client secrets. Use short assertion expiration times (e.g. 300 seconds via `sasl.oauthbearer.assertion.claim.exp.seconds`) and enable JTI inclusion (`sasl.oauthbearer.assertion.claim.jti.include=true`) to protect against replay attacks.
* For more details on OAuth 2 security considerations in general, refer to [RFC 6749, Section 10](https://tools.ietf.org/html/rfc6749#section-10).
### Enabling multiple SASL mechanisms in a broker

Expand Down