Chapter 4. Using Kafka in KRaft mode


KRaft (Kafka Raft metadata) mode replaces Kafka’s dependency on ZooKeeper for cluster management. KRaft mode simplifies the deployment and management of Kafka clusters by bringing metadata management and coordination of clusters into Kafka.

Kafka in KRaft mode is designed to offer enhanced reliability, scalability, and throughput. Metadata operations become more efficient as they are directly integrated. And by removing the need to maintain a ZooKeeper cluster, there’s also a reduction in the operational and security overhead.

Through Kafka configuration, nodes are assigned the role of broker, controller, or both:

  • Controller nodes operate in the control plane to manage cluster metadata and the state of the cluster using a Raft-based consensus protocol.
  • Broker nodes operate in the data plane to manage the streaming of messages, receiving and storing data in topic partitions.
  • Dual-role nodes fulfill the responsibilities of controllers and brokers.

You can use a dynamic or static quorum of controllers. Dynamic is recommended as it supports dynamic scaling.

Controllers use a metadata log, stored as a single-partition topic (__cluster_metadata) on every node, which records the state of the cluster. When requests are made to change the cluster configuration, an active (lead) controller manages updates to the metadata log, and follower controllers replicate these updates. The metadata log stores information on brokers, replicas, topics, and partitions, including the state of in-sync replicas and partition leadership. Kafka uses this metadata to coordinate changes and manage the cluster effectively.

Broker nodes act as observers, storing the metadata log passively to stay up-to-date with the cluster’s state. Each node fetches updates to the log independently. If you are using JBOD storage, you can change the directory that stores the metadata log.

Note

The KRaft metadata version used in the Kafka cluster must be supported by the Kafka version in use.

In the following example, a Kafka cluster comprises a quorum of controller and broker nodes for fault tolerance and high availability.

Figure 4.1. Example cluster with separate broker and controller nodes

In a typical production environment, use dedicated broker and controller nodes. However, you might want to use nodes in a dual-role configuration for development or testing.

You can use a combination of nodes that combine roles with nodes that perform a single role. In the following example, three nodes perform a dual role and two nodes act only as brokers.

Figure 4.2. Example cluster with dual-role nodes and dedicated broker nodes

4.1. Migrating to KRaft mode

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

To migrate your cluster, do as follows:

  • Install a quorum of controller nodes to replace ZooKeeper for cluster management.
  • Enable KRaft migration in the controller configuration by setting the zookeeper.metadata.migration.enable property to true.
  • Start the controllers and enable KRaft migration on the current cluster brokers using the same configuration property.
  • Perform a rolling restart of the brokers to apply the configuration changes.
  • When migration is complete, switch the brokers to KRaft mode and disable migration on the controllers.
Important

Once KRaft mode has been finalized, rollback to ZooKeeper is not possible. Carefully consider this before proceeding with the migration.

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

  • Migration is only supported on dedicated controller nodes, not on nodes with dual roles as brokers and controllers.
  • Throughout the migration process, ZooKeeper and KRaft controller nodes operate in parallel, requiring sufficient compute resources in your cluster.
Warning

If you previously rolled back a KRaft migration, ensure that all required cleanup steps were completed before attempting to migrate again.

  • The dedicated controller nodes must have been deleted.
  • The /migration znode in ZooKeeper must have been removed manually using the zookeeper-shell.sh command: delete /migration.

Failing to perform this cleanup before reattempting the migration can result in metadata loss and migration failure.

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.7 or newer with Kafka 3.7.0 or newer. If you are using an earlier version of Streams for Apache Kafka, upgrade before migrating to KRaft mode.
  • Logging is enabled to check the migration process.

    Set DEBUG level in log4j.properties for the root logger on the controllers and brokers in the cluster. For detailed migration-specific logs, set TRACE for the migration logger:

    Controller logging configuration

    log4j.rootLogger=DEBUG
    log4j.logger.org.apache.kafka.metadata.migration=TRACE
    Copy to Clipboard Toggle word wrap

