Open mensfeld opened 1 month ago
Ok I can reproduce it. As long as I have same consumer group different consumers subscribed to different topics and try to upscale, the assignor gets stuck in this loop. When I make the topics use different CGs, it no longer hangs.
I am not sure at the moment whether this is a multi-process issue or an issue that only affects a single process (some state shared?, name collision?) setup with multiple librdkafka consumers subscribed to multiple topics within the same consumer group. Will continue the investigation.
Here are some extensive debug logs. The moment I see the 100% CPU spike I kill -9 the whole process. I cannot however pinpoint (yet) where it starts in the logs exactly.
What is also interesting is that it is responding (I can poll data) but it hangs when I attempt to close it.
Kafka logs from the moment of the hanging assignment:
kafka | [2024-07-17 12:00:52,235] INFO [GroupCoordinator 1]: Dynamic member with unknown member id joins group eabcf19b-efb8-4f5f-8598-faf7cacb4dd1 in PreparingRebalance state. Created a new member id karafka_production-87f747f2-f652-4df6-b05c-51c380f2f4ec and request the member to rejoin with this id. (kafka.coordinator.group.GroupCoordinator)
kafka | [2024-07-17 12:00:56,238] INFO [GroupCoordinator 1]: Stabilized group eabcf19b-efb8-4f5f-8598-faf7cacb4dd1 generation 1 (__consumer_offsets-5) with 12 members (kafka.coordinator.group.GroupCoordinator)
kafka | [2024-07-17 12:00:56,240] INFO [GroupCoordinator 1]: Assignment received from leader karafka_production-20dfb758-1b0f-49ba-aa40-f3cfe97c1593 for group eabcf19b-efb8-4f5f-8598-faf7cacb4dd1 for generation 1. The group has 12 members, 0 of which are static. (kafka.coordinator.group.GroupCoordinator)
kafka | [2024-07-17 12:00:57,236] INFO [GroupCoordinator 1]: Dynamic member with unknown member id joins group eabcf19b-efb8-4f5f-8598-faf7cacb4dd1 in Stable state. Created a new member id karafka_production-299aaadc-80d4-4852-a022-e9370c1abf9f and request the member to rejoin with this id. (kafka.coordinator.group.GroupCoordinator)
kafka | [2024-07-17 12:00:57,236] INFO [GroupCoordinator 1]: Preparing to rebalance group eabcf19b-efb8-4f5f-8598-faf7cacb4dd1 in state PreparingRebalance with old generation 1 (__consumer_offsets-5) (reason: Adding new member karafka_production-299aaadc-80d4-4852-a022-e9370c1abf9f with group instance id None; client reason: not provided) (kafka.coordinator.group.GroupCoordinator)
kafka | [2024-07-17 12:00:59,243] INFO [GroupCoordinator 1]: Stabilized group eabcf19b-efb8-4f5f-8598-faf7cacb4dd1 generation 2 (__consumer_offsets-5) with 13 members (kafka.coordinator.group.GroupCoordinator)
kafka | [2024-07-17 12:00:59,244] INFO [GroupCoordinator 1]: Assignment received from leader karafka_production-20dfb758-1b0f-49ba-aa40-f3cfe97c1593 for group eabcf19b-efb8-4f5f-8598-faf7cacb4dd1 for generation 2. The group has 13 members, 0 of which are static. (kafka.coordinator.group.GroupCoordinator)
kafka | [2024-07-17 12:00:59,245] INFO [GroupCoordinator 1]: Preparing to rebalance group eabcf19b-efb8-4f5f-8598-faf7cacb4dd1 in state PreparingRebalance with old generation 2 (__consumer_offsets-5) (reason: Updating metadata for member karafka_production-06a7ceb2-c734-4904-b002-acbbdfcbcf68 during Stable; client reason: not provided) (kafka.coordinator.group.GroupCoordinator)
kafka | [2024-07-17 12:01:02,237] INFO [GroupCoordinator 1]: Dynamic member with unknown member id joins group eabcf19b-efb8-4f5f-8598-faf7cacb4dd1 in PreparingRebalance state. Created a new member id karafka_production-befd7019-b4e1-495d-9449-430659f20326 and request the member to rejoin with this id. (kafka.coordinator.group.GroupCoordinator)
kafka | [2024-07-17 12:01:02,246] INFO [GroupCoordinator 1]: Stabilized group eabcf19b-efb8-4f5f-8598-faf7cacb4dd1 generation 3 (__consumer_offsets-5) with 14 members (kafka.coordinator.group.GroupCoordinator)
kafka | [2024-07-17 12:01:07,239] INFO [GroupCoordinator 1]: Dynamic member with unknown member id joins group eabcf19b-efb8-4f5f-8598-faf7cacb4dd1 in CompletingRebalance state. Created a new member id karafka_production-f47116b2-d13c-4466-a6c2-83c49edf6cbc and request the member to rejoin with this id. (kafka.coordinator.group.GroupCoordinator)
kafka | [2024-07-17 12:01:07,239] INFO [GroupCoordinator 1]: Preparing to rebalance group eabcf19b-efb8-4f5f-8598-faf7cacb4dd1 in state PreparingRebalance with old generation 3 (__consumer_offsets-5) (reason: Adding new member karafka_production-f47116b2-d13c-4466-a6c2-83c49edf6cbc with group instance id None; client reason: not provided) (kafka.coordinator.group.GroupCoordinator)
what is crazy, is that when I use a random client id that is time based, it does not hang. So I suspect it is somehow related to metadata requests. I can not reproduce it within Karafka always.
What is SUPER interesting to me, is the fact that this does not happen if I set client.id
as follows (timestamps):
"1721229018.5395756"
"1721229018.539616"
"1721229018.539644"
"1721229018.5396647"
"1721229018.5396793"
"1721229018.5397322"
"1721229023.5395145"
"1721229028.5397005"
"1721229033.5398915"
"1721229038.5401068"
"1721229043.5417292"
"1721229048.5419412"
"1721229053.5420904"
"1721229058.5422208"
"1721229063.5424514"
"1721229068.5425892"
"1721229073.5427814"
"1721229078.5429163"
"1721229083.54326"
"1721229088.5432706"
"1721229093.5435174"
"1721229098.5436523"
"1721229103.5438313"
"1721229108.544021"
"1721229113.5441961"
"1721229118.5444007"
"1721229123.5445986"
"1721229128.5449526"
"1721229133.5449734"
"1721229138.5451102"
but when I use a constant client.id
or a random string (like uuid), it fails :man_shrugging:
One more update: I wanted to check if this isn't a Kafka issue with how it caches (incorrectly) some of the metadata responses but with RedPanda librdkafka presents the same behaviour and the same mitigation.
Thanks a lot @mensfeld ! Could reproduce it and found the cause. It's because of using the same variable i
in one nested loop here:
https://github.com/confluentinc/librdkafka/blob/6eaf89fb124c421b66b43b195879d458a3a31f86/src/rdkafka_sticky_assignor.c#L821
@emasab you are welcome. Can you explain to me why was it mitigated by client id randomization?
Not in all cases, I could reproduce it in Python even with "client.id": str(time.time())
or with "client.id": str(random.randint(1,1000000))
. It happens when number of potential partitions in inner loop is less than number of members in outer loop minus one and members after potential partition count don't have consumerPartitions->cnt == potentialTopicPartitions->cnt
that causes a continue and doesn't reset the index.
@emasab do you have an ETA for this maybe? Just a PR would help me because I could temporarily cherry-pick this and release as a special release for ppl affected on my side.
this seems to work
--- rdkafka_sticky_assignor.c 2024-07-08 09:47:43.000000000 +0200
+++ rdkafka_sticky_assignor_m.c 2024-07-30 09:44:38.529759640 +0200
@@ -769,7 +769,7 @@
const rd_kafka_topic_partition_list_t *partitions;
const char *consumer;
const rd_map_elem_t *elem;
- int i;
+ int i, j;
/* The assignment is balanced if minimum and maximum numbers of
* partitions assigned to consumers differ by at most one. */
@@ -836,9 +836,9 @@
/* Otherwise make sure it can't get any more partitions */
- for (i = 0; i < potentialTopicPartitions->cnt; i++) {
+ for (j = 0; j < potentialTopicPartitions->cnt; j++) {
const rd_kafka_topic_partition_t *partition =
- &potentialTopicPartitions->elems[i];
+ &potentialTopicPartitions->elems[j];
const char *otherConsumer;
int otherConsumerPartitionCount;
Description
I do not know why, but the cooperative-sticky assignment can get stuck in an infinite loop causing 100% CPU usage and hanging.
How to reproduce
At the moment I am able to reproduce it fully in a stable manner.
I subscribe few consumers from the same consumer group to two topics. Each consumer is subscribed to one out of two topics. I start with 1 consumer instance per topic and then every 5 seconds I add one more consumer instance. Everything is fine until I start adding consumers subscribed to the second topic.
Reproduction:
when
client.id
is not set (or set to the same value) it will cause the described behaviour. If I randomize it it will not.Topics configuration:
Checklist
IMPORTANT: We will close issues where the checklist has not been completed.
Please provide the following information:
2.4.0
and2.5.0
confluentinc/cp-kafka:7.6.1
partition.assignment.strategy: cooperative.sticky, client.id: same_for_all_instances
Linux shinra 5.15.0-113-generic #123-Ubuntu SMP Mon Jun 10 08:16:17 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux
debug=..
as necessary) from librdkafka (will be provided, trying to repro with logs)Full librdkafka config
Broker logs
Since I'm running multi-threaded setup with few consumer instances in one ruby process to simulate it and with randomness of them joining and rejoning there are some logs but they do not differ from when the instance is not stuck
Additional info
watch ps -T -p pid
that shows extreme usage of CPU orrdk:main
thread:gdb of this thread:
it looks like this code never exits:
https://github.com/confluentinc/librdkafka/blob/6eaf89fb124c421b66b43b195879d458a3a31f86/src/rdkafka_sticky_assignor.c#L899
I suspect (though I did not confirm this yet) that it may be caused by a consumer instance leaving during the sticky rebalance or something similar causing this to run forever.
I tried to stop with GDB several times and it's always in the same place (when tracking the backtrace)
frame details:
locals for this frame
Forcing a rebalance by keeping GBD beyond max.poll.interval.ms does not trigger an exit from this loop. It keeps running forever.
assignor logs from the moment it happens:
More detailed logs (assignor,generic,broker,cgrp,interceptor):
Once in a while when I try to shutdown such hanging consumer I get: