diff --git a/docs/ops.html b/docs/ops.html index 09842ee2084..529bce159bb 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -3983,20 +3983,56 @@ controller.listener.names=CONTROLLER
Every broker and controller must set the controller.quorum.bootstrap.servers
property.
-
Apache Kafka 4.1 added support for upgrading a cluster from a static controller configuration to a dynamic controller configuration. Dynamic controller configuration allows users to add controller to and remove controller from the cluster. See the Controller membership changes section for more details.
+ +This feature upgrade is done by upgrading the KRaft feature version and updating the nodes' configuration.
+ +Dynamic controller cluster was added in kraft.version=1
or release-version 4.1
. To determine which kraft feature version the cluster is using you can execute the following CLI command:
$ bin/kafka-features.sh --bootstrap-controller localhost:9093 describe
+...
+Feature: kraft.version SupportedMinVersion: 0 SupportedMaxVersion: 1 FinalizedVersionLevel: 0 Epoch: 7
+Feature: metadata.version SupportedMinVersion: 3.3-IV3 SupportedMaxVersion: 4.0-IV3 FinalizedVersionLevel: 4.0-IV3 Epoch: 7
+
+ If the FinalizedVersionLevel
for Feature: kraft.version
is 0
, the version needs to be upgraded to at least 1
to support a dynamic controller cluster.
The KRaft feature version can be upgraded to support dynamic controller clusters by using the kafka-feature
CLI command. To upgrade all of the feature versions to the latest version:
$ bin/kafka-features.sh --bootstrap-server localhost:9092 upgrade --release-version 4.1
+
+ To upgrade just the KRaft feature version:
+ +$ bin/kafka-features.sh --bootstrap-server localhost:9092 upgrade --feature kraft.version=1
+
+ KRaft version 1 deprecated the controller.quorum.voters
property and added the controller.quorum.bootstrap.servers
property. After checking that the KRaft version has been successfully upgraded to at least version 1
, remove the controller.quorum.voters
property and add the controller.quorum.bootstrap.servers
to all of the nodes (controllers and brokers) in the cluster.
process.roles=...
+node.id=...
+controller.quorum.bootstrap.servers=controller1.example.com:9093,controller2.example.com:9093,controller3.example.com:9093
+controller.listener.names=CONTROLLER
+
+ bin/kafka-storage.sh random-uuid
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 bin/kafka-storage.sh format
command.
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.
-$ bin/kafka-storage.sh format --cluster-id <CLUSTER_ID> --standalone --config config/controller.properties
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.
- CLUSTER_ID="$(bin/kafka-storage.sh random-uuid)"
@@ -4013,7 +4049,7 @@ This command is similar to the standalone version but the snapshot at 0000000000
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.
- Formatting Brokers and New Controllers
+ Formatting Brokers and New Controllers
When provisioning new broker and controller nodes that we want to add to an existing Kafka cluster, use the kafka-storage.sh format
command with the --no-initial-controllers flag.
$ bin/kafka-storage.sh format --cluster-id <CLUSTER_ID> --config config/server.properties --no-initial-controllers
@@ -4077,7 +4113,7 @@ Feature: metadata.version SupportedMinVersion: 3.3-IV3 SupportedMaxVers
use a dynamic controller quorum. This function will be supported in the future release.
Add New Controller
- If a dynamic controller cluster already exists, it can be expanded by first provisioning a new controller using the kafka-storage.sh tool and starting the controller.
+ If a dynamic controller cluster already exists, it can be expanded by first provisioning a new controller using the kafka-storage.sh tool and starting the controller.
After starting the controller, the replication to the new controller can be monitored using the bin/kafka-metadata-quorum.sh describe --replication
command. Once the new controller has caught up to the active controller, it can be added to the cluster using the bin/kafka-metadata-quorum.sh add-controller
command.
diff --git a/docs/toc.html b/docs/toc.html
index ccc824c4962..032228de3d2 100644
--- a/docs/toc.html
+++ b/docs/toc.html
@@ -167,7 +167,9 @@
6.8 KRaft