[
https://issues.apache.org/jira/browse/SAMZA-1069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15773624#comment-15773624
]
ASF GitHub Bot commented on SAMZA-1069:
---------------------------------------
GitHub user xinyuiscool opened a pull request:
https://github.com/apache/samza/pull/37
SAMZA-1069: Fix Deadlock between KafkaSystemProducer and KafkaProducer
Moving the producer.close() and sources.flush() outside the lock so it
won't have race condition with the kafka network thread callbacks.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/xinyuiscool/samza SAMZA-1069
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/samza/pull/37.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #37
----
commit 7dad5ad19df01911c109d1a6f3667558eda4d8e0
Author: Xinyu Liu <[email protected]>
Date: 2016-12-23T20:21:34Z
SAMZA-1069: Fix Deadlock between KafkaSystemProducer and KafkaProducer
----
> Deadlock between KafkaSystemProducer and KafkaProducer from kafka-clients lib
> -----------------------------------------------------------------------------
>
> Key: SAMZA-1069
> URL: https://issues.apache.org/jira/browse/SAMZA-1069
> Project: Samza
> Issue Type: Bug
> Affects Versions: 0.11.0
> Reporter: Yi Pan (Data Infrastructure)
> Assignee: Xinyu Liu
> Fix For: 0.12.0
>
>
> We have identified one deadlock scenario between the main thread that calls
> KafkaSystemProducer.close() vs the KafkaProducer client lib's network thread
> that calls the callback function within KafkaSystemProducer.send().
> The scenario is the following:
> # SamzaContainer main thread caught an exception from previous commit and
> container initiated shutdown, which calls KafkaSystemProducer.stop(),
> grabbing the synchronized producerLock in KafkaSystemProducer and call
> KafkaProducer.flush() to wait for all pending requests to be done.
> # KafkaProducer network I/O thread then calls KafkaSystemProducer’s callback
> function (in RecordBatch.done()), which is waiting on the same producerLock
> in KafkaSystemProducer before it can return and call producerFuture.done()
> and release the CountDownLatch that the main thread
> KafkaSystemProducer.close() is waiting on. Hence, deadlock!
> We need to make sure the KafkaSystemProducer.close() won't have race
> condition w/ the callbacks triggered by the KafkaProducer's network thread.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)