[ 
https://issues.apache.org/jira/browse/KAFKA-10134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17151465#comment-17151465
 ] 

Guozhang Wang commented on KAFKA-10134:
---------------------------------------

[~seanguo] thanks for confirming! As the root cause is known now I've thought 
about an alternative solution to fix it: I think ideally we would split 
{{updateAssignmentMetadataIfNeeded}} into three different logic: 1) discover 
coordinator if necessary, 2) join-group if necessary, 3) refresh metadata and 
fetch position if necessary. Then we can just make 2) to be best-effort if 
there are still some fetchable partitions.


But that’s a rather big change to make as a last minute blocker fix for 2.6, so 
I made a smaller change to make updateAssignmentMetadataIfNeeded has an 
optional boolean flag to indicate if 2) above should wait until either expired 
or complete, otherwise do not wait on the join-group future and just try once 
with the timer which would return if there’s anything written on the socket. 
I’ve updated the PR for this, if people agree this would be a reasonable fix 
for 2.6 I can add the test coverage and merge it. LMK.

> High CPU issue during rebalance in Kafka consumer after upgrading to 2.5
> ------------------------------------------------------------------------
>
>                 Key: KAFKA-10134
>                 URL: https://issues.apache.org/jira/browse/KAFKA-10134
>             Project: Kafka
>          Issue Type: Bug
>          Components: clients
>    Affects Versions: 2.5.0
>            Reporter: Sean Guo
>            Assignee: Guozhang Wang
>            Priority: Blocker
>             Fix For: 2.6.0, 2.5.1
>
>
> We want to utilize the new rebalance protocol to mitigate the stop-the-world 
> effect during the rebalance as our tasks are long running task.
> But after the upgrade when we try to kill an instance to let rebalance happen 
> when there is some load(some are long running tasks >30S) there, the CPU will 
> go sky-high. It reads ~700% in our metrics so there should be several threads 
> are in a tight loop. We have several consumer threads consuming from 
> different partitions during the rebalance. This is reproducible in both the 
> new CooperativeStickyAssignor and old eager rebalance rebalance protocol. The 
> difference is that with old eager rebalance rebalance protocol used the high 
> CPU usage will dropped after the rebalance done. But when using cooperative 
> one, it seems the consumers threads are stuck on something and couldn't 
> finish the rebalance so the high CPU usage won't drop until we stopped our 
> load. Also a small load without long running task also won't cause continuous 
> high CPU usage as the rebalance can finish in that case.
>  
> "executor.kafka-consumer-executor-4" #124 daemon prio=5 os_prio=0 
> cpu=76853.07ms elapsed=841.16s tid=0x00007fe11f044000 nid=0x1f4 runnable  
> [0x00007fe119aab000]"executor.kafka-consumer-executor-4" #124 daemon prio=5 
> os_prio=0 cpu=76853.07ms elapsed=841.16s tid=0x00007fe11f044000 nid=0x1f4 
> runnable  [0x00007fe119aab000]   java.lang.Thread.State: RUNNABLE at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:467)
>  at 
> org.apache.kafka.clients.consumer.KafkaConsumer.updateAssignmentMetadataIfNeeded(KafkaConsumer.java:1275)
>  at 
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1241) 
> at 
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1216) 
> at
>  
> By debugging into the code we found it looks like the clients are  in a loop 
> on finding the coordinator.
> I also tried the old rebalance protocol for the new version the issue still 
> exists but the CPU will be back to normal when the rebalance is done.
> Also tried the same on the 2.4.1 which seems don't have this issue. So it 
> seems related something changed between 2.4.1 and 2.5.0.
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to