Hadoop Map/Reduce
  1. Hadoop Map/Reduce
  2. MAPREDUCE-4576

Large dist cache can block tasktracker heartbeat

    Details

    • Type: Bug Bug
    • Status: Closed
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: 0.20.205.0, 1.0.0
    • Fix Version/s: 1.2.0
    • Component/s: None
    • Labels:
      None
    • Target Version/s:
    1. MR-4576.txt
      5 kB
      Robert Joseph Evans

      Issue Links

        Activity

        Hide
        Matt Foley added a comment -

        Closed upon release of Hadoop 1.2.0.

        Show
        Matt Foley added a comment - Closed upon release of Hadoop 1.2.0.
        Hide
        Thomas Graves added a comment -

        +1. Thanks Bobby.

        Note the findbugs exist without this patch.

        [exec] -1 overall.
        [exec]
        [exec] +1 @author. The patch does not contain any @author tags.
        [exec]
        [exec] -1 tests included. The patch doesn't appear to include any new or modified tests.
        [exec] Please justify why no tests are needed for this patch.
        [exec]
        [exec] +1 javadoc. The javadoc tool did not generate any warning messages.
        [exec]
        [exec] +1 javac. The applied patch does not increase the total number of javac compiler warnings.
        [exec]
        [exec] -1 findbugs. The patch appears to introduce 8 new Findbugs (version 1.3.9) warnings.
        [exec]
        [exec]

        Show
        Thomas Graves added a comment - +1. Thanks Bobby. Note the findbugs exist without this patch. [exec] -1 overall. [exec] [exec] +1 @author. The patch does not contain any @author tags. [exec] [exec] -1 tests included. The patch doesn't appear to include any new or modified tests. [exec] Please justify why no tests are needed for this patch. [exec] [exec] +1 javadoc. The javadoc tool did not generate any warning messages. [exec] [exec] +1 javac. The applied patch does not increase the total number of javac compiler warnings. [exec] [exec] -1 findbugs. The patch appears to introduce 8 new Findbugs (version 1.3.9) warnings. [exec] [exec]
        Hide
        Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12542634/MR-4576.txt
        against trunk revision .

        -1 patch. The patch command could not apply the patch.

        Console output: https://builds.apache.org/job/PreCommit-MAPREDUCE-Build/2784//console

        This message is automatically generated.

        Show
        Hadoop QA added a comment - -1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12542634/MR-4576.txt against trunk revision . -1 patch. The patch command could not apply the patch. Console output: https://builds.apache.org/job/PreCommit-MAPREDUCE-Build/2784//console This message is automatically generated.
        Hide
        Robert Joseph Evans added a comment -

        I have "verified" with some code changes that this fixes the issue. It is very difficult to reproduce the error. I had to modify the code to cause the issue to show up more often. I had to make it so the dist cache always was trying to delete things, and it never actually deleted them, I also had to modify it so that all dist cache entries take a long time to download, and on top of that I had to hit it just right so that a job was finishing at the same time a dist cache entry was being deleted. ugg.

        Show
        Robert Joseph Evans added a comment - I have "verified" with some code changes that this fixes the issue. It is very difficult to reproduce the error. I had to modify the code to cause the issue to show up more often. I had to make it so the dist cache always was trying to delete things, and it never actually deleted them, I also had to modify it so that all dist cache entries take a long time to download, and on top of that I had to hit it just right so that a job was finishing at the same time a dist cache entry was being deleted. ugg.
        Hide
        Robert Joseph Evans added a comment -

        This patch changes the locking on CacheStatus. I have not added in any new tests because it is a complicated bug, that is difficult to reproduce outside of a full cluster. branch-1 seems to have a build issue so I have tested it against 1.0.2. The commit tests pass, and the smoke tests are still running. I will also verify that it fixes the issue manually once the other tests finish.

        Show
        Robert Joseph Evans added a comment - This patch changes the locking on CacheStatus. I have not added in any new tests because it is a complicated bug, that is difficult to reproduce outside of a full cluster. branch-1 seems to have a build issue so I have tested it against 1.0.2. The commit tests pass, and the smoke tests are still running. I will also verify that it fixes the issue manually once the other tests finish.
        Hide
        Robert Joseph Evans added a comment -

        Moved it to MAPREDUCE. I don't know how it ended up being created in HDFS, I guess I just put it in the wrong spot.

        Show
        Robert Joseph Evans added a comment - Moved it to MAPREDUCE. I don't know how it ended up being created in HDFS, I guess I just put it in the wrong spot.
        Hide
        Robert Joseph Evans added a comment -

        I forgot to add in that I tested this on 0.23, and mrv2 does not have this issue at all. I added in a dist cache entry that takes 30 min to download and the job succeeded.

        Show
        Robert Joseph Evans added a comment - I forgot to add in that I tested this on 0.23, and mrv2 does not have this issue at all. I added in a dist cache entry that takes 30 min to download and the job succeeded.
        Hide
        Robert Joseph Evans added a comment -

        MAPREDUCE-2494 introduced a new lock when releasing a dist cache entry that introduced this problem. Thanks to Koji for finding and debugging this.

        Essentially the heartbeat thread holds a lock on the TaskTracker object.
        So does the job cleanup thread. Which also holds a lock on the TrackerDistributedCacheMenager's big list lock (this is the lock that MAPREDUCE-2494 added in).
        The thread that deletes things from the dist cache also grabs that big lock, and at the same time grabs locks in turn for every entry in the dist cache.
        While an entry in the dist cache is being downloaded it also holds the lock for the dist cache entry.

        So this can result in

        Downloading thread holds dist cache lock which blocks the dist cache delete thread which holds the full dist cache map lock that blocks the job cleanup thread that holds that Task Tracker lock which blocks the heartbeat thread. This can be seen below.

        I think it is probably best to change the DistCache entries' locks so that when we go to delete them if the lock is held we skip that entry instead of having it block.

        Here, tracing from the heartbeat thread.
        1=================================================
        "main" prio=10 tid=0x0875c400 nid=0x3fca waiting for monitor entry [0xf73e6000]
           java.lang.Thread.State: BLOCKED (on object monitor)
          at
        org.apache.hadoop.mapred.TaskTracker.transmitHeartBeat(TaskTracker.java:1790)
          - waiting to lock <0xb4299248> (a org.apache.hadoop.mapred.TaskTracker)
          at org.apache.hadoop.mapred.TaskTracker.offerService(TaskTracker.java:1653)
          at org.apache.hadoop.mapred.TaskTracker.run(TaskTracker.java:2503)
          at org.apache.hadoop.mapred.TaskTracker.main(TaskTracker.java:3744)
        
        Looking for lock <0xb4299248> 
        2=================================================
        "taskCleanup" daemon prio=10 tid=0x0949ac00 nid=0x405c waiting for monitor
        entry [0xadead000]
           java.lang.Thread.State: BLOCKED (on object monitor)
          at
        org.apache.hadoop.filecache.TrackerDistributedCacheManager$CacheStatus.decRefCount(TrackerDistributedCacheManager.java:597)
          - waiting to lock <0xb4214308> (a java.util.LinkedHashMap)
          at
        org.apache.hadoop.filecache.TrackerDistributedCacheManager.releaseCache(TrackerDistributedCacheManager.java:233)
          at
        org.apache.hadoop.filecache.TaskDistributedCacheManager.release(TaskDistributedCacheManager.java:254)
          at org.apache.hadoop.mapred.TaskTracker.purgeJob(TaskTracker.java:2066)
          - locked <0xb51e5d78> (a org.apache.hadoop.mapred.TaskTracker$RunningJob)
          - locked <0xb4299248> (a org.apache.hadoop.mapred.TaskTracker)
          at org.apache.hadoop.mapred.TaskTracker$1.run(TaskTracker.java:439)
          at java.lang.Thread.run(Thread.java:619)
        
        
        Looking for the lock <0xb4214308>
        
        
        3=================================================
        "Thread-27" prio=10 tid=0xae501400 nid=0x4021 waiting for monitor entry
        [0xae4ad000]
           java.lang.Thread.State: BLOCKED (on object monitor)
          at
        org.apache.hadoop.filecache.TrackerDistributedCacheManager$BaseDirManager.checkAndCleanup(TrackerDistributedCacheManager.java:1019)
          - waiting to lock <0xb52776c0> (a
        org.apache.hadoop.filecache.TrackerDistributedCacheManager$CacheStatus)
          - locked <0xb4214308> (a java.util.LinkedHashMap)
          at
        org.apache.hadoop.filecache.TrackerDistributedCacheManager$CleanupThread.run(TrackerDistributedCacheManager.java:948)
        
        Looking for the lock <0xb52776c0>
        
        4=================================================
        "Thread-187419" daemon prio=10 tid=0xaa103400 nid=0x3758 runnable [0xad75c000]
           java.lang.Thread.State: RUNNABLE
          at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
          at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
          at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
          at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
          - locked <0xb52998d0> (a sun.nio.ch.Util$1)
          - locked <0xb52998e0> (a java.util.Collections$UnmodifiableSet)
          - locked <0xb5299880> (a sun.nio.ch.EPollSelectorImpl)
          at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
          at
        org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:332)
          at
        org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157)
          at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155)
          at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128)
          at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
          at java.io.BufferedInputStream.read1(BufferedInputStream.java:258)
          at java.io.BufferedInputStream.read(BufferedInputStream.java:317)
          - locked <0xb5505ec8> (a java.io.BufferedInputStream)
          at java.io.DataInputStream.read(DataInputStream.java:132)
          at org.apache.hadoop.io.IOUtils.readFully(IOUtils.java:153)
          at
        org.apache.hadoop.hdfs.DFSClient$BlockReader.readChunk(DFSClient.java:1598)
          - locked <0xb5505ef8> (a org.apache.hadoop.hdfs.DFSClient$BlockReader)
          at
        org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(FSInputChecker.java:237)
          at org.apache.hadoop.fs.FSInputChecker.read1(FSInputChecker.java:189)
          at org.apache.hadoop.fs.FSInputChecker.read(FSInputChecker.java:158)
          - locked <0xb5505ef8> (a org.apache.hadoop.hdfs.DFSClient$BlockReader)
          at org.apache.hadoop.hdfs.DFSClient$BlockReader.read(DFSClient.java:1457)
          - locked <0xb5505ef8> (a org.apache.hadoop.hdfs.DFSClient$BlockReader)
          at
        org.apache.hadoop.hdfs.DFSClient$DFSInputStream.readBuffer(DFSClient.java:2172)
          - locked <0xb5505fd8> (a org.apache.hadoop.hdfs.DFSClient$DFSInputStream)
          at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DFSClient.java:2224)
          - locked <0xb5505fd8> (a org.apache.hadoop.hdfs.DFSClient$DFSInputStream)
          at java.io.DataInputStream.read(DataInputStream.java:83)
          at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:74)
          at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:47)
          at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:100)  at
        org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:230)
          at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:220)
          at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:220)
          at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:220)
          at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:220)
          at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:163)
          at org.apache.hadoop.fs.FileSystem.copyToLocalFile(FileSystem.java:1222)
          at org.apache.hadoop.fs.FileSystem.copyToLocalFile(FileSystem.java:1203)
          at
        org.apache.hadoop.filecache.TrackerDistributedCacheManager.downloadCacheObject(TrackerDistributedCacheManager.java:416)
          at
        org.apache.hadoop.filecache.TrackerDistributedCacheManager.localizePublicCacheObject(TrackerDistributedCacheManager.java:464)
          at
        org.apache.hadoop.filecache.TrackerDistributedCacheManager.getLocalCache(TrackerDistributedCacheManager.java:191)
          - locked <0xb52776c0> (a
        org.apache.hadoop.filecache.TrackerDistributedCacheManager$CacheStatus)
          at
        org.apache.hadoop.filecache.TaskDistributedCacheManager.setupCache(TaskDistributedCacheManager.java:182)
          at org.apache.hadoop.mapred.TaskTracker$4.run(TaskTracker.java:1212)
          at java.security.AccessController.doPrivileged(Native Method)
          at javax.security.auth.Subject.doAs(Subject.java:396)
          at
        org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1093)
          at org.apache.hadoop.mapred.TaskTracker.initializeJob(TaskTracker.java:1203)
          at org.apache.hadoop.mapred.TaskTracker.localizeJob(TaskTracker.java:1118)
          at org.apache.hadoop.mapred.TaskTracker$5.run(TaskTracker.java:2430)
          at java.lang.Thread.run(Thread.java:619)
        
        Show
        Robert Joseph Evans added a comment - MAPREDUCE-2494 introduced a new lock when releasing a dist cache entry that introduced this problem. Thanks to Koji for finding and debugging this. Essentially the heartbeat thread holds a lock on the TaskTracker object. So does the job cleanup thread. Which also holds a lock on the TrackerDistributedCacheMenager's big list lock (this is the lock that MAPREDUCE-2494 added in). The thread that deletes things from the dist cache also grabs that big lock, and at the same time grabs locks in turn for every entry in the dist cache. While an entry in the dist cache is being downloaded it also holds the lock for the dist cache entry. So this can result in Downloading thread holds dist cache lock which blocks the dist cache delete thread which holds the full dist cache map lock that blocks the job cleanup thread that holds that Task Tracker lock which blocks the heartbeat thread. This can be seen below. I think it is probably best to change the DistCache entries' locks so that when we go to delete them if the lock is held we skip that entry instead of having it block. Here, tracing from the heartbeat thread. 1================================================= "main" prio=10 tid=0x0875c400 nid=0x3fca waiting for monitor entry [0xf73e6000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.hadoop.mapred.TaskTracker.transmitHeartBeat(TaskTracker.java:1790) - waiting to lock <0xb4299248> (a org.apache.hadoop.mapred.TaskTracker) at org.apache.hadoop.mapred.TaskTracker.offerService(TaskTracker.java:1653) at org.apache.hadoop.mapred.TaskTracker.run(TaskTracker.java:2503) at org.apache.hadoop.mapred.TaskTracker.main(TaskTracker.java:3744) Looking for lock <0xb4299248> 2================================================= "taskCleanup" daemon prio=10 tid=0x0949ac00 nid=0x405c waiting for monitor entry [0xadead000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.hadoop.filecache.TrackerDistributedCacheManager$CacheStatus.decRefCount(TrackerDistributedCacheManager.java:597) - waiting to lock <0xb4214308> (a java.util.LinkedHashMap) at org.apache.hadoop.filecache.TrackerDistributedCacheManager.releaseCache(TrackerDistributedCacheManager.java:233) at org.apache.hadoop.filecache.TaskDistributedCacheManager.release(TaskDistributedCacheManager.java:254) at org.apache.hadoop.mapred.TaskTracker.purgeJob(TaskTracker.java:2066) - locked <0xb51e5d78> (a org.apache.hadoop.mapred.TaskTracker$RunningJob) - locked <0xb4299248> (a org.apache.hadoop.mapred.TaskTracker) at org.apache.hadoop.mapred.TaskTracker$1.run(TaskTracker.java:439) at java.lang.Thread.run(Thread.java:619) Looking for the lock <0xb4214308> 3================================================= "Thread-27" prio=10 tid=0xae501400 nid=0x4021 waiting for monitor entry [0xae4ad000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.hadoop.filecache.TrackerDistributedCacheManager$BaseDirManager.checkAndCleanup(TrackerDistributedCacheManager.java:1019) - waiting to lock <0xb52776c0> (a org.apache.hadoop.filecache.TrackerDistributedCacheManager$CacheStatus) - locked <0xb4214308> (a java.util.LinkedHashMap) at org.apache.hadoop.filecache.TrackerDistributedCacheManager$CleanupThread.run(TrackerDistributedCacheManager.java:948) Looking for the lock <0xb52776c0> 4================================================= "Thread-187419" daemon prio=10 tid=0xaa103400 nid=0x3758 runnable [0xad75c000] java.lang.Thread.State: RUNNABLE at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method) at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210) at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65) at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69) - locked <0xb52998d0> (a sun.nio.ch.Util$1) - locked <0xb52998e0> (a java.util.Collections$UnmodifiableSet) - locked <0xb5299880> (a sun.nio.ch.EPollSelectorImpl) at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80) at org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:332) at org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157) at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155) at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128) at java.io.BufferedInputStream.fill(BufferedInputStream.java:218) at java.io.BufferedInputStream.read1(BufferedInputStream.java:258) at java.io.BufferedInputStream.read(BufferedInputStream.java:317) - locked <0xb5505ec8> (a java.io.BufferedInputStream) at java.io.DataInputStream.read(DataInputStream.java:132) at org.apache.hadoop.io.IOUtils.readFully(IOUtils.java:153) at org.apache.hadoop.hdfs.DFSClient$BlockReader.readChunk(DFSClient.java:1598) - locked <0xb5505ef8> (a org.apache.hadoop.hdfs.DFSClient$BlockReader) at org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(FSInputChecker.java:237) at org.apache.hadoop.fs.FSInputChecker.read1(FSInputChecker.java:189) at org.apache.hadoop.fs.FSInputChecker.read(FSInputChecker.java:158) - locked <0xb5505ef8> (a org.apache.hadoop.hdfs.DFSClient$BlockReader) at org.apache.hadoop.hdfs.DFSClient$BlockReader.read(DFSClient.java:1457) - locked <0xb5505ef8> (a org.apache.hadoop.hdfs.DFSClient$BlockReader) at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.readBuffer(DFSClient.java:2172) - locked <0xb5505fd8> (a org.apache.hadoop.hdfs.DFSClient$DFSInputStream) at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DFSClient.java:2224) - locked <0xb5505fd8> (a org.apache.hadoop.hdfs.DFSClient$DFSInputStream) at java.io.DataInputStream.read(DataInputStream.java:83) at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:74) at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:47) at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:100) at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:230) at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:220) at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:220) at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:220) at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:220) at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:163) at org.apache.hadoop.fs.FileSystem.copyToLocalFile(FileSystem.java:1222) at org.apache.hadoop.fs.FileSystem.copyToLocalFile(FileSystem.java:1203) at org.apache.hadoop.filecache.TrackerDistributedCacheManager.downloadCacheObject(TrackerDistributedCacheManager.java:416) at org.apache.hadoop.filecache.TrackerDistributedCacheManager.localizePublicCacheObject(TrackerDistributedCacheManager.java:464) at org.apache.hadoop.filecache.TrackerDistributedCacheManager.getLocalCache(TrackerDistributedCacheManager.java:191) - locked <0xb52776c0> (a org.apache.hadoop.filecache.TrackerDistributedCacheManager$CacheStatus) at org.apache.hadoop.filecache.TaskDistributedCacheManager.setupCache(TaskDistributedCacheManager.java:182) at org.apache.hadoop.mapred.TaskTracker$4.run(TaskTracker.java:1212) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:396) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1093) at org.apache.hadoop.mapred.TaskTracker.initializeJob(TaskTracker.java:1203) at org.apache.hadoop.mapred.TaskTracker.localizeJob(TaskTracker.java:1118) at org.apache.hadoop.mapred.TaskTracker$5.run(TaskTracker.java:2430) at java.lang.Thread.run(Thread.java:619)

          People

          • Assignee:
            Robert Joseph Evans
            Reporter:
            Robert Joseph Evans
          • Votes:
            0 Vote for this issue
            Watchers:
            7 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development