Status
Current state: ["DISCUSSION"].
Discussion thread: here
JIRA: KAFKA-1696
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
We introduced support for security in kafka version 0.9.0. using kerberos as authentication layer. Kafka is designed to work with a lot of producers and consumers so in a secure environment all these clients will need access to a keytab or a TGT to ensure they can communicate with a secure kafka broker. This has few disadvantages:
Performance/load on KDC as each client has to go to KDC to get the ticket.
Renewal needs to go through KDC and this renewed TGT’s need to be redistributed to all the clients.
Blast Radius is large if the TGT is compromised as TGT may grant access to more than just kafka service
Only compatible with kerberos authentication scheme.
Administration cost as for any new client to work it must have access to keytab or some way to get a TGT from some other node.
Please read http://carfield.com.hk:8080/document/distributed/hadoop-security-design.pdf HDFS section for more detailed explanation of all the disadvatages above. To address the problems listed above we propose to add support for delegation tokens to secure Kafka. Delegation tokens are shared secret between kafka brokers and clients so authentication can be done without having to go through KDC.
Delegation tokens will help processing frameworks to distribute workload to available workers in a secure environment without the added cost of distributing keytabs or TGT. i.e. In case of Storm, Storm’s master (nimbus) is the only node that needs a keytab. Using this keytab Nimbus will authenticate with kafka broker and acquire a delegation token. Nimbus can then distribute this delegation token to all of its worker hosts and all workers will be able to authenticate to kafka using tokens and will have all the access that nimbus keytab principal has.
getDelegationToken(request: DelegationTokenRequest): DelegationTokenResponse
class DelegationTokenRequest(renewer: Set[KafkaPrincipal] = Set.empty, maxLifeTime: long = -1)
class DelegationTokenResponse(owner: KafkaPrincipal, expiryTimeMillis: long, renewer: Set[KafkaPrincipal], maxLifeTime: long, tokenId: String, hmac: byte[])
renewDelegationToken(request: RenewDelegationTokenRequest): DelegationTokenResponse
class RenewDelegationTokenRequest(hmac: byte[], expiryTimeMillis: long)
expireToken(request: ExpireTokenRequest)
class ExpireTokenRequest(hmac: byte[], expireAt: long = Systemtime.currentTimeMillis)
DelegationTokenRequest => [Renewer] MaxDateMs Renewer => string MaxDateMs => INT64 |
Field | Description |
---|---|
Renewer | Renewer is an Kafka Principal, which is allowed to renew this token before the max lifetime expires. If Renewer list is empty, then Renewer will default to the owner (Principal which requested this token). |
MaxDateMs | Max lifetime for token in milli seconds. if value is -1, then MaxLifeTime will default to a server side config value. |
DelegationTokenResponse => ErrorCode TokenDetails ErrorCode => INT16 TokenDetails => Owner IssueDateMs ExpiryDateMs TokenId HMAC [Renewer] Owner => String IssueDateMs => INT64 ExpiryDateMs => INT64 TokenId => String HMAC => bytes Renewer => String |
Field | Description |
---|---|
Owner | Kakfa Principal which requested the delegation token |
IssueDateMs | timestamp (in msec) when this token was generated. Unit is milliseconds since beginning of the epoch (midnight Jan 1, 1970 (UTC)). |
ExpiryDateMs | timestamp (in msec) at which this token expires. Unit is milliseconds since beginning of the epoch (midnight Jan 1, 1970 (UTC)). |
TokenId | Sequence number/UUID to ensure uniqueness |
HMAC | Keyed-hash message authentication code |
Renewer | Renewers list |
* DelegationTokenDisabledException
RenewDelegationTokenRequest => HMAC RenewPeriodMs HMAC => bytes RenewPeriodMs => INT64 |
Field | Description |
---|---|
HMAC | HMAC of the delegation token to be renewed |
RenewPeriodMs | Renew Time period in milli seconds |
RenewDelegationTokenResponse => ErrorCode TokenDetails ErrorCode => INT32 TokenDetails => Owner IssueDateMs ExpiryDateMs TokenId HMAC [Renewer] Owner => String IssueDateMs => INT64 ExpiryDateMs => INT64 TokenId => String HMAC => bytes Renewer => String |
* DelegationTokenDisabledException
* TokenRenewerMismatchException
* TokenNotFoundException
ExpireTokenRequest => HMAC expiryDateMs HMAC => bytes expiryDateMs => INT64 |
Field | Description |
---|---|
HMAC | HMAC of the delegation token to be renewed |
expiryDateMs | Token expiry timestamp. Unit is milliseconds since beginning of the epoch (midnight Jan 1, 1970 (UTC)) |
ExpireTokenResponse => ErrorCode ErroCode => INT32 |
* DelegationTokenDisabledException
* TokenRenewerMismatchException
* TokenNotFoundException
The following options will be added to KafkaConfig.java
and can be configured as properties for Kafka server:
delegation.token.max.lifetime.ms : The token has a maximum lifetime beyond which it cannot be renewed any more. Default value 7 days.
delegation.token.expiry.time.ms : The token validity time in seconds before the token needs to be renewed. Default value 1 day.
delegation.token.master.key : Secret/masterKey to generate and verify delegation tokens. This masterKey needs to be configured with all the brokers. If the secret is not set or set to empty string, brokers will disable the delegation token support.
The Kafka authentication token is modeled after the Hadoop user delegation token. The token will consist of:
TokenID:
TokenAuthenticator(HMAC) := HMAC_SHA1(master key, TokenUID)
Authentication Token := (TokenID, TokenAuthenticator(HMAC))
The secret is used to generate and verify delegation tokens. This is supplied using config option. This secret needs to be configured with all the brokers. The current proposal does not support rotation of secret. If the secret is not set or set to empty string, brokers will disable the delegation token support. We require a re-deployment when the secret needs to be rotated.
Following steps describe how tokens can be acquired:
A (Admin/DelegationToken) client connects with one of the kafka broker. Client must be authenticated using any of the available secure channels so it must have a way to authenticate, i.e. Kerberos keytab or TGT.
Once a client is authenticated, it will make a broker side call to issue a delegation token. The request for delegation token will have to contain an optional renewer identity and max lifetime for token. The renewer is the user that is allowed to renew this token before the max lifetime expires. Renewer will default to the owner if not provided and Max life time will default to a server side config value (default days) Brokers will allow a token to be renewed until maxLifeTime but a token will still expire if not renewed by the expiry time. The expiry time will be a broker side configuration and will default to min (24 hours, maxlifeTime) . A Delegation Token request can be represented as class DelegationTokenRequest(renewer: Set[KafkaPrincipal], maxLifeTime: long). The owner is implicit in the request connection as the user who requested the delegation token.
The broker generates a shared secret based on HMAC-SASM(a Password/Secret shared between all brokers, randomly generated tokenId). We can represent a token as scala case class DelegationToken(owner: KafkaPrincipal, renewer: Set[KafkaPrincipal], maxLifeTime: long, id: String, hmac: String, expirationTime: long)
Broker stores this token in its in memory cache. Broker also stores the DelegationToken without the hmac in the zookeeper. As all brokers share the Password/Secret to generate the HMAC-SASM, they can read the request info from zookeeper , generate the hmac and store the delegation token in local cache.
All brokers will have a cache backed by zookeeper so they will all get notified whenever a new token is generated and they will update their local cache whenever token state changes.
Broker returns the token to Client. Client is expected to only make delegation token request over an encrypted channel so the token in encrypted over the wire.
Client is free to distribute this token to other Kafka clients (Producer/Consumers). It is the client’s responsibility to distribute the token securely.
We will reuse the current SASL channel for delegation token based authentication.
SCRAM is a suitable mechanism for authentication using delegation tokens. KIP-84 proposes to support SASL SCRAM mechanisms. Kafka clients can authenticate using SCRAM-SHA-256, providing the delegation token HMAC as password.
Server will look up the token from its token cache, if it finds a match and token is not expired it will authenticate the client and the identity will be established as the owner of the delegation token.
If the token is not matched or token is expired, broker throws appropriate exception back and does not allow the client to continue.
The (Admin/Delegation Token) client authenticates using Kerberos or any other available authentication scheme. A token can not be renewed if the initial authentication is done through delegation token, client must use a different auth scheme.
Client sends a request to renew a token with an optional renew life time which must be < max life time of token.
Broker looks up the token, if token is expired or if the renewer’s identity does not match with the token’s renewers, or if token renewal is beyond the Max life time of token, broker disallows the operation by throwing an appropriate exception.
If none of the above conditions are matched, broker updates token’s expiry. Note that the HMAC-SASM is unchanged so the token on client side is unchanged. Broker updates the expiration in its local cache and on zookeeper so other brokers also get notified and their cache statuses are updated as well.
If a token is not renewed by the token’s expiration time or if token is beyond the max life time, it will be deleted from all broker caches as well as from zookeeper. Alternatively an owner or renewer can issue a expiration/cancellation by following a similar process as renewal.
Token are stored in Zookeeper as properties in the path /tokenauth/tokens/<tokenUID>
//Delegation Token Details for tokenID token123: Zookeeper persistence path /tokenauth/tokens/token123 { "version":1, "owner" : "owner", "renewer" : "renewer", "issueDate" : "issueDate", "tokenUID" : "tokenUID", //Store SCRAM credentials also here as per KIP-84 }; |
SCRAM messages have an optional extensions field which is a comma-separated list of key=value pairs.
After KIP-84 implementation , an extension will be added to the first client SCRAM message to indicate
that authentication is being requested for a delegation token. This will enable Kafka broker to obtain
credentials and principal using a different code path for delegation tokens.
KafkaClient { org.apache.kafka.common.security.scram.ScramLoginModule required username="test123" password="ab24267ac3e827e00e57cdf98465baccecbbeced512e90a719026177e04e547e"; }; |
We will be providing a DelegationToken Client using which users can generate, renew and expire the tokens. This may part of AdminClient implementation (KIP-4).
public class DelegationTokenClient { public TokenDetails generateToken(List<String> renewers, long maxLifeTime); public boolean renewToken(bytes[] hmac, long expiryTime); public boolean expireToken(bytes[] hmac, long expireTimeStamp); public boolean invalidateToken(bytes[] hmac); public void close(); } |
We will provide a CLI to acquire delegation tokens, renew tokens and to invalidate/expire tokens.
KIP-85 allows dynamic JAAS configuration for Kafka clients. After this we can easily configure the
delegation token for SCRAM-SHA-256 authentication.
Below diagram shows the steps required to use the delegation tokens.
Q1. Is there any dependency on Hadoop APIs/Libraries?
A. No.
Originally we considered to not have any shared Secret at config level. This required us to chose one of the 3 options: