kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From gwens...@apache.org
Subject kafka git commit: KAFKA-2681: Added SASL documentation
Date Mon, 02 Nov 2015 18:17:03 GMT
Repository: kafka
Updated Branches:
  refs/heads/trunk 9d8dd9f10 -> 34775bd3e


KAFKA-2681: Added SASL documentation

… info from the wiki

Author: Gwen Shapira <cshapi@gmail.com>

Reviewers: Sriharsha Chintalapani, Jun Rao

Closes #401 from gwenshap/KAFKA-2681


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/34775bd3
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/34775bd3
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/34775bd3

Branch: refs/heads/trunk
Commit: 34775bd3ed7b8eb4ca3532d30a9d8e6bf7c0b738
Parents: 9d8dd9f
Author: Gwen Shapira <cshapi@gmail.com>
Authored: Mon Nov 2 10:16:39 2015 -0800
Committer: Gwen Shapira <cshapi@gmail.com>
Committed: Mon Nov 2 10:16:39 2015 -0800

----------------------------------------------------------------------
 docs/documentation.html |  11 ++++
 docs/security.html      | 117 +++++++++++++++++++++++++++++++++++++++++++
 2 files changed, 128 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/34775bd3/docs/documentation.html
----------------------------------------------------------------------
diff --git a/docs/documentation.html b/docs/documentation.html
index 860f276..ce63469 100644
--- a/docs/documentation.html
+++ b/docs/documentation.html
@@ -101,6 +101,13 @@ Prior releases: <a href="/07/documentation.html">0.7.x</a>,
<a href="/08/documen
                     <li><a href="#zkops">Operationalization</a>
                 </ul>
         </ul>
+    <li><a href="#security">7. Security</a></li>
+        <ul>
+            <li><a href="#security_overview">7.1 Security Overview</a></li>
+            <li><a href="#security_ssl">7.2 Encryption and Authentication using
SSL</a></li>
+            <li><a href="#security_sasl">7.3 Authentication using SASL</a></li>
+            <li><a href="#security_authz">7.4 Authorization and ACLs</a></li>
+        </ul>
 </ul>
 
 <h2><a id="gettingStarted">1. Getting Started</a></h2>
@@ -130,4 +137,8 @@ Prior releases: <a href="/07/documentation.html">0.7.x</a>,
<a href="/08/documen
 
 <!--#include virtual="ops.html" -->
 
+<h2><a id="security">7. Security</a></h2>
+<!--#include virtual="security.html" -->
+
 <!--#include virtual="../includes/footer.html" -->
+

