You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-issues@hadoop.apache.org by "Ravi Gummadi (Commented) (JIRA)" <ji...@apache.org> on 2011/12/08 06:05:40 UTC

[jira] [Commented] (MAPREDUCE-3519) Deadlock in LocalDirsHandlerService and ShuffleHandler

    [ https://issues.apache.org/jira/browse/MAPREDUCE-3519?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13164994#comment-13164994 ] 

Ravi Gummadi commented on MAPREDUCE-3519:
-----------------------------------------

Here is one deadlock that is observed:


Found one Java-level deadlock:
=============================
"ContainersLauncher #32":
  waiting to lock monitor 0x08e84674 (object 0xb5bec810, a org.apache.hadoop.conf.Configuration),
  which is held by "ContainersLauncher #27"
"ContainersLauncher #27":
  waiting to lock monitor 0x08ea65b0 (object 0xb5cd2898, a org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext),
  which is held by "New I/O server worker #1-3"
"New I/O server worker #1-3":
  waiting to lock monitor 0x08e84674 (object 0xb5bec810, a org.apache.hadoop.conf.Configuration),
  which is held by "ContainersLauncher #27"

Java stack information for the threads listed above:
===================================================
"ContainersLauncher #32":
        at org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService.getLogPathForWrite(LocalDirsHandlerService.java:283)
        - waiting to lock <0xb5bec810> (a org.apache.hadoop.conf.Configuration)
        at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:126)
        at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:68)
        at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
        at java.util.concurrent.FutureTask.run(FutureTask.java:138)
        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:619)
"ContainersLauncher #27":
        at org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:331)
        - waiting to lock <0xb5cd2898> (a org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext)
        at org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:150)
        at org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:131)
        at org.apache.hadoop.fs.LocalDirAllocator.getLocalPathForWrite(LocalDirAllocator.java:115)
        at org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService.getLocalPathForWrite(LocalDirsHandlerService.java:262)
        - locked <0xb5bec810> (a org.apache.hadoop.conf.Configuration)
        at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:150)
        at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:68)
        at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
        at java.util.concurrent.FutureTask.run(FutureTask.java:138)
        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:619)
"New I/O server worker #1-3":
        at org.apache.hadoop.conf.Configuration.getProps(Configuration.java:1373)
        - waiting to lock <0xb5bec810> (a org.apache.hadoop.conf.Configuration)
        at org.apache.hadoop.conf.Configuration.get(Configuration.java:569)
        at org.apache.hadoop.conf.Configuration.getTrimmed(Configuration.java:586)
        at org.apache.hadoop.conf.Configuration.getLong(Configuration.java:730)
        at org.apache.hadoop.fs.FileSystem.getDefaultBlockSize(FileSystem.java:1805)
        at org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:429)
        at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:315)
        at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1069)
        at org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathToRead(LocalDirAllocator.java:425)
        - locked <0xb5cd2898> (a org.apache.hadoop.fs.LocalDirAllocator$AllocatorPerContext)
        at org.apache.hadoop.fs.LocalDirAllocator.getLocalPathToRead(LocalDirAllocator.java:164)
        at org.apache.hadoop.mapred.ShuffleHandler$Shuffle.sendMapOutput(ShuffleHandler.java:464)
        at org.apache.hadoop.mapred.ShuffleHandler$Shuffle.messageReceived(ShuffleHandler.java:397)
        at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:80)
        at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:545)
        at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:754)
        at org.jboss.netty.handler.stream.ChunkedWriteHandler.handleUpstream(ChunkedWriteHandler.java:148)
        at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:545)
        at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:754)
        at org.jboss.netty.handler.codec.http.HttpChunkAggregator.messageReceived(HttpChunkAggregator.java:116)
        at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:80)
        at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:545)
        at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:754)
        at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:302)
        at org.jboss.netty.handler.codec.replay.ReplayingDecoder.unfoldAndfireMessageReceived(ReplayingDecoder.java:522)
        at org.jboss.netty.handler.codec.replay.ReplayingDecoder.callDecode(ReplayingDecoder.java:506)
        at org.jboss.netty.handler.codec.replay.ReplayingDecoder.messageReceived(ReplayingDecoder.java:443)
        at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:80)
        at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:545)
        at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:540)
        at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:274)
        at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:261)
        at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:349)
        at org.jboss.netty.channel.socket.nio.NioWorker.processSelectedKeys(NioWorker.java:280)
        at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:200)
        at org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108)
        at org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:44)
        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:619)

Found 1 deadlock.
                
> Deadlock in LocalDirsHandlerService and ShuffleHandler
> ------------------------------------------------------
>
>                 Key: MAPREDUCE-3519
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-3519
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>          Components: nodemanager
>    Affects Versions: 0.23.1, 0.24.0
>            Reporter: Ravi Gummadi
>            Assignee: Ravi Gummadi
>             Fix For: 0.23.1
>
>
> MAPREDUCE-3121 cloned Configuration object in LocalDirsHandlerService.init() to avoid others to access that configuration object. But since it is set in the base class of LocalDirsHandlerService using super.init(conf), it is reflected and is accessible to some other services. This is causing a deadlock when accessing this configuration object from LocalDirsHandlerService and ShuffleHandler along with AllocatorPerContext object.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira