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

Task stuck in JobLocalizer prevented other tasks on the same node from committing

    Details

    • Type: Bug Bug
    • Status: Closed
    • Priority: Critical Critical
    • Resolution: Fixed
    • Affects Version/s: 0.20.205.0
    • Fix Version/s: 1.1.0
    • Component/s: mrv1
    • Labels:
      None
    • Target Version/s:

      Description

      We saw that as a result of HADOOP-6963, one task was stuck in this

      Thread 23668: (state = IN_NATIVE)

      • java.io.UnixFileSystem.getBooleanAttributes0(java.io.File) @bci=0 (Compiled frame; information may be imprecise)
      • java.io.UnixFileSystem.getBooleanAttributes(java.io.File) @bci=2, line=228 (Compiled frame)
      • java.io.File.exists() @bci=20, line=733 (Compiled frame)
      • org.apache.hadoop.fs.FileUtil.getDU(java.io.File) @bci=3, line=446 (Compiled frame)
      • org.apache.hadoop.fs.FileUtil.getDU(java.io.File) @bci=52, line=455 (Compiled frame)
      • org.apache.hadoop.fs.FileUtil.getDU(java.io.File) @bci=52, line=455 (Compiled frame)
        ....
        .... TONS MORE OF THIS SAME LINE
      • org.apache.hadoop.fs.FileUtil.getDU(java.io.File) @bci=52, line=455 (Compiled frame)
        .....
        .....
      • org.apache.hadoop.fs.FileUtil.getDU(java.io.File) @bci=52, line=455 (Compiled frame)
      • org.apache.hadoop.fs.FileUtil.getDU(java.io.File) @bci=52, line=455 (Interpreted frame)
        ne=451 (Interpreted frame)
      • org.apache.hadoop.mapred.JobLocalizer.downloadPrivateCacheObjects(org.apache.hadoop.conf.Configuration, java.net.URI[], org.apache.hadoop.fs.Path[], long[], boolean[], boolean) @bci=150, line=324 (Interpreted frame)
      • org.apache.hadoop.mapred.JobLocalizer.downloadPrivateCache(org.apache.hadoop.conf.Configuration) @bci=40, line=349 (Interpreted frame) 51, line=383 (Interpreted frame)
      • org.apache.hadoop.mapred.JobLocalizer.runSetup(java.lang.String, java.lang.String, org.apache.hadoop.fs.Path, org.apache.hadoop.mapred.TaskUmbilicalProtocol) @bci=46, line=477 (Interpreted frame)
      • org.apache.hadoop.mapred.JobLocalizer$3.run() @bci=20, line=534 (Interpreted frame)
      • org.apache.hadoop.mapred.JobLocalizer$3.run() @bci=1, line=531 (Interpreted frame)
      • java.security.AccessController.doPrivileged(java.security.PrivilegedExceptionAction, java.security.AccessControlContext) @bci=0 (Interpreted frame)
      • javax.security.auth.Subject.doAs(javax.security.auth.Subject, java.security.PrivilegedExceptionAction) @bci=42, line=396 (Interpreted frame)
      • org.apache.hadoop.security.UserGroupInformation.doAs(java.security.PrivilegedExceptionAction) @bci=14, line=1082 (Interpreted frame)
      • org.apache.hadoop.mapred.JobLocalizer.main(java.lang.String[]) @bci=266, line=530 (Interpreted frame)

      While all other tasks on the same node were stuck in
      Thread 32141: (state = BLOCKED)

      • java.lang.Thread.sleep(long) @bci=0 (Interpreted frame)
      • org.apache.hadoop.mapred.Task.commit(org.apache.hadoop.mapred.TaskUmbilicalProtocol, org.apache.hadoop.mapred.Task$TaskReporter, org.apache.hadoop.mapreduce.OutputCommitter) @bci=24, line=980 (Compiled frame)
      • org.apache.hadoop.mapred.Task.done(org.apache.hadoop.mapred.TaskUmbilicalProtocol, org.apache.hadoop.mapred.Task$TaskReporter) @bci=146, line=871 (Interpreted frame)
      • org.apache.hadoop.mapred.ReduceTask.run(org.apache.hadoop.mapred.JobConf, org.apache.hadoop.mapred.TaskUmbilicalProtocol) @bci=470, line=423 (Interpreted frame)
      • org.apache.hadoop.mapred.Child$4.run() @bci=29, line=255 (Interpreted frame)
      • java.security.AccessController.doPrivileged(java.security.PrivilegedExceptionAction, java.security.AccessControlContext) @bci=0 (Interpreted frame)
      • javax.security.auth.Subject.doAs(javax.security.auth.Subject, java.security.PrivilegedExceptionAction) @bci=42, line=396 (Interpreted frame)
      • org.apache.hadoop.security.UserGroupInformation.doAs(java.security.PrivilegedExceptionAction) @bci=14, line=1082 (Interpreted frame)
      • org.apache.hadoop.mapred.Child.main(java.lang.String[]) @bci=738, line=249 (Interpreted frame)

      This should never happen. A stuck task should never prevent other tasks from different jobs on the same node from committing.

      1. MAPREDUCE-4088.branch-1.patch
        4 kB
        Ravi Prakash
      2. MAPREDUCE-4088.branch-1.patch
        4 kB
        Ravi Prakash
      3. MAPREDUCE-4088.patch
        4 kB
        Ravi Prakash
      4. MAPREDUCE-4088.patch
        2 kB
        Ravi Prakash

        Issue Links

          Activity

          Hide
          Ravi Prakash added a comment -

          Courtesy Koji

          Each TaskTracker has a single thread for taskCleanup taking work from
          tasksToCleanup queue.

          For each task to cleanup, it firsts call
          checkJobStatusAndWait(action);

          And inside

            private void checkJobStatusAndWait(TaskTrackerAction action)
          ...
              synchronized (runningJobs) {
                rjob = runningJobs.get(jobId);
              }
              if (rjob != null) {
                synchronized (rjob) {
                  while (rjob.localizing) {
                    rjob.wait();
                  }
                }
              }
           

          So this thread would wait while the task is being localized.
          Even if one task is hung on localization, entire cleanup is stopped.

          East or west! Koji is the best!
          Soda lemon ginger pop! Koji is on the top!
          Yyaayyy yaayyyy yaayyyy for Koji!

          Show
          Ravi Prakash added a comment - Courtesy Koji Each TaskTracker has a single thread for taskCleanup taking work from tasksToCleanup queue. For each task to cleanup, it firsts call checkJobStatusAndWait(action); And inside private void checkJobStatusAndWait(TaskTrackerAction action) ... synchronized (runningJobs) { rjob = runningJobs.get(jobId); } if (rjob != null) { synchronized (rjob) { while (rjob.localizing) { rjob.wait(); } } } So this thread would wait while the task is being localized. Even if one task is hung on localization, entire cleanup is stopped. East or west! Koji is the best! Soda lemon ginger pop! Koji is on the top! Yyaayyy yaayyyy yaayyyy for Koji!
          Hide
          Ravi Prakash added a comment -

          Even though the TT heartbeats back now even when downloading (infinitely big files see HADOOP-6963), the taskCleanupThread blocks on a single job which might be localizing, thus preventing others from cleaning up. Eventually the TT fills up with tasks waiting for cleanup and the node is rendered useless.

          Show
          Ravi Prakash added a comment - Even though the TT heartbeats back now even when downloading (infinitely big files see HADOOP-6963 ), the taskCleanupThread blocks on a single job which might be localizing, thus preventing others from cleaning up. Eventually the TT fills up with tasks waiting for cleanup and the node is rendered useless.
          Hide
          Ravi Prakash added a comment -

          Does anyone have any suggestions on how to fix this? I'm thinking of this:

          • We could have a timeout in the wait for checkJobStatusAndWait.
          • If we time out, we simply put the action back into the queue (hoping next time around it succeeds)
          • This might make the isIdle method more complicated
          Show
          Ravi Prakash added a comment - Does anyone have any suggestions on how to fix this? I'm thinking of this: We could have a timeout in the wait for checkJobStatusAndWait. If we time out, we simply put the action back into the queue (hoping next time around it succeeds) This might make the isIdle method more complicated
          Hide
          Ravi Prakash added a comment -

          I'm thinking of a patch like this. If you have any concerns, speak now. Or forever hold thy peace!

          Show
          Ravi Prakash added a comment - I'm thinking of a patch like this. If you have any concerns, speak now. Or forever hold thy peace!
          Hide
          Ravi Prakash added a comment -

          https://issues.apache.org/jira/browse/HADOOP-912 details why isIdle() checks tasksToCleanup . Going by Arun's first comment, isIdle() need not check tasksToCleanup (since they are being consumed offline in a separate thread). I've chosen to add a new API isIdleAndClean()

          Can someone please review and commit this patch now?

          Show
          Ravi Prakash added a comment - https://issues.apache.org/jira/browse/HADOOP-912 details why isIdle() checks tasksToCleanup . Going by Arun's first comment, isIdle() need not check tasksToCleanup (since they are being consumed offline in a separate thread). I've chosen to add a new API isIdleAndClean() Can someone please review and commit this patch now?
          Hide
          Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12524865/MAPREDUCE-4088.patch
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

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

          Console output: https://builds.apache.org/job/PreCommit-MAPREDUCE-Build/2319//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/12524865/MAPREDUCE-4088.patch against trunk revision . +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 1 new or modified test files. -1 patch. The patch command could not apply the patch. Console output: https://builds.apache.org/job/PreCommit-MAPREDUCE-Build/2319//console This message is automatically generated.
          Hide
          Ravi Prakash added a comment -

          0.23 and later is not afflicted with this problem because of the different architecture (TaskCleanerImpl tries to abortTask)

          Show
          Ravi Prakash added a comment - 0.23 and later is not afflicted with this problem because of the different architecture (TaskCleanerImpl tries to abortTask)
          Hide
          Ravi Prakash added a comment -

          Hadoop QA is complaining because its trying to apply the patch on trunk. This patch applies ONLY to branch-1. Uploading again after appending the branch in the patch name

          Show
          Ravi Prakash added a comment - Hadoop QA is complaining because its trying to apply the patch on trunk. This patch applies ONLY to branch-1. Uploading again after appending the branch in the patch name
          Hide
          Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12524866/MAPREDUCE-4088.branch-1.patch
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

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

          Console output: https://builds.apache.org/job/PreCommit-MAPREDUCE-Build/2320//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/12524866/MAPREDUCE-4088.branch-1.patch against trunk revision . +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 1 new or modified test files. -1 patch. The patch command could not apply the patch. Console output: https://builds.apache.org/job/PreCommit-MAPREDUCE-Build/2320//console This message is automatically generated.
          Hide
          Robert Joseph Evans added a comment -

          Jenkins is not smart enough to apply the patch properly to branch-1 instead of trunk and run the tests. You need to run test-patch yourself manually.

          I looked at the patch, and it looks OK for the most part.

          Inside taskCleanupThread you are generating a StringBuffer, and appending things to it, but that string buffer is never used anywhere. Did you miss a log statement or something there? Because I would like a log statement saying what is happening and why the cleanup was suspended.

          Have you run the tests?

          Show
          Robert Joseph Evans added a comment - Jenkins is not smart enough to apply the patch properly to branch-1 instead of trunk and run the tests. You need to run test-patch yourself manually. I looked at the patch, and it looks OK for the most part. Inside taskCleanupThread you are generating a StringBuffer, and appending things to it, but that string buffer is never used anywhere. Did you miss a log statement or something there? Because I would like a log statement saying what is happening and why the cleanup was suspended. Have you run the tests?
          Hide
          Ravi Prakash added a comment -

          Thanks Bobby!

          I ran test-patch myself.

          -1 overall.  
              +1 @author.  The patch does not contain any @author tags.
              +1 tests included.  The patch appears to include 3 new or modified tests.
              +1 javadoc.  The javadoc tool did not generate any warning messages.
              +1 javac.  The applied patch does not increase the total number of javac compiler warnings.
              -1 findbugs.  The patch appears to introduce 8 new Findbugs (version 1.3.9) warnings.
          

          It claims there were 8 new findbugs warnings, but I searched for TaskTracker and MiniMRCluster in the findbugs report, and none of them had anything to do with my changes. I then ran findbugs target without any patch, and sure enough, there were already 219 warnings present

           [findbugs] Warnings generated: 219
          

          I also ran the test Arun mentioned in HADOOP-912 (TestMiniMRWithDFS) and it passed.

          Yes I had indeed forgot to log the StringBuffer. I've added that in the latest patch.

          Show
          Ravi Prakash added a comment - Thanks Bobby! I ran test-patch myself. -1 overall. +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 3 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. -1 findbugs. The patch appears to introduce 8 new Findbugs (version 1.3.9) warnings. It claims there were 8 new findbugs warnings, but I searched for TaskTracker and MiniMRCluster in the findbugs report, and none of them had anything to do with my changes. I then ran findbugs target without any patch, and sure enough, there were already 219 warnings present [findbugs] Warnings generated: 219 I also ran the test Arun mentioned in HADOOP-912 (TestMiniMRWithDFS) and it passed. Yes I had indeed forgot to log the StringBuffer. I've added that in the latest patch.
          Hide
          Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12524929/MAPREDUCE-4088.branch-1.patch
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 1 new or modified test files.

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

          Console output: https://builds.apache.org/job/PreCommit-MAPREDUCE-Build/2325//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/12524929/MAPREDUCE-4088.branch-1.patch against trunk revision . +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 1 new or modified test files. -1 patch. The patch command could not apply the patch. Console output: https://builds.apache.org/job/PreCommit-MAPREDUCE-Build/2325//console This message is automatically generated.
          Hide
          Robert Joseph Evans added a comment -

          The patch looks good but I am not an expert on this part of the code have you run it on a cluster at all? if so what tests have you run with it?

          Show
          Robert Joseph Evans added a comment - The patch looks good but I am not an expert on this part of the code have you run it on a cluster at all? if so what tests have you run with it?
          Hide
          Ravi Prakash added a comment -

          I ran the patched branch-1 build on my local dev box. The tasks were cleaned up successfully as usual after a simple wordcount job. Are you looking for specific tests?

          Show
          Ravi Prakash added a comment - I ran the patched branch-1 build on my local dev box. The tasks were cleaned up successfully as usual after a simple wordcount job. Are you looking for specific tests?
          Hide
          Robert Joseph Evans added a comment -

          No I am looking mostly at being sure that the code is working as expected. Thanks Ravi, +1

          Show
          Robert Joseph Evans added a comment - No I am looking mostly at being sure that the code is working as expected. Thanks Ravi, +1
          Hide
          Robert Joseph Evans added a comment -

          Thanks Ravi,

          I put this into branch-1. If you want this in 1.0.3 you should ask Matt Foley about it.

          Show
          Robert Joseph Evans added a comment - Thanks Ravi, I put this into branch-1. If you want this in 1.0.3 you should ask Matt Foley about it.
          Hide
          Matt Foley added a comment -

          Closed upon release of Hadoop-1.1.0.

          Show
          Matt Foley added a comment - Closed upon release of Hadoop-1.1.0.

            People

            • Assignee:
              Ravi Prakash
              Reporter:
              Ravi Prakash
            • Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development