http://git-wip-us.apache.org/repos/asf/kafka/blob/34775bd3/docs/security.html
----------------------------------------------------------------------
diff --git a/docs/security.html b/docs/security.html
new file mode 100644
index 0000000..2776ad4
--- /dev/null
+++ b/docs/security.html
@@ -0,0 +1,117 @@
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<h3><a id="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>Authenticating clients (Producers and consumers) connections to brokers, using
either SSL or SASL (Kerberos)</li>
+    <li>Authorizing read / write operations by clients</li>
+    <li>Encryption of data sent between brokers and clients, or between brokers, using
SSL</li>
+    <li>Security is optional - non-secured clusters are supported, as well as a mix
of authenticated, unauthenticated, encrypted and non-encrypted clients.</li>
+    <li>Authorization is pluggable and supports integration with external authorization
services</li>
+</ol>
+
+The guides below explain how to configure and use the security features in both clients and
brokers.
+
+<h3><a id="security_ssl">7.2 Encryption and Authentication using SSL</a></h3>
+
+<h3><a id="security_sasl">7.3 Authentication using SASL</a></h3>
+
+<ol>
+    <li><h4>Prerequisites</h4><br>
+    <ol>
+        <li><b>Kerberos</b><br>
+        If your organization is already using a Kerberos server (for example, by using Active
Directory), there is no need to install a new server just for Kafka. Otherwise you will need
to install one, your Linux vendor likely has packages for Kerberos and a short guide on how
to install and configure it (<a href="https://help.ubuntu.com/community/Kerberos">Ubuntu</a>,
<a href="https://access.redhat.com/documentation/en-US/Red_Hat_Enterprise_Linux/6/html/Managing_Smart_Cards/installing-kerberos.html">Redhat</a>).
Note that if you are using Oracle Java, you will need to download JCE policy files for your
Java version and copy them to $JAVA_HOME/jre/lib/security.</li>
+        <li><b>Create Kerberos Principals</b><br>
+        If you are using the organization's Kerberos or Active Directory server, ask your
Kerberos administrator for a principal for each Kafka broker in your cluster and for every
Linux user that will access Kafka with Kerberos authentication.</br>
+        If you installed your own Kerberos, you will need to create these principals yourself:</br>
+            <code>sudo /usr/sbin/kadmin.local -q 'addprinc -randkey kafka/hostname@domainname'<br>
+                sudo /usr/sbin/kadmin.local -q "ktadd -k /etc/security/keytabs/kafka.keytab
kafka/hostname@domainname"</code></li>
+        <li><b>Make sure all hosts can be reachable using hostnames</b>
- It is important in case of kerberos all your hosts can be resolved with their FQDNs.</li>
+        <li><b><a name="jaas_config_file">Creating JAAS Config File</a></b><br>
+            Each node in the cluster should have a JAAS file similar to the example below.
Add this file to kafka/config dir:
+        <pre>
+            KafkaServer {
+                com.sun.security.auth.module.Krb5LoginModule required
+                useKeyTab=true
+                storeKey=true
+                serviceName="kafka"
+                keyTab="/etc/security/keytabs/kafka1.keytab"
+                principal="kafka/kafka1.hostname.com@DOMAIN.COM";
+            };
+
+            Client {
+               com.sun.security.auth.module.Krb5LoginModule required
+               useKeyTab=true
+               storeKey=true
+               serviceName="zookeeper"
+               keyTab="/etc/security/keytabs/kafka1.keytab"
+               principal="kafka/kafka1.hostname.com@DOMAIN.COM";
+            };
+
+            KafkaClient {
+               com.sun.security.auth.module.Krb5LoginModule required
+               useTicketCache=true
+               serviceName="kafka";
+            };
+        </pre>
+            <u>Important notes:</u>
+            <ol>
+                <li>KafkaServer is a section name in JAAS file used by KafkaServer/Broker.
This section tells Kafka Server which principal to use and which keytab this principal is
stored. It allows Kafka Server to login using the keytab specified in this section.</li>
+                <li>Client section is used to authenticate a SASL connection with zookeeper.
It also allows a broker to set SASL ACL on zookeeper nodes which locks these nodes down so
that only kafka broker can modify. It is important to have the same principal name across
all the brokers.</li>
+                <li>KafkaClient section here describes how the clients like producer
and consumer can connect to the Kafka Broker. Here we specified "useTicketCache=true" not
a keytab this allows user to do kinit and run a kafka-console-consumer or kafka-console-producer
to connect to broker. For a long running process one should create KafkaClient section similar
to KafkaServer.</li>
+                <li>In KafkaServer and KafkaClient sections we've "serviceName" this
should match principal name with which kafka broker is running. In the above example principal="kafka/kafka1.hostname.com@DOMAIN.com"
so we've "kafka" which is matching the principalName.</li>
+            </ol>
+        </li>
+        <li><b><a name="jaas_client">Creating Client Side JAAS Config</a></b><br>
+        Clients (producers, consumers, copycat workers, etc) will authenticate to the cluster
with their own principal (usually with the same name as the user used for running the client),
so obtain or create these principals as needed. Then create a JAAS file as follows:
+            <pre>
+                KafkaClient {
+                    com.sun.security.auth.module.Krb5LoginModule required
+                    useKeyTab=true
+                    storeKey=true
+                    serviceName="kafka"
+                    keyTab="/etc/security/keytabs/kafka1.keytab"
+                    principal="kafkaproducer/hostname@DOMAIN.COM";
+                };
+            </pre>
+        </li>
+    </ol></li>
+    <li><h4>Configuring Kafka Brokers</h4>
+    <ol>
+        <li>Pass the name of the jaas file you created in <a href="#jaas_config_file">Creating
JAAS Config File"</a> as a JVM parameter to the kafka broker: <pre>-Djava.security.auth.login.config=/etc/kafka/kafka_jaas.conf</pre></li>
+        <li>Make sure the keytabs configured in the kafka_jaas.conf are readable by
the linux user who is starting kafka broker.</li>
+        <li>Configure a SASL port in server.properties, by adding the following 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 SASL port (or if you are very paranoid and want the Kafka
brokers to authenticate each other using SASL) then make sure you set same SASL protocol for
inter-broker communication:
+        <pre>security.inter.broker.protocol=SASL_PLAINTEXT</pre></li>
+
+    </ol>
+    </li>
+    <li><h4>Configuring Kafka Clients</h4>
+        SASL authentication is only supported for new kafka producer and consumer, the older
API is not supported.>br>
+        To configure SASL authentication on the clients:
+        <ol>
+            <li>pass the name of the jaas file you created in <a href="#jaas_client">Creating
Client Side JAAS Config"</a> as a JVM parameter to the client JVM:
+        <pre>-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 linux user who is starting kafka client.</li>
+            <li>Configure the following property in producer.properties or consumer.properties:
+                <pre>security.protocol=SASL_PLAINTEXT</pre></li>
+        </ol></li>
+</ol>
+
+<h3><a id="security_authz">7.4 Authorization and ACLs</a></h3>
\ No newline at end of file


Mime
View raw message