This is an automated email from the ASF dual-hosted git repository. davidarthur pushed a commit to branch asf-site in repository https://gitbox.apache.org/repos/asf/kafka-site.git
The following commit(s) were added to refs/heads/asf-site by this push: new 6edca846 Add initial ZK to KRaft migration docs (#485) 6edca846 is described below commit 6edca84661d69f74ef4534b1795dcc738362e3e9 Author: David Arthur <mum...@gmail.com> AuthorDate: Tue Feb 7 10:51:39 2023 -0500 Add initial ZK to KRaft migration docs (#485) Reviewers: Ismael Juma <ism...@juma.me.uk> --- 34/ops.html | 185 +++++++++++++++++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 184 insertions(+), 1 deletion(-) diff --git a/34/ops.html b/34/ops.html index c3ac673f..9f35e49e 100644 --- a/34/ops.html +++ b/34/ops.html @@ -3553,6 +3553,7 @@ foo <li>Kafka server's <code>process.role</code> should be set to either <code>broker</code> or <code>controller</code> but not both. Combined mode can be used in development enviroment but it should be avoided in critical deployment evironments.</li> <li>For redundancy, a Kafka cluster should use 3 controllers. More than 3 servers is not recommended in critical environments. In the rare case of a partial network failure it is possible for the cluster metadata quorum to become unavailable. This limitation will be addresses in a future release of Kafka.</li> <li>The Kafka controllers store all of the metadata for the cluster in memory and on disk. We believe that for a typical Kafka cluster 5GB of main memory and 5GB of disk space on the metadata log director is sufficient.</li> + </ul> <h4 class="anchor-heading"><a id="kraft_missing" class="anchor-link"></a><a href="#kraft_missing">Missing Features</a></h4> @@ -3563,9 +3564,191 @@ foo <li>Supporting JBOD configurations with multiple storage directories</li> <li>Modifying certain dynamic configurations on the standalone KRaft controller</li> <li>Delegation tokens</li> - <li>Upgrade from ZooKeeper mode</li> </ul> + <h4 class="anchor-heading"><a id="kraft_zk_migration" class="anchor-link"></a><a href="#kraft_zk_migration">ZooKeeper to KRaft Migration</a></h4> + + <p> + <b>The ZooKeeper to KRaft migration feature is considered Early Access in 3.4.0. It is not recommended for production clusters.</b> + </p> + + <p>The following features are not yet supported for ZK to KRaft migrations:</p> + + <ul> + <li>Downgrading to ZooKeeper mode during or after the migration</li> + <li>Migration of ACLs</li> + <li>Other features <a href="#kraft_missing">not yet supported in KRaft</a></li> + </ul> + + <h3>Terminology</h3> + <p> + In this documentation, we use the term "migration" to refer to the process to changing a Kafka cluster's metadata + system from ZooKeeper to KRaft. An "upgrade" refers to installing a newer version of Kafka. It is not recommended to + upgrade the software at the same time as performing a migration. + </p> + + <p> + This documentation uses the term "ZK mode" to refer to Kafka brokers which are using ZooKeeper as their metadata + system. "KRaft mode" refers Kafka brokers which are using a KRaft controller quorum as their metadata system. + </p> + + <h3>Preparing for migration</h3> + <p> + Before beginning the migration, the Kafka brokers must be upgraded to software version 3.4.0 and have the + "inter.broker.protocol.version" configuration set to "3.4". See <a href="#upgrade_3_4_0">Upgrading to 3.4.0</a> for + upgrade instructions. + </p> + + <p> + It is recommended to enable TRACE level logging for the migration components while the migration is active. This can + be done by adding the following log4j configuration to each KRaft controller's "log4j.properties" file. + </p> + + <pre> + log4j.logger.org.apache.kafka.metadata.migration=TRACE + </pre> + + <p> + It may generally useful to enable DEBUG logging on the KRaft controllers and the ZK brokers during the migration. + </p> + + <h3>Provisioning the KRaft controller quorum</h3> + <p> + Two things are needed before the migration can begin. The brokers must be configured to support the migration and + a KRaft controller quorum must be deployed. For the KRaft deployment, please refer to <a href="#kraft_config">the above documentation</a>. + The KRaft controllers should be provisioned with the latest <code>metadata.version</code> of "3.4". + In addition to the standard KRaft configuration, the KRaft controllers will need to enable support for the migration + as well as provide ZooKeeper connection configuration. + </p> + + <pre> + # Sample KRaft cluster controller.properties listening on 9093 + process.roles=controller + node.id=3000 + controller.quorum.voters=1@localhost:9093 + controller.listener.names=CONTROLLER + listeners=CONTROLLER://:9093 + + # Enable the migration + zookeeper.metadata.migration.enable=true + + # ZooKeeper client configuration + zookeeper.connect=localhost:2181 + + # Other configs ... + </pre> + + <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> + + <h3>Enabling the migration on the brokers</h3> + <p> + Once the KRaft controller quorum has been started, the brokers will need to be reconfigured and restarted. Brokers + may be restarted in a rolling fashion to avoid impacting cluster availability. Each broker will need to add the + following configurations to allow it to communicate with the KRaft controllers and to enable the migration. + </p> + + <ul> + <li><a href="#brokerconfigs_controller.quorum.voters">controller.quorum.voters</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> + </ul> + + <p>Here is a sample config for a broker that is ready for migration:</p> + + <pre> + # Sample ZK broker server.properties listening on 9092 + broker.id=0 + listeners=PLAINTEXT://:9092 + advertised.listeners=PLAINTEXT://localhost:9092 + listener.security.protocol.map=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT + + # Set the IBP + inter.broker.protocol.version=3.4 + + # Enable the migration + zookeeper.metadata.migration.enable=true + + # ZooKeeper client configuration + zookeeper.connect=localhost:2181 + + # KRaft controller quorum configuration + controller.quorum.voters=3000@localhost:9093 + controller.listener.names=CONTROLLER + </pre> + + <p> + <em>Note: Once the final ZK broker has been restarted with the necessary configuration, the migration will automatically begin.</em> + When the migration is complete, a INFO level log can be observed on the active controller. + </p> + + <pre>Completed migration of metadata from Zookeeper to KRaft</pre> + + <h3>Migrating brokers to KRaft</h3> + <p> + Once the KRaft controller completes the metadata migration, the brokers will still be running in ZK mode. While the + KRaft controller is in migration mode, it will continue sending controller RPCs to the ZK mode brokers. This includes + RPCs like UpdateMetadata and LeaderAndIsr. + </p> + + <p> + To migrate the brokers to KRaft, they simply need to be reconfigured as KRaft brokers and restarted. Using the above + broker configuration as an example, we would replace the <code>broker.id</code> with <code>node.id</code> and add + <code>process.roles=broker</code>. It is important that the broker maintain the same Broker/Node ID when it is restarted. + The zookeeper configurations should be removed at this point. + </p> + + <pre> + # Sample KRaft broker server.properties listening on 9092 + process.roles=broker + node.id=0 + listeners=PLAINTEXT://:9092 + advertised.listeners=PLAINTEXT://localhost:9092 + listener.security.protocol.map=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT + + # Don't set the IBP, KRaft uses "metadata.version" feature flag + # inter.broker.protocol.version=3.4 + + # Keep the migration enabled + zookeeper.metadata.migration.enable=true + + # Remove ZooKeeper client configuration + # zookeeper.connect=localhost:2181 + + # Keep the KRaft controller quorum configuration + controller.quorum.voters=3000@localhost:9093 + controller.listener.names=CONTROLLER + </pre> + + <p> + Each broker is restarted with a KRaft configuration until the entire cluster is running in KRaft mode. + </p> + + <h3>Finalizing the migration</h3> + <p> + Once all brokers have been restarted in KRaft mode, the last step to finalize the migration is to take the + KRaft controllers out of migration mode. This is done by removing the "zookeeper.metadata.migration.enable" + property from each of their configs and restarting them one at a time. + </p> + + <pre> + # Sample KRaft cluster controller.properties listening on 9093 + process.roles=controller + node.id=3000 + controller.quorum.voters=1@localhost:9093 + controller.listener.names=CONTROLLER + listeners=CONTROLLER://:9093 + + # Disable the migration + # zookeeper.metadata.migration.enable=true + + # Remove ZooKeeper client configuration + # zookeeper.connect=localhost:2181 + + # Other configs ... + </pre> + </script> <div class="p-ops"></div>