KAFKA-6452; Add documentation for delegation token authentication

Author: Manikumar Reddy <manikumar.reddy@gmail.com>

Reviewers: Jun Rao <junrao@gmail.com>

Closes #4490 from omkreddy/KAFKA-6452-TOKEN-DOCS
This commit is contained in:
Manikumar Reddy 2018-02-05 11:07:18 -08:00 committed by Jun Rao
parent 7fe1c2b3d3
commit ed971fd434
1 changed files with 102 additions and 0 deletions

View File

@ -661,6 +661,108 @@
old mechanism from JAAS config file. Incrementally bounce the cluster again.</li>
</ol>
</li>
<li><h4><a id="security_delegation_token" href="#security_delegation_token">Authentication using Delegation Tokens</a></h4>
<p>Delegation token based authentication is a lightweight authentication mechanism to complement existing SASL/SSL
methods. Delegation tokens are shared secrets between kafka brokers and clients. Delegation tokens will help processing
frameworks to distribute the workload to available workers in a secure environment without the added cost of distributing
Kerberos TGT/keytabs or keystores when 2-way SSL is used. See <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-48+Delegation+token+support+for+Kafka">KIP-48</a>
for more details.</p>
<p>Typical steps for delegation token usage are:</p>
<ol>
<li>User authenticates with the Kafka cluster via SASL or SSL, and obtains a delegation token. This can be done
using AdminClient APIs or using <tt>kafka-delegation-token.sh</tt> script.</li>
<li>User securely passes the delegation token to Kafka clients for authenticating with the Kafka cluster.</li>
<li>Token owner/renewer can renew/expire the delegation tokens.</li>
</ol>
<ol>
<li><h5><a id="security_token_management" href="#security_token_management">Token Management</a></h5>
<p> A master key/secret is used to generate and verify delegation tokens. This is supplied using config
option <tt>delegation.token.master.key</tt>. Same secret key must be configured across all the brokers.
If the secret is not set or set to empty string, brokers will disable the delegation token authentication.</p>
<p>In current implementation, token details are stored in Zookeeper and is suitable for use in Kafka installations where
Zookeeper is on a private network. Also currently, master key/secret is stored as plain text in server.properties
config file. We intend to make these configurable in a future Kafka release.</p>
<p>A token has a current life, and a maximum renewable life. By default, tokens must be renewed once every 24 hours
for up to 7 days. These can be configured using <tt>delegation.token.expiry.time.ms</tt>
and <tt>delegation.token.max.lifetime.ms</tt> config options.</p>
<p>Tokens can also be cancelled explicitly. If a token is not renewed by the tokens expiration time or if token
is beyond the max life time, it will be deleted from all broker caches as well as from zookeeper.</p>
</li>
<li><h5><a id="security_sasl_create_tokens" href="#security_sasl_create_tokens">Creating Delegation Tokens</a></h5>
<p>Tokens can be created by using AdminClient APIs or using <tt>kafka-delegation-token.sh</tt> script.
Delegation token requests (create/renew/expire/describe) should be issued only on SASL or SSL authenticated channels.
Tokens can not be requests if the initial authentication is done through delegation token.
<tt>kafka-delegation-token.sh</tt> script examples are given below.</p>
<p>Create a delegation token:
<pre class="brush: bash;">
> bin/kafka-delegation-tokens.sh --bootstrap-server localhost:9092 --create --max-life-time-period -1 --command-config client.properties --renewer-principal User:user1
</pre>
<p>Renew a delegation token:
<pre class="brush: bash;">
> bin/kafka-delegation-tokens.sh --bootstrap-server localhost:9092 --renew --renew-time-period -1 --command-config client.properties --hmac ABCDEFGHIJK
</pre>
<p>Expire a delegation token:
<pre class="brush: bash;">
> bin/kafka-delegation-tokens.sh --bootstrap-server localhost:9092 --expire --expiry-time-period -1 --command-config client.properties --hmac ABCDEFGHIJK
</pre>
<p>Existing tokens can be described using the --describe option:
<pre class="brush: bash;">
> bin/kafka-delegation-tokens.sh --bootstrap-server localhost:9092 --describe --command-config client.properties --owner-principal User:user1
</pre>
</li>
<li><h5><a id="security_token_authentication" href="#security_token_authentication">Token Authentication</a></h5>
<p>Delegation token authentication piggybacks on the current SASL/SCRAM authentication mechanism. We must enable
SASL/SCRAM mechanism on Kafka cluster as described in <a href="#security_sasl_scram">here</a>.</p>
<p>Configuring Kafka Clients:</p>
<ol>
<li>Configure the JAAS configuration property for each client in producer.properties or consumer.properties.
The login module describes how the clients like producer and consumer can connect to the Kafka Broker.
The following is an example configuration for a client for the token authentication:
<pre class="brush: text;">
sasl.jaas.config=org.apache.kafka.common.security.scram.ScramLoginModule required \
username="tokenID123" \
password="lAYYSFmLs4bTjf+lTZ1LCHR/ZZFNA==" \
tokenauth="true";</pre>
<p>The options <tt>username</tt> and <tt>password</tt> are used by clients to configure the token id and
token HMAC. And the option <tt>tokenauth</tt> is used to indicate the server about token authentication.
In this example, clients connect to the broker using token id: <i>tokenID123</i>. Different clients within a
JVM may connect using different tokens by specifying different token details in <code>sasl.jaas.config</code>.</p>
<p>JAAS configuration for clients may alternatively be specified as a JVM parameter similar to brokers
as described <a href="#security_client_staticjaas">here</a>. Clients use the login section named
<tt>KafkaClient</tt>. This option allows only one user for all client connections from a JVM.</p></li>
</ol>
</li>
<li><h5><a id="security_token_secret_rotation" href="#security_token_secret_rotation">Procedure to manually rotate the secret:</a></h5>
<p>We require a re-deployment when the secret needs to be rotated. During this process, already connected clients
will continue to work. But any new connection requests and renew/expire requests with old tokens can fail. Steps are given below.</p>
<ol>
<li>Expire all existing tokens.</li>
<li>Rotate the secret by rolling upgrade, and</li>
<li>Generate new tokens</li>
</ol>
<p>We intend to automate this in a future Kafka release.</p>
</li>
<li><h5><a id="security_token_notes" href="#security_token_notes">Notes on Delegation Tokens</a></h5>
<ul>
<li>Currently, we only allow a user to create delegation token for that user only. Owner/Renewers can renew or expire tokens.
Owner/renewers can always describe their own tokens. To describe others tokens, we need to add DESCRIBE permission on Token Resource.</li>
</ul>
</li>
</ol>
</li>
</ol>
<h3><a id="security_authz" href="#security_authz">7.4 Authorization and ACLs</a></h3>