Procedure

  1. Retrieve the cluster ID of your Kafka cluster.

    Use the zookeeper-shell tool:

    ./bin/zookeeper-shell.sh localhost:2181 get /cluster/id
    Copy to Clipboard Toggle word wrap

    The command returns the cluster ID.

  2. Install a KRaft controller quorum to the cluster.

    1. Configure a controller node on each host using the controller.properties file.

      At a minimum, each controller requires the following configuration:

      • A unique node ID
      • The migration enabled flag set to true
      • ZooKeeper connection details
      • Listener name used by the controller quorum
      • A quorum of controllers (dynamic is recommended)
      • Listener name for inter-broker communication

        Example controller configuration

        process.roles=controller
        node.id=1
        
        zookeeper.metadata.migration.enable=true
        zookeeper.connect=zoo1.my-domain.com:2181,zoo2.my-domain.com:2181,zoo3.my-domain.com:2181
        
        listeners=CONTROLLER://0.0.0.0:9090
        controller.listener.names=CONTROLLER
        listener.security.protocol.map=CONTROLLER:PLAINTEXT
        controller.quorum.bootstrap.servers=localhost:9090
        inter.broker.listener.name=PLAINTEXT
        Copy to Clipboard Toggle word wrap

        The format for the controller quorum is <node_id>@<hostname>:<port> in a comma-separated list. The inter-broker listener name is required for the KRaft controller to initiate the migration.

    2. Set up log directories for each controller node:

      ./bin/kafka-storage.sh format -t <uuid> -c ./config/kraft/controller.properties
      Copy to Clipboard Toggle word wrap

      Returns:

      Formatting /tmp/kraft-controller-logs
      Copy to Clipboard Toggle word wrap

      Replace <uuid> with the cluster ID you retrieved. Use the same cluster ID for each controller node in your cluster.

      By default, the log directory (log.dirs) specified in the controller.properties configuration file is set to /tmp/kraft-controller-logs. The /tmp directory is typically cleared on each system reboot, making it suitable for development environments only. Set multiple log directories using a comma-separated list, if needed.

    3. Start each controller.

      ./bin/kafka-server-start.sh -daemon ./config/kraft/controller.properties
      Copy to Clipboard Toggle word wrap
    4. Check that Kafka is running:

      jcmd | grep kafka
      Copy to Clipboard Toggle word wrap

      Returns:

      process ID kafka.Kafka ./config/kraft/controller.properties
      Copy to Clipboard Toggle word wrap

      Check the logs of each controller to ensure that they have successfully joined the KRaft cluster:

      tail -f ./logs/controller.log
      Copy to Clipboard Toggle word wrap
  3. Enable migration on each broker.

    1. If running, stop the Kafka broker running on the host.

      ./bin/kafka-server-stop.sh
      jcmd | grep kafka
      Copy to Clipboard Toggle word wrap

      If using a multi-node cluster, refer to Section 3.7, “Performing a graceful rolling restart of Kafka brokers”.

    2. Enable migration using the server.properties file.

      At a minimum, each broker requires the following additional configuration:

      • Inter-broker protocol version set to version 3.9
      • The migration enabled flag
      • Controller configuration that matches the controller nodes
      • A quorum of controllers

      Example broker configuration

      broker.id=0
      inter.broker.protocol.version=3.9
      
      zookeeper.metadata.migration.enable=true
      zookeeper.connect=zoo1.my-domain.com:2181,zoo2.my-domain.com:2181,zoo3.my-domain.com:2181
      
      listeners=CONTROLLER://0.0.0.0:9090
      controller.listener.names=CONTROLLER
      listener.security.protocol.map=CONTROLLER:PLAINTEXT
      controller.quorum.bootstrap.servers=localhost:9090
      Copy to Clipboard Toggle word wrap

      The ZooKeeper connection details should already be present.

    3. Restart the updated broker:

      ./bin/kafka-server-start.sh -daemon ./config/kraft/server.properties
      Copy to Clipboard Toggle word wrap

      The migration starts automatically and can take some time depending on the number of topics and partitions in the cluster.

    4. Check that Kafka is running:

      jcmd | grep kafka
      Copy to Clipboard Toggle word wrap

      Returns:

      process ID kafka.Kafka ./config/kraft/server.properties
      Copy to Clipboard Toggle word wrap
  4. Check the log on the active controller to confirm that the migration is complete:

    ./bin/zookeeper-shell.sh localhost:2181 get /controller
    Copy to Clipboard Toggle word wrap

    Look for an INFO log entry that says the following: Completed migration of metadata from ZooKeeper to KRaft.

  5. Switch each broker to KRaft mode.

    1. Stop the broker, as before.
    2. Update the broker configuration in the server.properties file:

      • Replace the broker.id with a node.id using the same ID
      • Add a broker KRaft role for the broker
      • Remove the inter-broker protocol version (inter.broker.protocol.version)
      • Remove the migration enabled flag (zookeeper.metadata.migration.enable)
      • Remove ZooKeeper configuration
      • Remove the listener for controller and broker communication (control.plane.listener.name)

      Example broker configuration for KRaft

      node.id=0
      process.roles=broker
      
      listeners=CONTROLLER://0.0.0.0:9090
      controller.listener.names=CONTROLLER
      listener.security.protocol.map=CONTROLLER:PLAINTEXT
      controller.quorum.bootstrap.servers=localhost:9090
      Copy to Clipboard Toggle word wrap

    3. If you are using ACLS in your broker configuration, update the authorizer using the authorizer.class.name property to the KRaft-based standard authorizer.

      ZooKeeper-based brokers use authorizer.class.name=kafka.security.authorizer.AclAuthorizer.

      When migrating to KRaft-based brokers, specify authorizer.class.name=org.apache.kafka.metadata.authorizer.StandardAuthorizer.

    4. Restart the broker, as before.
  6. Switch each controller out of migration mode.

    1. Stop the controller in the same way as the broker, as described previously.
    2. Update the controller configuration in the controller.properties file:

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

      Example controller configuration following 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=localhost:9090
      Copy to Clipboard Toggle word wrap

    3. Restart the controller in the same way as the broker, as described previously.

4.1.1. 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 migration 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, 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.
      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 you’ve begun migrating brokers to KRaft, 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.
Back to top
Red Hat logoGithubredditYoutubeTwitter

Learn

Try, buy, & sell

Communities

About Red Hat Documentation

We help Red Hat users innovate and achieve their goals with our products and services with content they can trust. Explore our recent updates.

Making open source more inclusive

Red Hat is committed to replacing problematic language in our code, documentation, and web properties. For more details, see the Red Hat Blog.

About Red Hat

We deliver hardened solutions that make it easier for enterprises to work across platforms and environments, from the core datacenter to the network edge.

Theme

© 2025 Red Hat