ahuang98 commented on code in PR #17076: URL: https://github.com/apache/kafka/pull/17076#discussion_r1745972615
########## docs/ops.html: ########## @@ -3776,25 +3776,71 @@ <h5 class="anchor-heading"><a id="kraft_voter" class="anchor-link"></a><a href=" <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.bootrtrap.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 [email protected]:9093,[email protected]:9093,[email protected]:9093</code></pre> +controller.quorum.bootstrap.servers=controller1.example.com:9093,controller2.example.com:9093,controller3.example.com:9093</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 dyncamically <a href="#kraft_reconfig_add">add the rest of the controllers</a>. This can be done with the following CLI command: + + <pre><code class="language-bash">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 Controller</a></h5> Review Comment: nit: `Controllers` ########## docs/ops.html: ########## @@ -3776,25 +3776,71 @@ <h5 class="anchor-heading"><a id="kraft_voter" class="anchor-link"></a><a href=" <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.bootrtrap.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> Review Comment: typo `bootstrap.servers` ########## docs/ops.html: ########## @@ -3776,25 +3776,71 @@ <h5 class="anchor-heading"><a id="kraft_voter" class="anchor-link"></a><a href=" <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.bootrtrap.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 [email protected]:9093,[email protected]:9093,[email protected]:9093</code></pre> +controller.quorum.bootstrap.servers=controller1.example.com:9093,controller2.example.com:9093,controller3.example.com:9093</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 dyncamically <a href="#kraft_reconfig_add">add the rest of the controllers</a>. This can be done with the following CLI command: Review Comment: since bootstrapping the first voter and dynamically adding the rest are separate steps, should we change `This can be done with the following CLI command:` to `Bootstrapping the first voter can be done with the following CLI command:` ########## docs/ops.html: ########## @@ -3776,25 +3776,71 @@ <h5 class="anchor-heading"><a id="kraft_voter" class="anchor-link"></a><a href=" <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.bootrtrap.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 [email protected]:9093,[email protected]:9093,[email protected]:9093</code></pre> +controller.quorum.bootstrap.servers=controller1.example.com:9093,controller2.example.com:9093,controller3.example.com:9093</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 dyncamically <a href="#kraft_reconfig_add">add the rest of the controllers</a>. This can be done with the following CLI command: + + <pre><code class="language-bash">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 Controller</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">kafka-storage format --cluster-id <cluster-id> --initial-controller 0@controller-0:1234:3Db5QLSqSZieL3rJBUUegA,1@controller-1:1234:L3rJBUUegA3Db5QLSqSZie,2@controller-2:1234:UegA3Db5QLSqSZieL3rJBU --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. Just like the controller.quorum.voters properties, it is important that the set of voters is equal 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. + +To generate a random UUID the user can execute the kafka-storage random-uuid command. + + <pre><code class="language-bash">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> \ + --controller-quorum-voters "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> + + <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 brokers nodes and new controller nodes joining 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">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 exist, 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 sarting 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. + + <pre><code class="language-bash">kafka-metadata-quorum --command-config controller.properties --bootstrap-server localhost:9092 add-controller</code></pre> Review Comment: this must be run from a broker, should we give the bootstrap-controller alternative command as well? ########## docs/ops.html: ########## @@ -3776,25 +3776,71 @@ <h5 class="anchor-heading"><a id="kraft_voter" class="anchor-link"></a><a href=" <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.bootrtrap.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 [email protected]:9093,[email protected]:9093,[email protected]:9093</code></pre> +controller.quorum.bootstrap.servers=controller1.example.com:9093,controller2.example.com:9093,controller3.example.com:9093</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 dyncamically <a href="#kraft_reconfig_add">add the rest of the controllers</a>. This can be done with the following CLI command: + + <pre><code class="language-bash">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 Controller</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">kafka-storage format --cluster-id <cluster-id> --initial-controller 0@controller-0:1234:3Db5QLSqSZieL3rJBUUegA,1@controller-1:1234:L3rJBUUegA3Db5QLSqSZie,2@controller-2:1234:UegA3Db5QLSqSZieL3rJBU --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. Just like the controller.quorum.voters properties, it is important that the set of voters is equal in all of the controllers with the same cluster id. Review Comment: did you mean `controller.quorum.bootstrap.servers` instead of `controller.quorum.voters`? if you _did_ mean `controller.quorum.voters`, it might not make sense to readers since it's not mentioned prior to this line anymore ########## docs/ops.html: ########## @@ -3776,25 +3776,71 @@ <h5 class="anchor-heading"><a id="kraft_voter" class="anchor-link"></a><a href=" <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.bootrtrap.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 [email protected]:9093,[email protected]:9093,[email protected]:9093</code></pre> +controller.quorum.bootstrap.servers=controller1.example.com:9093,controller2.example.com:9093,controller3.example.com:9093</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 dyncamically <a href="#kraft_reconfig_add">add the rest of the controllers</a>. This can be done with the following CLI command: + + <pre><code class="language-bash">kafka-storage format --cluster-id <cluster-id> --standalone --config controller.properties</code></pre> Review Comment: do we want to talk at all about what information must be in `controller.properties`? Perhaps in a separate section? ########## docs/ops.html: ########## @@ -3776,25 +3776,71 @@ <h5 class="anchor-heading"><a id="kraft_voter" class="anchor-link"></a><a href=" <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.bootrtrap.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 [email protected]:9093,[email protected]:9093,[email protected]:9093</code></pre> +controller.quorum.bootstrap.servers=controller1.example.com:9093,controller2.example.com:9093,controller3.example.com:9093</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 dyncamically <a href="#kraft_reconfig_add">add the rest of the controllers</a>. This can be done with the following CLI command: + + <pre><code class="language-bash">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 Controller</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">kafka-storage format --cluster-id <cluster-id> --initial-controller 0@controller-0:1234:3Db5QLSqSZieL3rJBUUegA,1@controller-1:1234:L3rJBUUegA3Db5QLSqSZie,2@controller-2:1234:UegA3Db5QLSqSZieL3rJBU --config controller.properties</code></pre> Review Comment: comment says `--initial-controllers` and the command says `--initial-controller` ########## docs/security.html: ########## @@ -116,36 +116,36 @@ <h3 class="anchor-heading"><a id="listener_configuration" class="anchor-link"></ (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><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> <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 + comes from the <code>controller.quorum.bootstrap.servers</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><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 for 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> Review Comment: `It is a requirement that the... is routed to` ########## docs/ops.html: ########## @@ -3776,25 +3776,71 @@ <h5 class="anchor-heading"><a id="kraft_voter" class="anchor-link"></a><a href=" <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.bootrtrap.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 [email protected]:9093,[email protected]:9093,[email protected]:9093</code></pre> +controller.quorum.bootstrap.servers=controller1.example.com:9093,controller2.example.com:9093,controller3.example.com:9093</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 dyncamically <a href="#kraft_reconfig_add">add the rest of the controllers</a>. This can be done with the following CLI command: + + <pre><code class="language-bash">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 Controller</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">kafka-storage format --cluster-id <cluster-id> --initial-controller 0@controller-0:1234:3Db5QLSqSZieL3rJBUUegA,1@controller-1:1234:L3rJBUUegA3Db5QLSqSZie,2@controller-2:1234:UegA3Db5QLSqSZieL3rJBU --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. Just like the controller.quorum.voters properties, it is important that the set of voters is equal 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. + +To generate a random UUID the user can execute the kafka-storage random-uuid command. + + <pre><code class="language-bash">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> \ + --controller-quorum-voters "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> + + <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 brokers nodes and new controller nodes joining 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">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 exist, 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 sarting 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. Review Comment: typo: `starting` ########## docs/ops.html: ########## @@ -3776,25 +3776,71 @@ <h5 class="anchor-heading"><a id="kraft_voter" class="anchor-link"></a><a href=" <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.bootrtrap.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 [email protected]:9093,[email protected]:9093,[email protected]:9093</code></pre> +controller.quorum.bootstrap.servers=controller1.example.com:9093,controller2.example.com:9093,controller3.example.com:9093</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 dyncamically <a href="#kraft_reconfig_add">add the rest of the controllers</a>. This can be done with the following CLI command: Review Comment: typo: `dynamically` ########## docs/ops.html: ########## @@ -3776,25 +3776,71 @@ <h5 class="anchor-heading"><a id="kraft_voter" class="anchor-link"></a><a href=" <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.bootrtrap.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 [email protected]:9093,[email protected]:9093,[email protected]:9093</code></pre> +controller.quorum.bootstrap.servers=controller1.example.com:9093,controller2.example.com:9093,controller3.example.com:9093</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 dyncamically <a href="#kraft_reconfig_add">add the rest of the controllers</a>. This can be done with the following CLI command: + + <pre><code class="language-bash">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 Controller</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">kafka-storage format --cluster-id <cluster-id> --initial-controller 0@controller-0:1234:3Db5QLSqSZieL3rJBUUegA,1@controller-1:1234:L3rJBUUegA3Db5QLSqSZie,2@controller-2:1234:UegA3Db5QLSqSZieL3rJBU --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. Just like the controller.quorum.voters properties, it is important that the set of voters is equal 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. + +To generate a random UUID the user can execute the kafka-storage random-uuid command. Review Comment: suggestion: `To generate random UUIDs for replica directory ids, the user can execute...` ########## docs/ops.html: ########## @@ -3776,25 +3776,71 @@ <h5 class="anchor-heading"><a id="kraft_voter" class="anchor-link"></a><a href=" <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.bootrtrap.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 [email protected]:9093,[email protected]:9093,[email protected]:9093</code></pre> +controller.quorum.bootstrap.servers=controller1.example.com:9093,controller2.example.com:9093,controller3.example.com:9093</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 dyncamically <a href="#kraft_reconfig_add">add the rest of the controllers</a>. This can be done with the following CLI command: + + <pre><code class="language-bash">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 Controller</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">kafka-storage format --cluster-id <cluster-id> --initial-controller 0@controller-0:1234:3Db5QLSqSZieL3rJBUUegA,1@controller-1:1234:L3rJBUUegA3Db5QLSqSZie,2@controller-2:1234:UegA3Db5QLSqSZieL3rJBU --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. Just like the controller.quorum.voters properties, it is important that the set of voters is equal 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. + +To generate a random UUID the user can execute the kafka-storage random-uuid command. + + <pre><code class="language-bash">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> \ + --controller-quorum-voters "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> Review Comment: this is confusing since we already give a command on `L3803` that's slightly different ``` <pre><code class="language-bash">kafka-storage format --cluster-id <cluster-id> --initial-controller 0@controller-0:1234:3Db5QLSqSZieL3rJBUUegA, 1@controller-1:1234:L3rJBUUegA3Db5QLSqSZie,2@controller-2:1234:UegA3Db5QLSqSZieL3rJBU --config controller.properties</code></pre> ``` ########## docs/ops.html: ########## @@ -3776,25 +3776,71 @@ <h5 class="anchor-heading"><a id="kraft_voter" class="anchor-link"></a><a href=" <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.bootrtrap.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 [email protected]:9093,[email protected]:9093,[email protected]:9093</code></pre> +controller.quorum.bootstrap.servers=controller1.example.com:9093,controller2.example.com:9093,controller3.example.com:9093</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 dyncamically <a href="#kraft_reconfig_add">add the rest of the controllers</a>. This can be done with the following CLI command: + + <pre><code class="language-bash">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 Controller</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">kafka-storage format --cluster-id <cluster-id> --initial-controller 0@controller-0:1234:3Db5QLSqSZieL3rJBUUegA,1@controller-1:1234:L3rJBUUegA3Db5QLSqSZie,2@controller-2:1234:UegA3Db5QLSqSZieL3rJBU --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. Just like the controller.quorum.voters properties, it is important that the set of voters is equal 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. + +To generate a random UUID the user can execute the kafka-storage random-uuid command. + + <pre><code class="language-bash">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> \ + --controller-quorum-voters "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> + + <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 brokers nodes and new controller nodes joining an existing Kafka cluster use the <code>kafka-storage.sh format</code> command without the --standalone or --initial-controllers flags. Review Comment: nit: `new broker and controller nodes that we want to add to an existing Kafka cluster,` ########## docs/ops.html: ########## @@ -3776,25 +3776,71 @@ <h5 class="anchor-heading"><a id="kraft_voter" class="anchor-link"></a><a href=" <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.bootrtrap.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 [email protected]:9093,[email protected]:9093,[email protected]:9093</code></pre> +controller.quorum.bootstrap.servers=controller1.example.com:9093,controller2.example.com:9093,controller3.example.com:9093</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 dyncamically <a href="#kraft_reconfig_add">add the rest of the controllers</a>. This can be done with the following CLI command: + + <pre><code class="language-bash">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 Controller</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">kafka-storage format --cluster-id <cluster-id> --initial-controller 0@controller-0:1234:3Db5QLSqSZieL3rJBUUegA,1@controller-1:1234:L3rJBUUegA3Db5QLSqSZie,2@controller-2:1234:UegA3Db5QLSqSZieL3rJBU --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. Just like the controller.quorum.voters properties, it is important that the set of voters is equal 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. + +To generate a random UUID the user can execute the kafka-storage random-uuid command. + + <pre><code class="language-bash">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> \ + --controller-quorum-voters "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> + + <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 brokers nodes and new controller nodes joining 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">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 exist, 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 sarting 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. + + <pre><code class="language-bash">kafka-metadata-quorum --command-config controller.properties --bootstrap-server 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 exist, the cluster can be shrunk using the <code>kafka-metadata-quorum remove-controller</code> command. Until KIP-996: Pre-vote has been implemented and release, it is recommended to shutdown the controller that will be removed. Review Comment: suggestion to make this extremely explicit: `it is recommended to shutdown the controller that will be removed before running the remove-controller command.` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
