[ https://issues.apache.org/jira/browse/KAFKA-1710?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14173368#comment-14173368 ]
Bhavesh Mistry commented on KAFKA-1710: --------------------------------------- Here is property file used for testing: {code} # THIS IS FOR NEW PRODUCERS API TRUNK Please see the configuration at https://kafka.apache.org/documentation.html#newproducerconfigs # Broker List bootstrap.servers=dare-msgq00.sv.walmartlabs.com:9092,dare-msgq01.sv.walmartlabs.com:9092,dare-msgq02.sv.walmartlabs.com:9092 #Data Acks acks=0 # 64MB of Buffer for log lines (including all messages). buffer.memory=134217728 compression.type=snappy retries=3 # DEFAULT FROM THE KAFKA... # batch size = ((buffer.memory) / (number of partitions)) (so we can have in progress batch size created for each partition.). batch.size=1048576 #2MiB max.request.size=1048576 send.buffer.bytes=2097152 # We do not want to block the buffer Full so application thread will not be blocked but logs lines will be dropped... block.on.buffer.full=false #2MiB send.buffer.bytes=2097152 #wait... linger.ms=3600000 {code} > [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 > Assignee: Jun Rao > 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)