Este conteúdo não está disponível no idioma selecionado.

Chapter 21. Migrating ZooKeeper-based Kafka clusters


If you are using ZooKeeper for metadata management of your Kafka cluster, you must migrate to using Kafka in KRaft mode. KRaft mode replaces ZooKeeper for distributed coordination, offering enhanced reliability, scalability, and throughput.

Note

Kafka 4.0 and later runs exclusively in KRaft mode, with no ZooKeeper integration. As a result of this change, Streams for Apache Kafka removed support for ZooKeeper-based Kafka clusters starting with version 3.0.

To complete the migration, follow the procedures in the order presented in this section.

21.1. Migration overview

Migrating an existing Kafka cluster from ZooKeeper to KRaft mode happens in four main phases. The diagrams show the logical transition at each step.

21.1.1. Phase 1: ZooKeeper-based cluster

Brokers store metadata in ZooKeeper and operate in ZooKeeper mode.

Diagram showing three broker nodes above three ZooKeeper nodes in a ZooKeeper-based cluster.

21.1.2. Phase 2: Deploy the KRaft controller quorum

A new set of dedicated controller nodes is deployed. They run in migration mode, connect to ZooKeeper, and wait for brokers to register.

Key changes:

  • Controllers added
  • Controllers connect to ZooKeeper
  • Migration flag enabled
Diagram showing the KRaft controller quorum added below ZooKeeper

21.1.3. Phase 3: Enable migration on brokers (metadata migration in progress)

Brokers are rolled one at a time with updated configuration. On restart, each broker registers with the KRaft controller quorum.

When all brokers have registered, the KRaft controller leader begins the metadata migration process: it reads the existing metadata stored in ZooKeeper and writes it into the KRaft metadata log.

During this phase, the controllers also perform dual-write by copying new metadata updates back to ZooKeeper to maintain rollback compatibility.

Key changes:

  • Brokers register with the KRaft controller quorum
  • The KRaft controller leader migrates existing metadata from ZooKeeper into KRaft
  • Brokers send new metadata updates to the KRaft controllers (not to ZooKeeper)
  • KRaft controllers dual-write updates back to ZooKeeper to support rollback
  • KRaft becomes the source of truth for cluster metadata
  • ZooKeeper receives replicated metadata only to support rollback
Diagram showing brokers in migration mode

21.1.4. Phase 4: Restart brokers in full KRaft mode

Brokers are rolled again with ZooKeeper configuration removed.

Key changes:

  • Brokers operate fully in KRaft mode
  • The KRaft controllers continue dual-write to ZooKeeper to support rollback
  • This is the last point where rollback is possible
Diagram showing brokers running in KRaft mode

21.1.5. Phase 5: Finalize migration (controllers exit migration mode)

Controllers are rolled with ZooKeeper settings removed.

Key changes:

  • Controllers stop dual-write to ZooKeeper
  • ZooKeeper is no longer used by the cluster
  • Rollback to ZooKeeper is no longer possible
  • The cluster’s metadata is managed by the KRaft controller quorum
Diagram showing brokers running in KRaft mode
Note

When the controllers exit migration mode, ZooKeeper can be decommissioned if it is not needed for other services.

21.2. Verify the ZooKeeper-Based Kafka Cluster is running

This procedure verifies that your ZooKeeper-based Kafka cluster is running and functioning correctly before you begin the migration. You check that ZooKeeper is available, that the brokers are connected, and that basic messaging works. An operational cluster is required for a successful migration to KRaft mode.

Prerequisites

  • You are logged in to Red Hat Enterprise Linux as the Kafka user.
  • Streams for Apache Kafka is installed on each host, and the configuration files are available.
  • You are using Streams for Apache Kafka 2.9 (Kafka 3.9.x), which includes important fixes for KRaft migration. Migration is supported only on Streams for Apache Kafka 2.9. If you are using an earlier version, upgrade to 2.9 before migrating to KRaft mode.
  • All brokers must be configured with an inter-broker protocol version supported by Kafka 3.9.x (for example, inter.broker.protocol.version=3.9). Using an older inter-broker protocol can prevent migration from starting or cause unexpected failures.

