[jira] [Commented] (KAFKA-5970) Deadlock due to locking of DelayedProduce and group

2017-09-25 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on KAFKA-5970:
---

GitHub user rajinisivaram opened a pull request:

https://github.com/apache/kafka/pull/3956

KAFKA-5970: Avoid locking group in operations that lock DelayedProduce



You can merge this pull request into a Git repository by running:

$ git pull https://github.com/rajinisivaram/kafka 
KAFKA-5970-delayedproduce-deadlock

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/3956.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 #3956


commit 27238fe16a31bad09cc82785a7bb62d347f2c7ac
Author: Rajini Sivaram 
Date:   2017-09-25T11:40:28Z

KAFKA-5970: Avoid locking group in operations that lock DelayedProduce




> Deadlock due to locking of DelayedProduce and group
> ---
>
> Key: KAFKA-5970
> URL: https://issues.apache.org/jira/browse/KAFKA-5970
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Reporter: Rajini Sivaram
>Assignee: Rajini Sivaram
>Priority: Critical
> Fix For: 1.0.0
>
>
> From a local run of TransactionsBounceTest. Looks like we hold group lock 
> while completing DelayedProduce, which in turn may acquire group lock.
> {quote}
> Found one Java-level deadlock:
> =
> "kafka-request-handler-7":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> "kafka-request-handler-4":
>   waiting to lock monitor 0x7fe0869e4408 (object 0x000749be7bb8, a 
> kafka.server.DelayedProduce),
>   which is held by "kafka-request-handler-3"
> "kafka-request-handler-3":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> Java stack information for the threads listed above:
> ===
> "kafka-request-handler-7":
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:752)
>   waiting to lock <0x00074a9fbc50> (a 
> kafka.coordinator.group.GroupMetadata)
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:750)
> at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
> at 
> kafka.coordinator.group.GroupMetadataManager.handleTxnCompletion(GroupMetadataManager.scala:750)
> at 
> kafka.coordinator.group.GroupCoordinator.handleTxnCompletion(GroupCoordinator.scala:439)
> at 
> kafka.server.KafkaApis.kafka$server$KafkaApis$$maybeSendResponseCallback$1(KafkaApis.scala:1556)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:134)
> at 
> kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:66)
> at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:116)
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:76)
>   locked <0x00074b21c968> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:284)
> at 
> kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:434)
> at 
> kafka.cluster.Partition.updateReplicaLogReadResult(Partition.scala:285)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1290)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1286)
> at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> at 
> kafka.server.ReplicaManager.updateFollowerLogReadResults(ReplicaManager.scala:1286)
> at kafka.server.ReplicaManager.fetchMessages(ReplicaManager.scala:786)
> at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:598)

[jira] [Commented] (KAFKA-5970) Deadlock due to locking of DelayedProduce and group

2017-09-25 Thread Peter Davis (JIRA)

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

Peter Davis commented on KAFKA-5970:


It looks like we are seeing this repeatedly in a production cluster after 
upgrading to Confluent Platform 3.3.0.

After the deadlock, the affected broker appears to drop all connections, 
including replication to other brokers.  Result is the deadlocked broker 
becomes the only member of ISR for partitions it was leading, and due to 
unclean.leader.election.enabled=false and min.insync.replicas=2, the partitions 
become unavailable and won't recover without restarting the deadlocked broker.  
So this is quite serious.

Any hints of what unusual condition might be causing this for us or a 
workaround?

> Deadlock due to locking of DelayedProduce and group
> ---
>
> Key: KAFKA-5970
> URL: https://issues.apache.org/jira/browse/KAFKA-5970
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Reporter: Rajini Sivaram
>Assignee: Rajini Sivaram
>Priority: Critical
> Fix For: 1.0.0
>
>
> From a local run of TransactionsBounceTest. Looks like we hold group lock 
> while completing DelayedProduce, which in turn may acquire group lock.
> {quote}
> Found one Java-level deadlock:
> =
> "kafka-request-handler-7":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> "kafka-request-handler-4":
>   waiting to lock monitor 0x7fe0869e4408 (object 0x000749be7bb8, a 
> kafka.server.DelayedProduce),
>   which is held by "kafka-request-handler-3"
> "kafka-request-handler-3":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> Java stack information for the threads listed above:
> ===
> "kafka-request-handler-7":
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:752)
>   waiting to lock <0x00074a9fbc50> (a 
> kafka.coordinator.group.GroupMetadata)
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:750)
> at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
> at 
> kafka.coordinator.group.GroupMetadataManager.handleTxnCompletion(GroupMetadataManager.scala:750)
> at 
> kafka.coordinator.group.GroupCoordinator.handleTxnCompletion(GroupCoordinator.scala:439)
> at 
> kafka.server.KafkaApis.kafka$server$KafkaApis$$maybeSendResponseCallback$1(KafkaApis.scala:1556)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:134)
> at 
> kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:66)
> at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:116)
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:76)
>   locked <0x00074b21c968> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:284)
> at 
> kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:434)
> at 
> kafka.cluster.Partition.updateReplicaLogReadResult(Partition.scala:285)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1290)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1286)
> at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> at 
> kafka.server.ReplicaManager.updateFollowerLogReadResults(ReplicaManager.scala:1286)
> at kafka.server.ReplicaManager.fetchMessages(ReplicaManager.scala:786)
> at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:598)
> at kafka.server.KafkaApis.handle(KafkaApis.scala:100)
> at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:65)
> at java.lang.Thread.run(Thread.java:748)
> 

[jira] [Commented] (KAFKA-5970) Deadlock due to locking of DelayedProduce and group

2017-09-25 Thread Ismael Juma (JIRA)

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

Ismael Juma commented on KAFKA-5970:


