Dieser Inhalt ist in der von Ihnen ausgewählten Sprache nicht verfügbar.

Chapter 14. Scaling clusters by adding or removing nodes


Kafka clusters can be scaled by adding or removing nodes, which serve as brokers, controllers, or both. In production, broker and controller roles are typically separated.

14.1. Scaling brokers

Adding brokers improves performance and reliability by increasing available resources, enabling larger workloads, and enhancing fault tolerance through more replicas. Removing underutilized brokers optimizes resource consumption and efficiency.

Operations for scaling brokers:

Adding brokers
Set up the new broker node, making sure its storage is formatted using the existing Cluster ID. When started, rebalance partitions to distribute load onto the new broker.
Removing brokers
Before shutting down brokers, rebalance partitions to migrate their data to other brokers in the cluster to prevent data loss.

Cruise Control can automate partition reassignments:

  • add_broker mode redistributes partitions to new brokers.
  • remove_broker mode moves partitions off brokers marked for removal.
Note

Broker scaling or partition scaling? Increasing partitions can enhance topic throughput by distributing workload, but if brokers are constrained, such as by I/O limitations, adding partitions alone won’t help. Instead, more brokers are needed.

The number of brokers impacts replication settings. Choose values based on your fault tolerance requirements. For example, a replication factor of 3 ensures that each partition is replicated across three brokers, helping maintain availability during failures. For data durability, a minimum in-sync value of 2 requires at least two replicas acknowledge a write of the partition data.

Example replica configuration

default.replication.factor = 3
min.insync.replicas = 2

14.2. Adding new brokers

To add a new broker to a Kafka cluster, configure the broker, start it, and rebalance partition replicas to distribute load evenly across all brokers.

Prerequisites

  • Streams for Apache Kafka is installed on the host, and the configuration files and tools are available.
    This procedure uses kafka-server-start.sh and Cruise Control for rebalancing.
  • Administrative access to the broker nodes.

For more information on using Cruise Control, see Chapter 15, Using Cruise Control for cluster rebalancing.

Procedure

  1. Configure the new broker using a broker.properties file.

    At a minimum, the broker requires the following configuration:

    • A unique node ID
    • A valid listener configuration
    • Bootstrap information for the KRaft controller quorum

      Example broker configuration

      process.roles=broker
      node.id=5
      
      listeners=PLAINTEXT://0.0.0.0:9094
      listener.security.protocol.map=PLAINTEXT:PLAINTEXT
      
      log.dirs=/var/lib/kafka/data
      controller.quorum.bootstrap.servers=node1:9093,node2:9093,node3:9093

  2. Format the broker:

    ./bin/kafka-storage.sh format --cluster-id <uuid> --config ./config/broker.properties --no-initial-controllers

    Use the appropriate cluster ID.

  3. Start the broker:

    ./bin/kafka-server-start.sh ./config/broker.properties
  4. Check the logs of the broker to ensure that is has successfully joined the KRaft cluster:

    tail -f ./logs/server.log
  5. Rebalance partitions to include the new broker using Cruise Control’s add_broker mode:

    curl -v -X POST 'http://<cruise_control_-_host>:9090/kafkacruisecontrol/add_broker?brokerid=5&dryrun=false'
    Tip

    Check the rebalance with dryrun=true first to review the proposed changes before running the command with dryrun=false.

  6. Monitor the rebalance progress using the Cruise Control /user_tasks endpoint:

    curl 'http://<cruise_control_host>:9090/kafkacruisecontrol/user_tasks'

    Wait for the rebalance to complete.

  7. Verify that the broker has been added to the active cluster:

    kafka-cluster.sh --bootstrap-server <kafka_host>:9092 --describe

    This displays the list of brokers and their status. <kafka_host> must be an active broker in the cluster with a known, accessible listener port.

14.3. Removing brokers

To remove a broker from a Kafka cluster, rebalance partition replicas off the broker, shut it down, and remove it from service.

