Browse Source

KAFKA-4363; Documentation for sasl.jaas.config property

Author: Rajini Sivaram <rajinisivaram@googlemail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #2316 from rajinisivaram/KAFKA-4363
pull/2387/head
Rajini Sivaram 8 years ago committed by Ismael Juma
parent
commit
b4d8668d6d
  1. 147
      docs/security.html

147
docs/security.html

@ -263,23 +263,65 @@ @@ -263,23 +263,65 @@
<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. To configure SASL authentication
on the clients:
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>Add a JAAS config file 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>. <tt>KafkaClient</tt> is the
section name in the JAAS file used by Kafka clients.</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>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>
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>
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 {
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>
</ol>
</li>
</ol>
</li>
<li><h4><a id="security_sasl_kerberos" href="#security_sasl_kerberos">Authentication using SASL/Kerberos</a></h4>
@ -320,7 +362,7 @@ @@ -320,7 +362,7 @@
</li>
<tt>KafkaServer</tt> section in the JAAS file tells the broker which principal to use and the location of the keytab where this principal is stored. It
allows the broker to login using the keytab specified in this section. See <a href="#security_sasl_brokernotes">notes</a> for more details on Zookeeper SASL configuration.
<li>Pass the JAAS and optionally the krb5 file locations as JVM parameters to each Kafka broker (see <a href="https://docs.oracle.com/javase/8/docs/technotes/guides/security/jgss/tutorials/KerberosReq.html">here</a> for more details):
<li>Pass the JAAS and optionally the krb5 file locations as JVM parameters to each Kafka broker (see <a href="https://docs.oracle.com/javase/8/docs/technotes/guides/security/jgss/tutorials/KerberosReq.html">here</a> for more details):
<pre> -Djava.security.krb5.conf=/etc/kafka/krb5.conf
-Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf</pre>
</li>
@ -331,7 +373,7 @@ @@ -331,7 +373,7 @@
sasl.mechanism.inter.broker.protocol=GSSAPI
sasl.enabled.mechanisms=GSSAPI
</pre>
</li>We must also configure the service name in server.properties, which should match the principal name of the kafka brokers. In the above example, principal is "kafka/kafka1.hostname.com@EXAMPLE.com", so:
</li>We must also configure the service name in server.properties, which should match the principal name of the kafka brokers. In the above example, principal is "kafka/kafka1.hostname.com@EXAMPLE.com", so:
<pre> sasl.kerberos.service.name=kafka</pre>
</ol></li>
@ -339,37 +381,43 @@ @@ -339,37 +381,43 @@
To configure SASL authentication on the clients:
<ol>
<li>
Clients (producers, consumers, connect workers, etc) will authenticate to the cluster with their own principal (usually with the same name as the user running the client), so obtain or create these principals as needed. Then create a JAAS file for each principal.
The KafkaClient section describes how the clients like producer and consumer can connect to the Kafka Broker. The following is an example configuration for a client using a keytab (recommended for long-running processes):
Clients (producers, consumers, connect workers, etc) will authenticate to the cluster with their
own principal (usually with the same name as the user running the client), so obtain or create
these principals as needed. Then configure the JAAS configuration property for each client.
Different clients within a JVM may run as different users by specifiying different principals.
The property <code>sasl.jaas.config</code> in producer.properties or consumer.properties describes
how clients like producer and consumer can connect to the Kafka Broker. The following is an example
configuration for a client using a keytab (recommended for long-running processes):
<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>
For command-line utilities like kafka-console-consumer or kafka-console-producer, kinit can be used along with "useTicketCache=true" as in:
sasl.jaas.config=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>
For command-line utilities like kafka-console-consumer or kafka-console-producer, kinit can be used
along with "useTicketCache=true" as in:
<pre>
KafkaClient {
com.sun.security.auth.module.Krb5LoginModule required
useTicketCache=true;
};</pre>
sasl.jaas.config=com.sun.security.auth.module.Krb5LoginModule required \
useTicketCache=true;</pre>
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.</li>
<li>Make sure the keytabs configured in the JAAS configuration are readable by the operating system user who is starting kafka client.</li>
</li>
<li>Pass the JAAS and optionally krb5 file locations as JVM parameters to each client JVM (see <a href="https://docs.oracle.com/javase/8/docs/technotes/guides/security/jgss/tutorials/KerberosReq.html">here</a> for more details):
<pre> -Djava.security.krb5.conf=/etc/kafka/krb5.conf
-Djava.security.auth.login.config=/etc/kafka/kafka_client_jaas.conf</pre></li>
<li>Make sure the keytabs configured in the kafka_client_jaas.conf are readable by the operating system user who is starting kafka client.</li>
<li>Configure the following properties in producer.properties or consumer.properties:
<pre> security.protocol=SASL_PLAINTEXT (or SASL_SSL)
sasl.mechanism=GSSAPI
sasl.kerberos.service.name=kafka</pre></li>
<li>Optionally pass the krb5 file locations as JVM parameters to each client JVM (see <a href="https://docs.oracle.com/javase/8/docs/technotes/guides/security/jgss/tutorials/KerberosReq.html">here</a> for more details):
<pre> -Djava.security.krb5.conf=/etc/kafka/krb5.conf</pre></li>
<li>Configure the following properties in producer.properties or consumer.properties:
<pre>
security.protocol=SASL_PLAINTEXT (or SASL_SSL)
sasl.mechanism=GSSAPI
sasl.kerberos.service.name=kafka</pre></li>
</ol>
</li>
</ol>
</li>
<li><h4><a id="security_sasl_plain" href="#security_sasl_plain">Authentication using SASL/PLAIN</a></h4>
<p>SASL/PLAIN is a simple username/password authentication mechanism that is typically used with TLS for encryption to implement secure authentication.
Kafka supports a default implementation for SASL/PLAIN which can be extended for production use as described <a href="#security_sasl_plain_production">here</a>.</p>
@ -404,22 +452,25 @@ @@ -404,22 +452,25 @@
<li><h5><a id="security_sasl_plain_clientconfig" href="#security_sasl_plain_clientconfig">Configuring Kafka Clients</a></h5>
To configure SASL authentication on the clients:
<ol>
<li>The <tt>KafkaClient</tt> section 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 PLAIN mechanism:
<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 PLAIN mechanism:
<pre>
KafkaClient {
org.apache.kafka.common.security.plain.PlainLoginModule required
username="alice"
password="alice-secret";
};</pre>
The properties <tt>username</tt> and <tt>password</tt> in the <tt>KafkaClient</tt> section are used by clients to configure
sasl.jaas.config=org.apache.kafka.common.security.plain.PlainLoginModule 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>.
</li>
<li>Pass the JAAS config file location as JVM parameter to each client JVM:
<pre> -Djava.security.auth.login.config=/etc/kafka/kafka_client_jaas.conf</pre></li>
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>Configure the following properties in producer.properties or consumer.properties:
<pre> security.protocol=SASL_SSL
sasl.mechanism=PLAIN</pre></li>
<pre>
security.protocol=SASL_SSL
sasl.mechanism=PLAIN</pre></li>
</ol>
</li>
<li><h5><a id="security_sasl_plain_production" href="#security_sasl_plain_production">Use of SASL/PLAIN in production</a></h5>

Loading…
Cancel
Save