[~davispw], are you using transactions? The deadlock in this JIRA seems to be 
related to the handling of transactions.

> Deadlock due to locking of DelayedProduce and group
> ---
>
> Key: KAFKA-5970
> URL: https://issues.apache.org/jira/browse/KAFKA-5970
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Reporter: Rajini Sivaram
>Assignee: Rajini Sivaram
>Priority: Critical
> Fix For: 1.0.0
>
>
> From a local run of TransactionsBounceTest. Looks like we hold group lock 
> while completing DelayedProduce, which in turn may acquire group lock.
> {quote}
> Found one Java-level deadlock:
> =
> "kafka-request-handler-7":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> "kafka-request-handler-4":
>   waiting to lock monitor 0x7fe0869e4408 (object 0x000749be7bb8, a 
> kafka.server.DelayedProduce),
>   which is held by "kafka-request-handler-3"
> "kafka-request-handler-3":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> Java stack information for the threads listed above:
> ===
> "kafka-request-handler-7":
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:752)
>   waiting to lock <0x00074a9fbc50> (a 
> kafka.coordinator.group.GroupMetadata)
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:750)
> at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
> at 
> kafka.coordinator.group.GroupMetadataManager.handleTxnCompletion(GroupMetadataManager.scala:750)
> at 
> kafka.coordinator.group.GroupCoordinator.handleTxnCompletion(GroupCoordinator.scala:439)
> at 
> kafka.server.KafkaApis.kafka$server$KafkaApis$$maybeSendResponseCallback$1(KafkaApis.scala:1556)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:134)
> at 
> kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:66)
> at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:116)
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:76)
>   locked <0x00074b21c968> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:284)
> at 
> kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:434)
> at 
> kafka.cluster.Partition.updateReplicaLogReadResult(Partition.scala:285)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1290)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1286)
> at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> at 
> kafka.server.ReplicaManager.updateFollowerLogReadResults(ReplicaManager.scala:1286)
> at kafka.server.ReplicaManager.fetchMessages(ReplicaManager.scala:786)
> at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:598)
> at kafka.server.KafkaApis.handle(KafkaApis.scala:100)
> at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:65)
> at java.lang.Thread.run(Thread.java:748)
> "kafka-request-handler-4":
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:75)
>   waiting to lock <0x000749be7bb8> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.

[jira] [Commented] (KAFKA-5970) Deadlock due to locking of DelayedProduce and group

2017-09-25 Thread Peter Davis (JIRA)

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

Peter Davis commented on KAFKA-5970:


I ended up here because of commonality of deadlock in GroupCoordinator and 
DelayedProduce, and version 0.11.0.0 (CP 3.3.0).

Other possibly-similar JIRAs found after a quick search all reportedly 
resolved: KAFKA-4478, KAFKA-3994, KAFKA-2821, KAFKA-4562

> Deadlock due to locking of DelayedProduce and group
> ---
>
> Key: KAFKA-5970
> URL: https://issues.apache.org/jira/browse/KAFKA-5970
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Reporter: Rajini Sivaram
>Assignee: Rajini Sivaram
>Priority: Critical
> Fix For: 1.0.0
>
> Attachments: jstack.txt
>
>
> From a local run of TransactionsBounceTest. Looks like we hold group lock 
> while completing DelayedProduce, which in turn may acquire group lock.
> {quote}
> Found one Java-level deadlock:
> =
> "kafka-request-handler-7":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> "kafka-request-handler-4":
>   waiting to lock monitor 0x7fe0869e4408 (object 0x000749be7bb8, a 
> kafka.server.DelayedProduce),
>   which is held by "kafka-request-handler-3"
> "kafka-request-handler-3":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> Java stack information for the threads listed above:
> ===
> "kafka-request-handler-7":
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:752)
>   waiting to lock <0x00074a9fbc50> (a 
> kafka.coordinator.group.GroupMetadata)
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:750)
> at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
> at 
> kafka.coordinator.group.GroupMetadataManager.handleTxnCompletion(GroupMetadataManager.scala:750)
> at 
> kafka.coordinator.group.GroupCoordinator.handleTxnCompletion(GroupCoordinator.scala:439)
> at 
> kafka.server.KafkaApis.kafka$server$KafkaApis$$maybeSendResponseCallback$1(KafkaApis.scala:1556)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:134)
> at 
> kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:66)
> at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:116)
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:76)
>   locked <0x00074b21c968> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:284)
> at 
> kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:434)
> at 
> kafka.cluster.Partition.updateReplicaLogReadResult(Partition.scala:285)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1290)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1286)
> at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> at 
> kafka.server.ReplicaManager.updateFollowerLogReadResults(ReplicaManager.scala:1286)
> at kafka.server.ReplicaManager.fetchMessages(ReplicaManager.scala:786)
> at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:598)
> at kafka.server.KafkaApis.handle(KafkaApis.scala:100)
> at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:65)
> at java.lang.Thread.run(Thread.java:748)
> "kafka-request-handler-4":
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:75)
>   waiting to lock <0x000749be7bb8> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 

[jira] [Commented] (KAFKA-5970) Deadlock due to locking of DelayedProduce and group

2017-09-25 Thread Rajini Sivaram (JIRA)

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

Rajini Sivaram commented on KAFKA-5970:
---

[~davispw], I think it is the same issue in a slightly different codepath, 
which should be fixed by the proposed solution from the PR here.