Prerequisites

  • Streams for Apache Kafka is installed on the host, and the configuration files and tools are available.
    This procedure uses Cruise Control for rebalancing and assumes the broker was originally configured using a broker.properties file.
  • Administrative access to the broker nodes.
  • The node ID of the broker to be removed is known.
    In this procedure, we remove the node with ID 5.

For more information on using Cruise Control, see Chapter 15, Using Cruise Control for cluster rebalancing.

Procedure

  1. Rebalance partitions off the broker that you want to remove using Cruise Control’s remove_broker mode:

    curl -X POST 'http://<cruise_control_-_host>:9090/kafkacruisecontrol/remove_broker?brokerid=5&dryrun=false'
    Tip

    Check the rebalance with dryrun=true first to review the proposed changes before running the command with dryrun=false.

  2. Monitor the rebalance progress using the Cruise Control /user_tasks endpoint:

    curl 'http://<cruise_control_host>:9090/kafkacruisecontrol/user_tasks'

    Wait for the rebalance to complete.

  3. Shut down the broker:

    ./bin/kafka-server-stop.sh
  4. Unregister the broker to remove it from the cluster metadata.

    For more information on unregistering, see Section 14.4, “Unregistering brokers after scale-down operations”.

  5. Verify that the broker has been removed from the active cluster:

    kafka-cluster.sh --bootstrap-server <kafka_host>:9092 --describe

    This displays the list of brokers and their status. <kafka_host> must be an active broker in the cluster with a known, accessible listener port.

14.4. Unregistering brokers after scale-down operations

After removing a broker from a Kafka cluster, use the kafka-cluster.sh script to unregister the broker from the cluster metadata. Failing to unregister removed nodes leads to stale metadata, which causes operational issues.

Prerequisites

Before unregistering a broker, ensure the following tasks are completed:

  1. Reassign the partitions from the broker you plan to remove to the remaining brokers using the Cruise control remove-nodes operation.
  2. Update the cluster configuration, if necessary, to adjust the replication factor for topics (default.replication.factor) and the minimum required number of in-sync replica acknowledgements (min.insync.replicas).
  3. Stop the Kafka broker service on the broker and remove the broker from the cluster.

Procedure

  1. Unregister the removed broker from the cluster:

    ./bin/kafka-cluster.sh unregister \
      --bootstrap-server <broker_host>:<port> \
      --id <node_id_number>
  2. Verify the current state of the cluster by describing the topics:

    ./bin/kafka-topics.sh \
      --bootstrap-server <broker_host>:<port> \
      --describe

14.5. Scaling controller quorums dynamically

Dynamic controller quorums support scaling without requiring system downtime. Dynamic scaling is useful not only for adding or removing controllers, but supports the following:

  • Replacing controllers because of hardware failure
  • Migrating clusters onto new machines
  • Moving nodes from dedicated controller roles to combined roles or vice versa

A dynamic quorum is specified in the controller configuration using the controller.quorum.bootstrap.servers property to list host:port endpoints for each controller. Only one controller can be added or removed from the cluster at a time, so complex quorum changes are implemented as a series of single changes. New controllers join as observers, replicating the metadata log but not counting towards the quorum. When caught up with the active controller, the new controller is eligible to join the quorum.

When removing controllers, it’s recommended that they are first shutdown to avoid unnecessary leader elections. If the removed controller is the active one, it will step down from the quorum only after the new quorum is confirmed. However, it will not include itself when calculating the last commit position in the __cluster_metadata log.

In a dynamic quorum, the active Kraft version is at 1 or above for all cluster nodes. Find the active KRaft version using the kafka-features.sh tool:

./bin/kafka-features.sh --bootstrap-controller localhost:9093 describe | grep kraft.version

In this example output, the active version (FinalizedVersionLevel) in the Kafka cluster is 1:

