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 to...@apache.org on 2008/03/10 17:51:21 UTC

svn commit: r635598 - in /hadoop/core/trunk: ./ src/contrib/streaming/src/java/org/apache/hadoop/streaming/ src/contrib/streaming/src/test/org/apache/hadoop/streaming/

Author: tomwhite
Date: Mon Mar 10 09:51:09 2008
New Revision: 635598

URL: http://svn.apache.org/viewvc?rev=635598&view=rev
Log:
HADOOP-2057.  Streaming should optionally treat a non-zero exit status of a child process as a failed task.  Contributed by Rick Cox.

Added:
    hadoop/core/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/FailApp.java
    hadoop/core/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingExitStatus.java
Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapRed.java
    hadoop/core/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java
    hadoop/core/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/UtilTest.java

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=635598&r1=635597&r2=635598&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Mon Mar 10 09:51:09 2008
@@ -70,6 +70,9 @@
 
     HADOOP-2810. Updated the Hadoop Core logo. (nigel)
 
+    HADOOP-2057.  Streaming should optionally treat a non-zero exit status
+    of a child process as a failed task.  (Rick Cox via tomwhite)
+
   OPTIMIZATIONS
 
     HADOOP-2790.  Fixed inefficient method hasSpeculativeTask by removing

Modified: hadoop/core/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapRed.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapRed.java?rev=635598&r1=635597&r2=635598&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapRed.java (original)
+++ hadoop/core/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapRed.java Mon Mar 10 09:51:09 2008
@@ -124,8 +124,9 @@
       this.reduceOutFieldSeparator = reduceOutputFieldSeparator.charAt(0);
       this.numOfMapOutputKeyFields = job_.getInt("stream.num.map.output.key.fields", 1);
       this.numOfReduceOutputKeyFields = job_.getInt("stream.num.reduce.output.key.fields", 1);
-      
 
+      nonZeroExitIsFailure_ = job_.getBoolean("stream.non.zero.exit.is.failure", false);
+      
       doPipe_ = getDoPipe();
       if (!doPipe_) return;
 
@@ -293,8 +294,14 @@
       int exitVal = sim.waitFor();
       // how'd it go?
       if (exitVal != 0) {
-	  logprintln("PipeMapRed.waitOutputThreads(): subprocess failed with code " + exitVal + " in " + PipeMapRed.class.getName());
-      };
+        if (nonZeroExitIsFailure_) {
+          throw new RuntimeException("PipeMapRed.waitOutputThreads(): subprocess failed with code "
+                                     + exitVal);
+        } else {
+          logprintln("PipeMapRed.waitOutputThreads(): subprocess exited with code " + exitVal
+                     + " in " + PipeMapRed.class.getName());
+        }
+      }
       if (outThread_ != null) {
         outThread_.join(joinDelay_);
       }
@@ -541,6 +548,8 @@
   boolean debugFailDuring_;
   boolean debugFailLate_;
 
+  boolean nonZeroExitIsFailure_;
+  
   Process sim;
   MROutputThread outThread_;
   String jobLog_;

Modified: hadoop/core/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java?rev=635598&r1=635597&r2=635598&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java (original)
+++ hadoop/core/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java Mon Mar 10 09:51:09 2008
@@ -515,6 +515,8 @@
     System.out.println("  -jobconf mapred.local.dir=/tmp/local");
     System.out.println("  -jobconf mapred.system.dir=/tmp/system");
     System.out.println("  -jobconf mapred.temp.dir=/tmp/temp");
+    System.out.println("To treat tasks with non-zero exit status as FAILED:");    
+    System.out.println("  -jobconf stream.non.zero.exit.is.failure=true");
     System.out.println("Use a custom hadoopStreaming build along a standard hadoop install:");
     System.out.println("  $HADOOP_HOME/bin/hadoop jar /path/my-hadoop-streaming.jar [...]\\");
     System.out

Added: hadoop/core/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/FailApp.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/FailApp.java?rev=635598&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/FailApp.java (added)
+++ hadoop/core/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/FailApp.java Mon Mar 10 09:51:09 2008
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming;
+
+import java.io.*;
+
+/**
+ * A simple Java app that will consume all input from stdin, echoing
+ * it to stdout, and then optionally throw an exception (which should
+ * cause a non-zero exit status for the process).
+ */
+public class FailApp
+{
+
+  public FailApp() {
+  }
+
+  public void go(boolean fail) throws IOException {
+    BufferedReader in = new BufferedReader(new InputStreamReader(System.in));
+    String line;
+
+    // Consume all input (to make sure streaming will still count this
+    // task as failed even if all input was consumed).
+    while ((line = in.readLine()) != null) {
+      System.out.println(line);
+    }
+
+    if (fail) {
+      throw new RuntimeException("Intentionally failing task");
+    }
+  }
+
+  public static void main(String[] args) throws IOException {
+    boolean fail = true;
+    if (args.length >= 1 && "false".equals(args[0])) {
+      fail = false;
+    }
+    
+    FailApp app = new FailApp();
+    app.go(fail);
+  }
+}