> Deadlock due to locking of DelayedProduce and group
> ---
>
> Key: KAFKA-5970
> URL: https://issues.apache.org/jira/browse/KAFKA-5970
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Reporter: Rajini Sivaram
>Assignee: Rajini Sivaram
>Priority: Critical
> Fix For: 1.0.0
>
> Attachments: jstack.txt
>
>
> From a local run of TransactionsBounceTest. Looks like we hold group lock 
> while completing DelayedProduce, which in turn may acquire group lock.
> {quote}
> Found one Java-level deadlock:
> =
> "kafka-request-handler-7":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> "kafka-request-handler-4":
>   waiting to lock monitor 0x7fe0869e4408 (object 0x000749be7bb8, a 
> kafka.server.DelayedProduce),
>   which is held by "kafka-request-handler-3"
> "kafka-request-handler-3":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> Java stack information for the threads listed above:
> ===
> "kafka-request-handler-7":
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:752)
>   waiting to lock <0x00074a9fbc50> (a 
> kafka.coordinator.group.GroupMetadata)
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:750)
> at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
> at 
> kafka.coordinator.group.GroupMetadataManager.handleTxnCompletion(GroupMetadataManager.scala:750)
> at 
> kafka.coordinator.group.GroupCoordinator.handleTxnCompletion(GroupCoordinator.scala:439)
> at 
> kafka.server.KafkaApis.kafka$server$KafkaApis$$maybeSendResponseCallback$1(KafkaApis.scala:1556)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:134)
> at 
> kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:66)
> at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:116)
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:76)
>   locked <0x00074b21c968> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:284)
> at 
> kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:434)
> at 
> kafka.cluster.Partition.updateReplicaLogReadResult(Partition.scala:285)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1290)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1286)
> at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> at 
> kafka.server.ReplicaManager.updateFollowerLogReadResults(ReplicaManager.scala:1286)
> at kafka.server.ReplicaManager.fetchMessages(ReplicaManager.scala:786)
> at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:598)
> at kafka.server.KafkaApis.handle(KafkaApis.scala:100)
> at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:65)
> at java.lang.Thread.run(Thread.java:748)
> "kafka-request-handler-4":
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:75)
>   waiting to lock <0x000749be7bb8> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.

[jira] [Commented] (KAFKA-5970) Deadlock due to locking of DelayedProduce and group

2017-09-25 Thread Peter Davis (JIRA)

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

Peter Davis commented on KAFKA-5970:


[~rsivaram] Thanks for looking into this so quickly.  We agree, looks like the 
same codepath.

Theorizing about our problem and whether this is a smoking gun, it's possible 
this deadlock is an effect, not a cause, of our issue.  As I mentioned we're 
seeing a broker repeatedly end up as the only member of ISR for partitions it 
is leading, which includes some __consumer_offsets partitions.  If replication 
to __consumer_offsets failed first, then the GroupCoordinator's producer would 
get hung trying to publish committed offsets and we might see this.  On the 
other hand, a deadlocked GroupCoordinator which hangs up request processing 
threads could have all kinds of cascading effects, right?  Sigh...

We are just spinning here trying to theorize how this is biting us so badly.  
If the GroupCoordinator can deadlock on merely committing offsets, this'd be 
extremely serious and affect a whole lot more people, right?

> Deadlock due to locking of DelayedProduce and group
> ---
>
> Key: KAFKA-5970
> URL: https://issues.apache.org/jira/browse/KAFKA-5970
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Reporter: Rajini Sivaram
>Assignee: Rajini Sivaram
>Priority: Critical
> Fix For: 1.0.0
>
> Attachments: jstack.txt
>
>
> From a local run of TransactionsBounceTest. Looks like we hold group lock 
> while completing DelayedProduce, which in turn may acquire group lock.
> {quote}
> Found one Java-level deadlock:
> =
> "kafka-request-handler-7":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> "kafka-request-handler-4":
>   waiting to lock monitor 0x7fe0869e4408 (object 0x000749be7bb8, a 
> kafka.server.DelayedProduce),
>   which is held by "kafka-request-handler-3"
> "kafka-request-handler-3":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> Java stack information for the threads listed above:
> ===
> "kafka-request-handler-7":
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:752)
>   waiting to lock <0x00074a9fbc50> (a 
> kafka.coordinator.group.GroupMetadata)
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:750)
> at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
> at 
> kafka.coordinator.group.GroupMetadataManager.handleTxnCompletion(GroupMetadataManager.scala:750)
> at 
> kafka.coordinator.group.GroupCoordinator.handleTxnCompletion(GroupCoordinator.scala:439)
> at 
> kafka.server.KafkaApis.kafka$server$KafkaApis$$maybeSendResponseCallback$1(KafkaApis.scala:1556)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:134)
> at 
> kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:66)
> at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:116)
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:76)
>   locked <0x00074b21c968> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:284)
> at 
> kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:434)
> at 
> kafka.cluster.Partition.updateReplicaLogReadResult(Partition.scala:285)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1290)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1286)
> at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> at 
> kafka.server.ReplicaManager.updateFollowerLogReadResults(ReplicaManager.scala:1

[jira] [Commented] (KAFKA-5970) Deadlock due to locking of DelayedProduce and group

2017-09-25 Thread Rajini Sivaram (JIRA)

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

Rajini Sivaram commented on KAFKA-5970:
---

[~davispw] I am surprised this was not reported earlier. We hold a group lock 
in several places where a purgatory operation may complete and attempt to 
acquire a potentially different group lock. [~hachikuji] may know why this 
doesn't deadlock more often.

> Deadlock due to locking of DelayedProduce and group
> ---
>
> Key: KAFKA-5970
> URL: https://issues.apache.org/jira/browse/KAFKA-5970
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Reporter: Rajini Sivaram
>Assignee: Rajini Sivaram
>Priority: Critical
> Fix For: 1.0.0
>
> Attachments: jstack.txt
>
>
> From a local run of TransactionsBounceTest. Looks like we hold group lock 
> while completing DelayedProduce, which in turn may acquire group lock.
> {quote}
> Found one Java-level deadlock:
> =
> "kafka-request-handler-7":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> "kafka-request-handler-4":
>   waiting to lock monitor 0x7fe0869e4408 (object 0x000749be7bb8, a 
> kafka.server.DelayedProduce),
>   which is held by "kafka-request-handler-3"
> "kafka-request-handler-3":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> Java stack information for the threads listed above:
> ===
> "kafka-request-handler-7":
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:752)
>   waiting to lock <0x00074a9fbc50> (a 
> kafka.coordinator.group.GroupMetadata)
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:750)
> at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
> at 
> kafka.coordinator.group.GroupMetadataManager.handleTxnCompletion(GroupMetadataManager.scala:750)
> at 
> kafka.coordinator.group.GroupCoordinator.handleTxnCompletion(GroupCoordinator.scala:439)
> at 
> kafka.server.KafkaApis.kafka$server$KafkaApis$$maybeSendResponseCallback$1(KafkaApis.scala:1556)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:134)
> at 
> kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:66)
> at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:116)
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:76)
>   locked <0x00074b21c968> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:284)
> at 
> kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:434)
> at 
> kafka.cluster.Partition.updateReplicaLogReadResult(Partition.scala:285)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1290)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1286)
> at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> at 
> kafka.server.ReplicaManager.updateFollowerLogReadResults(ReplicaManager.scala:1286)
> at kafka.server.ReplicaManager.fetchMessages(ReplicaManager.scala:786)
> at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:598)
> at kafka.server.KafkaApis.handle(KafkaApis.scala:100)
> at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:65)
> at java.lang.Thread.run(Thread.java:748)
> "kafka-request-handler-4":
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:75)
>   waiting to lock <0x000749be7bb8> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
>   

