[Live Workshop] Streams on Tour: Hands-On Deep Dive into Confluent | Register Now

Introducing KIP-848: The Next Generation of the Consumer Rebalance Protocol

Écrit par

Consumer Groups and the Rebalance Protocol

The consumer group is a cornerstone of Apache Kafka®, enabling scalable and fault-tolerant data consumption by allowing multiple consumer instances to share the workload of reading from topic partitions. The consumer rebalance protocol is the mechanism that coordinates which partitions are assigned to which consumers within a group.

Historically, Kafka has relied on what we now call the "classic" rebalance protocol. This protocol evolved, as it was initially dominated by the "eager" assignment strategy. Eager rebalancing worked on a stop-the-world principle: Any change in group membership (consumer joining/leaving) or topic metadata triggered a complete halt. All consumers revoked their partitions, a leader computed a new assignment, and partitions were redistributed before processing could resume. This caused significant downtime, especially in dynamic environments.

To mitigate this, the cooperative assignment strategy was introduced within the classic protocol. Cooperative rebalancing reduced downtime by allowing consumers to keep partitions unaffected by the rebalance, revoking only those needing reassignment. While an improvement, the classic protocol, even with cooperative assignment, still relied on a group-wide synchronization barrier and client-driven logic. Rebalances could involve multiple communication rounds, introduce delays, and increase operational complexity, particularly for large groups with many partitions. These delays and complexities became significant pain points for applications demanding high availability and low latency.

Addressing these challenges head-on, KIP-848 introduces a fundamentally new, optional consumer rebalance protocol, built on lessons learned and designed for modern, large-scale stream processing.

The New Consumer Rebalance Protocol (KIP-848)

Available in Apache Kafka 4.0, Confluent Cloud, and Confluent Platform 8.0, the KIP-848 protocol redesigns the consumer-broker interaction for rebalancing. KIP-848, with its core innovations, is a truly incremental/async protocol that shifts coordination logic from the client to the broker-side group coordinator.

Summary of Key Differences

The key processing differences between classic and consumer is summarized below.

Classic - Eager

Classic - Cooperative

Consumer

Application Impact

Full stop-the-world events occur

Partial stop-the-world consumer lag builds because commit cannot progress

No stop–the-world pause

No impact to consumers  with partitions not included in shuffling

Fetch Processing

Paused during rebalance processing

Able to progress during rebalance processing

Able to progress during rebalance processing

Commit Processing

Paused during rebalance processing

Paused during rebalance processing

Able to progress during rebalance processing

Consumer Impact

All consumers in a group are impacted

All consumers in a group are impacted

Select consumers in a group are impacted

How It Works: Server-Driven Reconciliation

Instead of clients coordinating through multiple JoinGroup/SyncGroup phases, the new protocol uses a continuous heartbeat mechanism and a server-driven reconciliation process.

  1. Declarative state: Consumers declare their subscriptions and acknowledge partition assignments/revocations via the heartbeat mechanism. They don't compute assignments themselves (unless using the extended protocol).

  2. Coordinator orchestration: The group coordinator becomes the central intelligence. It maintains group membership, monitors topic metadata (including handling wildcard subscriptions), computes the target assignment using a server-side assignor (pluggable, with range and uniform assignors provided by default), and orchestrates the transition toward this target state.

  3. Incremental reconciliation: The coordinator drives rebalances incrementally. It compares the current state (partitions owned by each consumer) with the target assignment. It then issues specific revoke/assign commands to individual consumers via the heartbeat response.

  4. No global synchronization: Crucially, there's no group-wide stop-the-world pause. A consumer pauses processing only for the specific partitions it's asked to revoke. Other consumers continue processing unaffected partitions. The coordinator manages dependencies, ensuring that a partition is fully revoked by one consumer before being assigned to another. Consumers move through assignment “epochs” independently as they complete revocations.

  5. Server-side configuration: Key parameters such as session timeout (group.consumer.session.timeout.ms) and heartbeat interval (group.consumer.heartbeat.interval.ms) are now configured on the server dynamically per group, simplifying client setup. The client provides a rebalance.timeout.ms (based on max.poll.interval.ms) indicating how long it needs to revoke partitions and an optional group.remote.assignor to specify the server-side assigner to use.

