This is an automated email from the ASF dual-hosted git repository.

lianetm pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 9f9836c0c4a KAFKA-20376: Update consumer rebalance docs (#21930)
9f9836c0c4a is described below

commit 9f9836c0c4a09910821fe0572a98d6eb0187a87e
Author: Lianet Magrans <[email protected]>
AuthorDate: Tue Apr 7 11:47:59 2026 -0400

    KAFKA-20376: Update consumer rebalance docs (#21930)
    
    Adding more info about assignors, evolution  timeline. Updating existing
    info with latest state.
    
    Reviewers: David Jacot <[email protected]>, Andrew Schofield
     <[email protected]>
---
 docs/operations/consumer-rebalance-protocol.md | 49 ++++++++++++++++++--------
 1 file changed, 35 insertions(+), 14 deletions(-)

diff --git a/docs/operations/consumer-rebalance-protocol.md 
b/docs/operations/consumer-rebalance-protocol.md
index 97ac44605ad..2ee2b7c20d2 100644
--- a/docs/operations/consumer-rebalance-protocol.md
+++ b/docs/operations/consumer-rebalance-protocol.md
@@ -28,13 +28,14 @@ type: docs
 
 ## Overview
 
-Starting from Apache Kafka 4.0, the Next Generation of the Consumer Rebalance 
Protocol ([KIP-848](https://cwiki.apache.org/confluence/x/HhD1D)) is Generally 
Available (GA). It improves the scalability of consumer groups while 
simplifying consumers. It also decreases rebalance times, thanks to its fully 
incremental design, which no longer relies on a global synchronization barrier.
+Starting from Apache Kafka 4.0, the Next Generation of the Consumer Rebalance 
Protocol ([KIP-848](https://cwiki.apache.org/confluence/x/HhD1D)) is Generally 
Available (GA), ready for production workloads. 
+It improves the scalability of consumer groups while simplifying consumers. It 
also decreases rebalance times, thanks to its fully incremental design, which 
no longer relies on a global synchronization barrier.
 
-Consumer Groups using the new protocol are now referred to as `Consumer` 
groups, while groups using the old protocol are referred to as `Classic` 
groups. Note that Classic groups can still be used to form consumer groups 
using the old protocol.
+There are now two types of consumer groups, named depending upon whether the 
consumers are using the new "consumer" group protocol or the earlier "classic" 
group protocol.
 
 ## Server
 
-The new consumer protocol is automatically enabled on the server since Apache 
Kafka 4.0. Enabling and disabling the protocol is controlled by the 
`group.version` feature flag.
+The new consumer protocol is automatically enabled on the server since Apache 
Kafka 4.0. Enabling and disabling the protocol on the server side is controlled 
by the `group.version` feature flag.
 
 The consumer heartbeat interval and the session timeout are controlled by the 
server now with the following configs:
 
@@ -43,15 +44,28 @@ The consumer heartbeat interval and the session timeout are 
controlled by the se
 
 
 
-The assignment strategy is also controlled by the server. The 
`group.consumer.assignors` configuration can be used to specify the list of 
available assignors for `Consumer` groups. By default, the `uniform` assignor 
and the `range` assignor are configured. The first assignor in the list is used 
by default unless the Consumer selects a different one. It is also possible to 
implement custom assignment strategies on the server side by implementing the 
`ConsumerGroupPartitionAssignor` interf [...]
+The assignment strategy is also controlled by the server. The 
`group.consumer.assignors` configuration can be used to specify the list of 
available assignors for `Consumer` groups. 
+* `uniform` and `range` assignors are provided by default
+* `uniform` is the default one (first assignor in the list of 
`group.consumer.assignors`) unless the Consumer selects a different one (via 
the client config `group.remote.assignor`)
+* it is possible to implement custom assignment strategies on the server side, 
by implementing the `ConsumerGroupPartitionAssignor` interface and specifying 
the full class name in the `group.consumer.assignors` configuration.
 
-## Consumer
+### Migrating from Client-side Assignors
+
+The following table shows the mapping from client-side assignors to the new 
server-side assignors:
+
+| Client-side Assignor      | Server-side Assignor |
+|---------------------------|----------------------|
+| RangeAssignor             | range                |
+| CooperativeStickyAssignor | uniform              |
+| StickyAssignor            | uniform              |
+| RoundRobinAssignor        | uniform              |
 
-Since Apache Kafka 4.0, the Consumer supports the new consumer rebalance 
protocol. However, the protocol is not enabled by default. The `group.protocol` 
configuration must be set to `consumer` to enable it. When enabled, the new 
consumer protocol is used alongside an improved threading model.
 
-The `group.remote.assignor` configuration is introduced as an optional 
configuration to overwrite the default assignment strategy configured on the 
server side.
+## Consumer
+
+Since Apache Kafka 4.0, the Consumer fully supports the new Consumer rebalance 
protocol. However, the protocol is not enabled by default. The `group.protocol` 
configuration must be set to `consumer` to enable it. When enabled, the new 
consumer protocol is used alongside an improved threading model.
 
-The `subscribe(SubscriptionPattern)` and `subscribe(SubscriptionPattern, 
ConsumerRebalanceListener)` methods have been added to subscribe to a regular 
expression with the new consumer rebalance protocol. With these methods, the 
regular expression uses the RE2J format and is now evaluated on the server side.
+The `subscribe(SubscriptionPattern)` and `subscribe(SubscriptionPattern, 
ConsumerRebalanceListener)` methods have been added to subscribe to a regular 
expression with the new Consumer rebalance protocol. With these methods, the 
regular expression uses the RE2J format and is now evaluated on the server side.
 
 New metrics have been added to the Consumer when using the new rebalance 
protocol, mainly providing visibility over the improved threading model. See 
[New Consumer Metrics](https://cwiki.apache.org/confluence/x/lQ_TEg).
 
@@ -72,15 +86,22 @@ Consumer groups are automatically converted from `Classic` 
to `Consumer` and vic
 
 ### Online
 
-Consumer groups can be upgraded without downtime by rolling out the consumer 
with the `group.protocol=consumer` configuration. When the first consumer using 
the new consumer rebalance protocol joins the group, the group is converted 
from `Classic` to `Consumer`, and the classic rebalance protocol is 
interoperated to work with the new consumer rebalance protocol. This is only 
possible when the classic group uses an assignor that does not embed custom 
metadata.
+Consumer groups can be upgraded without downtime by rolling out the consumer 
with the `group.protocol=consumer` configuration. When the first consumer using 
the new Consumer rebalance protocol joins the group, the group is converted 
from `Classic` to `Consumer`, and the Classic rebalance protocol is 
interoperated to work with the new Consumer rebalance protocol. This is only 
possible when the classic group uses an assignor that does not embed custom 
metadata.
 
-Consumer groups can be downgraded using the opposite process. In this case, 
the group is converted from `Consumer` to `Classic` when the last consumer 
using the new consumer rebalance protocol leaves the group.
+Consumer groups can be downgraded using the opposite process. In this case, 
the group is converted from `Consumer` to `Classic` when the last consumer 
using the new Consumer rebalance protocol leaves the group.
 
-## Limitations
+## Evolution Timeline
+
+The evolution timeline of the Consumer rebalance protocol is described in 
[KIP-1274](https://cwiki.apache.org/confluence/display/KAFKA/KIP-1274%3A+Deprecate+and+remove+support+for+Classic+rebalance+protocol+in+KafkaConsumer),
 and expected to be as follows:
 
-While the new consumer rebalance protocol works for most use cases, it is 
still important to be aware of the following limitations:
+  * **Apache Kafka 3.7**: Early Access
+  * **Apache Kafka 4.0**: GA (production-ready).
+  * **Apache Kafka 5.0**: `KafkaConsumer` defaults to `Consumer` protocol, 
while still supporting `Classic` 
+  * **Apache Kafka 6.0**: `KafkaConsumer` only supports `Consumer` as 
rebalance protocol, while the broker still supports `Classic` for backward 
compatibility.
+
+## Limitations
 
-  * Client-side assignors are not supported. (see 
[KAFKA-18327](https://issues.apache.org/jira/browse/KAFKA-18327))
-  * Rack-aware assignment strategies are not fully supported. (see 
[KAFKA-17747](https://issues.apache.org/jira/browse/KAFKA-17747))
+  * Client-side assignors are not supported and not in scope at the moment. 
Use [KAFKA-18327](https://issues.apache.org/jira/browse/KAFKA-18327) to provide 
feedback if you have custom assignment strategies that may not be covered.
+  * Rack-aware assignment strategies are not fully supported yet (work is in 
progress, see [KAFKA-19387](https://issues.apache.org/jira/browse/KAFKA-19387)).
 
 

Reply via email to