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

Ewen Cheslack-Postava commented on KAFKA-1710:
----------------------------------------------

bq. The dead lock will occur something depending on Thread scheduling and how 
log the are blocked. 

Dead lock has a specific definition -- two or more threads that are both 
waiting on each other such that neither can make any forward progress -- and as 
far as I can tell this isn't triggering a deadlock. From what I've seen this is 
simply an issue of trying of anywhere from 50 - 200 threads trying to access a 
shared, synchronized resource. This is just contention, everything continues to 
make progress. The test program runs to completion just fine.

As for performance, I have no doubt there are improvements to be made in the 
Producer implementation, but you'll get a far bigger performance boost with 
careful design in your system. I already mentioned multiple ways you can 
improve performance that, based on your current test code, shouldn't affect 
anything else. Here's a quick example (using a lightly modified version of your 
code against a local test cluster):

{quote}
Existing setup (4 producers, 1 partition):
All Producers done...!
All done...!

real    1m50.135s
user    1m45.019s
sys     1m53.219s
{quote}

{quote}
8 Producers, 1 partition (and parameters adjusted to generate same # of msgs):
All Producers done...!
All done...!

real    0m55.465s
user    1m27.132s
sys     1m1.144s
{quote}

Nothing surprising, but since you haven't specified a constraint on the # of 
producers this seems like the simplest solution to improve performance.

> [New Java Producer Potential Deadlock] Producer Deadlock when all messages is 
> being sent to single partition
> ------------------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-1710
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1710
>             Project: Kafka
>          Issue Type: Bug
>          Components: producer 
>         Environment: Development
>            Reporter: Bhavesh Mistry
>            Priority: Critical
>              Labels: performance
>         Attachments: Screen Shot 2014-10-13 at 10.19.04 AM.png, Screen Shot 
> 2014-10-15 at 9.09.06 PM.png, Screen Shot 2014-10-15 at 9.14.15 PM.png, 
> TestNetworkDownProducer.java, th1.dump, th10.dump, th11.dump, th12.dump, 
> th13.dump, th14.dump, th15.dump, th2.dump, th3.dump, th4.dump, th5.dump, 
> th6.dump, th7.dump, th8.dump, th9.dump
>
>
> Hi Kafka Dev Team,
> When I run the test to send message to single partition for 3 minutes or so 
> on, I have encounter deadlock (please see the screen attached) and thread 
> contention from YourKit profiling.  
> Use Case:
> 1)  Aggregating messages into same partition for metric counting. 
> 2)  Replicate Old Producer behavior for sticking to partition for 3 minutes.
> Here is output:
> Frozen threads found (potential deadlock)
>  
> It seems that the following threads have not changed their stack for more 
> than 10 seconds.
> These threads are possibly (but not necessarily!) in a deadlock or hung.
>  
> pool-1-thread-128 <--- Frozen for at least 2m
> org.apache.kafka.clients.producer.internals.RecordAccumulator.append(TopicPartition,
>  byte[], byte[], CompressionType, Callback) RecordAccumulator.java:139
> org.apache.kafka.clients.producer.KafkaProducer.send(ProducerRecord, 
> Callback) KafkaProducer.java:237
> org.kafka.test.TestNetworkDownProducer$MyProducer.run() 
> TestNetworkDownProducer.java:84
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor$Worker) 
> ThreadPoolExecutor.java:1145
> java.util.concurrent.ThreadPoolExecutor$Worker.run() 
> ThreadPoolExecutor.java:615
> java.lang.Thread.run() Thread.java:744
> pool-1-thread-159 <--- Frozen for at least 2m 1 sec
> org.apache.kafka.clients.producer.internals.RecordAccumulator.append(TopicPartition,
>  byte[], byte[], CompressionType, Callback) RecordAccumulator.java:139
> org.apache.kafka.clients.producer.KafkaProducer.send(ProducerRecord, 
> Callback) KafkaProducer.java:237
> org.kafka.test.TestNetworkDownProducer$MyProducer.run() 
> TestNetworkDownProducer.java:84
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor$Worker) 
> ThreadPoolExecutor.java:1145
> java.util.concurrent.ThreadPoolExecutor$Worker.run() 
> ThreadPoolExecutor.java:615
> java.lang.Thread.run() Thread.java:744
> pool-1-thread-55 <--- Frozen for at least 2m
> org.apache.kafka.clients.producer.internals.RecordAccumulator.append(TopicPartition,
>  byte[], byte[], CompressionType, Callback) RecordAccumulator.java:139
> org.apache.kafka.clients.producer.KafkaProducer.send(ProducerRecord, 
> Callback) KafkaProducer.java:237
> org.kafka.test.TestNetworkDownProducer$MyProducer.run() 
> TestNetworkDownProducer.java:84
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor$Worker) 
> ThreadPoolExecutor.java:1145
> java.util.concurrent.ThreadPoolExecutor$Worker.run() 
> ThreadPoolExecutor.java:615
> java.lang.Thread.run() Thread.java:744
> Thanks,
> Bhavesh 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to