You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "little brother ma (JIRA)" <ji...@apache.org> on 2019/05/05 01:48:01 UTC
[jira] [Comment Edited] (KAFKA-7697) Possible deadlock in
kafka.cluster.Partition
[ https://issues.apache.org/jira/browse/KAFKA-7697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16808327#comment-16808327 ]
little brother ma edited comment on KAFKA-7697 at 5/5/19 1:47 AM:
------------------------------------------------------------------
we also hit the same issue with 2.1.1 !
Get the metric "kafka.network:type=RequestChannel,name=RequestQueueSize" value is always 1000, and we config queued.max.requests=1000
kafka-network-thread-5-ListenerName(PLAINTEXT)-PLAINTEXT-4" #97 prio=5 os_prio=0 tid=0x00007fb7ce0ba800 nid=0x2d5 waiting on condition [0x00007fad6e5f8000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00000004530783a0> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
at java.util.concurrent.ArrayBlockingQueue.put(ArrayBlockingQueue.java:353)
at kafka.network.RequestChannel.sendRequest(RequestChannel.scala:310)
at kafka.network.Processor.$anonfun$processCompletedReceives$1(SocketServer.scala:709)
at kafka.network.Processor.$anonfun$processCompletedReceives$1$adapted(SocketServer.scala:699)
at kafka.network.Processor$$Lambda$877/855310793.apply(Unknown Source)
at scala.collection.Iterator.foreach(Iterator.scala:937)
at scala.collection.Iterator.foreach$(Iterator.scala:937)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1425)
at scala.collection.IterableLike.foreach(IterableLike.scala:70)
at scala.collection.IterableLike.foreach$(IterableLike.scala:69)
at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
at kafka.network.Processor.processCompletedReceives(SocketServer.scala:699)
at kafka.network.Processor.run(SocketServer.scala:595)
at java.lang.Thread.run(Thread.java:748)
Locked ownable synchronizers:
- None
"kafka-request-handler-15" #87 daemon prio=5 os_prio=0 tid=0x00007fb7ceee6800 nid=0x2cb waiting on condition [0x00007fad71af4000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00000004540423f0> (a java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireShared(AbstractQueuedSynchronizer.java:967)
at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireShared(AbstractQueuedSynchronizer.java:1283)
at java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.lock(ReentrantReadWriteLock.java:727)
at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:249)
at kafka.utils.CoreUtils$.inReadLock(CoreUtils.scala:257)
at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:729)
at kafka.server.ReplicaManager.$anonfun$appendToLocalLog$2(ReplicaManager.scala:735)
at kafka.server.ReplicaManager$$Lambda$1567/915411568.apply(Unknown Source)
at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:233)
at scala.collection.TraversableLike$$Lambda$12/811760110.apply(Unknown Source)
at scala.collection.immutable.Map$Map1.foreach(Map.scala:125)
at scala.collection.TraversableLike.map(TraversableLike.scala:233)
at scala.collection.TraversableLike.map$(TraversableLike.scala:226)
at scala.collection.AbstractTraversable.map(Traversable.scala:104)
at kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:723)
at kafka.server.ReplicaManager.appendRecords(ReplicaManager.scala:470)
at kafka.coordinator.group.GroupMetadataManager.appendForGroup(GroupMetadataManager.scala:280)
at kafka.coordinator.group.GroupMetadataManager.storeOffsets(GroupMetadataManager.scala:423)
at kafka.coordinator.group.GroupCoordinator.$anonfun$doCommitOffsets$1(GroupCoordinator.scala:518)
at kafka.coordinator.group.GroupCoordinator$$Lambda$1816/513285617.apply$mcV$sp(Unknown Source)
at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:12)
at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251)
at kafka.coordinator.group.GroupMetadata.inLock(GroupMetadata.scala:197)
at kafka.coordinator.group.GroupCoordinator.doCommitOffsets(GroupCoordinator.scala:503)
at kafka.coordinator.group.GroupCoordinator.handleCommitOffsets(GroupCoordinator.scala:482)
at kafka.server.KafkaApis.handleOffsetCommitRequest(KafkaApis.scala:365)
at kafka.server.KafkaApis.handle(KafkaApis.scala:114)
at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:69)
at java.lang.Thread.run(Thread.java:748)
Locked ownable synchronizers:
- <0x0000000794ea4248> (a java.util.concurrent.locks.ReentrantLock$NonfairSync)
-
-
- [^kafka_jstack.txt]
was (Author: little brother ma):
we also hit the same issue with 2.1.1 !
Get the metric "kafka.network:type=RequestChannel,name=RequestQueueSize" value is always 1000, and we config queued.max.requests=1000
kafka-network-thread-5-ListenerName(PLAINTEXT)-PLAINTEXT-4" #97 prio=5 os_prio=0 tid=0x00007fb7ce0ba800 nid=0x2d5 waiting on condition [0x00007fad6e5f8000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00000004530783a0> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
at java.util.concurrent.ArrayBlockingQueue.put(ArrayBlockingQueue.java:353)
at kafka.network.RequestChannel.sendRequest(RequestChannel.scala:310)
at kafka.network.Processor.$anonfun$processCompletedReceives$1(SocketServer.scala:709)
at kafka.network.Processor.$anonfun$processCompletedReceives$1$adapted(SocketServer.scala:699)
at kafka.network.Processor$$Lambda$877/855310793.apply(Unknown Source)
at scala.collection.Iterator.foreach(Iterator.scala:937)
at scala.collection.Iterator.foreach$(Iterator.scala:937)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1425)
at scala.collection.IterableLike.foreach(IterableLike.scala:70)
at scala.collection.IterableLike.foreach$(IterableLike.scala:69)
at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
at kafka.network.Processor.processCompletedReceives(SocketServer.scala:699)
at kafka.network.Processor.run(SocketServer.scala:595)
at java.lang.Thread.run(Thread.java:748)
Locked ownable synchronizers:
- None
"kafka-request-handler-15" #87 daemon prio=5 os_prio=0 tid=0x00007fb7ceee6800 nid=0x2cb waiting on condition [0x00007fad71af4000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00000004540423f0> (a java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireShared(AbstractQueuedSynchronizer.java:967)
at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireShared(AbstractQueuedSynchronizer.java:1283)
at java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.lock(ReentrantReadWriteLock.java:727)
at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:249)
at kafka.utils.CoreUtils$.inReadLock(CoreUtils.scala:257)
at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:729)
at kafka.server.ReplicaManager.$anonfun$appendToLocalLog$2(ReplicaManager.scala:735)
at kafka.server.ReplicaManager$$Lambda$1567/915411568.apply(Unknown Source)
at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:233)
at scala.collection.TraversableLike$$Lambda$12/811760110.apply(Unknown Source)
at scala.collection.immutable.Map$Map1.foreach(Map.scala:125)
at scala.collection.TraversableLike.map(TraversableLike.scala:233)
at scala.collection.TraversableLike.map$(TraversableLike.scala:226)
at scala.collection.AbstractTraversable.map(Traversable.scala:104)
at kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:723)
at kafka.server.ReplicaManager.appendRecords(ReplicaManager.scala:470)
at kafka.coordinator.group.GroupMetadataManager.appendForGroup(GroupMetadataManager.scala:280)
at kafka.coordinator.group.GroupMetadataManager.storeOffsets(GroupMetadataManager.scala:423)
at kafka.coordinator.group.GroupCoordinator.$anonfun$doCommitOffsets$1(GroupCoordinator.scala:518)
at kafka.coordinator.group.GroupCoordinator$$Lambda$1816/513285617.apply$mcV$sp(Unknown Source)
at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:12)
at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251)
at kafka.coordinator.group.GroupMetadata.inLock(GroupMetadata.scala:197)
at kafka.coordinator.group.GroupCoordinator.doCommitOffsets(GroupCoordinator.scala:503)
at kafka.coordinator.group.GroupCoordinator.handleCommitOffsets(GroupCoordinator.scala:482)
at kafka.server.KafkaApis.handleOffsetCommitRequest(KafkaApis.scala:365)
at kafka.server.KafkaApis.handle(KafkaApis.scala:114)
at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:69)
at java.lang.Thread.run(Thread.java:748)
Locked ownable synchronizers:
- <0x0000000794ea4248> (a java.util.concurrent.locks.ReentrantLock$NonfairSync)
> Possible deadlock in kafka.cluster.Partition
> --------------------------------------------
>
> Key: KAFKA-7697
> URL: https://issues.apache.org/jira/browse/KAFKA-7697
> Project: Kafka
> Issue Type: Bug
> Affects Versions: 2.1.0
> Reporter: Gian Merlino
> Assignee: Rajini Sivaram
> Priority: Blocker
> Fix For: 2.2.0, 2.1.1
>
> Attachments: kafka.log, kafka_jstack.txt, threaddump.txt
>
>
> After upgrading a fairly busy broker from 0.10.2.0 to 2.1.0, it locked up within a few minutes (by "locked up" I mean that all request handler threads were busy, and other brokers reported that they couldn't communicate with it). I restarted it a few times and it did the same thing each time. After downgrading to 0.10.2.0, the broker was stable. I attached a thread dump from the last attempt on 2.1.0 that shows lots of kafka-request-handler- threads trying to acquire the leaderIsrUpdateLock lock in kafka.cluster.Partition.
> It jumps out that there are two threads that already have some read lock (can't tell which one) and are trying to acquire a second one (on two different read locks: 0x0000000708184b88 and 0x000000070821f188): kafka-request-handler-1 and kafka-request-handler-4. Both are handling a produce request, and in the process of doing so, are calling Partition.fetchOffsetSnapshot while trying to complete a DelayedFetch. At the same time, both of those locks have writers from other threads waiting on them (kafka-request-handler-2 and kafka-scheduler-6). Neither of those locks appear to have writers that hold them (if only because no threads in the dump are deep enough in inWriteLock to indicate that).
> ReentrantReadWriteLock in nonfair mode prioritizes waiting writers over readers. Is it possible that kafka-request-handler-1 and kafka-request-handler-4 are each trying to read-lock the partition that is currently locked by the other one, and they're both parked waiting for kafka-request-handler-2 and kafka-scheduler-6 to get write locks, which they never will, because the former two threads own read locks and aren't giving them up?
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)