Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-6665

LeaderChangeListener.handleDataDeleted deadlock

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Duplicate
    • 0.10.1.0
    • 0.10.2.0
    • core
    • None

    Description

      Leader election logic may cause the following deadlock:

      ZkClient-EventThread triggers kafka.utils.KafkaScheduler.shutdown under a global lock(kafka.utils.CoreUtils.inLock() on controllerContext.controllerLock):

      "ZkClient-EventThread-20-cube:2181" #20 daemon prio=5 os_prio=0 tid=0x00007f656cb9c000 nid=0x4f1 waiting on condition [0x00007f652df28000]
         java.lang.Thread.State: TIMED_WAITING (parking)
              at sun.misc.Unsafe.park(Native Method)
              - parking to wait for  <0x00000000d437a898> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
              at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
              at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
              at java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1465)
              at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:98)
              at kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:373)
              at kafka.controller.KafkaController$$anonfun$2.apply$mcV$sp(KafkaController.scala:168)
              at kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply$mcZ$sp(ZookeeperLeaderElector.scala:145)
              at kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:141)
              at kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:141)
              at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:234)
              at kafka.server.ZookeeperLeaderElector$LeaderChangeListener.handleDataDeleted(ZookeeperLeaderElector.scala:141)
              at org.I0Itec.zkclient.ZkClient$9.run(ZkClient.java:824)
              at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)
       

      The shutdown awaits for tasks termination with a huge timeout under the lock: https://github.com/apache/kafka/blob/0.10.1.0/core/src/main/scala/kafka/utils/KafkaScheduler.scala#L98

      cachedExecutor.awaitTermination(1, TimeUnit.DAYS)
      

      Tasks within the stopping scheduler thread pool tries to get the same global lock and blocked, so the pool is not terminating:

      "kafka-scheduler-293" #1249 daemon prio=5 os_prio=0 tid=0x00007f64d8054800 nid=0x7c72 waiting on condition [0x00007f6406ce3000]
         java.lang.Thread.State: WAITING (parking)
              at sun.misc.Unsafe.park(Native Method)
              - parking to wait for  <0x00000000c84f7ef8> (a java.util.concurrent.locks.ReentrantLock$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.acquireQueued(AbstractQueuedSynchronizer.java:870)
              at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199)
              at java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:209)
              at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:285)
              at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:232)
              at kafka.controller.KafkaController.isActive(KafkaController.scala:400)
              at kafka.controller.KafkaController.kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance(KafkaController.scala:1178)
              at kafka.controller.KafkaController$$anonfun$onControllerFailover$1.apply$mcV$sp(KafkaController.scala:347)
              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:748)
      

      In total, we have a lock loop for a long time.

      As a side effect retrieval of some JMX attributes is stuck as well, for example
      isActive check is also executed under the same lock:
      https://github.com/apache/kafka/blob/0.10.1.0/core/src/main/scala/kafka/controller/KafkaController.scala#L399

      "pool-1-thread-2" #59 daemon prio=5 os_prio=0 tid=0x00007f6504007000 nid=0x55f waiting on condition [0x00007f64072e6000]
         java.lang.Thread.State: WAITING (parking)
      	at sun.misc.Unsafe.park(Native Method)
      	- parking to wait for  <0x00000000c84f7ef8> (a java.util.concurrent.locks.ReentrantLock$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.acquireQueued(AbstractQueuedSynchronizer.java:870)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199)
      	at java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:209)
      	at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:285)
      	at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:232)
      	at kafka.controller.KafkaController$$anon$3.value(KafkaController.scala:208)
      	at kafka.controller.KafkaController$$anon$3.value(KafkaController.scala:206)
      	at com.yammer.metrics.reporting.JmxReporter$Gauge.getValue(JmxReporter.java:63)
      	at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source)
      	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      	at java.lang.reflect.Method.invoke(Method.java:498)
      	at sun.reflect.misc.Trampoline.invoke(MethodUtil.java:71)
      	at sun.reflect.GeneratedMethodAccessor1.invoke(Unknown Source)
      	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      	at java.lang.reflect.Method.invoke(Method.java:498)
      	at sun.reflect.misc.MethodUtil.invoke(MethodUtil.java:275)
      	at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:112)
      	at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:46)
      	at com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:237)
      	at com.sun.jmx.mbeanserver.PerInterface.getAttribute(PerInterface.java:83)
      	at com.sun.jmx.mbeanserver.MBeanSupport.getAttribute(MBeanSupport.java:206)
      	at com.sun.jmx.mbeanserver.MBeanSupport.getAttributes(MBeanSupport.java:213)
      	at com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.getAttributes(DefaultMBeanServerInterceptor.java:709)
      	at com.sun.jmx.mbeanserver.JmxMBeanServer.getAttributes(JmxMBeanServer.java:705)
      

      Attachments

        1. thread_dump.txt
          65 kB
          Dmitry Konstantinov

        Issue Links

          Activity

            People

              Unassigned Unassigned
              dnk Dmitry Konstantinov
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: