Переглянути джерело

HADOOP-4163. Report FSErrors from map output fetch threads instead of
merely logging them. Contributed by Sharad Agarwal.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/branches/branch-0.19@702293 13f79535-47bb-0310-9956-ffa450edef68

Christopher Douglas 16 роки тому
батько
коміт
a23049471b
2 змінених файлів з 22 додано та 2 видалено
  1. 4 1
      CHANGES.txt
  2. 18 1
      src/mapred/org/apache/hadoop/mapred/ReduceTask.java

+ 4 - 1
CHANGES.txt

@@ -798,7 +798,10 @@ Release 0.19.0 - Unreleased
     retries for fetching map-outputs; also fixed the case where the reducer
     automatically kills on too many unique map-outputs could not be fetched
     for small jobs. (Amareshwari Sri Ramadasu via acmurthy)  
-    
+
+    HADOOP-4163. Report FSErrors from map output fetch threads instead of
+    merely logging them. (Sharad Agarwal via cdouglas)
+
 Release 0.18.2 - Unreleased
 
   BUG FIXES

+ 18 - 1
src/mapred/org/apache/hadoop/mapred/ReduceTask.java

@@ -49,6 +49,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ChecksumFileSystem;
+import org.apache.hadoop.fs.FSError;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
@@ -358,7 +359,14 @@ class ReduceTask extends Task {
     if (!isLocal) {
       reduceCopier = new ReduceCopier(umbilical, job);
       if (!reduceCopier.fetchOutputs()) {
-        throw new IOException(getTaskID() + "The reduce copier failed");
+        if(reduceCopier.mergeThrowable instanceof FSError) {
+          LOG.error("Task: " + getTaskID() + " - FSError: " + 
+              StringUtils.stringifyException(reduceCopier.mergeThrowable));
+          umbilical.fsError(getTaskID(), 
+              reduceCopier.mergeThrowable.getMessage());
+        }
+        throw new IOException("Task: " + getTaskID() + 
+            " - The reduce copier failed", reduceCopier.mergeThrowable);
       }
     }
     copyPhase.complete();                         // copy is already complete
@@ -1079,6 +1087,15 @@ class ReduceTask extends Task {
             }
           } catch (InterruptedException e) { 
             return; // ALL DONE
+          } catch (FSError e) {
+            LOG.error("Task: " + reduceTask.getTaskID() + " - FSError: " + 
+                      StringUtils.stringifyException(e));
+            try {
+              umbilical.fsError(reduceTask.getTaskID(), e.getMessage());
+            } catch (IOException io) {
+              LOG.error("Could not notify TT of FSError: " + 
+                      StringUtils.stringifyException(io));
+            }
           } catch (Throwable th) {
             LOG.error("Map output copy failure: " + 
                       StringUtils.stringifyException(th));