KAFKA-17048; Update docs for KIP-853 (#17076)

Change the configurations under config/kraft to use controller.quorum.bootstrap.servers instead of controller.quorum.voters. Add comments explaining how to use the older static quorum configuration where appropriate.

In docs/ops.html, remove the reference to "tentative timelines for ZooKeeper removal" and "Tiered storage is considered as an early access feature" since they are no longer up-to-date. Add KIP-853 information.

In docs/quickstart.html, move the ZK instructions to be after the KRaft instructions. Update the KRaft instructions to use KIP-853.

In docs/security.html, add an explanation of --bootstrap-controller and document controller.quorum.bootstrap.servers instead of controller.quorum.voters.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Alyssa Huang <ahuang@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
This commit is contained in:
José Armando García Sancio 2024-09-18 14:23:58 -04:00 committed by GitHub
parent 77e952687b
commit aee44efd40
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
6 changed files with 109 additions and 49 deletions

View File

@ -26,8 +26,8 @@ process.roles=broker
# The node id associated with this instance's roles
node.id=2
# The connect string for the controller quorum
controller.quorum.voters=1@localhost:9093
# Information about the KRaft controller quorum.
controller.quorum.bootstrap.servers=localhost:9093
############################# Socket Server Settings #############################

View File

@ -26,13 +26,15 @@ process.roles=controller
# The node id associated with this instance's roles
node.id=1
# The connect string for the controller quorum
controller.quorum.voters=1@localhost:9093
# Information about the KRaft controller quorum.
# Uncomment controller.quorum.voters to use a static controller quorum.
#controller.quorum.voters=1@localhost:9093
controller.quorum.bootstrap.servers=localhost:9093
############################# Socket Server Settings #############################
# The address the socket server listens on.
# Note that only the controller listeners are allowed here when `process.roles=controller`, and this listener should be consistent with `controller.quorum.voters` value.
# Note that only the controller listeners are allowed here when `process.roles=controller`
# FORMAT:
# listeners = listener_name://host_name:port
# EXAMPLE:

View File

@ -26,7 +26,9 @@ process.roles=broker,controller
# The node id associated with this instance's roles
node.id=1
# The connect string for the controller quorum
# Information about the KRaft controller quorum.
# Uncomment controller.quorum.voters to use a static controller quorum.
#controller.quorum.voters=1@localhost:9093
controller.quorum.voters=1@localhost:9093
############################# Socket Server Settings #############################

View File

@ -3736,9 +3736,6 @@ customized state stores; for built-in state stores, currently we have:
<h5 class="anchor-heading"><a id="zk_depr_3xsupport" class="anchor-link"></a><a href="#zk_depr_3xsupport">3.x and ZooKeeper Support</a></h5>
<p>The final 3.x minor release, that supports ZooKeeper mode, will receive critical bug fixes and security fixes for 12 months after its release.</p>
<h5 class="anchor-heading"><a id="zk_depr_timeline" class="anchor-link"></a><a href="#zk_depr_timeline">ZooKeeper and KRaft timeline</a></h5>
<p>For details and updates on tentative timelines for ZooKeeper removal and planned KRaft feature releases, refer to <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-833%3A+Mark+KRaft+as+Production+Ready">KIP-833</a>.</p>
<h4 class="anchor-heading"><a id="zkops" class="anchor-link"></a><a href="#zkops">Operationalizing ZooKeeper</a></h4>
Operationally, we do the following for a healthy ZooKeeper installation:
<ul>
@ -3776,25 +3773,77 @@ customized state stores; for built-in state stores, currently we have:
<p>A Kafka admin will typically select 3 or 5 servers for this role, depending on factors like cost and the number of concurrent failures your system should withstand without availability impact. A majority of the controllers must be alive in order to maintain availability. With 3 controllers, the cluster can tolerate 1 controller failure; with 5 controllers, the cluster can tolerate 2 controller failures.</p>
<p>All of the servers in a Kafka cluster discover the quorum voters using the <code>controller.quorum.voters</code> property. This identifies the quorum controller servers that should be used. All the controllers must be enumerated. Each controller is identified with their <code>id</code>, <code>host</code> and <code>port</code> information. For example:</p>
<p>All of the servers in a Kafka cluster discover the active controller using the <code>controller.quorum.bootstrap.servers</code> property. All the controllers should be enumerated in this property. Each controller is identified with their <code>host</code> and <code>port</code> information. For example:</p>
<pre><code class="language-bash">controller.quorum.voters=id1@host1:port1,id2@host2:port2,id3@host3:port3</code></pre>
<pre><code class="language-bash">controller.quorum.bootstrap.servers=host1:port1,host2:port2,host3:port3</code></pre>
<p>If a Kafka cluster has 3 controllers named controller1, controller2 and controller3, then controller1 may have the following configuration:</p>
<pre><code class="language-bash">process.roles=controller
node.id=1
listeners=CONTROLLER://controller1.example.com:9093
controller.quorum.voters=1@controller1.example.com:9093,2@controller2.example.com:9093,3@controller3.example.com:9093</code></pre>
controller.quorum.bootstrap.servers=controller1.example.com:9093,controller2.example.com:9093,controller3.example.com:9093
controller.listener.names=CONTROLLER</code></pre>
<p>Every broker and controller must set the <code>controller.quorum.voters</code> property. The node ID supplied in the <code>controller.quorum.voters</code> property must match the corresponding id on the controller servers. For example, on controller1, node.id must be set to 1, and so forth. Each node ID must be unique across all the servers in a particular cluster. No two servers can have the same node ID regardless of their <code>process.roles</code> values.
<p>Every broker and controller must set the <code>controller.quorum.bootstrap.servers</code> property.
<h4 class="anchor-heading"><a id="kraft_storage" class="anchor-link"></a><a href="#kraft_storage">Storage Tool</a></h4>
<h4 class="anchor-heading"><a id="kraft_storage" class="anchor-link"></a><a href="#kraft_storage">Provisioning Nodes</a></h4>
<p></p>
The <code>kafka-storage.sh random-uuid</code> command can be used to generate a cluster ID for your new cluster. This cluster ID must be used when formatting each server in the cluster with the <code>kafka-storage.sh format</code> command.
<p>This is different from how Kafka has operated in the past. Previously, Kafka would format blank storage directories automatically, and also generate a new cluster ID automatically. One reason for the change is that auto-formatting can sometimes obscure an error condition. This is particularly important for the metadata log maintained by the controller and broker servers. If a majority of the controllers were able to start with an empty log directory, a leader might be able to be elected with missing committed data.</p>
<h5 class="anchor-heading"><a id="kraft_storage_standalone" class="anchor-link"></a><a href="#kraft_storage_standalone">Bootstrap a Standalone Controller</a></h5>
The recommended method for creating a new KRaft controller cluster is to bootstrap it with one voter and dynamically <a href="#kraft_reconfig_add">add the rest of the controllers</a>. Bootstrapping the first controller can be done with the following CLI command:
<pre><code class="language-bash">$ bin/kafka-storage format --cluster-id <cluster-id> --standalone --config controller.properties</code></pre>
This command will 1) create a meta.properties file in metadata.log.dir with a randomly generated directory.id, 2) create a snapshot at 00000000000000000000-0000000000.checkpoint with the necessary control records (KRaftVersionRecord and VotersRecord) to make this Kafka node the only voter for the quorum.
<h5 class="anchor-heading"><a id="kraft_storage_voters" class="anchor-link"></a><a href="#kraft_storage_voters">Bootstrap with Multiple Controllers</a></h5>
The KRaft cluster metadata partition can also be bootstrapped with more than one voter. This can be done by using the --initial-controllers flag:
<pre><code class="language-bash">cluster-id=$(kafka-storage random-uuid)
controller-0-uuid=$(kafka-storage random-uuid)
controller-1-uuid=$(kafka-storage random-uuid)
controller-2-uuid=$(kafka-storage random-uuid)
# In each controller execute
kafka-storage format --cluster-id ${cluster-id} \
--initial-controllers "0@controller-0:1234:${controller-0-uuid},1@controller-1:1234:${controller-1-uuid},2@controller-2:1234:${controller-2-uuid}" \
--config controller.properties</code></pre>
This command is similar to the standalone version but the snapshot at 00000000000000000000-0000000000.checkpoint will instead contain a VotersRecord that includes information for all of the controllers specified in --initial-controllers. It is important that the value of this flag is the same in all of the controllers with the same cluster id.
In the replica description 0@controller-0:1234:3Db5QLSqSZieL3rJBUUegA, 0 is the replica id, 3Db5QLSqSZieL3rJBUUegA is the replica directory id, controller-0 is the replica's host and 1234 is the replica's port.
<h5 class="anchor-heading"><a id="kraft_storage_observers" class="anchor-link"></a><a href="#kraft_storage_observers">Formatting Brokers and New Controllers</a></h5>
When provisioning new broker and controller nodes that we want to add to an existing Kafka cluster, use the <code>kafka-storage.sh format</code> command without the --standalone or --initial-controllers flags.
<pre><code class="language-bash">$ bin/kafka-storage format --cluster-id <cluster-id> --config server.properties</code></pre>
<h4 class="anchor-heading"><a id="kraft_reconfig" class="anchor-link"></a><a href="#kraft_reconfig">Controller membership changes</a></h4>
<h5 class="anchor-heading"><a id="kraft_reconfig_add" class="anchor-link"></a><a href="#kraft_reconfig_add">Add New Controller</a></h5>
If the KRaft Controller cluster already exists, the cluster can be expanded by first provisioning a new controller using the <a href="#kraft_storage_observers">kafka-storage tool</a> and starting the controller.
After starting the controller, the replication to the new controller can be monitored using the <code>kafka-metadata-quorum describe --replication</code> command. Once the new controller has caught up to the active controller, it can be added to the cluster using the <code>kafka-metadata-quorum add-controller</code> command.
When using broker endpoints use the --bootstrap-server flag:
<pre><code class="language-bash">$ bin/kafka-metadata-quorum --command-config controller.properties --bootstrap-server localhost:9092 add-controller</code></pre>
When using controller endpoints use the --bootstrap-controller flag:
<pre><code class="language-bash">$ bin/kafka-metadata-quorum --command-config controller.properties --bootstrap-controller localhost:9092 add-controller</code></pre>
<h5 class="anchor-heading"><a id="kraft_reconfig_remove" class="anchor-link"></a><a href="#kraft_reconfig_remove">Remove Controller</a></h5>
If the KRaft Controller cluster already exists, the cluster can be shrunk using the <code>kafka-metadata-quorum remove-controller</code> command. Until KIP-996: Pre-vote has been implemented and released, it is recommended to shutdown the controller that will be removed before running the remove-controller command.
When using broker endpoints use the --bootstrap-server flag:
<pre><code class="language-bash">$ bin/kafka-metadata-quorum --bootstrap-server localhost:9092 remove-controller --controller-id <id> --controller-directory-id <directory-id></code></pre>
When using controller endpoints use the --bootstrap-controller flag:
<pre><code class="language-bash">$ bin/kafka-metadata-quorum --bootstrap-controller localhost:9092 remove-controller --controller-id <id> --controller-directory-id <directory-id></code></pre>
<h4 class="anchor-heading"><a id="kraft_debug" class="anchor-link"></a><a href="#kraft_debug">Debugging</a></h4>
<h5 class="anchor-heading"><a id="kraft_metadata_tool" class="anchor-link"></a><a href="#kraft_metadata_tool">Metadata Quorum Tool</a></h5>
@ -3808,8 +3857,12 @@ LeaderEpoch: 2
HighWatermark: 10
MaxFollowerLag: 0
MaxFollowerLagTimeMs: -1
CurrentVoters: [3000,3001,3002]
CurrentObservers: [0,1,2]</code></pre>
CurrentVoters: [{"id": 3000, "directoryId": "ILZ5MPTeRWakmJu99uBJCA", "endpoints": ["CONTROLLER://localhost:9093"]},
{"id": 3001, "directoryId": "b-DwmhtOheTqZzPoh52kfA", "endpoints": ["CONTROLLER://localhost:9094"]},
{"id": 3002, "directoryId": "g42deArWBTRM5A1yuVpMCg", "endpoints": ["CONTROLLER://localhost:9095"]}]
CurrentObservers: [{"id": 0, "directoryId": "3Db5QLSqSZieL3rJBUUegA"},
{"id": 1, "directoryId": "UegA3Db5QLSqSZieL3rJBU"},
{"id": 2, "directoryId": "L3rJBUUegA3Db5QLSqSZie"}]</code></pre>
<h5 class="anchor-heading"><a id="kraft_dump_log" class="anchor-link"></a><a href="#kraft_dump_log">Dump Log Tool</a></h5>
@ -3940,7 +3993,7 @@ foo
<pre><code class="language-text"># Sample KRaft cluster controller.properties listening on 9093
process.roles=controller
node.id=3000
controller.quorum.voters=3000@localhost:9093
controller.quorum.bootstrap.servers=localhost:9093
controller.listener.names=CONTROLLER
listeners=CONTROLLER://:9093
@ -3955,6 +4008,8 @@ inter.broker.listener.name=PLAINTEXT
# Other configs ...</code></pre>
<p>The new standalone controller in the example configuration above should be formatted using the <code>kafka-storage format --standalone</code>command.</p>
<p><em>Note: The KRaft cluster <code>node.id</code> values must be different from any existing ZK broker <code>broker.id</code>.
In KRaft-mode, the brokers and controllers share the same Node ID namespace.</em></p>
@ -3967,7 +4022,7 @@ inter.broker.listener.name=PLAINTEXT
<ul>
<li><a href="#brokerconfigs_broker.id">broker.id</a>: Ensure <code>broker.id</code> is set to a non-negative integer even if <code>broker.id.generation.enable</code> is enabled (default is enabled). Additionally, ensure <code>broker.id</code> does not exceed <code>reserved.broker.max.id</code> to avoid failure.</li>
<li><a href="#brokerconfigs_controller.quorum.voters">controller.quorum.voters</a></li>
<li><a href="#brokerconfigs_controller.quorum.bootstrap.servers">controller.quorum.bootstrap.servers</a></li>
<li><a href="#brokerconfigs_controller.listener.names">controller.listener.names</a></li>
<li>The controller.listener.name should also be added to <a href="#brokerconfigs_listener.security.protocol.map">listener.security.property.map</a></li>
<li><a href="#brokerconfigs_zookeeper.metadata.migration.enable">zookeeper.metadata.migration.enable</a></li>
@ -3991,7 +4046,7 @@ zookeeper.metadata.migration.enable=true
zookeeper.connect=localhost:2181
# KRaft controller quorum configuration
controller.quorum.voters=3000@localhost:9093
controller.quorum.bootstrap.servers=localhost:9093
controller.listener.names=CONTROLLER</code></pre>
<p>
@ -4039,7 +4094,7 @@ listener.security.protocol.map=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT
# zookeeper.connect=localhost:2181
# Keep the KRaft controller quorum configuration
controller.quorum.voters=3000@localhost:9093
controller.quorum.bootstrap.servers=localhost:9093
controller.listener.names=CONTROLLER</code></pre>
<p>
@ -4060,7 +4115,7 @@ controller.listener.names=CONTROLLER</code></pre>
<pre><code class="language-text"># Sample KRaft cluster controller.properties listening on 9093
process.roles=controller
node.id=3000
controller.quorum.voters=3000@localhost:9093
controller.quorum.bootstrap.servers=localhost:9093
controller.listener.names=CONTROLLER
listeners=CONTROLLER://:9093
@ -4135,7 +4190,7 @@ listeners=CONTROLLER://:9093
</li>
<li>
On each broker, remove the <code>zookeeper.metadata.migration.enable</code>,
<code>controller.listener.names</code>, and <code>controller.quorum.voters</code>
<code>controller.listener.names</code>, and <code>controller.quorum.bootstrap.servers</code>
configurations, and replace <code>node.id</code> with <code>broker.id</code>.
Then perform a rolling restart of all brokers.
</li>
@ -4172,7 +4227,7 @@ listeners=CONTROLLER://:9093
</li>
<li>
On each broker, remove the <code>zookeeper.metadata.migration.enable</code>,
<code>controller.listener.names</code>, and <code>controller.quorum.voters</code>
<code>controller.listener.names</code>, and <code>controller.quorum.bootstrap.servers</code>
configurations.
Then perform a second rolling restart of all brokers.
</li>