Feature: kraft.version  SupportedMinVersion: 0  SupportedMaxVersion: 1  FinalizedVersionLevel: 1 Epoch: 5

If the kraft.version property shows an active version level of 0 or is absent, you are using a static quorum. If it is 1 or above, you are using a dynamic quorum.

Note

It’s possible to configure a static quorum, but it is not a recommended approach as it requires downtime when scaling.

14.6. Adding new controllers

To add a new controller to an existing dynamic controller quorum in Kafka, create a new controller, monitor its replication status, and then integrate it into the cluster.

Prerequisites

  • Streams for Apache Kafka is installed on the host, and the configuration files and tools are available.
    This procedure uses the kafka-storage.sh, kafka-server-start.sh and kafka-metadata-quorum.sh tools.
  • Administrative access to the controller nodes.

Procedure

  1. Configure a new controller node using the controller.properties file.

    At a minimum, the new controller requires the following configuration:

    • A unique node ID
    • Listener name used by the controller quorum
    • A quorum of controllers

      Example controller configuration

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

      The controller.quorum.bootstrap.servers configuration includes the host and port of the new controller and each other controller already present in the cluster.

      Important

      If there is any further configuration required, such as authentication, make sure to include it in the controller.properties.

  2. Update controller.quorum.bootstrap.servers in the configuration of each node in the cluster with the host and port of the new controller.
  3. Set the log directory ID for the new controller:

    ./bin/kafka-storage.sh format --cluster-id <cluster_id> --config controller.properties --no-initial-controllers

    By using the no-initial-controllers option, the controller is initialized without it joining the controller quorum.

  4. Start the controller node

    ./bin/kafka-server-start.sh ./config/controller.properties
  5. Monitor the replication progress of the new controller:

    ./bin/kafka-metadata-quorum.sh --bootstrap-server localhost:9092 describe --replication

    Wait until the new controller has caught up with the active controller before proceeding.

  6. Add the new controller to the controller quorum:

    ./bin/kafka-metadata-quorum.sh --command-config controller.properties --bootstrap-controller localhost:9092 add-controller

14.7. Removing controllers

To remove a controller from an existing dynamic controller quorum in Kafka, use the kafka-metadata-quorum.sh tool.

Prerequisites

  • Streams for Apache Kafka is installed on the host, and the configuration files and tools are available.
    This procedure uses the kafka-server-stop.sh and kafka-metadata-quorum.sh tools.
  • Administrative access to the controller nodes.

Procedure

  1. Stop the controller node:

    ./bin/kafka-server-stop.sh
  2. Locate the ID of the controller and its directory ID to be able to remove it from the controller quorum. You can find this information in the meta.properties file of the metadata log.
  3. Remove the controller from the controller quorum:

    ./bin/kafka-metadata-quorum.sh --bootstrap-controller localhost:9092 remove-controller --controller-id <id> --controller-directory-id <directory_id>
  4. Update controller.quorum.bootstrap.servers in the configuration of each node in the cluster to remove the host and port of the controller removed from the controller quorum.
Red Hat logoGithubredditYoutubeTwitter

Lernen

Testen, kaufen und verkaufen

Communitys

Über Red Hat Dokumentation

Wir helfen Red Hat Benutzern, mit unseren Produkten und Diensten innovativ zu sein und ihre Ziele zu erreichen – mit Inhalten, denen sie vertrauen können. Entdecken Sie unsere neuesten Updates.

Mehr Inklusion in Open Source

Red Hat hat sich verpflichtet, problematische Sprache in unserem Code, unserer Dokumentation und unseren Web-Eigenschaften zu ersetzen. Weitere Einzelheiten finden Sie in Red Hat Blog.

Über Red Hat

Wir liefern gehärtete Lösungen, die es Unternehmen leichter machen, plattform- und umgebungsübergreifend zu arbeiten, vom zentralen Rechenzentrum bis zum Netzwerkrand.

Theme

© 2026 Red Hat
Nach oben