[ https://issues.apache.org/jira/browse/KAFKA-6103?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Ismael Juma resolved KAFKA-6103. -------------------------------- Resolution: Duplicate > one broker appear to dead lock after running serval hours with a fresh cluster > ------------------------------------------------------------------------------ > > Key: KAFKA-6103 > URL: https://issues.apache.org/jira/browse/KAFKA-6103 > Project: Kafka > Issue Type: Bug > Components: metrics > Affects Versions: 0.10.1.0 > Environment: brokers: 3 > Ubuntu 16.04.1 LTS (GNU/Linux 4.4.0-96-generic x86_64) > cpu: 8 core mem: 16G > Reporter: Peyton Peng > > today we recreated a refresh kafka cluster with three brokers, at the > beginning everything runs well without exception. main configuration list as > below: > num.io.threads=16 > num.network.threads=3 > offsets.commit.timeout.ms=10000 > #offsets.topic.num.partitions=60 > default.replication.factor=3 > offsets.topic.replication.factor=3 > num.replica.fetchers=4 > replica.fetch.wait.max.ms=1000 > replica.lag.time.max.ms=20000 > replica.socket.receive.buffer.bytes=1048576 > replica.socket.timeout.ms=60000 > socket.receive.buffer.bytes=1048576 > socket.send.buffer.bytes=1048576 > log.dirs=/data/kafka-logs > num.partitions=12 > log.retention.hours=48 > log.roll.hours=48 > zookeeper.connect=**** > listeners=PLAINTEXT://:9092 > advertised.listeners=PLAINTEXT://*:9092 > broker.id=3 > after serval hours we got the the LOOP exception from consumer layer as below: > "Marking the coordinator ####### dead". > checked and found one broker running with no IO, cpu rate is ok, memory ok > also, but other two brokers throws exception : > java.io.IOException: Connection to 3 was disconnected before the response was > read > at > kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1$$anonfun$apply$1.apply(NetworkClientBlockingOps.scala:115) > at > kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1$$anonfun$apply$1.apply(NetworkClientBlockingOps.scala:112) > at scala.Option.foreach(Option.scala:257) > at > kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1.apply(NetworkClientBlockingOps.scala:112) > at > kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1.apply(NetworkClientBlockingOps.scala:108) > at > kafka.utils.NetworkClientBlockingOps$.recursivePoll$1(NetworkClientBlockingOps.scala:137) > at > kafka.utils.NetworkClientBlockingOps$.kafka$utils$NetworkClientBlockingOps$$pollContinuously$extension(NetworkClientBlockingOps.scala:143) > at > kafka.utils.NetworkClientBlockingOps$.blockingSendAndReceive$extension(NetworkClientBlockingOps.scala:108) > at > kafka.server.ReplicaFetcherThread.sendRequest(ReplicaFetcherThread.scala:253) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:238) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:42) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:118) > at > kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:103) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63) > finally we found the jvm stacks show dead lock: > Found one Java-level deadlock: > ============================= > "executor-Heartbeat": > waiting to lock monitor 0x00007f747c038db8 (object 0x00000005886a5ae8, a > kafka.coordinator.GroupMetadata), > which is held by "group-metadata-manager-0" > "group-metadata-manager-0": > waiting to lock monitor 0x00007f757c0085e8 (object 0x00000005a71fccb0, a > java.util.LinkedList), > which is held by "kafka-request-handler-7" > "kafka-request-handler-7": > waiting to lock monitor 0x00007f747c038db8 (object 0x00000005886a5ae8, a > kafka.coordinator.GroupMetadata), > which is held by "group-metadata-manager-0" > Java stack information for the threads listed above: > =================================================== > "executor-Heartbeat": > at > kafka.coordinator.GroupCoordinator.onExpireHeartbeat(GroupCoordinator.scala:739) > - waiting to lock <0x00000005886a5ae8> (a > kafka.coordinator.GroupMetadata) > at > kafka.coordinator.DelayedHeartbeat.onExpiration(DelayedHeartbeat.scala:33) > at kafka.server.DelayedOperation.run(DelayedOperation.scala:107) > 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:745) > "group-metadata-manager-0": > at > kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:308) > - waiting to lock <0x00000005a71fccb0> (a java.util.LinkedList) > at > kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:234) > at > kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:199) > at > kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:374) > at kafka.cluster.Partition.appendMessagesToLeader(Partition.scala:457) > at > kafka.coordinator.GroupMetadataManager$$anonfun$cleanupGroupMetadata$1$$anonfun$apply$10.apply(GroupMetadataManager.scala:600) > at > kafka.coordinator.GroupMetadataManager$$anonfun$cleanupGroupMetadata$1$$anonfun$apply$10.apply(GroupMetadataManager.scala:593) > at scala.Option.foreach(Option.scala:257) > at > kafka.coordinator.GroupMetadataManager$$anonfun$cleanupGroupMetadata$1.apply(GroupMetadataManager.scala:593) > - locked <0x00000005886a5ae8> (a kafka.coordinator.GroupMetadata) > at > kafka.coordinator.GroupMetadataManager$$anonfun$cleanupGroupMetadata$1.apply(GroupMetadataManager.scala:579) > at scala.collection.Iterator$class.foreach(Iterator.scala:893) > at kafka.utils.Pool$$anon$1.foreach(Pool.scala:89) > at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) > at kafka.utils.Pool.foreach(Pool.scala:26) > at > kafka.coordinator.GroupMetadataManager.cleanupGroupMetadata(GroupMetadataManager.scala:579) > at > kafka.coordinator.GroupMetadataManager$$anonfun$1.apply$mcV$sp(GroupMetadataManager.scala:101) > at > kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(KafkaScheduler.scala:110) > at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:58) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) > 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:745) > "kafka-request-handler-7": > at > kafka.coordinator.GroupMetadataManager.kafka$coordinator$GroupMetadataManager$$putCacheCallback$2(GroupMetadataManager.scala:301) > - waiting to lock <0x00000005886a5ae8> (a > kafka.coordinator.GroupMetadata) > at > kafka.coordinator.GroupMetadataManager$$anonfun$prepareStoreOffsets$1.apply(GroupMetadataManager.scala:357) > at > kafka.coordinator.GroupMetadataManager$$anonfun$prepareStoreOffsets$1.apply(GroupMetadataManager.scala:357) > at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:123) > at > kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:70) > at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:105) > at > kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:315) > - locked <0x00000005a71fcce8> (a kafka.server.DelayedProduce) > - locked <0x00000005a71fccb0> (a java.util.LinkedList) > at > kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:234) > at > kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:199) > at > kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:909) > at > kafka.server.ReplicaManager$$anonfun$updateFollowerLogReadResults$2.apply(ReplicaManager.scala:902) > 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:902) > at kafka.server.ReplicaManager.fetchMessages(ReplicaManager.scala:475) > at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:523) > at kafka.server.KafkaApis.handle(KafkaApis.scala:79) > at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:60) > at java.lang.Thread.run(Thread.java:745) > Found 1 deadlock. > (not sure if the dead lock appear to when we restart the kafka server.) -- This message was sent by Atlassian JIRA (v6.4.14#64029)