[jira] [Commented] (KAFKA-5970) Deadlock due to locking of DelayedProduce and group

2017-09-25 Thread Peter Davis (JIRA)

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

Peter Davis commented on KAFKA-5970:


Thanks [~rsivaram] and [~hachikuji] for looking at this.  Even a theory why 
this *doesn't* bite people more often would be helpful, as it might give us a 
clue how to work around this.   As I said, it is quite seriously hurting us at 
the moment.

> Deadlock due to locking of DelayedProduce and group
> ---
>
> Key: KAFKA-5970
> URL: https://issues.apache.org/jira/browse/KAFKA-5970
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Reporter: Rajini Sivaram
>Assignee: Rajini Sivaram
>Priority: Critical
> Fix For: 1.0.0
>
> Attachments: jstack.txt
>
>
> From a local run of TransactionsBounceTest. Looks like we hold group lock 
> while completing DelayedProduce, which in turn may acquire group lock.
> {quote}
> Found one Java-level deadlock:
> =
> "kafka-request-handler-7":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> "kafka-request-handler-4":
>   waiting to lock monitor 0x7fe0869e4408 (object 0x000749be7bb8, a 
> kafka.server.DelayedProduce),
>   which is held by "kafka-request-handler-3"
> "kafka-request-handler-3":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> Java stack information for the threads listed above:
> ===
> "kafka-request-handler-7":
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:752)
>   waiting to lock <0x00074a9fbc50> (a 
> kafka.coordinator.group.GroupMetadata)
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:750)
> at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
> at 
> kafka.coordinator.group.GroupMetadataManager.handleTxnCompletion(GroupMetadataManager.scala:750)
> at 
> kafka.coordinator.group.GroupCoordinator.handleTxnCompletion(GroupCoordinator.scala:439)
> at 
> kafka.server.KafkaApis.kafka$server$KafkaApis$$maybeSendResponseCallback$1(KafkaApis.scala:1556)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:134)
> at 
> kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:66)
> at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:116)
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:76)
>   locked <0x00074b21c968> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:284)
> at 
> kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:434)
> at 
> kafka.cluster.Partition.updateReplicaLogReadResult(Partition.scala:285)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1290)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1286)
> at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> at 
> kafka.server.ReplicaManager.updateFollowerLogReadResults(ReplicaManager.scala:1286)
> at kafka.server.ReplicaManager.fetchMessages(ReplicaManager.scala:786)
> at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:598)
> at kafka.server.KafkaApis.handle(KafkaApis.scala:100)
> at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:65)
> at java.lang.Thread.run(Thread.java:748)
> "kafka-request-handler-4":
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:75)
>   waiting to lock <0x000749be7bb8> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka

[jira] [Commented] (KAFKA-5970) Deadlock due to locking of DelayedProduce and group

2017-09-27 Thread Scott Reynolds (JIRA)

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

Scott Reynolds commented on KAFKA-5970:
---

Ran into a very similar deadlock

{code}
Found one Java-level deadlock:
=
"executor-Heartbeat":
  waiting to lock monitor 0x7f8c9c297a98 (object 0x00042f7c55a0, a 
kafka.coordinator.group.GroupMetadata),
  which is held by "kafka-request-handler-5"
"kafka-request-handler-5":
  waiting to lock monitor 0x7f8d10125ba8 (object 0x00042f7b64e8, a 
kafka.coordinator.group.GroupMetadata),
  which is held by "kafka-request-handler-1"
"kafka-request-handler-1":
  waiting to lock monitor 0x7f8c9c297a98 (object 0x00042f7c55a0, a 
kafka.coordinator.group.GroupMetadata),
  which is held by "kafka-request-handler-5"

