mirror of https://github.com/apache/kafka.git
MINOR: Add ZK migration instructions to the operations documentation (#13257)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
This commit is contained in:
parent
b10beaae77
commit
66f0cbc424
206
docs/ops.html
206
docs/ops.html
|
@ -3594,22 +3594,216 @@ foo
|
||||||
<h4 class="anchor-heading"><a id="kraft_deployment" class="anchor-link"></a><a href="#kraft_deployment">Deploying Considerations</a></h4>
|
<h4 class="anchor-heading"><a id="kraft_deployment" class="anchor-link"></a><a href="#kraft_deployment">Deploying Considerations</a></h4>
|
||||||
|
|
||||||
<ul>
|
<ul>
|
||||||
<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>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 environments, but it should be avoided in critical deployment environments.</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>For redundancy, a Kafka cluster should use 3 controllers. More than 3 controllers 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 addressed 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>
|
<li>The Kafka controllers store all 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>
|
<h4 class="anchor-heading"><a id="kraft_missing" class="anchor-link"></a><a href="#kraft_missing">Missing Features</a></h4>
|
||||||
|
|
||||||
<p>The following features are not fullying implemented in KRaft mode:</p>
|
<p>The following features are not fully implemented in KRaft mode:</p>
|
||||||
|
|
||||||
<ul>
|
<ul>
|
||||||
<li>Configuring SCRAM users via the administrative API</li>
|
|
||||||
<li>Supporting JBOD configurations with multiple storage directories</li>
|
<li>Supporting JBOD configurations with multiple storage directories</li>
|
||||||
<li>Modifying certain dynamic configurations on the standalone KRaft controller</li>
|
<li>Modifying certain dynamic configurations on the standalone KRaft controller</li>
|
||||||
<li>Delegation tokens</li>
|
<li>Delegation tokens</li>
|
||||||
<li>Upgrade from ZooKeeper mode</li>
|
|
||||||
</ul>
|
</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>ZooKeeper to KRaft migration is considered an Early Access feature and 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>Other features <a href="#kraft_missing">not yet supported in KRaft</a></li>
|
||||||
|
</ul>
|
||||||
|
|
||||||
|
<p>
|
||||||
|
Please report issues with ZooKeeper to KRaft migration using the
|
||||||
|
<a href="https://issues.apache.org/jira/projects/KAFKA" target="_blank">project JIRA</a> and the "kraft" component.
|
||||||
|
</p>
|
||||||
|
|
||||||
|
<h3>Terminology</h3>
|
||||||
|
<p>
|
||||||
|
We use the term "migration" here to refer to the process of changing a Kafka cluster's metadata
|
||||||
|
system from ZooKeeper to KRaft and migrating existing metadata. 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 metadata migration.
|
||||||
|
</p>
|
||||||
|
|
||||||
|
<p>
|
||||||
|
We also use 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.5.0 and have the
|
||||||
|
"inter.broker.protocol.version" configuration set to "3.5". See <a href="#upgrade_3_5_0">Upgrading to 3.5.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 is 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. First, the brokers must be configured to support the migration and second,
|
||||||
|
a KRaft controller quorum must be deployed. The KRaft controllers should be provisioned with the same cluster ID as
|
||||||
|
the existing Kafka cluster. This can be found by examining one of the "meta.properties" files in the data directories
|
||||||
|
of the brokers, or by running the following command.
|
||||||
|
</p>
|
||||||
|
|
||||||
|
<pre>./bin/zookeeper-shell.sh localhost:2181 get /cluster/id</pre>
|
||||||
|
|
||||||
|
<p>
|
||||||
|
The KRaft controller quorum should also be provisioned with the latest <code>metadata.version</code> of "3.4".
|
||||||
|
For further instructions on KRaft deployment, please refer to <a href="#kraft_config">the above documentation</a>.
|
||||||
|
</p>
|
||||||
|
|
||||||
|
<p>
|
||||||
|
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>
|
||||||
|
|
||||||
|
<p>
|
||||||
|
Here is a sample config for a KRaft controller that is ready for migration:
|
||||||
|
</p>
|
||||||
|
<pre>
|
||||||
|
# Sample KRaft cluster controller.properties listening on 9093
|
||||||
|
process.roles=controller
|
||||||
|
node.id=3000
|
||||||
|
controller.quorum.voters=3000@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 requires the
|
||||||
|
following configuration 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.5
|
||||||
|
|
||||||
|
# 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, an 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.5
|
||||||
|
|
||||||
|
# Remove the migration enabled flag
|
||||||
|
# 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=3000@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>
|
</script>
|
||||||
|
|
||||||
<div class="p-ops"></div>
|
<div class="p-ops"></div>
|
||||||
|
|
|
@ -162,6 +162,7 @@
|
||||||
<li><a href="#kraft_debug">Debugging</a></li>
|
<li><a href="#kraft_debug">Debugging</a></li>
|
||||||
<li><a href="#kraft_deployment">Deploying Considerations</a></li>
|
<li><a href="#kraft_deployment">Deploying Considerations</a></li>
|
||||||
<li><a href="#kraft_missing">Missing Features</a></li>
|
<li><a href="#kraft_missing">Missing Features</a></li>
|
||||||
|
<li><a href="#kraft_zk_migration">ZooKeeper to KRaft Migration</a></li>
|
||||||
</ul>
|
</ul>
|
||||||
</li>
|
</li>
|
||||||
</ul>
|
</ul>
|
||||||
|
|
Loading…
Reference in New Issue