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

Sachin Mittal commented on KAFKA-4848:
--------------------------------------

The deadlock issue is like this.
If a thread has two partitions and while processing first partition it takes 
more than MAX_POLL_INTERVAL_MS_CONFIG time, then this thread is evicted from 
the group and both partitions are now migrated to some other thread. Now when 
it tries to process the second partition it tries to get the lock to rocks db. 
It won't get the lock since that partition is now moved to some other thread. 
So it keeps increasing the backoffTimeMs and keeps trying to get the lock 
forever. This reaching a deadlock.
To fix this we need some upper bound of the time limit till it tries to get 
that lock. And that upper bound has to be MAX_POLL_INTERVAL_MS_CONFIG, because 
if by that time it has not got the lock, we can see that this thread was 
evicted from the group and need to rejoin again to get new partitions.

See in attached file:
DEBUG 2017-03-01 18:17:42,465 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - stream-thread 
[StreamThread-1] creating new task 0_4

DEBUG 2017-03-01 18:24:19,202 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - stream-thread 
[StreamThread-1] creating new task 0_8

Note from above 2 lines it took more than 5 minutes to process task 0_4. As a 
result partitions moved to a different thread.

Next see following entries for 0_8

WARN 2017-03-01 18:24:20,205 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-01 18:24:21,257 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-01 18:24:22,360 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-01 18:24:23,563 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-01 18:24:24,966 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-01 18:24:26,768 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-01 18:24:29,371 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-01 18:24:34,435 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-01 18:24:41,837 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-01 18:24:55,640 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-01 18:25:22,242 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-01 18:26:14,445 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-01 18:27:57,848 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-01 18:31:23,689 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-01 18:38:14,294 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-01 18:51:54,497 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-01 19:19:13,900 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-01 20:13:53,014 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-01 22:03:07,629 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-02 01:41:35,831 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.
WARN 2017-03-02 08:58:31,234 [StreamThread-1]: 
org.apache.kafka.streams.processor.internals.StreamThread - Could not create 
task 0_8. Will retry.

>From 2017-03-01 18:24:20,205 to 017-03-02 08:58:31,234 it kept trying to get 
>the lock, hence deadlock.


> Stream thread getting into deadlock state while trying to get rocksdb lock in 
> retryWithBackoff
> ----------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-4848
>                 URL: https://issues.apache.org/jira/browse/KAFKA-4848
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 0.10.2.0
>            Reporter: Sachin Mittal
>         Attachments: thr-1
>
>
> We see a deadlock state when streams thread to process a task takes longer 
> than MAX_POLL_INTERVAL_MS_CONFIG time. In this case this threads partitions 
> are assigned to some other thread including rocksdb lock. When it tries to 
> process the next task it cannot get rocks db lock and simply keeps waiting 
> for that lock forever.
> in retryWithBackoff for AbstractTaskCreator we have a backoffTimeMs = 50L.
> If it does not get lock the we simply increase the time by 10x and keep 
> trying inside the while true loop.
> We need to have a upper bound for this backoffTimeM. If the time is greater 
> than  MAX_POLL_INTERVAL_MS_CONFIG and it still hasn't got the lock means this 
> thread's partitions are moved somewhere else and it may not get the lock 
> again.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to