Java stack information for the threads listed above:
===
"executor-Heartbeat":
at 
kafka.coordinator.group.GroupCoordinator.onExpireHeartbeat(GroupCoordinator.scala:777)
- waiting to lock <0x00042f7c55a0> (a 
kafka.coordinator.group.GroupMetadata)
at 
kafka.coordinator.group.DelayedHeartbeat.onExpiration(DelayedHeartbeat.scala:38)
at kafka.server.DelayedOperation.run(DelayedOperation.scala:113)
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:748)
"kafka-request-handler-5":
at kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:75)
- waiting to lock <0x00042f7b64e8> (a 
kafka.coordinator.group.GroupMetadata)
at 
kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
at 
kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
at 
kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:250)
at 
kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:418)
at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:500)
at 
kafka.server.ReplicaManager$$anonfun$appendToLocalLog$2.apply(ReplicaManager.scala:546)
at 
kafka.server.ReplicaManager$$anonfun$appendToLocalLog$2.apply(ReplicaManager.scala:532)
at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at scala.collection.immutable.Map$Map1.foreach(Map.scala:116)
at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
at scala.collection.AbstractTraversable.map(Traversable.scala:104)
at 
kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:532)
at kafka.server.ReplicaManager.appendRecords(ReplicaManager.scala:373)
at 
kafka.coordinator.group.GroupMetadataManager.appendForGroup(GroupMetadataManager.scala:239)
at 
kafka.coordinator.group.GroupMetadataManager.storeOffsets(GroupMetadataManager.scala:381)
at 
kafka.coordinator.group.GroupCoordinator.doCommitOffsets(GroupCoordinator.scala:465)
- locked <0x00042f7c55a0> (a kafka.coordinator.group.GroupMetadata)
at 
kafka.coordinator.group.GroupCoordinator.handleCommitOffsets(GroupCoordinator.scala:428)
at kafka.server.KafkaApis.handleOffsetCommitRequest(KafkaApis.scala:356)
at kafka.server.KafkaApis.handle(KafkaApis.scala:105)
at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:66)
at java.lang.Thread.run(Thread.java:748)
"kafka-request-handler-1":
at kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:75)
- waiting to lock <0x00042f7c55a0> (a 
kafka.coordinator.group.GroupMetadata)
at 
kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
at 
kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
at 
kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:250)
at 
kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:418)
at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:500)
at 
kafka.server.ReplicaManager$$anonfun$appendToLocalLog$2.apply(ReplicaManager.scala:546)
at 
kafka.server.ReplicaManager$$anonfun$appendToLocalLog$2.apply(ReplicaManager.scala:532)
at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.sca

[jira] [Commented] (KAFKA-5970) Deadlock due to locking of DelayedProduce and group

2017-09-27 Thread Peter Davis (JIRA)

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

Peter Davis commented on KAFKA-5970:


In our case, turns out there's a very chatty consumer calling commitAsync() 
after each message, about 30/s across about 8 consumer threads.  (Found by heap 
dump of the deadlocked JVM -> inspect the GroupMetadata object.)  Been running 
this way since 0.9.0.  Less commits -> less likely but still possible.

If I read this right, this is a ticking time bomb for anyone running 0.11 in 
production, transactions or no.  Can a fix go in 0.11.0.2?

> Deadlock due to locking of DelayedProduce and group
> ---
>
> Key: KAFKA-5970
> URL: https://issues.apache.org/jira/browse/KAFKA-5970
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Reporter: Rajini Sivaram
>Assignee: Rajini Sivaram
>Priority: Critical
> Fix For: 1.0.0
>
> Attachments: jstack.txt
>
>
> From a local run of TransactionsBounceTest. Looks like we hold group lock 
> while completing DelayedProduce, which in turn may acquire group lock.
> {quote}
> Found one Java-level deadlock:
> =
> "kafka-request-handler-7":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> "kafka-request-handler-4":
>   waiting to lock monitor 0x7fe0869e4408 (object 0x000749be7bb8, a 
> kafka.server.DelayedProduce),
>   which is held by "kafka-request-handler-3"
> "kafka-request-handler-3":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> Java stack information for the threads listed above:
> ===
> "kafka-request-handler-7":
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:752)
>   waiting to lock <0x00074a9fbc50> (a 
> kafka.coordinator.group.GroupMetadata)
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:750)
> at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
> at 
> kafka.coordinator.group.GroupMetadataManager.handleTxnCompletion(GroupMetadataManager.scala:750)
> at 
> kafka.coordinator.group.GroupCoordinator.handleTxnCompletion(GroupCoordinator.scala:439)
> at 
> kafka.server.KafkaApis.kafka$server$KafkaApis$$maybeSendResponseCallback$1(KafkaApis.scala:1556)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:134)
> at 
> kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:66)
> at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:116)
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:76)
>   locked <0x00074b21c968> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:284)
> at 
> kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:434)
> at 
> kafka.cluster.Partition.updateReplicaLogReadResult(Partition.scala:285)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1290)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1286)
> at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> at 
> kafka.server.ReplicaManager.updateFollowerLogReadResults(ReplicaManager.scala:1286)
> at kafka.server.ReplicaManager.fetchMessages(ReplicaManager.scala:786)
> at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:598)
> at kafka.server.KafkaApis.handle(KafkaApis.scala:100)
> at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:65)
> at java.lang.Thread.run(Thread.java:748)
> "kafka-request-handler-4":
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:75)
>   wa

[jira] [Commented] (KAFKA-5970) Deadlock due to locking of DelayedProduce and group

2017-10-04 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on KAFKA-5970:
---

Github user asfgit closed the pull request at:

https://github.com/apache/kafka/pull/3956


