You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cd...@apache.org on 2008/10/07 00:25:16 UTC
svn commit: r702292 - in /hadoop/core/trunk: CHANGES.txt
src/mapred/org/apache/hadoop/mapred/ReduceTask.java
Author: cdouglas
Date: Mon Oct 6 15:25:16 2008
New Revision: 702292
URL: http://svn.apache.org/viewvc?rev=702292&view=rev
Log:
HADOOP-4163. Report FSErrors from map output fetch threads instead of
merely logging them. Contributed by Sharad Agarwal.
Modified:
hadoop/core/trunk/CHANGES.txt
hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/ReduceTask.java
Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=702292&r1=702291&r2=702292&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Mon Oct 6 15:25:16 2008
@@ -852,7 +852,10 @@
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
Modified: hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/ReduceTask.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/ReduceTask.java?rev=702292&r1=702291&r2=702292&view=diff
==============================================================================
--- hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/ReduceTask.java (original)
+++ hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/ReduceTask.java Mon Oct 6 15:25:16 2008
@@ -49,6 +49,7 @@
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 @@
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 @@
}
} 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));