hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From tgra...@apache.org
Subject svn commit: r1328026 - in /hadoop/common/branches/branch-1: CHANGES.txt src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapRed.java src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingFailure.java
Date Thu, 19 Apr 2012 16:18:43 GMT
Author: tgraves
Date: Thu Apr 19 16:18:42 2012
New Revision: 1328026

URL: http://svn.apache.org/viewvc?rev=1328026&view=rev
Log:
MAPREDUCE-4154. streaming MR job succeeds even if the streaming command fails. (Devaraj Das
via tgraves)

Modified:
    hadoop/common/branches/branch-1/CHANGES.txt
    hadoop/common/branches/branch-1/src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapRed.java
    hadoop/common/branches/branch-1/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingFailure.java

Modified: hadoop/common/branches/branch-1/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1/CHANGES.txt?rev=1328026&r1=1328025&r2=1328026&view=diff
==============================================================================
--- hadoop/common/branches/branch-1/CHANGES.txt (original)
+++ hadoop/common/branches/branch-1/CHANGES.txt Thu Apr 19 16:18:42 2012
@@ -222,6 +222,9 @@ Release 1.1.0 - unreleased
 
     HDFS-119. Fix a bug in logSync(), which causes NameNode block forever. (shv)
 
+    MAPREDUCE-4154. streaming MR job succeeds even if the streaming command 
+    fails. (Devaraj Das via tgraves)
+
 Release 1.0.3 - unreleased
 
   NEW FEATURES

Modified: hadoop/common/branches/branch-1/src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapRed.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1/src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapRed.java?rev=1328026&r1=1328025&r2=1328026&view=diff
==============================================================================
--- hadoop/common/branches/branch-1/src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapRed.java
(original)
+++ hadoop/common/branches/branch-1/src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapRed.java
Thu Apr 19 16:18:42 2012
@@ -569,6 +569,10 @@ public abstract class PipeMapRed {
           clientOut_.flush();
           clientOut_.close();
         }
+      } catch (IOException io) {
+        LOG.warn(StringUtils.stringifyException(io));
+      }
+      try {
         waitOutputThreads();
       } catch (IOException io) {
         LOG.warn(StringUtils.stringifyException(io));

Modified: hadoop/common/branches/branch-1/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingFailure.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingFailure.java?rev=1328026&r1=1328025&r2=1328026&view=diff
==============================================================================
--- hadoop/common/branches/branch-1/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingFailure.java
(original)
+++ hadoop/common/branches/branch-1/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingFailure.java
Thu Apr 19 16:18:42 2012
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.ToolRunner;
 
 /**
  * This class tests if hadoopStreaming returns Exception 
@@ -82,6 +83,25 @@ public class TestStreamingFailure extend
       }
     }
   }
+  
+  public void testStreamingFailureForFailedProcess() throws Exception {
+    int ret = 0;
+    try {
+      createInput();
+      String[] args = {
+          "-input", INPUT_FILE.getAbsolutePath(),
+          "-output", OUTPUT_DIR.getAbsolutePath(),
+          "-mapper", "/bin/ls dsdsdsds-does-not-exist",
+          "-jobconf", "stream.tmpdir="+System.getProperty("test.build.data",
+              "/tmp"),
+      };
+      ret = ToolRunner.run(new StreamJob(), args);
+    } finally {
+      INPUT_FILE.delete();
+      FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
+    }
+    assertEquals("Streaming job failure code expected", 1, ret);
+  }
 
   public static void main(String[]args) throws Exception
   {



Mime
View raw message