KAFKA-4589; SASL/SCRAM documentation

Author: Rajini Sivaram <rajinisivaram@googlemail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>, Gwen Shapira <cshapi@gmail.com>, Sriharsha Chintalapani <harsha@hortonworks.com>, Jun Rao <junrao@gmail.com>

Closes #2369 from rajinisivaram/KAFKA-4589
This commit is contained in:
Rajini Sivaram 2017-01-19 08:35:39 -08:00 committed by Jun Rao
parent 8e2cbae8b1
commit 666abafcc5
1 changed files with 217 additions and 98 deletions

View File

@ -19,8 +19,12 @@
<h3><a id="security_overview" href="#security_overview">7.1 Security Overview</a></h3>
In release 0.9.0.0, the Kafka community added a number of features that, used either separately or together, increases security in a Kafka cluster. The following security measures are currently supported:
<ol>
<li>Authentication of connections to brokers from clients (producers and consumers), other brokers and tools, using either SSL or SASL (Kerberos).
SASL/PLAIN can also be used from release 0.10.0.0 onwards.</li>
<li>Authentication of connections to brokers from clients (producers and consumers), other brokers and tools, using either SSL or SASL. Kafka supports the following SASL mechanisms:
<ul>
<li>SASL/GSSAPI (Kerberos) - starting at version 0.9.0.0</li>
<li>SASL/PLAIN - starting at version 0.10.0.0</li>
<li>SASL/SCRAM-SHA-256 and SASL/SCRAM-SHA-512 - starting at version 0.10.2.0</li>
</ul></li>
<li>Authentication of connections from brokers to ZooKeeper</li>
<li>Encryption of data transferred between brokers and clients, between brokers, or between brokers and tools using SSL (Note that there is a performance degradation when SSL is enabled, the magnitude of which depends on the CPU type and the JVM implementation.)</li>
<li>Authorization of read / write operations by clients</li>
@ -211,117 +215,125 @@
<h3><a id="security_sasl" href="#security_sasl">7.3 Authentication using SASL</a></h3>
<ol>
<li><h4><a id="security_sasl_brokerconfig"
href="#security_sasl_brokerconfig">SASL configuration for Kafka brokers</a></h4>
<li><h4><a id="security_sasl_jaasconfig" href="#security_sasl_jaasconfig">JAAS configuration</a></h4>
<p>Kafka uses the Java Authentication and Authorization Service
(<a href="https://docs.oracle.com/javase/8/docs/technotes/guides/security/jaas/JAASRefGuide.html">JAAS</a>)
for SASL configuration.</p>
<ol>
<li>Select one or more supported mechanisms to enable in the broker. <tt>GSSAPI</tt>
and <tt>PLAIN</tt> are the mechanisms currently supported in Kafka.</li>
<li>Add a JAAS config file for the selected mechanisms as described in the examples
for setting up <a href="#security_sasl_kerberos_brokerconfig">GSSAPI (Kerberos)</a>
or <a href="#security_sasl_plain_brokerconfig">PLAIN</a>.</li>
<li>Pass the JAAS config file location as JVM parameter to each Kafka broker.
For example:
<pre> -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf</pre></li>
<li>Configure a SASL port in server.properties, by adding at least one of
SASL_PLAINTEXT or SASL_SSL to the <i>listeners</i> parameter, which
contains one or more comma-separated values:
<pre> listeners=SASL_PLAINTEXT://host.name:port</pre>
If SASL_SSL is used, then <a href="#security_ssl">SSL must also be
configured</a>. If you are only configuring a SASL port (or if you want
the Kafka brokers to authenticate each other using SASL) then make sure
you set the same SASL protocol for inter-broker communication:
<pre> security.inter.broker.protocol=SASL_PLAINTEXT (or SASL_SSL)</pre></li>
<li>Enable one or more SASL mechanisms in server.properties:
<pre> sasl.enabled.mechanisms=GSSAPI (,PLAIN)</pre></li>
<li>Configure the SASL mechanism for inter-broker communication in server.properties
if using SASL for inter-broker communication:
<pre> sasl.mechanism.inter.broker.protocol=GSSAPI (or PLAIN)</pre></li>
<li>Follow the steps in <a href="#security_sasl_kerberos_brokerconfig">GSSAPI (Kerberos)</a>
or <a href="#security_sasl_plain_brokerconfig">PLAIN</a> to configure SASL
for the enabled mechanisms. To enable multiple mechanisms in the broker, follow
the steps <a href="#security_sasl_multimechanism">here</a>.</li>
<u><a id="security_sasl_brokernotes" href="#security_sasl_brokernotes">Important notes:</a></u>
<ol>
<li><tt>KafkaServer</tt> is the section name in the JAAS file used by each
<li><h5><a id="security_jaas_broker"
href="#security_jaas_broker">JAAS configuration for Kafka brokers</a></h5>
<p><tt>KafkaServer</tt> is the section name in the JAAS file used by each
KafkaServer/Broker. This section provides SASL configuration options
for the broker including any SASL client connections made by the broker
for inter-broker communication.</li>
<li><tt>Client</tt> section is used to authenticate a SASL connection with
for inter-broker communication.</p>
<p><tt>Client</tt> section is used to authenticate a SASL connection with
zookeeper. It also allows the brokers to set SASL ACL on zookeeper
nodes which locks these nodes down so that only the brokers can
modify it. It is necessary to have the same principal name across all
brokers. If you want to use a section name other than Client, set the
system property <tt>zookeeper.sasl.client</tt> to the appropriate
name (<i>e.g.</i>, <tt>-Dzookeeper.sasl.client=ZkClient</tt>).</li>
<li>ZooKeeper uses "zookeeper" as the service name by default. If you
name (<i>e.g.</i>, <tt>-Dzookeeper.sasl.client=ZkClient</tt>).</p>
<p>ZooKeeper uses "zookeeper" as the service name by default. If you
want to change this, set the system property
<tt>zookeeper.sasl.client.username</tt> to the appropriate name
(<i>e.g.</i>, <tt>-Dzookeeper.sasl.client.username=zk</tt>).</li>
</ol>
</ol>
</li>
<li><h4><a id="security_sasl_clientconfig"
href="#security_sasl_clientconfig">SASL configuration for Kafka clients</a></h4>
SASL authentication is only supported for the new Java Kafka producer and
consumer, the older API is not supported. JAAS configuration for clients may
be specified as a static JAAS config file or using the client configuration property
<code>sasl.jaas.config</code>.
To configure SASL authentication on the clients:
<ol>
<li>Select a SASL mechanism for authentication.</li>
<li>Configure the following properties in producer.properties or
consumer.properties:
<pre> security.protocol=SASL_PLAINTEXT (or SASL_SSL)
sasl.mechanism=GSSAPI (or PLAIN)</pre></li>
<li>Follow the steps in <a href="#security_sasl_kerberos_clientconfig">GSSAPI (Kerberos)</a>
or <a href="#security_sasl_plain_clientconfig">PLAIN</a> to configure SASL
for the selected mechanism.</li>
<li>Configure JAAS using <a href="#security_client_dynamicjaas">client configuration property</a>
or <a href="#security_client_staticjaas">static JAAS config file</a> as described below.</li>
</ol>
<ol>
<li><h5><a id="security_client_dynamicjaas" href="#security_client_dynamicjaas">JAAS configuration using client configuration property</a></h5>
<p>Clients may specify JAAS configuration as a producer or consumer property without
creating a physical configuration file. This mode also enables different producers
and consumers within the same JVM to use different credentials by specifying
different properties for each client. If both static JAAS configuration system property
<code>java.security.auth.login.config</code> and client property <code>sasl.jaas.config</code>
are specified, the client property will be used.</p>
(<i>e.g.</i>, <tt>-Dzookeeper.sasl.client.username=zk</tt>).</p></li>
<li><h5><a id="security_jaas_client"
href="#security_jaas_client">JAAS configuration for Kafka clients</a></h5>
<p>Clients may configure JAAS using the client configuration property
<a href="#security_client_dynamicjaas">sasl.jaas.config</a>
or using the <a href="#security_client_staticjaas">static JAAS config file</a>
similar to brokers.</p>
To configure SASL authentication on the clients using configuration property:
<ol>
<li>Configure the property <code>sasl.jaas.config</code> in producer.properties or
consumer.properties to be the JAAS login module section of the selected mechanism.
For example, <a href="#security_sasl_plain_clientconfig">PLAIN</a>
credentials may be configured as:
<pre> sasl.jaas.config=org.apache.kafka.common.security.plain.PlainLoginModule required username="alice" password="alice-secret";</pre></li>
See <a href="#security_sasl_kerberos_clientconfig">GSSAPI (Kerberos)</a> or <a href="#security_sasl_plain_clientconfig">PLAIN</a>
for full example configurations.</li>
</ol>
</li>
<li><h5><a id="security_client_staticjaas" href="#security_client_staticjaas">JAAS configuration using static config file</a></h5>
To configure SASL authentication on the clients using static JAAS config file:
<ol>
<li>Add a JAAS config file with a client login section named <tt>KafkaClient</tt>. Configure
a login module in <tt>KafkaClient</tt> for the selected mechanism as described in the examples
for setting up <a href="#security_sasl_kerberos_clientconfig">GSSAPI (Kerberos)</a>
or <a href="#security_sasl_plain_clientconfig">PLAIN</a>.
For example, <a href="#security_sasl_gssapi_clientconfig">GSSAPI</a>
credentials may be configured as:
<pre>
KafkaClient {
<li><h6><a id="security_client_dynamicjaas"
href="#security_client_dynamicjaas">JAAS configuration using client configuration property</a></h6>
<p>Clients may specify JAAS configuration as a producer or consumer property without
creating a physical configuration file. This mode also enables different producers
and consumers within the same JVM to use different credentials by specifying
different properties for each client. If both static JAAS configuration system property
<code>java.security.auth.login.config</code> and client property <code>sasl.jaas.config</code>
are specified, the client property will be used.</p>
<p>See <a href="#security_sasl_kerberos_clientconfig">GSSAPI (Kerberos)</a>,
<a href="#security_sasl_plain_clientconfig">PLAIN</a> or
<a href="#security_sasl_scram_clientconfig">SCRAM</a> for example configurations.</p></li>
<li><h6><a id="security_client_staticjaas" href="#security_client_staticjaas">JAAS configuration using static config file</a></h6>
To configure SASL authentication on the clients using static JAAS config file:
<ol>
<li>Add a JAAS config file with a client login section named <tt>KafkaClient</tt>. Configure
a login module in <tt>KafkaClient</tt> for the selected mechanism as described in the examples
for setting up <a href="#security_sasl_kerberos_clientconfig">GSSAPI (Kerberos)</a>,
<a href="#security_sasl_plain_clientconfig">PLAIN</a> or
<a href="#security_sasl_scram_clientconfig">SCRAM</a>.
For example, <a href="#security_sasl_gssapi_clientconfig">GSSAPI</a>
credentials may be configured as:
<pre>
KafkaClient {
com.sun.security.auth.module.Krb5LoginModule required
useKeyTab=true
storeKey=true
keyTab="/etc/security/keytabs/kafka_client.keytab"
principal="kafka-client-1@EXAMPLE.COM";
};</pre>
See <a href="#security_sasl_kerberos_clientconfig">GSSAPI (Kerberos)</a> or <a href="#security_sasl_plain_clientconfig">PLAIN</a>
for example configurations of each mechanism.</li>
<li>Pass the JAAS config file location as JVM parameter to each client JVM. For example:
<pre> -Djava.security.auth.login.config=/etc/kafka/kafka_client_jaas.conf</pre></li>
</li>
<li>Pass the JAAS config file location as JVM parameter to each client JVM. For example:
<pre> -Djava.security.auth.login.config=/etc/kafka/kafka_client_jaas.conf</pre></li>
</ol>
</li>
</ol>
</li>
</ol>
</li>
<li><h4><a id="security_sasl_config"
href="#security_sasl_config">SASL configuration</a></h4>
<p>SASL may be used with PLAINTEXT or SSL as the transport layer using the
security protocol SASL_PLAINTEXT or SASL_SSL respectively. If SASL_SSL is
used, then <a href="#security_ssl">SSL must also be configured</a>.</p>
<ol>
<li><h5><a id="security_sasl_mechanism"
href="#security_sasl_mechanism">SASL mechanisms</a></h5>
Kafka supports the following SASL mechanisms:
<ul>
<li><a href="#security_sasl_kerberos">GSSAPI</a> (Kerberos)</li>
<li><a href="#security_sasl_plain">PLAIN</a></li>
<li><a href="#security_sasl_scram">SCRAM-SHA-256</a></li>
<li><a href="#security_sasl_scram">SCRAM-SHA-512</a></li>
</ul>
</li>
<li><h5><a id="security_sasl_brokerconfig"
href="#security_sasl_brokerconfig">SASL configuration for Kafka brokers</a></h5>
<ol>
<li>Configure a SASL port in server.properties, by adding at least one of
SASL_PLAINTEXT or SASL_SSL to the <i>listeners</i> parameter, which
contains one or more comma-separated values:
<pre> listeners=SASL_PLAINTEXT://host.name:port</pre>
If you are only configuring a SASL port (or if you want
the Kafka brokers to authenticate each other using SASL) then make sure
you set the same SASL protocol for inter-broker communication:
<pre> security.inter.broker.protocol=SASL_PLAINTEXT (or SASL_SSL)</pre></li>
<li>Select one or more <a href="#security_sasl_mechanism">supported mechanisms</a>
to enable in the broker and follow the steps to configure SASL for the mechanism.
To enable multiple mechanisms in the broker, follow the steps
<a href="#security_sasl_multimechanism">here</a>.</li>
</ol>
</li>
<li><h5><a id="security_sasl_clientconfig"
href="#security_sasl_clientconfig">SASL configuration for Kafka clients</a></h5>
<p>SASL authentication is only supported for the new Java Kafka producer and
consumer, the older API is not supported.</p>
<p>To configure SASL authentication on the clients, select a SASL
<a href="#security_sasl_mechanism">mechanism</a> that is enabled in
the broker for client authentication and follow the steps to configure SASL
for the selected mechanism.</p></li>
</ol>
</li>
<li><h4><a id="security_sasl_kerberos" href="#security_sasl_kerberos">Authentication using SASL/Kerberos</a></h4>
@ -502,6 +514,111 @@
</li>
</ol>
</li>
<li><h4><a id="security_sasl_scram" href="#security_sasl_scram">Authentication using SASL/SCRAM</a></h4>
<p>Salted Challenge Response Authentication Mechanism (SCRAM) is a family of SASL mechanisms that
addresses the security concerns with traditional mechanisms that perform username/password authentication
like PLAIN and DIGEST-MD5. The mechanism is defined in <a href="https://tools.ietf.org/html/rfc5802">RFC 5802</a>.
Kafka supports <a href="https://tools.ietf.org/html/rfc7677">SCRAM-SHA-256</a> and SCRAM-SHA-512 which
can be used with TLS to perform secure authentication. The username is used as the authenticated
<code>Principal</code> for configuration of ACLs etc. The default SCRAM implementation in Kafka
stores SCRAM credentials in Zookeeper and is suitable for use in Kafka installations where Zookeeper
is on a private network. Refer to <a href="#security_sasl_scram_security">Security Considerations</a>
for more details.</p>
<ol>
<li><h5><a id="security_sasl_scram_credentials" href="#security_sasl_scram_credentials">Creating SCRAM Credentials</a></h5>
<p>The SCRAM implementation in Kafka uses Zookeeper as credential store. Credentials can be created in
Zookeeper using <tt>kafka-configs.sh</tt>. For each SCRAM mechanism enabled, credentials must be created
by adding a config with the mechanism name. Credentials for inter-broker communication must be created
before Kafka brokers are started. Client credentials may be created and updated dynamically and updated
credentials will be used to authenticate new connections.</p>
<p>Create SCRAM credentials for user <i>alice</i> with password <i>alice-secret</i>:
<pre>
bin/kafka-configs.sh --zookeeper localhost:2181 --alter --add-config 'SCRAM-SHA-256=[iterations=8192,password=alice-secret],SCRAM-SHA-512=[password=alice-secret]' --entity-type users --entity-name alice
</pre>
<p>The default iteration count of 4096 is used if iterations are not specified. A random salt is created
and the SCRAM identity consisting of salt, iterations, StoredKey and ServerKey are stored in Zookeeper.
See <a href="https://tools.ietf.org/html/rfc5802">RFC 5802</a> for details on SCRAM identity and the individual fields.
<p>The following examples also require a user <i>admin</i> for inter-broker communication which can be created using:
<pre>
bin/kafka-configs.sh --zookeeper localhost:2181 --alter --add-config 'SCRAM-SHA-256=[password=admin-secret],SCRAM-SHA-512=[password=admin-secret]' --entity-type users --entity-name admin
</pre>
<p>Existing credentials may be listed using the <i>--describe</i> option:
<pre>
bin/kafka-configs.sh --zookeeper localhost:2181 --describe --entity-type users --entity-name alice
</pre>
<p>Credentials may be deleted for one or more SCRAM mechanisms using the <i>--delete</i> option:
<pre>
bin/kafka-configs.sh --zookeeper localhost:2181 --alter --delete-config 'SCRAM-SHA-512' --entity-type users --entity-name alice
</pre>
</li>
<li><h5><a id="security_sasl_scram_brokerconfig" href="#security_sasl_scram_brokerconfig">Configuring Kafka Brokers</a></h5>
<ol>
<li>Add a suitably modified JAAS file similar to the one below to each Kafka broker's config directory, let's call it kafka_server_jaas.conf for this example:
<pre>
KafkaServer {
org.apache.kafka.common.security.scram.ScramLoginModule required
username="admin"
password="admin-secret"
};</pre>
The properties <tt>username</tt> and <tt>password</tt> in the <tt>KafkaServer</tt> section are used by
the broker to initiate connections to other brokers. In this example, <i>admin</i> is the user for
inter-broker communication.</li>
<li>Pass the JAAS config file location as JVM parameter to each Kafka broker:
<pre> -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf</pre></li>
<li>Configure SASL port and SASL mechanisms in server.properties as described <a href="#security_sasl_brokerconfig">here</a>.</pre> For example:
<pre>
listeners=SASL_SSL://host.name:port
security.inter.broker.protocol=SASL_SSL
sasl.mechanism.inter.broker.protocol=SCRAM-SHA-256 (or SCRAM-SHA-512)
sasl.enabled.mechanisms=SCRAM-SHA-256 (or SCRAM-SHA-512)</pre></li>
</ol>
</li>
<li><h5><a id="security_sasl_scram_clientconfig" href="#security_sasl_scram_clientconfig">Configuring Kafka Clients</a></h5>
To configure SASL authentication on the clients:
<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 SCRAM mechanisms:
<pre>
sasl.jaas.config=org.apache.kafka.common.security.scram.ScramLoginModule required \
username="alice" \
password="alice-secret";</pre>
<p>The options <tt>username</tt> and <tt>password</tt> are used by clients to configure
the user for client connections. In this example, clients connect to the broker as user <i>alice</i>.
Different clients within a JVM may connect as different users by specifying different user names
and passwords 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>
</li>
<li>Configure the following properties in producer.properties or consumer.properties:
<pre>
security.protocol=SASL_SSL
sasl.mechanism=SCRAM-SHA-256 (or SCRAM-SHA-512)</pre></li>
</ol>
</li>
<li><h5><a id="security_sasl_scram_security" href="#security_sasl_scram_security">Security Considerations for SASL/SCRAM</a></h5>
<ul>
<li>The default implementation of SASL/SCRAM in Kafka stores SCRAM credentials in Zookeeper. This
is suitable for production use in installations where Zookeeper is secure and on a private network.</li>
<li>Kafka supports only the strong hash functions SHA-256 and SHA-512 with a minimum iteration count
of 4096. Strong hash functions combined with strong passwords and high iteration counts protect
against brute force attacks if Zookeeper security is compromised.</li>
<li>SCRAM should be used only with TLS-encryption to prevent interception of SCRAM exchanges. This
protects against dictionary or brute force attacks and against impersonation if Zookeeper is compromised.</li>
<li>The default SASL/SCRAM implementation may be overridden using custom login modules in installations
where Zookeeper is not secure. See <a href="#security_sasl_plain_production">here</a> for details.</li>
<li>For more details on security considerations, refer to
<a href="https://tools.ietf.org/html/rfc5802#section-9">RFC 5802</a>.
</ul>
</li>
</ol>
</li>
<li><h4><a id="security_sasl_multimechanism" href="#security_sasl_multimechanism">Enabling multiple SASL mechanisms in a broker</a></h4>
<ol>
<li>Specify configuration for the login modules of all enabled mechanisms in the <tt>KafkaServer</tt> section of the JAAS config file. For example:
@ -519,12 +636,14 @@
user_admin="admin-secret"
user_alice="alice-secret";
};</pre></li>
<li>Enable the SASL mechanisms in server.properties: <pre> sasl.enabled.mechanisms=GSSAPI,PLAIN</pre></li>
<li>Enable the SASL mechanisms in server.properties: <pre> sasl.enabled.mechanisms=GSSAPI,PLAIN,SCRAM-SHA-256,SCRAM-SHA-512</pre></li>
<li>Specify the SASL security protocol and mechanism for inter-broker communication in server.properties if required:
<pre> security.inter.broker.protocol=SASL_PLAINTEXT (or SASL_SSL)
sasl.mechanism.inter.broker.protocol=GSSAPI (or PLAIN)</pre></li>
<li>Follow the mechanism-specific steps in <a href="#security_sasl_kerberos_brokerconfig">GSSAPI (Kerberos)</a>
and <a href="#security_sasl_plain_brokerconfig">PLAIN</a> to configure SASL for the enabled mechanisms.</li>
<pre>
security.inter.broker.protocol=SASL_PLAINTEXT (or SASL_SSL)
sasl.mechanism.inter.broker.protocol=GSSAPI (or one of the other enabled mechanisms)</pre></li>
<li>Follow the mechanism-specific steps in <a href="#security_sasl_kerberos_brokerconfig">GSSAPI (Kerberos)</a>,
<a href="#security_sasl_plain_brokerconfig">PLAIN</a> and <a href="#security_sasl_scram_brokerconfig">SCRAM</a>
to configure SASL for the enabled mechanisms.</li>
</ol>
</li>
<li><h4><a id="saslmechanism_rolling_upgrade" href="#saslmechanism_rolling_upgrade">Modifying SASL mechanism in a Running Cluster</a></h4>