View File

@ -50,17 +50,35 @@ $ cd kafka_{{scalaVersion}}-{{fullDotVersion}}</code></pre>
<p>Kafka can be run using KRaft mode using local scripts and downloaded files or the docker image. Follow one of the sections below but not both to start the kafka server.</p>
<h5>Using downloaded files</h5>
<h6>Using downloaded files</h6>
<p>Generate a Cluster UUID</p>
<pre><code class="language-bash">$ KAFKA_CLUSTER_ID="$(bin/kafka-storage.sh random-uuid)"</code></pre>
<p>Format Log Directories</p>
<pre><code class="language-bash">$ bin/kafka-storage.sh format -t $KAFKA_CLUSTER_ID -c config/kraft/server.properties</code></pre>
<pre><code class="language-bash">$ bin/kafka-storage.sh format --standalone -t $KAFKA_CLUSTER_ID -c config/kraft/server.properties</code></pre>
<p>Start the Kafka Server</p>
<pre><code class="language-bash">$ bin/kafka-server-start.sh config/kraft/server.properties</code></pre>
<p>Once the Kafka server has successfully launched, you will have a basic Kafka environment running and ready to use.</p>
<h6>Using JVM Based Apache Kafka Docker Image</h6>
<p> Get the Docker image:</p>
<pre><code class="language-bash">$ docker pull apache/kafka:{{fullDotVersion}}</code></pre>
<p> Start the Kafka Docker container: </p>
<pre><code class="language-bash">$ docker run -p 9092:9092 apache/kafka:{{fullDotVersion}}</code></pre>
<h6>Using GraalVM Based Native Apache Kafka Docker Image</h6>
<p>Get the Docker image:</p>
<pre><code class="language-bash">$ docker pull apache/kafka-native:{{fullDotVersion}}</code></pre>
<p>Start the Kafka Docker container:</p>
<pre><code class="language-bash">$ docker run -p 9092:9092 apache/kafka-native:{{fullDotVersion}}</code></pre>
<h5>Kafka with ZooKeeper</h5>
<p>Run the following commands in order to start all services in the correct order:</p>
@ -73,23 +91,6 @@ $ bin/kafka-server-start.sh config/server.properties</code></pre>
<p>Once all services have successfully launched, you will have a basic Kafka environment running and ready to use.</p>
<h5>Using JVM Based Apache Kafka Docker Image</h5>
<p> Get the Docker image:</p>
<pre><code class="language-bash">$ docker pull apache/kafka:{{fullDotVersion}}</code></pre>
<p> Start the Kafka Docker container: </p>
<pre><code class="language-bash">$ docker run -p 9092:9092 apache/kafka:{{fullDotVersion}}</code></pre>
<h5>Using GraalVM Based Native Apache Kafka Docker Image</h5>
<p>Get the Docker image:</p>
<pre><code class="language-bash">$ docker pull apache/kafka-native:{{fullDotVersion}}</code></pre>
<p>Start the Kafka Docker container:</p>
<pre><code class="language-bash">$ docker run -p 9092:9092 apache/kafka-native:{{fullDotVersion}}</code></pre>
<p>Once the Kafka server has successfully launched, you will have a basic Kafka environment running and ready to use.</p>
</div>
<div class="quickstart-step">

View File

@ -120,7 +120,7 @@
<pre><code class="language-text">process.roles=broker
listeners=BROKER://localhost:9092
inter.broker.listener.name=BROKER
controller.quorum.voters=0@localhost:9093
controller.quorum.bootstrap.servers=localhost:9093
controller.listener.names=CONTROLLER
listener.security.protocol.map=BROKER:SASL_SSL,CONTROLLER:SASL_SSL</code></pre>
@ -137,15 +137,15 @@ listener.security.protocol.map=BROKER:SASL_SSL,CONTROLLER:SASL_SSL</code></pre>
<pre><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.quorum.bootstrap.servers=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>It is a requirement that the host and port defined in <code>controller.quorum.bootstrap.servers</code>
is routed to the exposed controller listeners. For example, here the <code>CONTROLLER</code>
listener is bound to localhost:9093. The connection string defined by
<code>controller.quorum.bootstrap.servers</code> must then also use localhost: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.