Key Benefits of KIP-848

  • Reduced/eliminated consumer downtime: Incremental changes minimize the stop-the-world effect, often eliminating pauses entirely for consumers keeping processing online.

  • Faster rebalances: Server-driven, incremental changes are significantly faster than the classic protocol. Multi-threaded processing on the broker further parallelizes large group rebalances.

  • Improved stability and scalability: The protocol is more resilient, and shifting complexity to the server makes clients lighter and easier to implement and manage. Broker-driven logic scales better with large numbers of partitions and consumers.

  • Simplified management: Centralized, server-side configuration streamlines setup.

  • Enhanced monitoring: New metrics provide better visibility into rebalance performance in tools such as Confluent Control Center and the Confluent Cloud user interface.

  • Foundation for future enhancements: KIP-848 supports existing assignors with performance improvements and the ability to extend the new server-side assignor implementation, and it paves the way for extensions of the protocol, such as KIP-1071, the Kafka Streams rebalance protocol. Note: Rack-aware strategies are not yet supported (see KAFKA-17747).

Adopting the New Protocol: An Easy Upgrade

Upgrading to the new consumer rebalance protocol is straightforward:

  1. Ensure cluster support: Your cluster must run Apache Kafka 4.0+ or Confluent Platform 8.0+ or must be on Confluent Cloud.

  2. Upgrade clients: Update your consumer applications to use Kafka client libraries that are compatible with the new protocol (e.g., Java clients for Apache Kafka 4.0+). Note: For librdkafka-based clients, KIP-848 is in Early Access with librdkafka 2.10.

  3. Configure consumers: Set the consumer configuration group.protocol=consumer

  4. Migrate consumer configs: Migrate consumer configs to remove unsupported properties now moved to the broker (partition.assignment.strategy, session.timeout.ms, and heartbeat.interval.ms).

The protocol supports live upgrades or offline upgrades. Live upgrades can be performed via a rolling restart of consumer instances. When the first consumer configured with group.protocol=consumer joins an existing classic group, the group coordinator automatically transitions the group. It internally proxies the classic protocol (join/sync/heartbeat) requests from older clients to work seamlessly alongside the new protocol messages from upgraded clients. This is possible if the classic group uses a standard assignor without custom metadata. (Note: Mixing consumer rebalance protocols in the same consumer group is intended only for temporary scenarios to support live upgrades.) 

What's Next?

The new consumer rebalance protocol is Generally Available in Apache Kafka, Confluent Cloud and (soon) Confluent Platform 8.0. KIP-848 is also in Early Access in librdkafka-based clients with librdkafka 2.10. Future enhancements include client-side performance improvements and full topic ID support when retrieving and committing offsets. Work is also underway to bring these benefits to Kafka Streams via KIP-1071. Kafka Connect and Schema Registry are also expected to adopt this protocol as the default in the future.

Given the substantial benefits it provides, the new KIP-848 protocol will become the default rebalance protocol in a future Apache Kafka release, likely version 5.0.

Conclusion

The KIP-848 consumer rebalance protocol represents a significant leap forward for Apache Kafka, as it addresses key limitations of the classic protocol. By shifting coordination to the broker and enabling incremental, server-driven reconciliation, it delivers faster rebalances and more stable and scalable groups. This enhancement reduces operational overhead and improves the availability and performance of Kafka applications. 

We encourage you to try the new protocol in your development environments on Confluent Cloud or compatible platforms and share your feedback!

‎ 

Apache®, Apache Kafka®, and Kafka® are registered trademarks of the Apache Software Foundation in the United States and/or other countries. No endorsement by the Apache Software Foundation is implied by using these marks. All other trademarks are the property of their respective owners.

  • Jonathan Lacefield is a Group Product Manager at Confluent with 20 years of experience leading product innovation in cloud and data technologies with a focus on Apache projects. He is passionate about helping developers solve complex, distributed problems through open source technology and translating intricate technical concepts into highly loved products.

Avez-vous aimé cet article de blog ? Partagez-le !