Procedure

  1. Verify that ZooKeeper is running.

    jcmd | grep zookeeper

    Returns:

    <process_id> org.apache.zookeeper.server.quorum.QuorumPeerMain ./config/zookeeper.properties
  2. Check that the Kafka brokers are running:

    jcmd | grep kafka

    Returns:

    <process_id> kafka.Kafka ./config/server.properties

    Each Kafka broker appears as a running process.

  3. Verify that each broker is connected to ZooKeeper.

    Use the ZooKeeper shell to list the registered brokers:

    ./bin/zookeeper-shell.sh <zk-host>:2181 ls /brokers/ids

    Returns a list of the IDs for the brokers currently registered with ZooKeeper. This command confirms the brokers are registered with ZooKeeper. Client connectivity is verified in the next step.

  4. Check the cluster is running correctly by sending and consuming messages.

    For more information, see Sending and receiving messages from a topic.

21.3. Migrating from ZooKeeper to KRaft mode

This is a single end-to-end procedure for migrating a ZooKeeper-based Kafka cluster to KRaft mode. The workflow applies to both single-node and multi-node deployments.

To migrate your Kafka cluster to KRaft mode, complete the following steps:

  • Install a quorum of dedicated controller nodes to replace ZooKeeper for metadata management.
  • Enable KRaft metadata migration on the controllers.
  • Start the controllers and enable migration on the existing brokers.
  • Perform a rolling restart of the brokers to apply the configuration changes.
  • After migration completes, switch the brokers to KRaft mode and disable migration on the controllers.

During the migration, the cluster enters a dual-write phase. ZooKeeper and the KRaft controllers store metadata in parallel until the final transition is complete.

Important

After you finalize KRaft mode, you cannot roll back to ZooKeeper. Review your environment and backup requirements before proceeding.

Before starting the migration, verify that your environment can support Kafka in KRaft mode:

  • Migration requires dedicated controller nodes. Nodes that act as both brokers and controllers are not supported.
  • ZooKeeper and the KRaft controller quorum run in parallel during migration, so ensure that the cluster has sufficient compute resources.
Warning

If you previously rolled back a KRaft migration, make sure that all cleanup steps were completed before attempting the migration again:

  • Delete the dedicated controller nodes.
  • Remove the /migration znode from ZooKeeper using zookeeper-shell.sh: delete /migration

Skipping this cleanup can lead to metadata loss and migration failure.

Prerequisites

  • The Kafka cluster is running.
  • Logging is enabled to monitor the migration process.

    Set the root logger to DEBUG on the controllers and brokers. For migration-specific logging, set TRACE for the migration logger.

    Controller logging configuration

    log4j.rootLogger=DEBUG
    log4j.logger.org.apache.kafka.metadata.migration=TRACE

