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 om...@apache.org on 2007/08/15 00:19:40 UTC

svn commit: r565948 - in /lucene/hadoop/branches/branch-0.14: ./ src/contrib/streaming/src/java/org/apache/hadoop/streaming/ src/contrib/streaming/src/test/org/apache/hadoop/streaming/

Author: omalley
Date: Tue Aug 14 15:19:36 2007
New Revision: 565948

URL: http://svn.apache.org/viewvc?view=rev&rev=565948
Log:
Merge -r 565945:565946 from trunk to branch-0.14. Fixes HADOOP-1663.

Added:
    lucene/hadoop/branches/branch-0.14/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingFailure.java
      - copied unchanged from r565946, lucene/hadoop/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingFailure.java
Modified:
    lucene/hadoop/branches/branch-0.14/CHANGES.txt
    lucene/hadoop/branches/branch-0.14/src/contrib/streaming/src/java/org/apache/hadoop/streaming/HadoopStreaming.java
    lucene/hadoop/branches/branch-0.14/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java

Modified: lucene/hadoop/branches/branch-0.14/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/branches/branch-0.14/CHANGES.txt?view=diff&rev=565948&r1=565947&r2=565948
==============================================================================
--- lucene/hadoop/branches/branch-0.14/CHANGES.txt (original)
+++ lucene/hadoop/branches/branch-0.14/CHANGES.txt Tue Aug 14 15:19:36 2007
@@ -523,6 +523,8 @@
 155. HADOOP-1714.  Fix TestDFSUpgradeFromImage to work on Windows.
      (Raghu Angadi via nigel)
 
+156. HADOOP-1663.  Return a non-zero exit code if streaming fails. (Lohit Renu
+     via omalley)
 
 Release 0.13.0 - 2007-06-08
 

Modified: lucene/hadoop/branches/branch-0.14/src/contrib/streaming/src/java/org/apache/hadoop/streaming/HadoopStreaming.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/branches/branch-0.14/src/contrib/streaming/src/java/org/apache/hadoop/streaming/HadoopStreaming.java?view=diff&rev=565948&r1=565947&r2=565948
==============================================================================
--- lucene/hadoop/branches/branch-0.14/src/contrib/streaming/src/java/org/apache/hadoop/streaming/HadoopStreaming.java (original)
+++ lucene/hadoop/branches/branch-0.14/src/contrib/streaming/src/java/org/apache/hadoop/streaming/HadoopStreaming.java Tue Aug 14 15:19:36 2007
@@ -28,7 +28,12 @@
 
   public static void main(String[] args) throws IOException {
     boolean mayExit = true;
+    int returnStatus = 0;
     StreamJob job = new StreamJob(args, mayExit);
-    job.go();
+    returnStatus = job.go();
+    if (returnStatus != 0) {
+      System.err.println("Streaming Job Failed!");
+      System.exit(returnStatus);
+    }
   }
 }

Modified: lucene/hadoop/branches/branch-0.14/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/branches/branch-0.14/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java?view=diff&rev=565948&r1=565947&r2=565948
==============================================================================
--- lucene/hadoop/branches/branch-0.14/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java (original)
+++ lucene/hadoop/branches/branch-0.14/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java Tue Aug 14 15:19:36 2007
@@ -98,7 +98,7 @@
    * to the jobtracker
    * @throws IOException
    */
-  public void go() throws IOException {
+  public int go() throws IOException {
     init();
 
     preProcessArgs();
@@ -106,7 +106,7 @@
     postProcessArgs();
 
     setJobConf();
-    submitAndMonitorJob();
+    return submitAndMonitorJob();
   }
   
   protected void init() {
@@ -868,7 +868,7 @@
   }
 
   // Based on JobClient
-  public void submitAndMonitorJob() throws IOException {
+  public int submitAndMonitorJob() throws IOException {
 
     if (jar_ != null && isLocalHadoop()) {
       // getAbs became required when shell and subvm have different working dirs...
@@ -906,28 +906,33 @@
       }
       if (!running_.isSuccessful()) {
         jobInfo();
-        throw new IOException("Job not Successful!");
+	LOG.error("Job not Successful!");
+	return 1;
       }
       LOG.info("Job complete: " + jobId_);
       LOG.info("Output: " + output_);
       error = false;
-    } catch(FileNotFoundException fe){
+    } catch(FileNotFoundException fe) {
       LOG.error("Error launching job , bad input path : " + fe.getMessage());
-    }catch(InvalidJobConfException je){
+      return 2;
+    } catch(InvalidJobConfException je) {
       LOG.error("Error launching job , Invalid job conf : " + je.getMessage());
-    }catch(FileAlreadyExistsException fae){
+      return 3;
+    } catch(FileAlreadyExistsException fae) {
       LOG.error("Error launching job , Output path already exists : " 
                 + fae.getMessage());
-    }catch(IOException ioe){
+      return 4;
+    } catch(IOException ioe) {
       LOG.error("Error Launching job : " + ioe.getMessage());
-    }
-    finally {
+      return 5;
+    } finally {
       if (error && (running_ != null)) {
         LOG.info("killJob...");
         running_.killJob();
       }
       jc_.close();
     }
+    return 0;
   }
   /** Support -jobconf x=y x1=y1 type options **/
   class MultiPropertyOption extends PropertyOption{