Added: hadoop/core/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingExitStatus.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingExitStatus.java?rev=635598&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingExitStatus.java (added)
+++ hadoop/core/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingExitStatus.java Mon Mar 10 09:51:09 2008
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming;
+
+import junit.framework.TestCase;
+import java.io.*;
+import java.util.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * This class tests if hadoopStreaming fails a job when the mapper or
+ * reducers have non-zero exit status and the
+ * stream.non.zero.exit.status.is.failure jobconf is set.
+ */
+public class TestStreamingExitStatus extends TestCase
+{
+  protected File INPUT_FILE = new File("input.txt");
+  protected File OUTPUT_DIR = new File("out");  
+
+  protected String failingTask = StreamUtil.makeJavaCommand(FailApp.class, new String[]{"true"});
+  protected String echoTask = StreamUtil.makeJavaCommand(FailApp.class, new String[]{"false"});
+
+  public TestStreamingExitStatus() throws IOException {
+    UtilTest utilTest = new UtilTest(getClass().getName());
+    utilTest.checkUserDir();
+    utilTest.redirectIfAntJunit();
+  }
+
+  protected String[] genArgs(boolean exitStatusIsFailure, boolean failMap) {
+    return new String[] {
+      "-input", INPUT_FILE.getAbsolutePath(),
+      "-output", OUTPUT_DIR.getAbsolutePath(),
+      "-mapper", (failMap ? failingTask : echoTask),
+      "-reducer", (failMap ? echoTask : failingTask),
+      "-jobconf", "keep.failed.task.files=true",
+      "-jobconf", "stream.non.zero.exit.is.failure=" + exitStatusIsFailure,
+      "-jobconf", "stream.tmpdir="+System.getProperty("test.build.data","/tmp")
+    };
+  }
+
+  public void setUp() throws IOException {
+    UtilTest.recursiveDelete(INPUT_FILE);
+    UtilTest.recursiveDelete(OUTPUT_DIR);
+    
+    FileOutputStream out = new FileOutputStream(INPUT_FILE.getAbsoluteFile());
+    out.write("hello\n".getBytes());
+    out.close();
+  }
+
+  public void tearDown() {
+    UtilTest.recursiveDelete(INPUT_FILE);
+    UtilTest.recursiveDelete(OUTPUT_DIR);    
+  }
+  
+  public void runStreamJob(boolean exitStatusIsFailure, boolean failMap) {
+    try {
+      boolean mayExit = false;
+      int returnStatus = 0;
+
+      StreamJob job = new StreamJob(genArgs(exitStatusIsFailure, failMap), mayExit);
+      returnStatus = job.go();
+      
+      if (exitStatusIsFailure) {
+        assertEquals("Streaming Job failure code expected", /*job not successful:*/1, returnStatus);
+      } else {
+        assertEquals("Streaming Job expected to succeed", 0, returnStatus);
+      }
+    } catch (Exception e) {
+      failTrace(e);
+    }
+  }
+  
+  public void testMapFailOk() {
+    runStreamJob(false, true);
+  }
+  
+  public void testMapFailNotOk() {
+    runStreamJob(true, true);
+  }
+  
+  public void testReduceFailOk() {
+    runStreamJob(false, false);
+  }
+  
+  public void testReduceFailNotOk() {
+    runStreamJob(true, false);
+  }  
+  
+  protected void failTrace(Exception e) {
+    StringWriter sw = new StringWriter();
+    e.printStackTrace(new PrintWriter(sw));
+    fail(sw.toString());
+  }
+}

Modified: hadoop/core/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/UtilTest.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/UtilTest.java?rev=635598&r1=635597&r2=635598&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/UtilTest.java (original)
+++ hadoop/core/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/UtilTest.java Mon Mar 10 09:51:09 2008
@@ -25,6 +25,30 @@
 
 class UtilTest {
 
+  /**
+   * Utility routine to recurisvely delete a directory.
+   * On normal return, the file does not exist.
+   *
+   * @param file File or directory to delete.
+   *
+   * @throws RuntimeException if the file, or some file within
+   * it, could not be deleted.
+   */
+  static void recursiveDelete(File file) {
+    file = file.getAbsoluteFile();
+
+    if (!file.exists()) return;
+    
+    if (file.isDirectory()) {
+      for (File child : file.listFiles()) {
+	recursiveDelete(child);
+      }
+    }
+    if (!file.delete()) {
+      throw new RuntimeException("Failed to delete " + file);
+    }
+  }
+  
   public UtilTest(String testName) {
     testName_ = testName;
     userDir_ = System.getProperty("user.dir");