You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Dmitry Bugaychenko (JIRA)" <ji...@apache.org> on 2014/04/28 23:37:15 UTC

[jira] [Created] (KAFKA-1429) Yet another deadlock in controller shutdown

Dmitry Bugaychenko created KAFKA-1429:
-----------------------------------------

             Summary: Yet another deadlock in controller shutdown
                 Key: KAFKA-1429
                 URL: https://issues.apache.org/jira/browse/KAFKA-1429
             Project: Kafka
          Issue Type: Bug
          Components: controller
    Affects Versions: 0.8.1
            Reporter: Dmitry Bugaychenko
            Assignee: Neha Narkhede


Found one more case of deadlock in controller during shutdown:

{code}
ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181 id=57 state=TIMED_WAITING
    - waiting on <0x288a66ec> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
    - locked <0x288a66ec> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
    at sun.misc.Unsafe.park(Native Method)
    at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226)
    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2082)
    at java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1468)
    at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:88)
    at kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply$mcV$sp(KafkaController.scala:339)
    at kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337)
    at kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337)
    at kafka.utils.Utils$.inLock(Utils.scala:538)
    at kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:337)
    at kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply$mcZ$sp(KafkaController.scala:1068)
    at kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067)
    at kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067)
    at kafka.utils.Utils$.inLock(Utils.scala:538)
    at kafka.controller.KafkaController$SessionExpirationListener.handleNewSession(KafkaController.scala:1067)
    at org.I0Itec.zkclient.ZkClient$4.run(ZkClient.java:472)
    at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)

    Locked synchronizers: count = 1
      - java.util.concurrent.locks.ReentrantLock$NonfairSync@22b9b31a

kafka-scheduler-0 id=172 state=WAITING
    - waiting on <0x22b9b31a> (a java.util.concurrent.locks.ReentrantLock$NonfairSync)
    - locked <0x22b9b31a> (a java.util.concurrent.locks.ReentrantLock$NonfairSync)
     owned by ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181 id=57
    at sun.misc.Unsafe.park(Native Method)
    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
    at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:834)
    at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:867)
    at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1197)
    at java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:214)
    at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:290)
    at kafka.utils.Utils$.inLock(Utils.scala:536)
    at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1110)
    at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1108)
    at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98)
    at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98)
    at scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226)
    at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39)
    at scala.collection.mutable.HashMap.foreach(HashMap.scala:98)
    at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1108)
    at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1087)
    at scala.collection.immutable.Map$Map4.foreach(Map.scala:181)
    at kafka.controller.KafkaController.kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance(KafkaController.scala:1087)
    at kafka.controller.KafkaController$$anonfun$onControllerFailover$1.apply$mcV$sp(KafkaController.scala:323)
    at kafka.utils.KafkaScheduler$$anon$1.run(KafkaScheduler.scala:100)
    at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
    at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304)
    at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178)
    at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
    at java.lang.Thread.run(Thread.java:744)

    Locked synchronizers: count = 1
      - java.util.concurrent.ThreadPoolExecutor$Worker@7d6e8aba
{code}

Just before the shutdown broker entered a state where it was not able to update ISR and replication was not progressing (this was the reason for restart):

{code}
Apr 29 01:09:52 srvd2229 odnoklassniki-databus[5571]: 2014-04-29 01:09:52,189  INFO [kafka-scheduler-1] Partition - Partition [stabilityTestSecond,9] on broker 4: Shrinking ISR for partition [stabilityTe
stSecond,9] from 4,2 to 4
Apr 29 01:09:52 srvd2229 odnoklassniki-databus[5571]: 2014-04-29 01:09:52,210 ERROR [kafka-scheduler-1] ZkUtils$ - Conditional update of path /brokers/topics/stabilityTestSecond/partitions/9/state with d
ata {"controller_epoch":20,"leader":4,"version":1,"leader_epoch":38,"isr":[4]} and expected version 134 failed due to org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = BadVersio
n for /brokers/topics/stabilityTestSecond/partitions/9/state
Apr 29 01:09:52 srvd2229 odnoklassniki-databus[5571]: 2014-04-29 01:09:52,210  INFO [kafka-scheduler-1] Partition - Partition [stabilityTestSecond,9] on broker 4: Cached zkVersion [134] not equal to that
 in zookeeper, skip updating ISR
{code}

Right before the broker entered this state there was a temporary connection loss to ZK due to GC fo 5 seconds.



--
This message was sent by Atlassian JIRA
(v6.2#6252)