From jira-return-10798-archive-asf-public=cust-asf.ponee.io@kafka.apache.org Thu Mar 15 20:25:06 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 7C422180654 for ; Thu, 15 Mar 2018 20:25:05 +0100 (CET) Received: (qmail 80752 invoked by uid 500); 15 Mar 2018 19:25:04 -0000 Mailing-List: contact jira-help@kafka.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: jira@kafka.apache.org Delivered-To: mailing list jira@kafka.apache.org Received: (qmail 80737 invoked by uid 99); 15 Mar 2018 19:25:04 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 15 Mar 2018 19:25:04 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id 1FA2EC0582 for ; Thu, 15 Mar 2018 19:25:04 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -110.311 X-Spam-Level: X-Spam-Status: No, score=-110.311 tagged_above=-999 required=6.31 tests=[ENV_AND_HDR_SPF_MATCH=-0.5, RCVD_IN_DNSWL_MED=-2.3, SPF_PASS=-0.001, T_RP_MATCHES_RCVD=-0.01, USER_IN_DEF_SPF_WL=-7.5, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id XrHzqS3GiVGx for ; Thu, 15 Mar 2018 19:25:02 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id 2BF675F167 for ; Thu, 15 Mar 2018 19:25:01 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 668F0E0CF1 for ; Thu, 15 Mar 2018 19:25:00 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 19F9B2148F for ; Thu, 15 Mar 2018 19:25:00 +0000 (UTC) Date: Thu, 15 Mar 2018 19:25:00 +0000 (UTC) From: "Dmitry Konstantinov (JIRA)" To: jira@kafka.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (KAFKA-6665) LeaderChangeListener.handleDataDeleted deadlock MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/KAFKA-6665?page=3Dcom.atlassia= n.jira.plugin.system.issuetabpanels:all-tabpanel ] Dmitry Konstantinov updated KAFKA-6665: --------------------------------------- Attachment: thread_dump.txt > LeaderChangeListener.handleDataDeleted deadlock > ----------------------------------------------- > > Key: KAFKA-6665 > URL: https://issues.apache.org/jira/browse/KAFKA-6665 > Project: Kafka > Issue Type: Bug > Components: core > Affects Versions: 0.10.1.0 > Environment: Linux 3.10.0-327.10.1.el7.x86_64 #1 SMP Sat Jan 23 0= 4:54:55 EST 2016 x86_64 x86_64 x86_64 GNU/Linux > java version "1.8.0_131" > Java(TM) SE Runtime Environment (build 1.8.0_131-b11) > Java HotSpot(TM) 64-Bit Server VM (build 25.131-b11, mixed mode) > Reporter: Dmitry Konstantinov > Priority: Major > Attachments: thread_dump.txt > > > 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.controller= Lock): > {code:java} > "ZkClient-EventThread-20-cube:2181" #20 daemon prio=3D5 os_prio=3D0 tid= =3D0x00007f656cb9c000 nid=3D0x4f1 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.concurre= nt.locks.AbstractQueuedSynchronizer$ConditionObject) > at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.j= ava:215) > at java.util.concurrent.locks.AbstractQueuedSynchronizer$Conditio= nObject.awaitNanos(AbstractQueuedSynchronizer.java:2078) > at java.util.concurrent.ThreadPoolExecutor.awaitTermination(Threa= dPoolExecutor.java:1465) > at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:98) > at kafka.controller.KafkaController.onControllerResignation(Kafka= Controller.scala:373) > at kafka.controller.KafkaController$$anonfun$2.apply$mcV$sp(Kafka= Controller.scala:168) > at kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anon= fun$handleDataDeleted$1.apply$mcZ$sp(ZookeeperLeaderElector.scala:145) > at kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anon= fun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:141) > at kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anon= fun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:141) > at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:234) > at kafka.server.ZookeeperLeaderElector$LeaderChangeListener.handl= eDataDeleted(ZookeeperLeaderElector.scala:141) > at org.I0Itec.zkclient.ZkClient$9.run(ZkClient.java:824) > at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71) > =C2=A0{code} > The shutdown awaits for tasks termination with a huge timeout under the l= ock: [https://github.com/apache/kafka/blob/0.10.1.0/core/src/main/scala/kaf= ka/utils/KafkaScheduler.scala#L98] > {code:java} > cachedExecutor.awaitTermination(1, TimeUnit.DAYS) > {code} > Tasks within the stopping scheduler thread pool tries to get the same glo= bal lock and blocked, so the pool is not terminating: > {code:java} > "kafka-scheduler-293" #1249 daemon prio=3D5 os_prio=3D0 tid=3D0x00007f64d= 8054800 nid=3D0x7c72 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.concurre= nt.locks.ReentrantLock$NonfairSync) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:1= 75) > at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndC= heckInterrupt(AbstractQueuedSynchronizer.java:836) > at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQ= ueued(AbstractQueuedSynchronizer.java:870) > at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(= AbstractQueuedSynchronizer.java:1199) > at java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(Reen= trantLock.java:209) > at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.ja= va:285) > at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:232) > at kafka.controller.KafkaController.isActive(KafkaController.scal= a:400) > at kafka.controller.KafkaController.kafka$controller$KafkaControl= ler$$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(KafkaSchedu= ler.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:30= 8) > at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutu= reTask.access$301(ScheduledThreadPoolExecutor.java:180) > at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutu= reTask.run(ScheduledThreadPoolExecutor.java:294) > at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolEx= ecutor.java:1142) > at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolE= xecutor.java:617) > at java.lang.Thread.run(Thread.java:748) > {code} > 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 e= xample=20 > 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] > {code:java} > "pool-1-thread-2" #59 daemon prio=3D5 os_prio=3D0 tid=3D0x00007f650400700= 0 nid=3D0x55f waiting on condition [0x00007f64072e6000] > java.lang.Thread.State: WAITING (parking) > =09at sun.misc.Unsafe.park(Native Method) > =09- parking to wait for <0x00000000c84f7ef8> (a java.util.concurrent.lo= cks.ReentrantLock$NonfairSync) > =09at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) > =09at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckI= nterrupt(AbstractQueuedSynchronizer.java:836) > =09at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued= (AbstractQueuedSynchronizer.java:870) > =09at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(Abstr= actQueuedSynchronizer.java:1199) > =09at java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(Reentrant= Lock.java:209) > =09at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:28= 5) > =09at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:232) > =09at kafka.controller.KafkaController$$anon$3.value(KafkaController.scal= a:208) > =09at kafka.controller.KafkaController$$anon$3.value(KafkaController.scal= a:206) > =09at com.yammer.metrics.reporting.JmxReporter$Gauge.getValue(JmxReporter= .java:63) > =09at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source) > =09at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAcc= essorImpl.java:43) > =09at java.lang.reflect.Method.invoke(Method.java:498) > =09at sun.reflect.misc.Trampoline.invoke(MethodUtil.java:71) > =09at sun.reflect.GeneratedMethodAccessor1.invoke(Unknown Source) > =09at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAcc= essorImpl.java:43) > =09at java.lang.reflect.Method.invoke(Method.java:498) > =09at sun.reflect.misc.MethodUtil.invoke(MethodUtil.java:275) > =09at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(Standard= MBeanIntrospector.java:112) > =09at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(Standard= MBeanIntrospector.java:46) > =09at com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector= .java:237) > =09at com.sun.jmx.mbeanserver.PerInterface.getAttribute(PerInterface.java= :83) > =09at com.sun.jmx.mbeanserver.MBeanSupport.getAttribute(MBeanSupport.java= :206) > =09at com.sun.jmx.mbeanserver.MBeanSupport.getAttributes(MBeanSupport.jav= a:213) > =09at com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.getAttributes= (DefaultMBeanServerInterceptor.java:709) > =09at com.sun.jmx.mbeanserver.JmxMBeanServer.getAttributes(JmxMBeanServer= .java:705) > {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)