MINOR: Add section on listener configuration (including kraft) to security docs (#12682)

This patch adds a section in security.html about listener configuration. This includes the basics of how to define the security mapping of each listener as well as the configurations to control inter-cluster traffic.

Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>, Luke Chen <showuon@gmail.com>
This commit is contained in:
Jason Gustafson 2022-09-26 19:33:32 -07:00 committed by GitHub
parent 1c0f8f90e2
commit 017868d8ac
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
1 changed files with 144 additions and 17 deletions

View File

@ -36,7 +36,136 @@
The guides below explain how to configure and use the security features in both clients and brokers.
<h3 class="anchor-heading"><a id="security_ssl" class="anchor-link"></a><a href="#security_ssl">7.2 Encryption and Authentication using SSL</a></h3>
<h3 class="anchor-heading"><a id="listener_configuration" class="anchor-link"></a><a href="#listener_configuration">7.2 Listener Configuration</a></h3>
<p>In order to secure a Kafka cluster, it is necessary to secure the channels that are used to
communicate with the servers. Each server must define the set of listeners that are used to
receive requests from clients as well as other servers. Each listener may be configured
to authenticate clients using various mechanisms and to ensure traffic between the
server and the client is encrypted. This section provides a primer for the configuration
of listeners.</p>
<p>Kafka servers support listening for connections on multiple ports. This is configured through
the <code>listeners</code> property in the server configuration, which accepts a comma-separated
list of the listeners to enable. At least one listener must be defined on each server. The format
of each listener defined in <code>listeners</code> is given below:</p>
<pre class="line-numbers"><code class="language-text">{LISTENER_NAME}://{hostname}:{port}</code></pre>
<p>The <code>LISTENER_NAME</code> is usually a descriptive name which defines the purpose of
the listener. For example, many configurations use a separate listener for client traffic,
so they might refer to the corresponding listener as <code>CLIENT</code> in the configuration:</p
<pre class="line-numbers"><code class="language-text">listeners=CLIENT://localhost:9092</code></pre>
<p>The security protocol of each listener is defined in a separate configuration:
<code>listener.security.protocol.map</code>. The value is a comma-separated list
of each listener mapped to its security protocol. For example, the follow value
configuration specifies that the <code>CLIENT</code> listener will use SSL while the
<code>BROKER</code> listener will use plaintext.</p>
<pre class="line-numbers"><code class="language-text">listener.security.protocol.map=CLIENT:SSL,BROKER:PLAINTEXT</code></pre>
<p>Possible options for the security protocol are given below:</p>
<ol>
<li>PLAINTEXT</li>
<li>SSL</li>
<li>SASL_PLAINTEXT</li>
<li>SASL_SSL</li>
</ol>
<p>The plaintext protocol provides no security and does not require any additional configuration.
In the following sections, this document covers how to configure the remaining protocols.</p>
<p>If each required listener uses a separate security protocol, it is also possible to use the
security protocol name as the listener name in <code>listeners</code>. Using the example above,
we could skip the definition of the <code>CLIENT</code> and <code>BROKER</code> listeners
using the following definition:</p>
<pre class="line-numbers"><code class="language-text">listeners=SSL://localhost:9092,PLAINTEXT://localhost:9093</code></pre>
<p>However, we recommend users to provide explicit names for the listeners since it
makes the intended usage of each listener clearer.</p>
<p>Among the listeners in this list, it is possible to declare the listener to be used for
inter-broker communication by setting the <code>inter.broker.listener.name</code> configuration
to the name of the listener. The primary purpose of the inter-broker listener is
partition replication. If not defined, then the inter-broker listener is determined
by the security protocol defined by <code>security.inter.broker.protocol</code>, which
defaults to <code>PLAINTEXT</code>.</p>
<p>For legacy clusters which rely on Zookeeper to store cluster metadata, it is possible to
declare a separate listener to be used for metadata propagation from the active controller
to the brokers. This is defined by <code>control.plane.listener.name</code>. The active controller
will use this listener when it needs to push metadata updates to the brokers in the cluster.
The benefit of using a control plane listener is that it uses a separate processing thread,
which makes it less likely for application traffic to impede timely propagation of metadata changes
(such as partition leader and ISR updates). Note that the default value is null, which
means that the controller will use the same listener defined by <code>inter.broker.listener</code></p>
<p>In a KRaft cluster, a broker is any server which has the <code>broker</code> role enabled
in <code>process.roles</code> and a controller is any server which has the <code>controller</code>
role enabled. Listener configuration depends on the role. The listener defined by
<code>inter.broker.listener.name</code> is used exclusively for requests between brokers.
Controllers, on the other hand, must use separate listener which is defined by the
<code>controller.listener.names</code> configuration. This cannot be set to the same
value as the inter-broker listener.</p>
<p>Controllers receive requests both from other controllers and from brokers. For
this reason, even if a server does not have the <code>controller</code> role enabled
(i.e. it is just a broker), it must still define the controller listener along with
any security properties that are needed to configure it. For example, we might
use the following configuration on a standalone broker:</p>
<pre class="line-numbers"><code class="language-text">process.roles=broker
listeners=BROKER://localhost:9092
inter.broker.listener.name=BROKER
controller.quorum.voters=0@localhost:9093
controller.listener.names=CONTROLLER
listener.security.protocol.map=BROKER:SASL_SSL,CONTROLLER:SASL_SSL</code></pre>
<p>The controller listener is still configured in this example to use the <code>SASL_SSL</code>
security protocol, but it is not included in <code>listeners</code> since the broker
does not expose the controller listener itself. The port that will be used in this case
comes from the <code>controller.quorum.voters</code> configuration, which defines
the complete list of controllers.</p>
<p>For KRaft servers which have both the broker and controller role enabled, the configuration
is similar. The only difference is that the controller listener must be included in
<code>listeners</code>:</p>
<pre class="line-numbers"><code class="language-text">process.roles=broker,controller
listeners=BROKER://localhost:9092,CONTROLLER://localhost:9093
inter.broker.listener.name=BROKER
controller.quorum.voters=0@localhost:9093
controller.listener.names=CONTROLLER
listener.security.protocol.map=BROKER:SASL_SSL,CONTROLLER:SASL_SSL</code></pre>
<p>It is a requirement for the port defined in <code>controller.quorum.voters</code> to
exactly match one of the exposed controller listeners. For example, here the
<code>CONTROLLER</code> listener is bound to port 9093. The connection string
defined by <code>controller.quorum.voters</code> must then also use port 9093,
as it does here.</p>
<p>The controller will accept requests on all listeners defined by <code>controller.listener.names</code>.
Typically there would be just one controller listener, but it is possible to have more.
For example, this provides a way to change the active listener from one port or security
protocol to another through a roll of the cluster (one roll to expose the new listener,
and one roll to remove the old listener). When multiple controller listeners are defined,
the first one in the list will be used for outbound requests.</p>
<p>It is conventional in Kafka to use a separate listener for clients. This allows the
inter-cluster listeners to be isolated at the network level. In the case of the controller
listener in KRaft, the listener should be isolated since clients do not work with it
anyway. Clients are expected to connect to any other listener configured on a broker.
Any requests that are bound for the controller will be forwarded as described
<a href="#kraft_principal_forwarding">below</a></p>
<p>In the following <a href="#security_ssl">section</a>, this document covers how to enable SSL
on a listener for encryption as well as authentication. The subsequent <a href="#security_sasl">section</a> will then
cover additional authentication mechanisms using SASL.</p>
<h3 class="anchor-heading"><a id="security_ssl" class="anchor-link"></a><a href="#security_ssl">7.3 Encryption and Authentication using SSL</a></h3>
Apache Kafka allows clients to use SSL for encryption of traffic as well as authentication. By default, SSL is disabled but can be turned on if needed.
The following paragraphs explain in detail how to set up your own PKI infrastructure, use it to create certificates and configure Kafka to use these.
@ -314,10 +443,8 @@ keyUsage = digitalSignature, keyEncipherment</code></pre>
</li>
</ol>
</li>
<li><h4 class="anchor-heading"><a id="security_configbroker" class="anchor-link"></a><a href="#security_configbroker">Configuring Kafka Brokers</a></h4>
Kafka Brokers support listening for connections on multiple ports.
We need to configure the following property in server.properties, which must have one or more comma-separated values:
<pre><code class="language-text">listeners</code></pre>
If SSL is not enabled for inter-broker communication (see below for how to enable it), both PLAINTEXT and SSL ports will be necessary.
<pre class="line-numbers"><code class="language-text">listeners=PLAINTEXT://host.name:port,SSL://host.name:port</code></pre>
@ -397,7 +524,7 @@ ssl.key.password=test1234</code></pre>
&gt; kafka-console-consumer.sh --bootstrap-server localhost:9093 --topic test --consumer.config client-ssl.properties</code></pre>
</li>
</ol>
<h3 class="anchor-heading"><a id="security_sasl" class="anchor-link"></a><a href="#security_sasl">7.3 Authentication using SASL</a></h3>
<h3 class="anchor-heading"><a id="security_sasl" class="anchor-link"></a><a href="#security_sasl">7.4 Authentication using SASL</a></h3>
<ol>
<li><h4 class="anchor-heading"><a id="security_sasl_jaasconfig" class="anchor-link"></a><a href="#security_sasl_jaasconfig">JAAS configuration</a></h4>
@ -1135,7 +1262,7 @@ sasl.mechanism.inter.broker.protocol=GSSAPI (or one of the other enabled mechani
</li>
</ol>
<h3 class="anchor-heading"><a id="security_authz" class="anchor-link"></a><a href="#security_authz">7.4 Authorization and ACLs</a></h3>
<h3 class="anchor-heading"><a id="security_authz" class="anchor-link"></a><a href="#security_authz">7.5 Authorization and ACLs</a></h3>
Kafka ships with a pluggable authorization framework, which is configured with the <tt>authorizer.class.name</tt> property in the server confgiuration.
Configured implementations must extend <code>org.apache.kafka.server.authorizer.Authorizer</code>.
Kafka provides default implementations which store ACLs in the cluster metadata (either Zookeeper or the KRaft metadata log).
@ -1959,7 +2086,7 @@ bin/kafka-acls.sh --bootstrap-server localhost:9092 --command-config /tmp/adminc
</tbody>
</table>
<h3 class="anchor-heading"><a id="security_rolling_upgrade" class="anchor-link"></a><a href="#security_rolling_upgrade">7.5 Incorporating Security Features in a Running Cluster</a></h3>
<h3 class="anchor-heading"><a id="security_rolling_upgrade" class="anchor-link"></a><a href="#security_rolling_upgrade">7.6 Incorporating Security Features in a Running Cluster</a></h3>
You can secure a running cluster via one or more of the supported protocols discussed previously. This is done in phases:
<p></p>
<ul>
@ -1969,7 +2096,7 @@ bin/kafka-acls.sh --bootstrap-server localhost:9092 --command-config /tmp/adminc
<li>A final incremental bounce to close the PLAINTEXT port.</li>
</ul>
<p></p>
The specific steps for configuring SSL and SASL are described in sections <a href="#security_ssl">7.2</a> and <a href="#security_sasl">7.3</a>.
The specific steps for configuring SSL and SASL are described in sections <a href="#security_ssl">7.3</a> and <a href="#security_sasl">7.4</a>.
Follow these steps to enable security for your desired protocol(s).
<p></p>
The security implementation lets you configure different protocols for both broker-client and broker-broker communication.
@ -2017,10 +2144,10 @@ security.inter.broker.protocol=SSL</code></pre>
<pre class="line-numbers"><code class="language-text">listeners=SSL://broker1:9092,SASL_SSL://broker1:9093
security.inter.broker.protocol=SSL</code></pre>
ZooKeeper can be secured independently of the Kafka cluster. The steps for doing this are covered in section <a href="#zk_authz_migration">7.6.2</a>.
ZooKeeper can be secured independently of the Kafka cluster. The steps for doing this are covered in section <a href="#zk_authz_migration">7.7.2</a>.
<h3 class="anchor-heading"><a id="zk_authz" class="anchor-link"></a><a href="#zk_authz">7.6 ZooKeeper Authentication</a></h3>
<h3 class="anchor-heading"><a id="zk_authz" class="anchor-link"></a><a href="#zk_authz">7.7 ZooKeeper Authentication</a></h3>
ZooKeeper supports mutual TLS (mTLS) authentication beginning with the 3.5.x versions.
Kafka supports authenticating to ZooKeeper with SASL and mTLS -- either individually or both together --
beginning with version 2.5. See
@ -2052,8 +2179,8 @@ security.inter.broker.protocol=SSL</code></pre>
Use the <tt>-zk-tls-config-file &lt;file&gt;</tt> option (note the single-dash rather than double-dash)
to set TLS configs for the <tt>zookeeper-shell.sh</tt> CLI tool.
</p>
<h4 class="anchor-heading"><a id="zk_authz_new" class="anchor-link"></a><a href="#zk_authz_new">7.6.1 New clusters</a></h4>
<h5 class="anchor-heading"><a id="zk_authz_new_sasl" class="anchor-link"></a><a href="#zk_authz_new_sasl">7.6.1.1 ZooKeeper SASL Authentication</a></h5>
<h4 class="anchor-heading"><a id="zk_authz_new" class="anchor-link"></a><a href="#zk_authz_new">7.7.1 New clusters</a></h4>
<h5 class="anchor-heading"><a id="zk_authz_new_sasl" class="anchor-link"></a><a href="#zk_authz_new_sasl">7.7.1.1 ZooKeeper SASL Authentication</a></h5>
To enable ZooKeeper SASL authentication on brokers, there are two necessary steps:
<ol>
<li> Create a JAAS login file and set the appropriate system property to point to it as described above</li>
@ -2062,7 +2189,7 @@ security.inter.broker.protocol=SSL</code></pre>
The metadata stored in ZooKeeper for the Kafka cluster is world-readable, but can only be modified by the brokers. The rationale behind this decision is that the data stored in ZooKeeper is not sensitive, but inappropriate manipulation of that data can cause cluster disruption. We also recommend limiting the access to ZooKeeper via network segmentation (only brokers and some admin tools need access to ZooKeeper).
<h5 class="anchor-heading"><a id="zk_authz_new_mtls" class="anchor-link"></a><a href="#zk_authz_new_mtls">7.6.1.2 ZooKeeper Mutual TLS Authentication</a></h5>
<h5 class="anchor-heading"><a id="zk_authz_new_mtls" class="anchor-link"></a><a href="#zk_authz_new_mtls">7.7.1.2 ZooKeeper Mutual TLS Authentication</a></h5>
ZooKeeper mTLS authentication can be enabled with or without SASL authentication. As mentioned above,
when using mTLS alone, every broker and any CLI tools (such as the <a href="#zk_authz_migration">ZooKeeper Security Migration Tool</a>)
must generally identify itself with the same Distinguished Name (DN) because it is the DN that is ACL'ed, which means
@ -2109,7 +2236,7 @@ zookeeper.set.acl=true</code></pre>
to a value different from the keystore password itself.
Be sure to set the key password to be the same as the keystore password.
<h4 class="anchor-heading"><a id="zk_authz_migration" class="anchor-link"></a><a href="#zk_authz_migration">7.6.2 Migrating clusters</a></h4>
<h4 class="anchor-heading"><a id="zk_authz_migration" class="anchor-link"></a><a href="#zk_authz_migration">7.7.2 Migrating clusters</a></h4>
If you are running a version of Kafka that does not support security or simply with security disabled, and you want to make the cluster secure, then you need to execute the following steps to enable ZooKeeper authentication with minimal disruption to your operations:
<ol>
<li>Enable SASL and/or mTLS authentication on ZooKeeper. If enabling mTLS, you would now have both a non-TLS port and a TLS port, like this:
@ -2139,17 +2266,17 @@ ssl.trustStore.password=zk-ts-passwd</code></pre>
<pre class="line-numbers"><code class="language-bash">&gt; bin/zookeeper-security-migration.sh --zookeeper.acl=secure --zookeeper.connect=localhost:2181</code></pre>
<p>Run this to see the full list of parameters:</p>
<pre class="line-numbers"><code class="language-bash">&gt; bin/zookeeper-security-migration.sh --help</code></pre>
<h4 class="anchor-heading"><a id="zk_authz_ensemble" class="anchor-link"></a><a href="#zk_authz_ensemble">7.6.3 Migrating the ZooKeeper ensemble</a></h4>
<h4 class="anchor-heading"><a id="zk_authz_ensemble" class="anchor-link"></a><a href="#zk_authz_ensemble">7.7.3 Migrating the ZooKeeper ensemble</a></h4>
It is also necessary to enable SASL and/or mTLS authentication on the ZooKeeper ensemble. To do it, we need to perform a rolling restart of the server and set a few properties. See above for mTLS information. Please refer to the ZooKeeper documentation for more detail:
<ol>
<li><a href="https://zookeeper.apache.org/doc/r3.5.7/zookeeperProgrammers.html#sc_ZooKeeperAccessControl">Apache ZooKeeper documentation</a></li>
<li><a href="https://cwiki.apache.org/confluence/display/ZOOKEEPER/Zookeeper+and+SASL">Apache ZooKeeper wiki</a></li>
</ol>
<h4 class="anchor-heading"><a id="zk_authz_quorum" class="anchor-link"></a><a href="#zk_authz_quorum">7.6.4 ZooKeeper Quorum Mutual TLS Authentication</a></h4>
<h4 class="anchor-heading"><a id="zk_authz_quorum" class="anchor-link"></a><a href="#zk_authz_quorum">7.7.4 ZooKeeper Quorum Mutual TLS Authentication</a></h4>
It is possible to enable mTLS authentication between the ZooKeeper servers themselves.
Please refer to the <a href="https://zookeeper.apache.org/doc/r3.5.7/zookeeperAdmin.html#Quorum+TLS">ZooKeeper documentation</a> for more detail.
<h3 class="anchor-heading"><a id="zk_encryption" class="anchor-link"></a><a href="#zk_encryption">7.7 ZooKeeper Encryption</a></h3>
<h3 class="anchor-heading"><a id="zk_encryption" class="anchor-link"></a><a href="#zk_encryption">7.8 ZooKeeper Encryption</a></h3>
ZooKeeper connections that use mutual TLS are encrypted.
Beginning with ZooKeeper version 3.5.7 (the version shipped with Kafka version 2.5) ZooKeeper supports a sever-side config
<tt>ssl.clientAuth</tt> (case-insensitively: <tt>want</tt>/<tt>need</tt>/<tt>none</tt> are the valid options, the default is <tt>need</tt>),