Procedure

  1. Retrieve the Kafka cluster ID using the zookeeper-shell tool:

    ./bin/zookeeper-shell.sh localhost:2181 get /cluster/id

    The command prints connection information before returning the cluster ID. The final JSON line in the output contains the cluster ID. For example:

    Connecting to localhost:2181
    
    WATCHER::
    WatchedEvent state:SyncConnected type:None path:null
    {"version":"1","id":"SY9T31EKSS6XeNO_Jio6yQ"}

    In this example, the cluster ID is SY9T31EKSS6XeNO_Jio6yQ.

  2. Install a KRaft controller quorum to the cluster.

    A KRaft controller quorum is required to manage metadata during the migration. Each controller runs in a dedicated role and must be configured separately.

    1. Create a controller.properties file for each controller node.

      Each controller requires:

      • A unique node.id. The node.id must be unique across the entire Kafka cluster, including the existing brokers (which still use broker.id). Brokers and controllers share a single ID namespace, so do not reuse an ID already assigned to a broker or another controller.
      • The migration flag set to true.
      • ZooKeeper connection details.
      • A listener for controller-quorum communication.
      • A list of all controllers in the quorum.
      • A listener name for inter-broker communication.
      • A dedicated controller listener port that does not conflict with broker listener ports on the same host.

        Note

        The controller listener is used for communication within the KRaft quorum and for controller-related requests from brokers and administrative clients. It is not used for regular client traffic such as producing or consuming messages.

        Example controller configuration

        process.roles=controller
        node.id=1
        
        # Enable KRaft metadata migration
        zookeeper.metadata.migration.enable=true
        zookeeper.connect=zoo1.example.com:2181,zoo2.example.com:2181,zoo3.example.com:2181
        
        # Listener for quorum communication
        listeners=CONTROLLER://0.0.0.0:9090
        controller.listener.names=CONTROLLER
        listener.security.protocol.map=CONTROLLER:PLAINTEXT
        
        # List all controllers in the quorum
        controller.quorum.bootstrap.servers=controller1:9090,controller2:9090,controller3:9090
        
        # Required for migration
        inter.broker.listener.name=PLAINTEXT

        Use a dedicated port for the controller listener that does not conflict with broker listeners on each host. In most deployments, the same controller listener port is used on every controller node, and the host name distinguishes each controller. Only controllers configure the quorum voters.

    2. Generate a directory ID for each controller node.

      CONTROLLER_1_DIR_ID="$(./bin/kafka-storage.sh random-uuid)"
      CONTROLLER_2_DIR_ID="$(./bin/kafka-storage.sh random-uuid)"
      CONTROLLER_3_DIR_ID="$(./bin/kafka-storage.sh random-uuid)"

      Each controller requires a unique directory ID for metadata storage initialization.

    3. Format the metadata directory on each controller node.

      Use the cluster ID retrieved from ZooKeeper and specify the initial controller quorum:

      ./bin/kafka-storage.sh format \
        --cluster-id <cluster-id> \
        --initial-controllers "1@controller1:9090:${CONTROLLER_1_DIR_ID},2@controller2:9090:${CONTROLLER_2_DIR_ID},3@controller3:9090:${CONTROLLER_3_DIR_ID}" \
        --config ./config/kraft/controller.properties

      Use the same cluster ID and --initial-controllers configuration on all controllers. Each controller entry uses the format <node.id>@<host>:<port>:<directory.id>.

      By default, log.dirs is /tmp/kraft-controller-logs, which is cleared on system reboot. Set a persistent directory in production environments.

    4. Start each controller node.

      ./bin/kafka-server-start.sh -daemon ./config/kraft/controller.properties
    5. Verify that the controllers are running.

      jcmd | grep kafka

      You should see one process per controller.

    6. Confirm that each controller has entered the PRE_MIGRATION state.

      Run the following command on each controller host:

      grep "NONE to PRE_MIGRATION" ./logs/controller.log

      A log entry similar to the following indicates that the controller has started successfully, formed a quorum, and is ready for brokers to register:

      Replayed a ZkMigrationStateRecord changing the migration state from NONE to PRE_MIGRATION

      This state confirms that the controllers are prepared to begin metadata migration when all brokers have been restarted with migration enabled.

  3. Enable migration on each broker.

    In this step, update each broker so it can register with the KRaft controller quorum and enter the migration process. Metadata migration does not begin immediately. The controllers start migrating metadata from ZooKeeper to KRaft only after all brokers have been restarted with migration enabled and have successfully registered with the controller quorum. During this phase, any new metadata created through brokers is written to KRaft first and then copied back to ZooKeeper by the controllers (dual-write) to maintain rollback compatibility. Controllers must be running before you restart any brokers.

    1. Stop the broker running on the host.

      ./bin/kafka-server-stop.sh
      jcmd | grep kafka

      If you are using a multi-node cluster, restart brokers one at a time. For guidance, see Section 3.8, “Performing a graceful rolling restart of Kafka brokers”.

    2. Update the broker configuration.

      Edit the server.properties file to enable metadata migration and to add a listener for communication with the KRaft controllers.

      Each broker requires:

      • inter.broker.protocol.version set to a supported version
      • The migration flag set to true
      • ZooKeeper connection details (already present on existing brokers)
      • A controller listener on a free port
      • Controller quorum connection details (must list all controllers)

        Example broker configuration

        broker.id=0
        
        # Set the inter-broker protocol version for migration
        inter.broker.protocol.version=3.9
        
        # Enable dual-write migration mode
        zookeeper.metadata.migration.enable=true
        zookeeper.connect=zoo1.my-domain.com:2181,zoo2.my-domain.com:2181,zoo3.my-domain.com:2181
        
        # Add a controller listener for KRaft quorum communication
        listeners=PLAINTEXT://0.0.0.0:9092,CONTROLLER://0.0.0.0:9093
        listener.security.protocol.map=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT
        
        # Connect the broker to the controller quorum
        controller.listener.names=CONTROLLER
        controller.quorum.bootstrap.servers=controller1:9090,controller2:9090,controller3:9090

        Use a unique controller listener port for each broker if required by your environment.

    3. Restart the broker with the updated configuration.

      ./bin/kafka-server-start.sh -daemon ./config/kraft/server.properties

      The broker starts with the migration-enabled configuration. Metadata migration begins after all brokers have restarted with migration enabled and have registered with the controller quorum. The time required for migration depends on the number of topics and partitions in the cluster.

    4. Verify that the broker is running.

      jcmd | grep kafka

      You should see a running kafka.Kafka process using the updated configuration file.

  4. Confirm that metadata migration is complete.

    After all brokers have registered with the KRaft controller quorum, the active controller begins migrating metadata from ZooKeeper into the KRaft metadata log.

    1. Verify that the controller has entered the MIGRATION state.

      Run the following command on the active controller host:

      grep "PRE_MIGRATION to MIGRATION" ./logs/controller.log

      A log entry similar to the following indicates that the migration phase has started:

      Replayed a ZkMigrationStateRecord changing the migration state from PRE_MIGRATION to MIGRATION
    2. Check the controller logs for the completion message.

      Run the following command on the active controller host:

      grep "migration.*ZooKeeper.*KRaft" ./logs/controller.log

      A log entry similar to the following indicates that the metadata migration phase has finished:

      • Completed migration of metadata from ZooKeeper to KRaft.
    3. (Optional) Check the ZooKeeper /migration znode.

      This znode contains information about the active KRaft controller during migration:

      ./bin/zookeeper-shell.sh localhost:2181 get /migration

      The output includes the controller ID currently coordinating migration. Once the migration is complete, this information should reflect the active controller and the final metadata state.

  5. Switch each broker to KRaft mode.

    After the metadata migration is complete, you must update each broker so it runs fully in KRaft mode. This step removes all ZooKeeper-specific settings and enables the broker role in KRaft.

    1. Stop the broker running on the host.

      ./bin/kafka-server-stop.sh
      jcmd | grep kafka

      If the cluster has multiple brokers, restart them one at a time.

    2. Update the broker configuration.

      Edit the server.properties file and make the following changes:

      • Replace broker.id with node.id using the same value
      • Add the KRaft broker role: process.roles=broker
      • Remove inter.broker.protocol.version
      • Remove zookeeper.metadata.migration.enable
      • Remove all ZooKeeper configuration (zookeeper.connect)
      • Remove any control-plane listener configuration (for example, control.plane.listener.name and its associated listener entry), if present.

        Example broker configuration for KRaft mode

        node.id=0
        process.roles=broker
        
        # Listener for client traffic
        listeners=PLAINTEXT://0.0.0.0:9092
        
        # Connect the broker to the controller quorum
        controller.listener.names=CONTROLLER
        controller.quorum.bootstrap.servers=controller1:9090,controller2:9090,controller3:9090

        This configuration removes all ZooKeeper dependencies and enables the standalone KRaft broker mode.

    3. Update the authorizer if you use ACLs.

      ZooKeeper-based brokers use:

      authorizer.class.name=kafka.security.authorizer.AclAuthorizer

      For KRaft-based brokers, replace it with:

      authorizer.class.name=org.apache.kafka.metadata.authorizer.StandardAuthorizer

    4. Restart the broker.

      ./bin/kafka-server-start.sh -daemon ./config/kraft/server.properties

      The broker starts in KRaft mode and connects to the controller quorum.

  6. Switch each controller out of migration mode.

    After all brokers are running in KRaft mode, update the controllers so that they stop interacting with ZooKeeper. This completes the migration process and places the cluster fully under KRaft control.

    1. Stop the controller.

      Stop the controller in the same way as the broker:

      ./bin/kafka-server-stop.sh
      jcmd | grep kafka
    2. Update the controller configuration.

      Edit the controller.properties file and remove all ZooKeeper-related migration settings:

      • Remove the ZooKeeper connection details
      • Remove the zookeeper.metadata.migration.enable property
      • Remove inter.broker.listener.name

        Example controller configuration after migration

        process.roles=controller
        node.id=1
        
        listeners=CONTROLLER://0.0.0.0:9090
        controller.listener.names=CONTROLLER
        listener.security.protocol.map=CONTROLLER:PLAINTEXT
        controller.quorum.bootstrap.servers=controller1:9090,controller2:9090,controller3:9090

    3. Restart the controller.

      ./bin/kafka-server-start.sh -daemon ./config/kraft/controller.properties
    4. Verify that the controller has exited migration mode.

      Check the controller log:

      grep "MIGRATION to POST_MIGRATION" ./logs/controller.log

      A log entry similar to the following indicates the transition out of migration:

      • Replayed a ZkMigrationStateRecord changing the migration state from MIGRATION to POST_MIGRATION.

        This confirms that the controller no longer relies on ZooKeeper and that the migration process is complete.

        Note

        After all brokers and controllers are running in KRaft mode and the controllers have exited migration mode, the cluster no longer relies on ZooKeeper for metadata management. If ZooKeeper is not required for other applications, you can decommission it.