> Deadlock due to locking of DelayedProduce and group
> ---
>
> Key: KAFKA-5970
> URL: https://issues.apache.org/jira/browse/KAFKA-5970
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Reporter: Rajini Sivaram
>Assignee: Rajini Sivaram
>Priority: Blocker
> Fix For: 1.0.0
>
> Attachments: jstack.txt
>
>
> From a local run of TransactionsBounceTest. Looks like we hold group lock 
> while completing DelayedProduce, which in turn may acquire group lock.
> {quote}
> Found one Java-level deadlock:
> =
> "kafka-request-handler-7":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> "kafka-request-handler-4":
>   waiting to lock monitor 0x7fe0869e4408 (object 0x000749be7bb8, a 
> kafka.server.DelayedProduce),
>   which is held by "kafka-request-handler-3"
> "kafka-request-handler-3":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> Java stack information for the threads listed above:
> ===
> "kafka-request-handler-7":
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:752)
>   waiting to lock <0x00074a9fbc50> (a 
> kafka.coordinator.group.GroupMetadata)
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:750)
> at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
> at 
> kafka.coordinator.group.GroupMetadataManager.handleTxnCompletion(GroupMetadataManager.scala:750)
> at 
> kafka.coordinator.group.GroupCoordinator.handleTxnCompletion(GroupCoordinator.scala:439)
> at 
> kafka.server.KafkaApis.kafka$server$KafkaApis$$maybeSendResponseCallback$1(KafkaApis.scala:1556)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:134)
> at 
> kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:66)
> at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:116)
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:76)
>   locked <0x00074b21c968> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:284)
> at 
> kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:434)
> at 
> kafka.cluster.Partition.updateReplicaLogReadResult(Partition.scala:285)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1290)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1286)
> at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> at 
> kafka.server.ReplicaManager.updateFollowerLogReadResults(ReplicaManager.scala:1286)
> at kafka.server.ReplicaManager.fetchMessages(ReplicaManager.scala:786)
> at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:598)
> at kafka.server.KafkaApis.handle(KafkaApis.scala:100)
> at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:65)
> at java.lang.Thread.run(Thread.java:748)
> "kafka-request-handler-4":
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:75)
>   waiting to lock <0x000749be7bb8> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.ReplicaManager.tryCompleteDelayedProduce

[jira] [Commented] (KAFKA-5970) Deadlock due to locking of DelayedProduce and group

2017-10-09 Thread Ismael Juma (JIRA)

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

Ismael Juma commented on KAFKA-5970:


Can anyone test the latest code from the 0.11.0 branch to verify that it no 
longer happens for them?

> Deadlock due to locking of DelayedProduce and group
> ---
>
> Key: KAFKA-5970
> URL: https://issues.apache.org/jira/browse/KAFKA-5970
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Reporter: Rajini Sivaram
>Assignee: Rajini Sivaram
>Priority: Blocker
> Fix For: 1.0.0, 0.11.0.2
>
> Attachments: jstack.txt
>
>
> From a local run of TransactionsBounceTest. Looks like we hold group lock 
> while completing DelayedProduce, which in turn may acquire group lock.
> {quote}
> Found one Java-level deadlock:
> =
> "kafka-request-handler-7":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> "kafka-request-handler-4":
>   waiting to lock monitor 0x7fe0869e4408 (object 0x000749be7bb8, a 
> kafka.server.DelayedProduce),
>   which is held by "kafka-request-handler-3"
> "kafka-request-handler-3":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> Java stack information for the threads listed above:
> ===
> "kafka-request-handler-7":
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:752)
>   waiting to lock <0x00074a9fbc50> (a 
> kafka.coordinator.group.GroupMetadata)
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:750)
> at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
> at 
> kafka.coordinator.group.GroupMetadataManager.handleTxnCompletion(GroupMetadataManager.scala:750)
> at 
> kafka.coordinator.group.GroupCoordinator.handleTxnCompletion(GroupCoordinator.scala:439)
> at 
> kafka.server.KafkaApis.kafka$server$KafkaApis$$maybeSendResponseCallback$1(KafkaApis.scala:1556)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:134)
> at 
> kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:66)
> at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:116)
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:76)
>   locked <0x00074b21c968> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:284)
> at 
> kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:434)
> at 
> kafka.cluster.Partition.updateReplicaLogReadResult(Partition.scala:285)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1290)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1286)
> at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> at 
> kafka.server.ReplicaManager.updateFollowerLogReadResults(ReplicaManager.scala:1286)
> at kafka.server.ReplicaManager.fetchMessages(ReplicaManager.scala:786)
> at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:598)
> at kafka.server.KafkaApis.handle(KafkaApis.scala:100)
> at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:65)
> at java.lang.Thread.run(Thread.java:748)
> "kafka-request-handler-4":
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:75)
>   waiting to lock <0x000749be7bb8> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.ReplicaManager.tryCompleteDel

[jira] [Commented] (KAFKA-5970) Deadlock due to locking of DelayedProduce and group

2017-10-16 Thread Karolis Pocius (JIRA)

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

Karolis Pocius commented on KAFKA-5970:
---

We've applied the patch on CP 3.3.0 as the issue started to hit us every few 
hours. Haven't recurred in the last 24 hours and hopefully that's the last we 
saw it.

