[ 
https://issues.apache.org/jira/browse/KAFKA-2454?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Joel Koshy updated KAFKA-2454:
------------------------------
    Resolution: Fixed
        Status: Resolved  (was: Patch Available)

Issue resolved by pull request 153
[https://github.com/apache/kafka/pull/153]

> Dead lock between delete log segment and shutting down.
> -------------------------------------------------------
>
>                 Key: KAFKA-2454
>                 URL: https://issues.apache.org/jira/browse/KAFKA-2454
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: Jiangjie Qin
>            Assignee: Jiangjie Qin
>             Fix For: 0.9.0.0
>
>
> When the broker shutdown, it will shutdown scheduler which grabs the 
> scheduler lock then wait for all the threads in scheduler to shutdown.
> The dead lock will happen when the scheduled task try to delete old log 
> segment, it will schedule a log delete task which also needs to acquire the 
> scheduler lock. In this case the shutdown thread will hold scheduler lock and 
> wait for the the log deletion thread to finish, but the log deletion thread 
> will block on waiting for the scheduler lock.
> Related stack trace:
> {noformat}
> "Thread-1" #21 prio=5 os_prio=0 tid=0x00007fe7601a7000 nid=0x1a4e waiting on 
> condition [0x00007fe7cf698000]
>    java.lang.Thread.State: TIMED_WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x0000000640d53540> (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:94)
>         - locked <0x0000000640b6d480> (a kafka.utils.KafkaScheduler)
>         at 
> kafka.server.KafkaServer$$anonfun$shutdown$4.apply$mcV$sp(KafkaServer.scala:352)
>         at kafka.utils.CoreUtils$.swallow(CoreUtils.scala:79)
>         at kafka.utils.Logging$class.swallowWarn(Logging.scala:92)
>         at kafka.utils.CoreUtils$.swallowWarn(CoreUtils.scala:51)
>         at kafka.utils.Logging$class.swallow(Logging.scala:94)
>         at kafka.utils.CoreUtils$.swallow(CoreUtils.scala:51)
>         at kafka.server.KafkaServer.shutdown(KafkaServer.scala:352)
>         at 
> kafka.server.KafkaServerStartable.shutdown(KafkaServerStartable.scala:42)
>         at com.linkedin.kafka.KafkaServer.notifyShutdown(KafkaServer.java:99)
>         at 
> com.linkedin.util.factory.lifecycle.LifeCycleMgr.notifyShutdownListener(LifeCycleMgr.java:123)
>         at 
> com.linkedin.util.factory.lifecycle.LifeCycleMgr.notifyListeners(LifeCycleMgr.java:102)
>         at 
> com.linkedin.util.factory.lifecycle.LifeCycleMgr.notifyStop(LifeCycleMgr.java:82)
>         - locked <0x0000000640b77bb0> (a java.util.ArrayDeque)
>         at com.linkedin.util.factory.Generator.stop(Generator.java:177)
>         - locked <0x0000000640b77bc8> (a java.lang.Object)
>         at 
> com.linkedin.offspring.servlet.OffspringServletRuntime.destroy(OffspringServletRuntime.java:82)
>         at 
> com.linkedin.offspring.servlet.OffspringServletContextListener.contextDestroyed(OffspringServletContextListener.java:51)
>         at 
> org.eclipse.jetty.server.handler.ContextHandler.doStop(ContextHandler.java:813)
>         at 
> org.eclipse.jetty.servlet.ServletContextHandler.doStop(ServletContextHandler.java:160)
>         at 
> org.eclipse.jetty.webapp.WebAppContext.doStop(WebAppContext.java:516)
>         at com.linkedin.emweb.WebappContext.doStop(WebappContext.java:35)
>         at 
> org.eclipse.jetty.util.component.AbstractLifeCycle.stop(AbstractLifeCycle.java:89)
>         - locked <0x00000006400018b8> (a java.lang.Object)
>         at 
> com.linkedin.emweb.ContextBasedHandlerImpl.doStop(ContextBasedHandlerImpl.java:112)
>         at 
> org.eclipse.jetty.util.component.AbstractLifeCycle.stop(AbstractLifeCycle.java:89)
>         - locked <0x0000000640001900> (a java.lang.Object)
>         at 
> com.linkedin.emweb.WebappDeployerImpl.stop(WebappDeployerImpl.java:349)
>         at 
> com.linkedin.emweb.WebappDeployerImpl.doStop(WebappDeployerImpl.java:414)
>         - locked <0x00000006400019c0> (a 
> com.linkedin.emweb.MapBasedHandlerImpl)
>         at 
> org.eclipse.jetty.util.component.AbstractLifeCycle.stop(AbstractLifeCycle.java:89)
>         - locked <0x00000006404ee8e8> (a java.lang.Object)
>         at 
> org.eclipse.jetty.util.component.AggregateLifeCycle.doStop(AggregateLifeCycle.java:107)
>         at 
> org.eclipse.jetty.server.handler.AbstractHandler.doStop(AbstractHandler.java:69)
>         at 
> org.eclipse.jetty.server.handler.HandlerWrapper.doStop(HandlerWrapper.java:108)
>         at org.eclipse.jetty.server.Server.doStop(Server.java:338)
>         at 
> org.eclipse.jetty.util.component.AbstractLifeCycle.stop(AbstractLifeCycle.java:89)
>         - locked <0x0000000640476ab0> (a java.lang.Object)
>         at com.linkedin.emweb.BaseRunner.destroy(BaseRunner.java:162)
>         at 
> com.linkedin.spring.core.TerminationHandler.destroy(TerminationHandler.java:151)
>         at 
> com.linkedin.spring.core.TerminationHandler.runTermination(TerminationHandler.java:113)
>         at 
> com.linkedin.spring.core.TerminationHandler.destroy(TerminationHandler.java:79)
>         at 
> com.linkedin.spring.core.SettableFactoryBean.destroy(SettableFactoryBean.java:68)
>         at 
> org.springframework.beans.factory.support.DisposableBeanAdapter.destroy(DisposableBeanAdapter.java:211)
>         at 
> org.springframework.beans.factory.support.DefaultSingletonBeanRegistry.destroyBean(DefaultSingletonBeanRegistry.java:500)
>         at 
> org.springframework.beans.factory.support.DefaultSingletonBeanRegistry.destroySingleton(DefaultSingletonBeanRegistry.java:476)
>         at 
> org.springframework.beans.factory.support.DefaultSingletonBeanRegistry.destroySingletons(DefaultSingletonBeanRegistry.java:445)
>         at 
> org.springframework.context.support.AbstractApplicationContext.destroyBeans(AbstractApplicationContext.java:1078)
>         at 
> org.springframework.context.support.AbstractApplicationContext.doClose(AbstractApplicationContext.java:1052)
>         at 
> org.springframework.context.support.AbstractApplicationContext.close(AbstractApplicationContext.java:1000)
>         - locked <0x000000064001c718> (a java.lang.Object)
>         at 
> com.linkedin.spring.cmdline.CmdLineAppRunner.terminate(CmdLineAppRunner.java:277)
>         at 
> com.linkedin.spring.cmdline.CmdLineAppRunner$1.run(CmdLineAppRunner.java:219)
> ...
> "kafka-scheduler-2" #272 daemon prio=5 os_prio=0 tid=0x00007fecd58be000 
> nid=0x7868 waiting for monitor entry [0x00007feb991d4000]
>    java.lang.Thread.State: BLOCKED (on object monitor)
>         at kafka.utils.KafkaScheduler.schedule(KafkaScheduler.scala:103)
>         - waiting to lock <0x0000000640b6d480> (a kafka.utils.KafkaScheduler)
>         at kafka.log.Log.kafka$log$Log$$asyncDeleteSegment(Log.scala:789)
>         at kafka.log.Log.kafka$log$Log$$deleteSegment(Log.scala:775)
>         - locked <0x0000000641d18030> (a java.lang.Object)
>         at kafka.log.Log$$anonfun$deleteOldSegments$1.apply(Log.scala:533)
>         at kafka.log.Log$$anonfun$deleteOldSegments$1.apply(Log.scala:533)
>         at scala.collection.immutable.List.foreach(List.scala:318)
>         at kafka.log.Log.deleteOldSegments(Log.scala:533)
>         - locked <0x0000000641d18030> (a java.lang.Object)
>         at 
> kafka.log.LogManager.kafka$log$LogManager$$cleanupExpiredSegments(LogManager.scala:421)
>         at 
> kafka.log.LogManager$$anonfun$cleanupLogs$3.apply(LogManager.scala:452)
>         at 
> kafka.log.LogManager$$anonfun$cleanupLogs$3.apply(LogManager.scala:450)
>         at 
> scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:772)
>         at scala.collection.Iterator$class.foreach(Iterator.scala:727)
>         at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
>         at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
>         at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
>         at 
> scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:771)
>         at kafka.log.LogManager.cleanupLogs(LogManager.scala:450)
>         at 
> kafka.log.LogManager$$anonfun$startup$1.apply$mcV$sp(LogManager.scala:190)
>         at 
> kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(KafkaScheduler.scala:108)
>         at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:60)
>         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)
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to