kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Aravind Dongara <adong...@yahoo.com.INVALID>
Subject Re: Group Coordinator stuck on PrepareRebalance state.
Date Tue, 16 Jul 2019 06:33:55 GMT
Hi Boyang,

Our initial assessment was not accurate, but after scanning through all the stream threads
on all machines we found that while most of the threads are participating in the joinRequest
call some are stuck at TransactionalRequestResult.await() while doing StreamTask.commit().
We think since these threads are not sending in the JoinRequest, the GroupCoordinator is not
able to make progress to next state.

"metric-extractor-stream-c1-d9ac8890-cd80-4b75-a85a-2ff39ea27961-StreamThread-27" #143 daemon
prio=5 os_prio=0 tid=0x00007f27c4365800 nid=0xb9 waiting on condition [0x00007f27736f7000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x0000000657b0b6e8> (a java.util.concurrent.CountDownLatch$Sync)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
        at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:231)
        at org.apache.kafka.clients.producer.internals.TransactionalRequestResult.await(TransactionalRequestResult.java:50)
        at org.apache.kafka.clients.producer.KafkaProducer.sendOffsetsToTransaction(KafkaProducer.java:675)
        at org.apache.kafka.streams.processor.internals.StreamTask.commit(StreamTask.java:487)
        at org.apache.kafka.streams.processor.internals.StreamTask.commit(StreamTask.java:459)
        at org.apache.kafka.streams.processor.internals.AssignedTasks.commit(AssignedTasks.java:286)
        at org.apache.kafka.streams.processor.internals.TaskManager.commitAll(TaskManager.java:412)
        at org.apache.kafka.streams.processor.internals.StreamThread.maybeCommit(StreamThread.java:1057)
        at org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:911)
        at org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:805)
        at org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:774)


Restarting the machines that had these stuck threads helped us unblock the GroupCoordinator
and our streams application.
We are not sure how and why some stream threads got into that stuck state. Any pointers will
be greatly appreciated.

Thanks
Aravind

> On Jul 11, 2019, at 10:08 AM, Thameem Ansari <thameema@gmail.com> wrote:
> 
> In addition to the session timeout try increasing the request timeout as well. We had
similar issue and resolved it by increasing the timeouts. As per my understanding, If you
have complex topology then it will take some time for kafka brokers to create the tasks and
assign them to consumers. In the mean time if any consumer try to join the group it will timeout
due to lower timeout values. Increasing the timeouts will give enough time for the brokers
to assign the tasks to consumers properly. 
> 
> request.timeout.ms=60000
> 
> Thanks
> Thameem
> 
> 
>> On Jul 11, 2019, at 7:56 PM, Aravind Dongara <adongara@yahoo.com.INVALID> wrote:
>> 
>> Hi Boyang,
>> 
>> Thanks for the quick response.
>> We are on version 2.2.0.
>> 
>> We are using the following properties on KStreams/consumer:
>> session.timeout.ms=15000
>> heartbeat.interval.ms=3000
>> 
>> I was wondering if a member might leak if it satisfies “shouldKeepAlive” condition
in "onExpireHeartbeat" and the consumer restarts or goes down right after that, before next
heartbeat is sent.
>> 
>> Thanks
>> Aravind
>> 
>> 
>>> On Jul 10, 2019, at 10:40 PM, Boyang Chen <reluctanthero104@gmail.com>
wrote:
>>> 
>>> Hey Aravind,
>>> 
>>> If your client/broker just upgraded to 2.3, Jason has filed a blocker for
>>> 2.3: https://issues.apache.org/jira/browse/KAFKA-8653
>>> 
>>> and a fix is on its way: https://github.com/apache/kafka/pull/7072/files
>>> 
>>> Let me know if you are actually on a different version.
>>> 
>>> Boyang
>>> 
>>> On Wed, Jul 10, 2019 at 7:43 PM Aravind Dongara <adongara@yahoo.com.invalid>
>>> wrote:
>>> 
>>>> Our kafka streams application is stuck and continuously emits
>>>> "(Re-)joining group” log message every 5 minutes without making any
>>>> progress.
>>>> 
>>>> Kafka-consumer-groups cmd line tool with “—members” option shows lots
of
>>>> stale members, in addition to expected member-ids shown on log msgs on
>>>> kafka-streams app and broker logs that were failing to join).
>>>> For some reason these old members didn’t get evicted from members list.
>>>> Looks like this is preventing the GroupCordinator from reaching
>>>> “CompletingRebalance” state.
>>>> 
>>>> Restart of Kafka streams app didn’t help either, it just replaced the
>>>> newer member-ids; but the old stale member-ids are still present in the
>>>> members-list.
>>>> 
>>>> Is there any way to resolve this without restarting the broker hosting the
>>>> GroupCoordinator for this group.
>>>> 
>>>> Thanks
>>>> Aravind
>> 
> 


Mime
View raw message