Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

fetch from follower causes frequent rebalances on rolling restarts #4742

Open
5 of 7 tasks
kphelps opened this issue Jun 4, 2024 · 1 comment
Open
5 of 7 tasks

fetch from follower causes frequent rebalances on rolling restarts #4742

kphelps opened this issue Jun 4, 2024 · 1 comment

Comments

@kphelps
Copy link

kphelps commented Jun 4, 2024

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:

  • librdkafka version (release number or git tag): 2.3.0 and 2.4.0 both tested
  • Apache Kafka version: 3.7.0
  • librdkafka client configuration: client.rack and partition.assignment.strategy = cooperative-sticky
  • Operating system: ubuntu 20.04
  • Provide logs (with debug=.. as necessary) from librdkafka
  • Provide broker log excerpts
  • Critical issue
@kphelps kphelps changed the title cooperative sticky with fetch from follower causes frequent rebalances on rolling restarts fetch from follower causes frequent rebalances on rolling restarts Jun 4, 2024
@emasab
Copy link
Collaborator

emasab commented Jun 11, 2024

@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.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

No branches or pull requests

2 participants