> Deadlock due to locking of DelayedProduce and group
> ---
>
> Key: KAFKA-5970
> URL: https://issues.apache.org/jira/browse/KAFKA-5970
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Reporter: Rajini Sivaram
>Assignee: Rajini Sivaram
>Priority: Blocker
> Fix For: 1.0.0, 0.11.0.2
>
> Attachments: jstack.txt
>
>
> From a local run of TransactionsBounceTest. Looks like we hold group lock 
> while completing DelayedProduce, which in turn may acquire group lock.
> {quote}
> Found one Java-level deadlock:
> =
> "kafka-request-handler-7":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> "kafka-request-handler-4":
>   waiting to lock monitor 0x7fe0869e4408 (object 0x000749be7bb8, a 
> kafka.server.DelayedProduce),
>   which is held by "kafka-request-handler-3"
> "kafka-request-handler-3":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> Java stack information for the threads listed above:
> ===
> "kafka-request-handler-7":
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:752)
>   waiting to lock <0x00074a9fbc50> (a 
> kafka.coordinator.group.GroupMetadata)
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:750)
> at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
> at 
> kafka.coordinator.group.GroupMetadataManager.handleTxnCompletion(GroupMetadataManager.scala:750)
> at 
> kafka.coordinator.group.GroupCoordinator.handleTxnCompletion(GroupCoordinator.scala:439)
> at 
> kafka.server.KafkaApis.kafka$server$KafkaApis$$maybeSendResponseCallback$1(KafkaApis.scala:1556)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:134)
> at 
> kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:66)
> at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:116)
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:76)
>   locked <0x00074b21c968> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:284)
> at 
> kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:434)
> at 
> kafka.cluster.Partition.updateReplicaLogReadResult(Partition.scala:285)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1290)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1286)
> at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> at 
> kafka.server.ReplicaManager.updateFollowerLogReadResults(ReplicaManager.scala:1286)
> at kafka.server.ReplicaManager.fetchMessages(ReplicaManager.scala:786)
> at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:598)
> at kafka.server.KafkaApis.handle(KafkaApis.scala:100)
> at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:65)
> at java.lang.Thread.run(Thread.java:748)
> "kafka-request-handler-4":
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:75)
>   waiting to lock <0x000749be7bb8> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.

[jira] [Commented] (KAFKA-5970) Deadlock due to locking of DelayedProduce and group

2017-10-16 Thread Guozhang Wang (JIRA)

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

Guozhang Wang commented on KAFKA-5970:
--

Glad to hear that, thanks Karolis!

> Deadlock due to locking of DelayedProduce and group
> ---
>
> Key: KAFKA-5970
> URL: https://issues.apache.org/jira/browse/KAFKA-5970
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Reporter: Rajini Sivaram
>Assignee: Rajini Sivaram
>Priority: Blocker
> Fix For: 1.0.0, 0.11.0.2
>
> Attachments: jstack.txt
>
>
> From a local run of TransactionsBounceTest. Looks like we hold group lock 
> while completing DelayedProduce, which in turn may acquire group lock.
> {quote}
> Found one Java-level deadlock:
> =
> "kafka-request-handler-7":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> "kafka-request-handler-4":
>   waiting to lock monitor 0x7fe0869e4408 (object 0x000749be7bb8, a 
> kafka.server.DelayedProduce),
>   which is held by "kafka-request-handler-3"
> "kafka-request-handler-3":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> Java stack information for the threads listed above:
> ===
> "kafka-request-handler-7":
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:752)
>   waiting to lock <0x00074a9fbc50> (a 
> kafka.coordinator.group.GroupMetadata)
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:750)
> at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
> at 
> kafka.coordinator.group.GroupMetadataManager.handleTxnCompletion(GroupMetadataManager.scala:750)
> at 
> kafka.coordinator.group.GroupCoordinator.handleTxnCompletion(GroupCoordinator.scala:439)
> at 
> kafka.server.KafkaApis.kafka$server$KafkaApis$$maybeSendResponseCallback$1(KafkaApis.scala:1556)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:134)
> at 
> kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:66)
> at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:116)
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:76)
>   locked <0x00074b21c968> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:284)
> at 
> kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:434)
> at 
> kafka.cluster.Partition.updateReplicaLogReadResult(Partition.scala:285)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1290)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1286)
> at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> at 
> kafka.server.ReplicaManager.updateFollowerLogReadResults(ReplicaManager.scala:1286)
> at kafka.server.ReplicaManager.fetchMessages(ReplicaManager.scala:786)
> at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:598)
> at kafka.server.KafkaApis.handle(KafkaApis.scala:100)
> at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:65)
> at java.lang.Thread.run(Thread.java:748)
> "kafka-request-handler-4":
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:75)
>   waiting to lock <0x000749be7bb8> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:284)
> at 
> kafka.clu

[jira] [Commented] (KAFKA-5970) Deadlock due to locking of DelayedProduce and group

2017-10-20 Thread Ben Corlett (JIRA)

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

Ben Corlett commented on KAFKA-5970:


Unfortunately we've had another incident today on broker 125 after applying the 
changes from pull request 3956. You can see the changes here 
https://github.com/corlettb/kafka/commits/deadlock. 

{noformat}
Found one Java-level deadlock:
=
"executor-Heartbeat":
  waiting to lock monitor 0x7fbd8c1834c8 (object 0x00068cccb590, a 
kafka.coordinator.group.GroupMetadata),
  which is held by "kafka-request-handler-7"
"kafka-request-handler-7":
  waiting to lock monitor 0x7fbe1942f698 (object 0x00068cd2c420, a 
kafka.coordinator.group.GroupMetadata),
  which is held by "kafka-request-handler-4"
"kafka-request-handler-4":
  waiting to lock monitor 0x7fbd8c1834c8 (object 0x00068cccb590, a 
kafka.coordinator.group.GroupMetadata),
  which is held by "kafka-request-handler-7"

Java stack information for the threads listed above:
===
"executor-Heartbeat":
at 
kafka.coordinator.group.GroupCoordinator.onExpireHeartbeat(GroupCoordinator.scala:776)
- waiting to lock <0x00068cccb590> (a 
kafka.coordinator.group.GroupMetadata)
at 
kafka.coordinator.group.DelayedHeartbeat.onExpiration(DelayedHeartbeat.scala:34)
at kafka.server.DelayedOperation.run(DelayedOperation.scala:120)
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:748)
"kafka-request-handler-7":
at 
kafka.coordinator.group.GroupMetadataManager.putCacheCallback$2(GroupMetadataManager.scala:311)
- waiting to lock <0x00068cd2c420> (a 
kafka.coordinator.group.GroupMetadata)
at 
kafka.coordinator.group.GroupMetadataManager.$anonfun$storeOffsets$10(GroupMetadataManager.scala:380)
at 
kafka.coordinator.group.GroupMetadataManager.$anonfun$storeOffsets$10$adapted(GroupMetadataManager.scala:380)
at 
kafka.coordinator.group.GroupMetadataManager$$Lambda$1045/747223912.apply(Unknown
 Source)