21.4. Performing a rollback on the migration

If you have finalized the migration, rollback is not possible.

Before finalizing the migration, while the cluster is still in dual-write mode, you can roll back to ZooKeeper mode. The steps depend on how far the migration has progressed.

  • If you’ve only completed the preparation and controller quorum setup, stop and remove the dedicated KRaft controller nodes from the cluster. No other changes are required, and the cluster continues to operate in ZooKeeper mode.
  • If you’ve enabled metadata migration on the brokers, follow these steps:

    1. Stop and remove the dedicated KRaft controller nodes from the cluster.
    2. Use zookeeper-shell.sh to do the following:

      • Run delete /controller to allow a ZooKeeper-based broker to become the active controller.
      • Run get /migration to confirm that the migration znode exists, then run delete /migration to remove it. Removing this znode resets ZooKeeper to a clean pre-migration state.
      Warning

      Run delete /controller promptly to minimize controller downtime. Temporary errors in broker logs are expected until rollback completes.

    3. On each broker:

      • Remove the following KRaft-related configurations:

        • zookeeper.metadata.migration.enable
        • controller.listener.names
        • controller.quorum.bootstrap.servers
      • Replace node.id with broker.id
    4. Perform a rolling restart of all brokers.
  • If brokers have already been restarted in KRaft mode, follow these steps:

    1. On each broker:

      • Remove process.roles.
      • Replace node.id with broker.id.
    2. Restore zookeeper.connect and any required ZooKeeper settings.
    3. Perform a first rolling restart of the brokers.

      Important

      Retain zookeeper.metadata.migration.enable=true for the first restart.

    4. Stop and remove the dedicated KRaft controller nodes from the cluster.
    5. Use zookeeper-shell.sh to do the following:

      • Run delete /controller to allow a ZooKeeper-based broker to become the active controller.
      • Run get /migration, then delete /migration to inspect and clear the migration metadata (stored in the znode). This restores a clean state in ZooKeeper for retrying the migration.
    6. On each broker, remove KRaft-related configuration:

      • zookeeper.metadata.migration.enable
      • controller.listener.names
      • controller.quorum.bootstrap.servers
    7. Perform a second rolling restart of the brokers.
Red Hat logoGithubredditYoutubeTwitter

Aprender

Experimente, compre e venda

Comunidades

Sobre a documentação da Red Hat

Ajudamos os usuários da Red Hat a inovar e atingir seus objetivos com nossos produtos e serviços com conteúdo em que podem confiar. Explore nossas atualizações recentes.

Tornando o open source mais inclusivo

A Red Hat está comprometida em substituir a linguagem problemática em nosso código, documentação e propriedades da web. Para mais detalhes veja o Blog da Red Hat.

Sobre a Red Hat

Fornecemos soluções robustas que facilitam o trabalho das empresas em plataformas e ambientes, desde o data center principal até a borda da rede.

Theme

© 2026 Red Hat
Voltar ao topo