fetch from follower causes frequent rebalances on rolling restarts
Description
When using cooperative sticky with fetch from follower, metadata changes during a rolling restart are triggering frequent rebalances. This seems to be related to KIP-881's behavior to rebalance when the set of racks changes. However, no reassignment is being performed and I'm only restarting the cluster.
%7|1717443488.390|METADATA|rdkafka#consumer-1| [thrd:main]: 10.132.158.32:9092/10005: 1/1 requested topic(s) seen in metadata
%7|1717443488.390|SUBSCRIPTION|rdkafka#consumer-1| [thrd:main]: Group "kphelps-rdkafka-test": effective subscription list changed from 1 to 1 topic(s):
%7|1717443488.390|SUBSCRIPTION|rdkafka#consumer-1| [thrd:main]: Topic loadgen with 32 partition(s)
%7|1717443488.390|REJOIN|rdkafka#consumer-1| [thrd:main]: Group "kphelps-rdkafka-test": subscription updated from metadata change: rejoining group in state steady
%7|1717443488.390|REJOIN|rdkafka#consumer-1| [thrd:main]: Group "kphelps-rdkafka-test": Rejoining group with 6 owned partition(s): Metadata for subscribed topic(s) has changed
How to reproduce
Setup a consumer group that is using fetch-from-follower (ie, set client.rack) and cooperative sticky assignment. Rolling restart the cluster. The consumer group will trigger rebalances when the metadata changes.
It may be reproducible with other assignment strategies, but I have not tested that yet.
Checklist
IMPORTANT: We will close issues where the checklist has not been completed.
Please provide the following information:
- [x] librdkafka version (release number or git tag):
2.3.0and2.4.0both tested - [x] Apache Kafka version:
3.7.0 - [x] librdkafka client configuration:
client.rackandpartition.assignment.strategy = cooperative-sticky - [x] Operating system:
ubuntu 20.04 - [x] Provide logs (with
debug=..as necessary) from librdkafka - [ ] Provide broker log excerpts
- [ ] Critical issue
@kphelps In case partition racks have changed the rack aware partitioner is run again.
Before the log effective subscription list changed from 1 to 1 topic(s):
There's this code:
/* Compare to existing to see if anything changed. */
if (!rd_list_cmp(rkcg->rkcg_subscribed_topics, tinfos,
rd_kafka_topic_info_cmp)) {
/* No change */
rd_list_destroy(tinfos);
return rd_false;
}
The topic is exactly the same if it
- has the same name
- same number of partitions
- each partition has the same set of racks as previously, depending on the replicas
In case of a rolling restart a broker is not reported by the metadata request and that can change the list of reported racks. Seems like when a broker is missing we could avoid considering the partition racks changed. Have to check what the Java client is doing.
It seems Java is doing the same for comparing MetadataSnapshot as here it compares a Map with a list of PartitionRackInfo.
https://github.com/apache/kafka/blob/80f31224aad543dbfc892bce1ad73b6bb693855a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java#L1548
and then when PartitionRackInfo is compared, it checks the Set of racks
https://github.com/apache/kafka/blob/80f31224aad543dbfc892bce1ad73b6bb693855a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java#L1599
at that point when a change is detected it's requesting a rejoin here
https://github.com/apache/kafka/blob/80f31224aad543dbfc892bce1ad73b6bb693855a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java#L883
this just by checking the code but have to see if the problem happens there too.
Reproduced the same issue in Java client and opened this issue: https://issues.apache.org/jira/browse/KAFKA-17237