at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:124)
at 
kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:68)
at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:106)
at 
kafka.server.DelayedOperation.maybeTryComplete(DelayedOperation.scala:107)
at 
kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:347)
at 
kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:253)
at 
kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:250)
at 
kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:418)
at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:500)
at 
kafka.server.ReplicaManager.$anonfun$appendToLocalLog$2(ReplicaManager.scala:545)
at kafka.server.ReplicaManager$$Lambda$909/475609331.apply(Unknown 
Source)
at 
scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:234)
at scala.collection.TraversableLike$$Lambda$14/1859039536.apply(Unknown 
Source)
at scala.collection.immutable.Map$Map1.foreach(Map.scala:120)
at scala.collection.TraversableLike.map(TraversableLike.scala:234)
at scala.collection.TraversableLike.map$(TraversableLike.scala:227)
at scala.collection.AbstractTraversable.map(Traversable.scala:104)
at 
kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:531)
at kafka.server.ReplicaManager.appendRecords(ReplicaManager.scala:373)
at 
kafka.coordinator.group.GroupMetadataManager.appendForGroup(GroupMetadataManager.scala:245)
at 
kafka.coordinator.group.GroupMetadataManager.storeOffsets(GroupMetadataManager.scala:380)
at 
kafka.coordinator.group.GroupCoordinator.doCommitOffsets(GroupCoordinator.scala:465)
- locked <0x00068cccb590> (a kafka.coordinator.group.GroupMetadata)
at 
kafka.coordinator.group.GroupCoordinator.handleCommitOffsets(GroupCoordinator.scala:429)
at kafka.server.KafkaApis.handleOffsetCommitRequest(KafkaApis.scala:361)
at kafka.server.KafkaApis.handle(KafkaApis.scala:105)
at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:66)
at java.lang.Thread.run(Thread.java:748)
"kafka-request-handler-4":
at 
kafka.coordinator.group.GroupMetadataManager.putCacheCallback$2(GroupMetadataManager.scala:311)

[jira] [Commented] (KAFKA-5970) Deadlock due to locking of DelayedProduce and group

2017-10-20 Thread Ismael Juma (JIRA)

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

Ismael Juma commented on KAFKA-5970:


[~rsivaram], if it's a separate deadlock, maybe you can track it via KAFKA-6042 
since we did fix one as part of this JIRA.

> Deadlock due to locking of DelayedProduce and group
> ---
>
> Key: KAFKA-5970
> URL: https://issues.apache.org/jira/browse/KAFKA-5970
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Reporter: Rajini Sivaram
>Assignee: Rajini Sivaram
>Priority: Blocker
> Fix For: 1.0.0, 0.11.0.2
>
> Attachments: jstack.txt
>
>
> From a local run of TransactionsBounceTest. Looks like we hold group lock 
> while completing DelayedProduce, which in turn may acquire group lock.
> {quote}
> Found one Java-level deadlock:
> =
> "kafka-request-handler-7":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> "kafka-request-handler-4":
>   waiting to lock monitor 0x7fe0869e4408 (object 0x000749be7bb8, a 
> kafka.server.DelayedProduce),
>   which is held by "kafka-request-handler-3"
> "kafka-request-handler-3":
>   waiting to lock monitor 0x7fe08891fb08 (object 0x00074a9fbc50, a 
> kafka.coordinator.group.GroupMetadata),
>   which is held by "kafka-request-handler-4"
> Java stack information for the threads listed above:
> ===
> "kafka-request-handler-7":
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:752)
>   waiting to lock <0x00074a9fbc50> (a 
> kafka.coordinator.group.GroupMetadata)
> at 
> kafka.coordinator.group.GroupMetadataManager$$anonfun$handleTxnCompletion$1.apply(GroupMetadataManager.scala:750)
> at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
> at 
> kafka.coordinator.group.GroupMetadataManager.handleTxnCompletion(GroupMetadataManager.scala:750)
> at 
> kafka.coordinator.group.GroupCoordinator.handleTxnCompletion(GroupCoordinator.scala:439)
> at 
> kafka.server.KafkaApis.kafka$server$KafkaApis$$maybeSendResponseCallback$1(KafkaApis.scala:1556)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at 
> kafka.server.KafkaApis$$anonfun$handleWriteTxnMarkersRequest$1$$anonfun$apply$20.apply(KafkaApis.scala:1614)
> at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:134)
> at 
> kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:66)
> at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:116)
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:76)
>   locked <0x00074b21c968> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:284)
> at 
> kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:434)
> at 
> kafka.cluster.Partition.updateReplicaLogReadResult(Partition.scala:285)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1290)
> at 
> kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:1286)
> at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> at 
> kafka.server.ReplicaManager.updateFollowerLogReadResults(ReplicaManager.scala:1286)
> at kafka.server.ReplicaManager.fetchMessages(ReplicaManager.scala:786)
> at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:598)
> at kafka.server.KafkaApis.handle(KafkaApis.scala:100)
> at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:65)
> at java.lang.Thread.run(Thread.java:748)
> "kafka-request-handler-4":
> at 
> kafka.server.DelayedProduce.safeTryComplete(DelayedProduce.scala:75)
>   waiting to lock <0x000749be7bb8> (a kafka.server.DelayedProduce)
> at 
> kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:338)
> at 
> kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:244)
> at 
> kafka.server.Replica