Преглед на файлове

HADOOP-542. Omnibus patch for contrib/streaming. Contributed by Michel.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@447626 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting преди 19 години
родител
ревизия
7617f69315
променени са 29 файла, в които са добавени 2967 реда и са изтрити 1432 реда
  1. 4 0
      CHANGES.txt
  2. 1 1
      build.xml
  3. 5 2
      src/contrib/build-contrib.xml
  4. 23 1
      src/contrib/streaming/build.xml
  5. 270 0
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/CompoundDirSpec.java
  6. 69 80
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/Environment.java
  7. 5 6
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/HadoopStreaming.java
  8. 154 169
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/JarBuilder.java
  9. 334 0
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/MergerInputFormat.java
  10. 50 61
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/MustangFile.java
  11. 172 0
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/MuxOutputFormat.java
  12. 2 4
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeCombiner.java
  13. 382 287
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapRed.java
  14. 29 35
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapper.java
  15. 19 18
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeReducer.java
  16. 41 54
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamBaseRecordReader.java
  17. 38 57
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamInputFormat.java
  18. 370 206
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java
  19. 48 48
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamLineRecordReader.java
  20. 17 19
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamOutputFormat.java
  21. 15 25
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamSequenceRecordReader.java
  22. 316 139
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamUtil.java
  23. 114 135
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamXmlRecordReader.java
  24. 86 0
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/TupleInputFormat.java
  25. 317 0
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamedMerge.java
  26. 14 62
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreaming.java
  27. 10 18
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TrApp.java
  28. 6 5
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/UniqApp.java
  29. 56 0
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/UtilTest.java

+ 4 - 0
CHANGES.txt

@@ -23,6 +23,10 @@ Trunk (unreleased changes)
 6. HADOOP-527.  Permit specification of the local address that various
    Hadoop daemons should bind to.  (Philippe Gassmann via cutting)
 
+7. HADOOP-542.  Updates to contrib/streaming: reformatted source code,
+   on-the-fly merge sort, a fix for HADOOP-540, etc.
+   (Michel Tourn via cutting)
+
 
 Release 0.6.2 (unreleased)
 

+ 1 - 1
build.xml

@@ -317,7 +317,7 @@
 
   </target>   
 
-  <target name="test-contrib" depends="compile-core">
+  <target name="test-contrib" depends="compile-core, compile-core-test">
     <subant target="test">
         <fileset file="${basedir}/src/contrib/build.xml"/>
     </subant>  	

+ 5 - 2
src/contrib/build-contrib.xml

@@ -43,7 +43,7 @@
     <fileset refid="lib.jars"/>
     <pathelement location="${hadoop.root}/build/classes"/>
     <fileset dir="${hadoop.root}/lib">
-      <include name="*.jar" />
+      <include name="**/*.jar" />
     </fileset>
   </path>
 
@@ -131,7 +131,7 @@
   <!-- ================================================================== -->
   <target name="test" depends="compile-test, deploy" if="test.available">
     <echo message="contrib: ${name}"/>
-    <junit 
+    <junit
       printsummary="withOutAndErr" haltonfailure="no" fork="yes"
       errorProperty="tests.failed" failureProperty="tests.failed">
       
@@ -143,6 +143,9 @@
       -->
       <sysproperty key="user.dir" value="${build.test}/data"/>
       
+      <sysproperty key="fs.default.name" value="${fs.default.name}"/>
+      <sysproperty key="hadoop.test.localoutputfile" value="${hadoop.test.localoutputfile}"/>
+   
       <classpath refid="test.classpath"/>
       <formatter type="plain" />
       <batchtest todir="${build.test}" unless="testcase">

+ 23 - 1
src/contrib/streaming/build.xml

@@ -1,5 +1,9 @@
 <?xml version="1.0"?>
 
+<!-- 
+Before you can run these subtargets directly, you need 
+to call at top-level: ant deploy-contrib compile-core-test
+-->
 <project name="streaming" default="jar">
 
   <import file="../build-contrib.xml"/>
@@ -15,5 +19,23 @@
 	</manifest>
     </jar>
   </target>
-  
+
+  <!-- Run only pure-Java unit tests. superdottest -->
+  <target name="test">
+   <antcall target="hadoopbuildcontrib.test"> 
+     <param name="test.exclude" value="TestStreamedMerge"/>
+   </antcall>
+  </target>  
+ 
+  <!-- Run all unit tests
+  This is not called as part of the nightly build
+  because it will only run on platforms that have standard 
+  Unix utilities available. 
+  -->
+ <target name="test-unix">
+   <antcall target="hadoopbuildcontrib.test">
+   </antcall>
+ </target>  
+
+
 </project>

+ 270 - 0
src/contrib/streaming/src/java/org/apache/hadoop/streaming/CompoundDirSpec.java

@@ -0,0 +1,270 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.File;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.mapred.JobConf;
+
+/** Parses a -input &lt;spec> that determines the DFS paths that will 
+ be accessed by a MergedInputFormat.<br>
+ CompoundDirSpec.getPaths() is a 2-D ragged array of DFS paths.<br>
+ One of the paths is the <b>primary</b> and can contain a globbing pattern
+ to match multiple files.<br>
+ The other paths are <b>secondary</b> and must indicate either a directory or a single file.
+ During execution secondary files are computed to be the secondary path 
+ plus the primary non-qualified filename.
+ Example: <tt>
+ -input "/filter/colsx NULL | +/batch1/colsx/* /batch1/colsy/"
+ -input "/filter/colsx NULL | +/batch2/colsx/* /batch2/colsy/"
+ </tt>
+ Files and contents:<tt>
+ /filter/colsx/part-00000:
+ /batch1/colsx/part-00000:
+ /batch1/colsy/part-00000:
+ /batch2/colsx/part-00000:
+ /batch2/colsy/part-00000:
+ </tt>
+ Mapper input:<tt>
+ </tt>
+ Side-effect outputs with Identity "mapper":<tt>
+
+ </tt>
+ @author Michel Tourn
+ */
+class CompoundDirSpec {
+
+  // Keep the Usage messages and docs in sync!
+  public final static String MERGEGLOB_PREFIX = "||";
+  public final static String MERGE_SEP = "|";
+  public final static String COL_SEP = " ";
+  public final static String PRIMARY_PREFIX = "+";
+
+  CompoundDirSpec(String argSpec, boolean isInputSpec) {
+    argSpec_ = argSpec;
+    isInputSpec_ = isInputSpec;
+
+    direction_ = isInputSpec_ ? "input" : "output";
+    parse();
+  }
+
+  public void parse() throws IllegalStateException {
+    String[] mergerSpecs = argSpec_.split(StreamUtil.regexpEscape(MERGE_SEP));
+
+    int msup = mergerSpecs.length;
+    paths_ = new String[msup][];
+
+    if (msup == 0) {
+      throw new IllegalStateException("A -" + direction_ + " spec needs at list one path");
+    }
+    if (false == isInputSpec_) {
+      if (msup > 1) {
+        throw new IllegalStateException("A -output spec cannot use merged streams ('" + MERGE_SEP
+            + "' delimiter)");
+      }
+    }
+    for (int m = 0; m < msup; m++) {
+      String merged = mergerSpecs[m];
+      merged = merged.trim();
+      String[] colSpecs = merged.split(StreamUtil.regexpEscape(COL_SEP));
+      int csup = colSpecs.length;
+      if (csup == 0) {
+        throw new IllegalStateException("A -input spec needs at list one path spec per |<column>|");
+      }
+      paths_[m] = new String[csup];
+      for (int c = 0; c < csup; c++) {
+        String spec = colSpecs[c];
+        if (spec.startsWith(PRIMARY_PREFIX)) {
+          // for (!isInputSpec_) the tuple paths should be symmetric.
+          // but we still allow user to specify one in case setOutputDir makes a difference
+          if (prow_ != NA) {
+            throwBadNumPrimaryInputSpecs();
+          }
+          spec = spec.substring(PRIMARY_PREFIX.length());
+          prow_ = m;
+          pcol_ = c;
+        }
+        paths_[m][c] = spec;
+      }
+    }
+    if (prow_ == NA) {
+      if (!isInputSpec_) {
+        // pick an 'arbitrary' one -- the tuple paths should be symmetric.
+        prow_ = 0;
+        pcol_ = 0;
+      } else if (msup == 1 && paths_[0].length == 1) {
+        // pick the only one available. That's also bw-compatible syntax
+        prow_ = 0;
+        pcol_ = 0;
+      } else {
+        throwBadNumPrimaryInputSpecs();
+      }
+    }
+  }
+
+  void throwBadNumPrimaryInputSpecs() throws IllegalStateException {
+    String msg = "A compound -input spec needs exactly one primary path prefixed with "
+        + PRIMARY_PREFIX;
+    msg += ":\n";
+    msg += toTableString();
+    throw new IllegalStateException(msg);
+  }
+
+  // TBD need to decide early whether they are dirs or files or globs?
+  public void validatePaths(FileSystem fs) {
+    int rsup = paths_.length;
+    for (int r = 0; r < rsup; r++) {
+      int csup = paths_[r].length;
+      for (int c = 0; c < csup; c++) {
+        String path = paths_[r][c];
+      }
+    }
+  }
+
+  public int primaryRow() {
+    return prow_;
+  }
+
+  public int primaryCol() {
+    return pcol_;
+  }
+
+  public String primarySpec() {
+    return paths_[prow_][pcol_];
+  }
+  
+  /*
+   Example input spec in table form:
+   <1 +[/input/part-00] 
+   <2  [/input/part-01] 
+   <3  [/input/part-02] 
+   Example output spec in table form:
+   +[/my.output] 
+   */
+  public String toTableString() {
+    StringBuffer buf = new StringBuffer();
+    int maxWid = 0;
+    for (int pass = 1; pass <= 2; pass++) {
+      int rsup = paths_.length;
+      for (int r = 0; r < rsup; r++) {
+        int csup = paths_[r].length;
+        for (int c = 0; c < csup; c++) {
+          String cell = "[" + paths_[r][c] + "]";
+          if (r == prow_ && c == pcol_) {
+            cell = PRIMARY_PREFIX + cell;
+          } else {
+            cell = StreamUtil.rjustify(cell, cell.length() + PRIMARY_PREFIX.length());
+          }
+          if (isInputSpec_) {
+            // channels are for tagged input streams: r-based
+            if (rsup > 1) {
+              String channel = "<" + (r + 1);
+              cell = channel + " " + cell;
+            }
+          } else {
+            // channels are for columns (multiple files) c-based
+            if (csup > 1) {
+              String channel = ">" + (c + 1);
+              cell = channel + " " + cell;
+            }
+          }
+          if (pass == 2) {
+            cell = StreamUtil.ljustify(cell, maxWid);
+            buf.append(cell);
+            buf.append(" ");
+          } else {
+            if (cell.length() > maxWid) {
+              maxWid = cell.length();
+            }
+          }
+        }
+        if (pass == 2) {
+          buf.append("\n");
+        }
+      }
+    }
+    return buf.toString();
+  }
+
+  /** 
+   @see #primaryRow 
+   @see #primaryCol
+   */
+  public String[][] getPaths() {
+    return paths_;
+  }
+
+  // ==== Static helpers that depend on a JobConf. ====
+  
+  // Unlike CompoundDirSpec.parse() which is reexecuted at Task runtime,
+  // this is expanded once in advance and relies on client-side DFS access.
+  // Main reason is we need to choose a primary input file at submission time. 
+  public static String expandGlobInputSpec(String inputSpec, JobConf job)
+  {
+    inputSpec = inputSpec.trim();
+    if(!inputSpec.startsWith(MERGEGLOB_PREFIX)) {
+      return inputSpec;
+    }
+    inputSpec = inputSpec.substring(MERGEGLOB_PREFIX.length());
+    // TODO use upcoming DFSShell wildcarding code..
+    return inputSpec;
+  }
+  
+  // find the -input statement that contains the job's split
+  // TODO test with globbing / directory /single file
+  public static CompoundDirSpec findInputSpecForPrimary(String primary, JobConf job) {
+    int num = job.getInt("stream.numinputspecs", -1);
+    for (int s = 0; s < num; s++) {
+      String specStr = job.get("stream.inputspecs." + s);
+      CompoundDirSpec spec = new CompoundDirSpec(specStr, true);
+      if (pathsMatch(spec.primarySpec(), primary, job)) {
+        return spec;
+      }
+    }
+    return null;
+  }
+
+  // There can be only one output spec but this provides some server-side validation
+  public static CompoundDirSpec findOutputSpecForPrimary(String primary, JobConf job) {
+    String specStr = job.get("stream.outputspec");
+    CompoundDirSpec spec = new CompoundDirSpec(specStr, false);
+    if (pathsMatch(spec.primarySpec(), primary, job)) {
+      return spec;
+    }
+    return spec;
+  }
+
+  static boolean pathsMatch(String s1, String s2, JobConf job) {
+    boolean isLocalFS = job.get("fs.default.name", "").equals("local");
+    if (isLocalFS) {
+      s1 = StreamUtil.safeGetCanonicalPath(new File(s1));
+      s2 = StreamUtil.safeGetCanonicalPath(new File(s2));
+    }
+    return (s1.equals(s2));
+  }
+
+  final static int NA = -1;
+
+  String argSpec_;
+  boolean isInputSpec_;
+
+  String direction_;
+  String[][] paths_;
+  int prow_ = NA;
+  int pcol_ = NA;
+}

+ 69 - 80
src/contrib/streaming/src/java/org/apache/hadoop/streaming/Environment.java

@@ -23,89 +23,78 @@ import java.util.*;
 /*
  * If we move to Java 1.5, we can get rid of this class and just use System.getenv
  */
-public class Environment extends Properties
-{
-   public Environment()
-      throws IOException
-   {
-      // Extend this code to fit all operating
-      // environments that you expect to run in
-      // http://lopica.sourceforge.net/os.html
-      String command = null;
-      String OS = System.getProperty("os.name");
-      String lowerOs = OS.toLowerCase();
-      if (OS.indexOf("Windows") > -1) {
-         command = "cmd /C set";
-      } else if (lowerOs.indexOf("ix") > -1 || lowerOs.indexOf("linux") > -1 
-        || lowerOs.indexOf("freebsd") > -1
-        || lowerOs.indexOf("sunos") > -1 || lowerOs.indexOf("solaris") > -1
-        || lowerOs.indexOf("hp-ux") > -1) {
-         command = "env";
-      } else if(lowerOs.startsWith("mac os x")) {
-         command = "env";
-      } else {
-         // Add others here
-      }
+public class Environment extends Properties {
 
-      if (command == null) {
-         throw new RuntimeException("Operating system " + OS
-            + " not supported by this class");
-      }
+  public Environment() throws IOException {
+    // Extend this code to fit all operating
+    // environments that you expect to run in
+    // http://lopica.sourceforge.net/os.html
+    String command = null;
+    String OS = System.getProperty("os.name");
+    String lowerOs = OS.toLowerCase();
+    if (OS.indexOf("Windows") > -1) {
+      command = "cmd /C set";
+    } else if (lowerOs.indexOf("ix") > -1 || lowerOs.indexOf("linux") > -1
+        || lowerOs.indexOf("freebsd") > -1 || lowerOs.indexOf("sunos") > -1
+        || lowerOs.indexOf("solaris") > -1 || lowerOs.indexOf("hp-ux") > -1) {
+      command = "env";
+    } else if (lowerOs.startsWith("mac os x")) {
+      command = "env";
+    } else {
+      // Add others here
+    }
 
-      // Read the environment variables
+    if (command == null) {
+      throw new RuntimeException("Operating system " + OS + " not supported by this class");
+    }
 
-      Process pid = Runtime.getRuntime().exec(command);
-      BufferedReader in =
-         new BufferedReader(
-         new InputStreamReader(
-         pid.getInputStream()));
-      while(true) {
-         String line = in.readLine();
-         if (line == null)
-            break;
-         int p = line.indexOf("=");
-         if (p != -1) {
-            String name = line.substring(0, p);
-            String value = line.substring(p+1);
-            setProperty(name, value);
-         }
+    // Read the environment variables
+
+    Process pid = Runtime.getRuntime().exec(command);
+    BufferedReader in = new BufferedReader(new InputStreamReader(pid.getInputStream()));
+    while (true) {
+      String line = in.readLine();
+      if (line == null) break;
+      int p = line.indexOf("=");
+      if (p != -1) {
+        String name = line.substring(0, p);
+        String value = line.substring(p + 1);
+        setProperty(name, value);
       }
-      in.close();
+    }
+    in.close();
+    try {
+      pid.waitFor();
+    } catch (InterruptedException e) {
+      throw new IOException(e.getMessage());
+    }
+  }
+
+  // to be used with Runtime.exec(String[] cmdarray, String[] envp) 
+  String[] toArray() {
+    String[] arr = new String[super.size()];
+    Enumeration it = super.keys();
+    int i = -1;
+    while (it.hasMoreElements()) {
+      String key = (String) it.nextElement();
+      String val = (String) get(key);
+      i++;
+      arr[i] = key + "=" + val;
+    }
+    return arr;
+  }
+
+  public String getHost() {
+    String host = getProperty("HOST");
+    if (host == null) {
+      // HOST isn't always in the environment
       try {
-         pid.waitFor();
-      }
-      catch (InterruptedException e) {
-         throw new IOException(e.getMessage());
+        host = InetAddress.getLocalHost().getHostName();
+      } catch (IOException io) {
+        io.printStackTrace();
       }
-   }
-   
-   // to be used with Runtime.exec(String[] cmdarray, String[] envp) 
-   String[] toArray()
-   {
-     String[] arr = new String[super.size()];
-     Enumeration it = super.keys();
-     int i = -1;
-     while(it.hasMoreElements()) {
-        String key = (String)it.nextElement();
-        String val = (String)get(key);
-        i++;   
-        arr[i] = key + "=" + val;
-     }     
-     return arr;
-   }
-   
-   public String getHost()
-   {
-     String host = getProperty("HOST");
-     if(host == null) {
-       // HOST isn't always in the environment
-       try {
-         host = InetAddress.getLocalHost().getHostName();
-       } catch(IOException io) {
-         io.printStackTrace();
-       }
-     }
-     return host;
-   }
-   
-} 
+    }
+    return host;
+  }
+
+}

+ 5 - 6
src/contrib/streaming/src/java/org/apache/hadoop/streaming/HadoopStreaming.java

@@ -19,13 +19,12 @@ package org.apache.hadoop.streaming;
 import java.io.IOException;
 
 /** The main entrypoint. Usually invoked with the script bin/hadoopStreaming
-*/
-public class HadoopStreaming 
-{
-  public static void main(String[] args) throws IOException
-  {
+ */
+public class HadoopStreaming {
+
+  public static void main(String[] args) throws IOException {
     boolean mayExit = true;
     StreamJob job = new StreamJob(args, mayExit);
     job.go();
-  }  
+  }
 }

+ 154 - 169
src/contrib/streaming/src/java/org/apache/hadoop/streaming/JarBuilder.java

@@ -21,190 +21,175 @@ import java.util.*;
 import java.util.jar.*;
 import java.util.zip.ZipException;
 
-public class JarBuilder
-{
-    public JarBuilder()
-    {
-    }
+public class JarBuilder {
 
-    public void setVerbose(boolean v)
-    {
-        this.verbose = v;
-    }
-    
-    public void merge(List srcNames, List srcUnjar, String dstJar) 
-        throws IOException
-    {
-        String source = null;
-        JarOutputStream jarOut = null;
-        JarFile jarSource = null;
-        jarOut = new JarOutputStream(new FileOutputStream(dstJar));
-        boolean throwing = false;
-        try {
-          if(srcNames != null) {
-            Iterator iter = srcNames.iterator(); 
-            while(iter.hasNext()) {
-                source = (String)iter.next();
-                File fsource = new File(source);
-                String base = getBasePathInJarOut(source);
-                if(!fsource.exists()) {
-                    throwing = true;
-                    throw new FileNotFoundException(fsource.getAbsolutePath());
-                }
-                if(fsource.isDirectory()) {
-                    addDirectory(jarOut, base, fsource, 0);
-                } else {
-                    addFileStream(jarOut, base, fsource);
-                }
-            }
+  public JarBuilder() {
+  }
+
+  public void setVerbose(boolean v) {
+    this.verbose = v;
+  }
+
+  public void merge(List srcNames, List srcUnjar, String dstJar) throws IOException {
+    String source = null;
+    JarOutputStream jarOut = null;
+    JarFile jarSource = null;
+    jarOut = new JarOutputStream(new FileOutputStream(dstJar));
+    boolean throwing = false;
+    try {
+      if (srcNames != null) {
+        Iterator iter = srcNames.iterator();
+        while (iter.hasNext()) {
+          source = (String) iter.next();
+          File fsource = new File(source);
+          String base = getBasePathInJarOut(source);
+          if (!fsource.exists()) {
+            throwing = true;
+            throw new FileNotFoundException(fsource.getAbsolutePath());
           }
-          if(srcUnjar != null) {
-            Iterator iter = srcUnjar.iterator(); 
-            while(iter.hasNext()) {
-                source = (String)iter.next();
-                jarSource = new JarFile(source);
-                addJarEntries(jarOut, jarSource);
-                jarSource.close();
-            }
-          
+          if (fsource.isDirectory()) {
+            addDirectory(jarOut, base, fsource, 0);
+          } else {
+            addFileStream(jarOut, base, fsource);
           }
-        } finally {
-            try {
-              jarOut.close();
-            } catch(ZipException z) {
-                if(! throwing) {
-                    throw new IOException(z.toString());
-                }
-            }
         }
+      }
+      if (srcUnjar != null) {
+        Iterator iter = srcUnjar.iterator();
+        while (iter.hasNext()) {
+          source = (String) iter.next();
+          jarSource = new JarFile(source);
+          addJarEntries(jarOut, jarSource);
+          jarSource.close();
+        }
+
+      }
+    } finally {
+      try {
+        jarOut.close();
+      } catch (ZipException z) {
+        if (!throwing) {
+          throw new IOException(z.toString());
+        }
+      }
     }
+  }
+
+  protected String fileExtension(String file) {
+    int leafPos = file.lastIndexOf('/');
+    if (leafPos == file.length() - 1) return "";
+    String leafName = file.substring(leafPos + 1);
+    int dotPos = leafName.lastIndexOf('.');
+    if (dotPos == -1) return "";
+    String ext = leafName.substring(dotPos + 1);
+    return ext;
+  }
 
-    protected String fileExtension(String file)
-    {
-    	int leafPos = file.lastIndexOf('/');
-    	if(leafPos == file.length()-1) return "";
-        String leafName = file.substring(leafPos+1);
-    	int dotPos = leafName.lastIndexOf('.');
-    	if(dotPos == -1) return "";
-        String ext = leafName.substring(dotPos+1);
-    	return ext;
+  /** @return empty or a jar base path. Must not start with '/' */
+  protected String getBasePathInJarOut(String sourceFile) {
+    // TaskRunner will unjar and append to classpath: .:classes/:lib/*    	
+    String ext = fileExtension(sourceFile);
+    if (ext.equals("class")) {
+      return "classes/"; // or ""
+    } else if (ext.equals("jar") || ext.equals("zip")) {
+      return "lib/";
+    } else {
+      return "";
     }
-    
-    /** @return empty or a jar base path. Must not start with '/' */
-    protected String getBasePathInJarOut(String sourceFile)
-    {
-        // TaskRunner will unjar and append to classpath: .:classes/:lib/*    	
-    	String ext = fileExtension(sourceFile);
-    	if(ext.equals("class")) {
-    		return "classes/"; // or ""
-        } else if(ext.equals("jar") || ext.equals("zip")) {
-    		return "lib/";
-    	} else {
-            return "";
-        }
+  }
+
+  private void addJarEntries(JarOutputStream dst, JarFile src) throws IOException {
+    Enumeration entries = src.entries();
+    JarEntry entry = null;
+    while (entries.hasMoreElements()) {
+      entry = (JarEntry) entries.nextElement();
+      //if(entry.getName().startsWith("META-INF/")) continue; 
+      InputStream in = src.getInputStream(entry);
+      addNamedStream(dst, entry.getName(), in);
     }
-    
-    private void addJarEntries(JarOutputStream dst, JarFile src)
-        throws IOException
-    {
-        Enumeration entries = src.entries();
-        JarEntry entry = null;
-        while(entries.hasMoreElements()) {
-            entry = (JarEntry)entries.nextElement();
-            //if(entry.getName().startsWith("META-INF/")) continue; 
-            InputStream in = src.getInputStream(entry);
-            addNamedStream(dst, entry.getName(), in);
-        }
+  }
+
+  /** @param name path in jar for this jar element. Must not start with '/' */
+  void addNamedStream(JarOutputStream dst, String name, InputStream in) throws IOException {
+    if (verbose) {
+      System.err.println("JarBuilder.addNamedStream " + name);
     }
-    
-    /** @param name path in jar for this jar element. Must not start with '/' */
-    void addNamedStream(JarOutputStream dst, String name, InputStream in) 
-        throws IOException
-    {
-        if(verbose) {
-            System.err.println("JarBuilder.addNamedStream " + name);
-        }
-        try {
-          dst.putNextEntry(new JarEntry(name));
-          int bytesRead = 0;
-          while((bytesRead = in.read(buffer, 0, BUFF_SIZE)) != -1) {
-              dst.write(buffer, 0, bytesRead);
-          }
-        } catch(ZipException ze) {
-            if(ze.getMessage().indexOf("duplicate entry") >= 0) {
-              if(verbose) {
-                  System.err.println(ze + " Skip duplicate entry " + name);
-              }
-            } else {
-                throw ze;
-            }
-        } finally {
-          in.close();
-          dst.flush();
-          dst.closeEntry();        
+    try {
+      dst.putNextEntry(new JarEntry(name));
+      int bytesRead = 0;
+      while ((bytesRead = in.read(buffer, 0, BUFF_SIZE)) != -1) {
+        dst.write(buffer, 0, bytesRead);
+      }
+    } catch (ZipException ze) {
+      if (ze.getMessage().indexOf("duplicate entry") >= 0) {
+        if (verbose) {
+          System.err.println(ze + " Skip duplicate entry " + name);
         }
+      } else {
+        throw ze;
+      }
+    } finally {
+      in.close();
+      dst.flush();
+      dst.closeEntry();
     }
+  }
 
-    void addFileStream(JarOutputStream dst, String jarBaseName, File file) 
-        throws IOException 
-    {
-    	FileInputStream in = new FileInputStream(file);
-    	String name = jarBaseName + file.getName();
-    	addNamedStream(dst, name, in);
-    	in.close();
-    }
-    
-    void addDirectory(JarOutputStream dst, String jarBaseName, File dir, int depth) 
-        throws IOException
-    {
-    	File[] contents = dir.listFiles();
-    	if(contents != null) {
-    		for(int i=0; i<contents.length; i++) {
-    			File f = contents[i];
-    			String fBaseName = (depth==0) ? "" : dir.getName();
-    			if(jarBaseName.length()>0) {
-    				fBaseName = jarBaseName + "/" + fBaseName;
-    			}
-    			if(f.isDirectory()) {
-    				addDirectory(dst, fBaseName, f, depth+1);
-    			} else {
-    				addFileStream(dst, fBaseName+"/", f);
-    			}
-    		}
-    	}
+  void addFileStream(JarOutputStream dst, String jarBaseName, File file) throws IOException {
+    FileInputStream in = new FileInputStream(file);
+    String name = jarBaseName + file.getName();
+    addNamedStream(dst, name, in);
+    in.close();
+  }
+
+  void addDirectory(JarOutputStream dst, String jarBaseName, File dir, int depth) throws IOException {
+    File[] contents = dir.listFiles();
+    if (contents != null) {
+      for (int i = 0; i < contents.length; i++) {
+        File f = contents[i];
+        String fBaseName = (depth == 0) ? "" : dir.getName();
+        if (jarBaseName.length() > 0) {
+          fBaseName = jarBaseName + "/" + fBaseName;
+        }
+        if (f.isDirectory()) {
+          addDirectory(dst, fBaseName, f, depth + 1);
+        } else {
+          addFileStream(dst, fBaseName + "/", f);
+        }
+      }
     }
+  }
 
-    /** Test program */    
-    public static void main(String args[])
-    {
-        // args = new String[] { "C:/Temp/merged.jar", "C:/jdk1.5.0/jre/lib/ext/dnsns.jar",  "/Temp/addtojar2.log", "C:/jdk1.5.0/jre/lib/ext/mtest.jar", "C:/Temp/base"};
-        if(args.length < 2) {
-            System.err.println("Usage: JarFiles merged.jar [src.jar | dir | file ]+");
+  /** Test program */
+  public static void main(String args[]) {
+    // args = new String[] { "C:/Temp/merged.jar", "C:/jdk1.5.0/jre/lib/ext/dnsns.jar",  "/Temp/addtojar2.log", "C:/jdk1.5.0/jre/lib/ext/mtest.jar", "C:/Temp/base"};
+    if (args.length < 2) {
+      System.err.println("Usage: JarFiles merged.jar [src.jar | dir | file ]+");
+    } else {
+      JarBuilder jarFiles = new JarBuilder();
+      List names = new ArrayList();
+      List unjar = new ArrayList();
+      for (int i = 1; i < args.length; i++) {
+        String f = args[i];
+        String ext = jarFiles.fileExtension(f);
+        boolean expandAsJar = ext.equals("jar") || ext.equals("zip");
+        if (expandAsJar) {
+          unjar.add(f);
         } else {
-            JarBuilder jarFiles = new JarBuilder();
-            List names = new ArrayList();
-            List unjar = new ArrayList();
-            for(int i = 1; i < args.length; i++) {
-                String f = args[i];
-                String ext = jarFiles.fileExtension(f);
-                boolean expandAsJar = ext.equals("jar") || ext.equals("zip");                
-                if(expandAsJar) {
-                    unjar.add(f);
-                } else {
-                    names.add(f);
-                }                
-            }
-            try {
-                jarFiles.merge(names, unjar, args[0]);
-                Date lastMod = new Date(new File(args[0]).lastModified());
-                System.out.println("Merge done to " + args[0] + " " + lastMod);
-            } catch(Exception ge) {
-                ge.printStackTrace(System.err);
-            }
+          names.add(f);
         }
+      }
+      try {
+        jarFiles.merge(names, unjar, args[0]);
+        Date lastMod = new Date(new File(args[0]).lastModified());
+        System.out.println("Merge done to " + args[0] + " " + lastMod);
+      } catch (Exception ge) {
+        ge.printStackTrace(System.err);
+      }
     }
-    
-    private static final int BUFF_SIZE = 32*1024;
-    private byte buffer[] = new byte[BUFF_SIZE];
-    protected boolean verbose = false;
+  }
+
+  private static final int BUFF_SIZE = 32 * 1024;
+  private byte buffer[] = new byte[BUFF_SIZE];
+  protected boolean verbose = false;
 }

+ 334 - 0
src/contrib/streaming/src/java/org/apache/hadoop/streaming/MergerInputFormat.java

@@ -0,0 +1,334 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.IOException;
+import java.io.ByteArrayOutputStream;
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.util.ArrayList;
+import org.apache.lucene.util.PriorityQueue;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapred.*;
+
+/**
+ Eventually will be fed TupleInputFormats. 
+ For now will be fed primitive InputFormats.
+ @author Michel Tourn
+ */
+public class MergerInputFormat extends InputFormatBase {
+
+  public MergerInputFormat() {
+  }
+
+  void checkReady(FileSystem fs, JobConf job) {
+    if (ready_) {
+      // could complain if fs / job changes
+      return;
+    }
+    fs_ = fs;
+    job_ = job;
+    debug_ = (job.get("stream.debug") != null);
+
+    String someInputSpec = job_.get("stream.inputspecs.0");
+    CompoundDirSpec someSpec = new CompoundDirSpec(someInputSpec, true);
+    fmts_ = new ArrayList();
+    int n = someSpec.paths_.length;
+    inputTagged_ = job.getBoolean("stream.inputtagged", false);
+    //  0 is primary
+    //  Curr. secondaries are NOT used for getSplits(), only as RecordReader factory
+    for (int i = 0; i < n; i++) {
+      // this ignores -inputreader.. 
+      // That's why if hasSimpleInputSpecs_=true (n=1) then StreamJob will set
+      // the top-level format to StreamInputFormat rather than MergeInputFormat.
+      // So we only support custom -inputformat for n=1. 
+      // Probably OK for now since custom inputformats would be constrained (no \t and \n in payload) 
+      fmts_.add(new StreamInputFormat()); // will be TupleInputFormat
+    }
+    primary_ = (InputFormat) fmts_.get(0);
+    ready_ = true;
+  }
+
+  /** This implementation always returns true. */
+  public boolean[] areValidInputDirectories(FileSystem fileSys, Path[] inputDirs) throws IOException {
+    // must do this without JobConf...
+    boolean[] b = new boolean[inputDirs.length];
+    for (int i = 0; i < inputDirs.length; ++i) {
+      b[i] = true;
+    }
+    return b;
+  }
+
+  /** Delegate to the primary InputFormat. 
+   Force full-file splits since there's no index to sync secondaries.
+   (and if there was, this index may need to be created for the first time
+   full file at a time...    )
+   */
+  public FileSplit[] getSplits(FileSystem fs, JobConf job, int numSplits) throws IOException {
+    checkReady(fs, job);
+    return ((StreamInputFormat) primary_).getFullFileSplits(fs, job);
+  }
+
+  /**
+   */
+  public RecordReader getRecordReader(FileSystem fs, FileSplit split, JobConf job, Reporter reporter) throws IOException {
+    checkReady(fs, job);
+
+    reporter.setStatus(split.toString());
+
+    ArrayList readers = new ArrayList();
+    String primary = split.getPath().toString();
+    CompoundDirSpec spec = CompoundDirSpec.findInputSpecForPrimary(primary, job);
+    if (spec == null) {
+      throw new IOException("Did not find -input spec in JobConf for primary:" + primary);
+    }
+    for (int i = 0; i < fmts_.size(); i++) {
+      InputFormat f = (InputFormat) fmts_.get(i);
+      Path path = new Path(spec.getPaths()[i][0]);
+      FileSplit fsplit = makeFullFileSplit(path);
+      RecordReader r = f.getRecordReader(fs, fsplit, job, reporter);
+      readers.add(r);
+    }
+
+    return new MergedRecordReader(readers);
+  }
+
+  private FileSplit makeFullFileSplit(Path path) throws IOException {
+    long len = fs_.getLength(path);
+    return new FileSplit(path, 0, len);
+  }
+
+  /*
+   private FileSplit relatedSplit(FileSplit primarySplit, int i, CompoundDirSpec spec) throws IOException
+   {
+   if(i == 0) {
+   return primarySplit;
+   }
+
+   // TODO based on custom JobConf (or indirectly: InputFormat-s?)
+   String path = primarySplit.getFile().getAbsolutePath();
+   Path rpath = new Path(path + "." + i);
+
+   long rlength = fs_.getLength(rpath);
+   FileSplit related = new FileSplit(rpath, 0, rlength);
+   return related;    
+   }*/
+
+  class MergedRecordReader implements RecordReader {
+
+    MergedRecordReader(ArrayList/*<RecordReader>*/readers) throws IOException {
+      try {
+        readers_ = readers;
+        primaryReader_ = (RecordReader) readers.get(0);
+        q_ = new MergeQueue(readers.size(), debug_);
+        for (int i = 0; i < readers_.size(); i++) {
+          RecordReader reader = (RecordReader) readers.get(i);
+          WritableComparable k = (WritableComparable) job_.getInputKeyClass().newInstance();
+          Writable v = (Writable) job_.getInputValueClass().newInstance();
+          MergeRecordStream si = new MergeRecordStream(i, reader, k, v);
+          if (si.next()) {
+            q_.add(si);
+          }
+        }
+      } catch (Exception e) {
+        e.printStackTrace();
+        throw new IOException(e.toString());
+      }
+    }
+
+    // 1. implements RecordReader
+
+    public boolean next(Writable key, Writable value) throws IOException {
+      boolean more = (q_.size() > 0);
+      if (!more) return false;
+
+      MergeRecordStream ms = (MergeRecordStream) q_.top();
+      int keyTag = inputTagged_ ? (ms.index_ + 1) : NOTAG;
+      assignTaggedWritable(key, ms.k_, keyTag);
+      assignTaggedWritable(value, ms.v_, NOTAG);
+
+      if (ms.next()) { // has another entry
+        q_.adjustTop();
+      } else {
+        q_.pop(); // done with this file
+        if (ms.reader_ == primaryReader_) {
+          primaryClosed_ = true;
+          primaryLastPos_ = primaryReader_.getPos();
+        }
+        ms.reader_.close();
+      }
+      return true;
+    }
+
+    public long getPos() throws IOException {
+      if (primaryClosed_) {
+        return primaryLastPos_;
+      } else {
+        return primaryReader_.getPos();
+      }
+    }
+
+    public void close() throws IOException {
+      IOException firstErr = null;
+
+      for (int i = 0; i < readers_.size(); i++) {
+        RecordReader r = (RecordReader) readers_.get(i);
+        try {
+          r.close();
+        } catch (IOException io) {
+          io.printStackTrace();
+          if (firstErr == null) {
+            firstErr = io;
+          }
+        }
+      }
+      if (firstErr != null) {
+        throw firstErr;
+      }
+    }
+
+    public WritableComparable createKey() {
+      return new Text();
+    }
+
+    public Writable createValue() {
+      return new Text();
+    }
+
+    // 2. utilities
+
+    final static int NOTAG = -1;
+
+    private void assignTaggedWritable(Writable dst, Writable src, int tag) {
+      try {
+        outBuf.reset();
+        if (tag != NOTAG) {
+          if (src instanceof UTF8) {
+            src = new UTF8(">" + tag + "\t" + src.toString()); // breaks anything?
+          } else if (src instanceof Text) {
+            src = new Text(">" + tag + "\t" + src.toString()); // breaks anything?
+          } else {
+            throw new UnsupportedOperationException("Cannot use with tags with key class "
+                + src.getClass());
+          }
+        }
+        src.write(outBuf);
+        inBuf.reset(outBuf.getData(), outBuf.getLength());
+        dst.readFields(inBuf); // throws..
+      } catch (IOException io) {
+        // streams are backed by buffers, but buffers can run out
+        throw new IllegalStateException(io);
+      }
+    }
+
+    private DataInputBuffer inBuf = new DataInputBuffer();
+    private DataOutputBuffer outBuf = new DataOutputBuffer();
+
+    ArrayList/*<RecordReader>*/readers_;
+
+    RecordReader primaryReader_;
+    boolean primaryClosed_;
+    long primaryLastPos_;
+
+    MergeQueue q_;
+
+  }
+
+  boolean ready_;
+  FileSystem fs_;
+  JobConf job_;
+  boolean debug_;
+
+  // we need the JobConf: the other delegated InputFormat-s 
+  // will only be created in the delegator RecordReader
+  InputFormat primary_;
+  boolean inputTagged_;
+  ArrayList/*<InputFormat>*/fmts_;
+}
+
+class MergeQueue extends PriorityQueue // <MergeRecordStream>
+{
+
+  private boolean done;
+  private boolean debug;
+
+  public void add(MergeRecordStream reader) throws IOException {
+    super.put(reader);
+  }
+
+  public MergeQueue(int size, boolean debug) throws IOException {
+    initialize(size);
+    this.debug = debug;
+  }
+
+  protected boolean lessThan(Object a, Object b) {
+    MergeRecordStream ra = (MergeRecordStream) a;
+    MergeRecordStream rb = (MergeRecordStream) b;
+    int cp = ra.k_.compareTo(rb.k_);
+    if (debug) {
+      System.err.println("MergerInputFormat:lessThan " + ra.k_ + ", " + rb.k_ + " cp=" + cp);
+    }
+    if (cp == 0) {
+      return (ra.index_ < rb.index_);
+    } else {
+      return (cp < 0);
+    }
+  }
+
+  public void close() throws IOException {
+    IOException firstErr = null;
+    MergeRecordStream mr;
+    while ((mr = (MergeRecordStream) pop()) != null) {
+      try {
+        mr.reader_.close();
+      } catch (IOException io) {
+        io.printStackTrace();
+        if (firstErr == null) {
+          firstErr = io;
+        }
+      }
+    }
+    if (firstErr != null) {
+      throw firstErr;
+    }
+  }
+}
+
+class MergeRecordStream {
+
+  int index_;
+  RecordReader reader_;
+  WritableComparable k_;
+  Writable v_;
+
+  public MergeRecordStream(int index, RecordReader reader, WritableComparable k, Writable v)
+      throws IOException {
+    index_ = index;
+    reader_ = reader;
+    k_ = k;
+    v_ = v;
+  }
+
+  public boolean next() throws IOException {
+    boolean more = reader_.next(k_, v_);
+    return more;
+  }
+}

+ 50 - 61
src/contrib/streaming/src/java/org/apache/hadoop/streaming/MustangFile.java

@@ -30,71 +30,60 @@ import java.util.*;
  * From man chmod: If no user specs are given, the effect is as if `a' were given. 
  * 
  */
-public class MustangFile extends File
-{
+public class MustangFile extends File {
 
-    public MustangFile(File parent, String child)
-    {
-      super(parent, child);
-    }
+  public MustangFile(File parent, String child) {
+    super(parent, child);
+  }
 
-    public MustangFile(String pathname)
-    {
-      super(pathname);
-    }
+  public MustangFile(String pathname) {
+    super(pathname);
+  }
 
-    public MustangFile(String parent, String child) 
-    {
-      super(parent, child);
-    }
+  public MustangFile(String parent, String child) {
+    super(parent, child);
+  }
 
-    public boolean setReadable(boolean readable, boolean ownerOnly) 
-    {
-      chmod("r", readable, ownerOnly);
-      return SUCCESS;
-    }
+  public boolean setReadable(boolean readable, boolean ownerOnly) {
+    chmod("r", readable, ownerOnly);
+    return SUCCESS;
+  }
 
-    public boolean setReadable(boolean readable)
-    {
-      chmod("r", readable, false);
-      return SUCCESS;
-    }
+  public boolean setReadable(boolean readable) {
+    chmod("r", readable, false);
+    return SUCCESS;
+  }
 
-    public boolean setWritable(boolean writable, boolean ownerOnly) 
-    {
-      chmod("w", writable, ownerOnly);
-      return SUCCESS;
-    }
-    
-    public boolean setWritable(boolean writable) 
-    {
-      chmod("w", writable, false);
-      return SUCCESS;
-    }
+  public boolean setWritable(boolean writable, boolean ownerOnly) {
+    chmod("w", writable, ownerOnly);
+    return SUCCESS;
+  }
 
-    public boolean setExecutable(boolean executable, boolean ownerOnly) 
-    {
-      chmod("x", executable, ownerOnly);
-      return SUCCESS;
-    }
-    
-    public boolean setExecutable(boolean executable)
-    {
-      chmod("x", executable, false);
-      return SUCCESS;
-    }
-    
-    void chmod(String perms, boolean plus, boolean ownerOnly)
-    {
-       String[] argv = new String[3];
-       argv[0] = "/bin/chmod";
-       String spec = ownerOnly ? "u" : "ugoa";
-       spec += (plus ? "+" : "-");
-       spec += perms;
-       argv[1] = spec;
-       argv[2] = getAbsolutePath();
-       StreamUtil.exec(argv, System.err);
-    }
-    
-    final static boolean SUCCESS = true;
-}    
+  public boolean setWritable(boolean writable) {
+    chmod("w", writable, false);
+    return SUCCESS;
+  }
+
+  public boolean setExecutable(boolean executable, boolean ownerOnly) {
+    chmod("x", executable, ownerOnly);
+    return SUCCESS;
+  }
+
+  public boolean setExecutable(boolean executable) {
+    chmod("x", executable, false);
+    return SUCCESS;
+  }
+
+  void chmod(String perms, boolean plus, boolean ownerOnly) {
+    String[] argv = new String[3];
+    argv[0] = "/bin/chmod";
+    String spec = ownerOnly ? "u" : "ugoa";
+    spec += (plus ? "+" : "-");
+    spec += perms;
+    argv[1] = spec;
+    argv[2] = getAbsolutePath();
+    StreamUtil.exec(argv, System.err);
+  }
+
+  final static boolean SUCCESS = true;
+}

+ 172 - 0
src/contrib/streaming/src/java/org/apache/hadoop/streaming/MuxOutputFormat.java

@@ -0,0 +1,172 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.IOException;
+import java.util.ArrayList;
+
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.UTF8;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * A multiplexed OutputFormat. The channel choice is encoded within the key.
+ * If channels are fed at the same rate then the data can be read back in 
+ * with a TupleInputFormat. (in a different Job)
+ * @see TupleInputFormat 
+ * @author Michel Tourn
+ */
+public class MuxOutputFormat implements OutputFormat {
+
+  public RecordWriter getRecordWriter(FileSystem fs, JobConf job, String name, Progressable progr) throws IOException {
+    fs_ = fs;
+    job_ = job;
+
+    String primary = job.getOutputPath().toString();
+    CompoundDirSpec spec = CompoundDirSpec.findOutputSpecForPrimary(primary, job);
+    if (spec == null) {
+      throw new IOException("Did not find -output spec in JobConf for primary:" + primary);
+    }
+    String[] outPaths = spec.getPaths()[0];
+    int n = outPaths.length;
+    RecordWriter[] writers = new RecordWriter[n];
+    Path[] paths = new Path[n];
+    for (int i = 0; i < n; i++) {
+      OutputFormat f = new StreamOutputFormat(); // the only one supported
+      writers[i] = f.getRecordWriter(fs, job, name, progr);
+      paths[i] = new Path(outPaths[i], name); // same leaf name in different dir
+    }
+    return new MuxRecordWriter(writers, paths);
+  }
+
+  class MuxRecordWriter implements RecordWriter {
+
+    MuxRecordWriter(RecordWriter[] writers, Path[] paths) throws IOException {
+      writers_ = writers;
+      paths_ = paths;
+      numChannels_ = writers_.length;
+      out_ = new FSDataOutputStream[numChannels_];
+      for (int i = 0; i < out_.length; i++) {
+        System.err.println("MuxRecordWriter [" + i + "] create: " + paths[i]);
+        out_[i] = fs_.create(paths[i]);
+      }
+    }
+
+    final static int ONE_BASED = 1;
+    final static char CHANOUT = '>';
+    final static char CHANIN = '<';
+    final static String BADCHANOUT = "Invalid output channel spec: ";
+
+    int parseOutputChannel(String s, int max) throws IOException {
+      try {
+        if (s.charAt(s.length() - 1) != CHANOUT) {
+          throw new IOException(BADCHANOUT + s);
+        }
+        String s1 = s.substring(0, s.length() - 1);
+        int c = Integer.parseInt(s1);
+        if (c < 1 || c > max) {
+          String msg = "Output channel '" + s + "': must be an integer between 1 and " + max
+              + " followed by '" + CHANOUT + "' and TAB";
+          throw new IndexOutOfBoundsException(msg);
+        }
+        return c;
+      } catch (Exception e) {
+        throw new IOException(BADCHANOUT + s + " cause:" + e);
+      }
+    }
+
+    // TODO after Text patch, share code with StreamLineRecordReader.next()
+    void splitFirstTab(String input, UTF8 first, UTF8 second) {
+      int tab = input.indexOf('\t');
+      if (tab == -1) {
+        ((UTF8) first).set(input);
+        ((UTF8) second).set("");
+      } else {
+        ((UTF8) first).set(input.substring(0, tab));
+        ((UTF8) second).set(input);
+      }
+
+    }
+
+    void writeKeyTabVal(Writable key, Writable val, FSDataOutputStream out) throws IOException {
+      out.write(key.toString().getBytes("UTF-8"));
+      out.writeByte('\t');
+      out.write(val.toString().getBytes("UTF-8"));
+      out.writeByte('\n');
+    }
+
+    public void write(WritableComparable key, Writable value) throws IOException {
+      // convention: Application code must put a channel spec in first column
+      // iff there is more than one (output) channel
+      if (numChannels_ == 1) {
+        writeKeyTabVal(key, value, out_[0]);
+      } else {
+        // StreamInputFormat does not know about channels 
+        // Now reinterpret key as channel and split value as new key-value
+        // A more general mechanism would still require Reader classes to know about channels. 
+        // (and encode it as part of key or value)
+        int channel = parseOutputChannel(key.toString(), numChannels_);
+        FSDataOutputStream oi = out_[channel - ONE_BASED];
+        splitFirstTab(value.toString(), key2, val2);
+        writeKeyTabVal(key2, val2, oi);
+      }
+    }
+
+    public void close(Reporter reporter) throws IOException {
+      IOException firstErr = null;
+
+      for (int i = 0; i < writers_.length; i++) {
+        FSDataOutputStream oi = out_[i];
+        RecordWriter r = writers_[i];
+        try {
+          oi.close();
+          r.close(reporter);
+        } catch (IOException io) {
+          System.err.println("paths_[" + i + "]: " + paths_[i]);
+          io.printStackTrace();
+          if (firstErr == null) {
+            firstErr = io;
+          }
+        }
+      }
+      if (firstErr != null) {
+        throw firstErr;
+      }
+    }
+
+    UTF8 key2 = new UTF8();
+    UTF8 val2 = new UTF8();
+
+    RecordWriter[] writers_;
+    Path[] paths_;
+    int numChannels_;
+    FSDataOutputStream[] out_;
+  }
+
+  public void checkOutputSpecs(FileSystem fs, JobConf job) throws IOException {
+    // allow existing data (for app-level restartability)
+  }
+
+  FileSystem fs_;
+  JobConf job_;
+}

+ 2 - 4
src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeCombiner.java

@@ -40,11 +40,9 @@ import org.apache.hadoop.io.WritableComparable;
  *  such a multipass aggregation.
  *  @author Michel Tourn
  */
-public class PipeCombiner extends PipeReducer
-{
+public class PipeCombiner extends PipeReducer {
 
-  String getPipeCommand(JobConf job)
-  {
+  String getPipeCommand(JobConf job) {
     return job.get("stream.combine.streamprocessor");
   }
 

+ 382 - 287
src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapRed.java

@@ -17,6 +17,9 @@
 package org.apache.hadoop.streaming;
 
 import java.io.*;
+import java.net.Socket;
+import java.net.URI;
+import java.nio.channels.*;
 import java.nio.charset.CharacterCodingException;
 import java.io.IOException;
 import java.util.Date;
@@ -29,6 +32,7 @@ import java.util.regex.*;
 
 import org.apache.commons.logging.*;
 
+import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.OutputCollector;
@@ -38,6 +42,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Writable;
 
+import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -53,17 +58,19 @@ public abstract class PipeMapRed {
    * Mapper/Reducer operations will delegate to it
    */
   abstract String getPipeCommand(JobConf job);
+
   /*
-  */
+   */
   abstract String getKeyColPropName();
 
   /** Write output as side-effect files rather than as map outputs.
-      This is useful to do "Map" tasks rather than "MapReduce" tasks. */
-  boolean getUseSideEffect()
-  {
+   This is useful to do "Map" tasks rather than "MapReduce" tasks. */
+  boolean getUseSideEffect() {
     return false;
   }
 
+  abstract boolean getDoPipe();
+
   /**
    * @returns how many TABS before the end of the key part
    * usually: 1 or "ALL"
@@ -71,29 +78,28 @@ public abstract class PipeMapRed {
    * configured via tool's argv: splitKeyVal=ALL or 1..
    * although it is interpreted here, not by tool
    */
-  int getKeyColsFromPipeCommand(String cmd)
-  {
+  int getKeyColsFromPipeCommand(String cmd) {
     String key = getKeyColPropName();
     Pattern kcPat = Pattern.compile(".*" + key + "=([^\\s]*).*");
     Matcher match = kcPat.matcher(cmd);
     String kc;
-    if(!match.matches()) {
+    if (!match.matches()) {
       kc = null;
     } else {
       kc = match.group(1);
     }
 
     int cols;
-    if(kc== null) {
+    if (kc == null) {
       // default value is 1 and the Stream applications could instead
       // add/remove the \t separator on lines to get the same effect as value 0, 1, ALL
       cols = 1;
-    } else if(kc.equals("ALL")) {
+    } else if (kc.equals("ALL")) {
       cols = ALL_COLS;
     } else {
       try {
         cols = Integer.parseInt(kc);
-      } catch(NumberFormatException nf) {
+      } catch (NumberFormatException nf) {
         cols = Integer.MAX_VALUE;
       }
     }
@@ -107,87 +113,135 @@ public abstract class PipeMapRed {
   final static int SINGLEQ = 2;
   final static int DOUBLEQ = 3;
 
-  static String[] splitArgs(String args)
-  {
+  static String[] splitArgs(String args) {
     ArrayList argList = new ArrayList();
     char[] ch = args.toCharArray();
     int clen = ch.length;
     int state = OUTSIDE;
     int argstart = 0;
-    for(int c=0; c<=clen; c++) {
-        boolean last = (c==clen);
-        int lastState = state;
-        boolean endToken = false;
-        if(!last) {
-          if(ch[c]=='\'') {
-            if(state == OUTSIDE) {
-              state = SINGLEQ;
-            } else if(state == SINGLEQ) {
-              state = OUTSIDE;
-            }
-            endToken = (state != lastState);
-          } else if(ch[c]=='"') {
-            if(state == OUTSIDE) {
-              state = DOUBLEQ;
-            } else if(state == DOUBLEQ) {
-              state = OUTSIDE;
-            }
-            endToken = (state != lastState);
-          } else if(ch[c]==' ') {
-            if(state == OUTSIDE) {
-              endToken = true;
-            }
+    for (int c = 0; c <= clen; c++) {
+      boolean last = (c == clen);
+      int lastState = state;
+      boolean endToken = false;
+      if (!last) {
+        if (ch[c] == '\'') {
+          if (state == OUTSIDE) {
+            state = SINGLEQ;
+          } else if (state == SINGLEQ) {
+            state = OUTSIDE;
           }
-        }
-        if(last || endToken) {
-          if(c == argstart) {
-            // unquoted space
-          } else {
-            String a;
-            a = args.substring(argstart, c);
-            argList.add(a);
+          endToken = (state != lastState);
+        } else if (ch[c] == '"') {
+          if (state == OUTSIDE) {
+            state = DOUBLEQ;
+          } else if (state == DOUBLEQ) {
+            state = OUTSIDE;
+          }
+          endToken = (state != lastState);
+        } else if (ch[c] == ' ') {
+          if (state == OUTSIDE) {
+            endToken = true;
           }
-          argstart = c+1;
-          lastState = state;
         }
+      }
+      if (last || endToken) {
+        if (c == argstart) {
+          // unquoted space
+        } else {
+          String a;
+          a = args.substring(argstart, c);
+          argList.add(a);
+        }
+        argstart = c + 1;
+        lastState = state;
+      }
+    }
+    return (String[]) argList.toArray(new String[0]);
+  }
+
+  OutputStream getURIOutputStream(URI uri, boolean allowSocket) throws IOException {
+    final String SOCKET = "socket";
+    if (uri.getScheme().equals(SOCKET)) {
+      if (!allowSocket) {
+        throw new IOException(SOCKET + " not allowed on outputstream " + uri);
+      }
+      final Socket sock = new Socket(uri.getHost(), uri.getPort());
+      OutputStream out = new FilterOutputStream(sock.getOutputStream()) {
+        public void close() throws IOException {
+          sock.close();
+          super.close();
+        }
+      };
+      return out;
+    } else {
+      // a FSDataOutputStreamm, localFS or HDFS.
+      // localFS file may be set up as a FIFO.
+      return sideFs_.create(new Path(uri.getSchemeSpecificPart()));
     }
-    return (String[])argList.toArray(new String[0]);
   }
 
-  public void configure(JobConf job)
-  {
+  String getSideEffectFileName() {
+    FileSplit split = StreamUtil.getCurrentSplit(job_);
+    String leaf = split.getPath().getName();
+    if (split.getStart() == 0) {
+      return leaf;
+    } else {
+      return new FileSplit(new Path(leaf), split.getStart(), split.getLength()).toString();
+    }
+  }
 
+  String makeUniqueFileSuffix() {
+    return "." + System.currentTimeMillis() + "." + job_.get("mapred.task.id");
+  }
+
+  public void configure(JobConf job) {
     try {
       String argv = getPipeCommand(job);
+
       keyCols_ = getKeyColsFromPipeCommand(argv);
 
       debug_ = (job.get("stream.debug") != null);
-      if(debug_) {
+      if (debug_) {
         System.out.println("PipeMapRed: stream.debug=true");
       }
-      
+
       joinDelay_ = job.getLong("stream.joindelay.milli", 0);
-      
+
       job_ = job;
+      fs_ = FileSystem.get(job_);
+      if (job_.getBoolean("stream.sideoutput.localfs", false)) {
+        //sideFs_ = new LocalFileSystem(job_);
+        sideFs_ = FileSystem.getNamed("local", job_);
+      } else {
+        sideFs_ = fs_;
+      }
 
-      // Currently: null is identity reduce. REDUCE_NONE is no-map-outputs.
-      doPipe_ = (argv != null) && !StreamJob.REDUCE_NONE.equals(argv);
-      if(!doPipe_) return;
+      if (debug_) {
+        System.out.println("kind   :" + this.getClass());
+        System.out.println("split  :" + StreamUtil.getCurrentSplit(job_));
+        System.out.println("fs     :" + fs_.toString());
+        System.out.println("sideFs :" + sideFs_.toString());
+      }
+
+      doPipe_ = getDoPipe();
+      if (!doPipe_) return;
 
       setStreamJobDetails(job);
       setStreamProperties();
 
+      if (debugFailEarly_) {
+        throw new RuntimeException("debugFailEarly_");
+      }
       String[] argvSplit = splitArgs(argv);
       String prog = argvSplit[0];
       String userdir = System.getProperty("user.dir");
-      if(new File(prog).isAbsolute()) {
+      if (new File(prog).isAbsolute()) {
         // we don't own it. Hope it is executable
       } else {
         new MustangFile(prog).setExecutable(true, true);
       }
 
-
-      if(job_.getInputValueClass().equals(BytesWritable.class)) {
+      if (job_.getInputValueClass().equals(BytesWritable.class)) {
         // TODO expose as separate config:
         // job or semistandard inputformat property
         optUseKey_ = false;
@@ -195,12 +249,28 @@ public abstract class PipeMapRed {
 
       optSideEffect_ = getUseSideEffect();
 
-      if(optSideEffect_) {
-        // in cluster local named: outnone/map_bw5nzv
-        String fileName = job_.get("mapred.task.id");
-        sideEffectPath_ = new Path(job_.getOutputPath(), fileName);
-        FileSystem fs = FileSystem.get(job_);
-        sideEffectOut_ = fs.create(sideEffectPath_);
+      if (optSideEffect_) {
+        // during work: use a completely unique filename to avoid HDFS namespace conflicts
+        // after work: rename to a filename that depends only on the workload (the FileSplit)
+        //   it's a friendly name and in case of reexecution it will clobber. 
+        // reexecution can be due to: other job, failed task and speculative task
+        // See StreamJob.setOutputSpec(): if reducerNone_ aka optSideEffect then: 
+        // client has renamed outputPath and saved the argv's original output path as:
+        if (useSingleSideOutputURI_) {
+          sideEffectURI_ = new URI(sideOutputURI_);
+          sideEffectPathFinal_ = null; // in-place, no renaming to final
+        } else {
+          String sideOutputPath = job_.get("stream.sideoutput.dir"); // was: job_.getOutputPath() 
+          String fileName = getSideEffectFileName(); // see HADOOP-444 for rationale
+          sideEffectPathFinal_ = new Path(sideOutputPath, fileName);
+          sideEffectURI_ = new URI(sideEffectPathFinal_ + makeUniqueFileSuffix()); // implicit dfs: 
+        }
+        // apply default scheme
+        if(sideEffectURI_.getScheme() == null) {
+          sideEffectURI_ = new URI("file", sideEffectURI_.getSchemeSpecificPart(), null);
+        }
+        boolean allowSocket = useSingleSideOutputURI_;
+        sideEffectOut_ = getURIOutputStream(sideEffectURI_, allowSocket);
       }
 
       // argvSplit[0]:
@@ -209,109 +279,131 @@ public abstract class PipeMapRed {
       // In this case, force an absolute path to make sure exec finds it.
       argvSplit[0] = new File(argvSplit[0]).getAbsolutePath();
       logprintln("PipeMapRed exec " + Arrays.asList(argvSplit));
-      logprintln("sideEffectPath_=" + sideEffectPath_);
+      logprintln("sideEffectURI_=" + sideEffectURI_);
 
-      Environment childEnv = (Environment)StreamUtil.env().clone();
+      Environment childEnv = (Environment) StreamUtil.env().clone();
       addJobConfToEnvironment(job_, childEnv);
       addEnvironment(childEnv, job_.get("stream.addenvironment"));
       sim = Runtime.getRuntime().exec(argvSplit, childEnv.toArray());
 
       /* // This way required jdk1.5
-      ProcessBuilder processBuilder = new ProcessBuilder(argvSplit);
-      Map<String, String> env = processBuilder.environment();
-      addEnvironment(env, job_.get("stream.addenvironment"));
-      sim = processBuilder.start();
-      */
+       Builder processBuilder = new ProcessBuilder(argvSplit);
+       Map<String, String> env = processBuilder.environment();
+       addEnvironment(env, job_.get("stream.addenvironment"));
+       sim = processBuilder.start();
+       */
 
       clientOut_ = new DataOutputStream(new BufferedOutputStream(sim.getOutputStream()));
-      clientIn_  = new DataInputStream(new BufferedInputStream(sim.getInputStream()));
+      clientIn_ = new DataInputStream(new BufferedInputStream(sim.getInputStream()));
       clientErr_ = new DataInputStream(new BufferedInputStream(sim.getErrorStream()));
       startTime_ = System.currentTimeMillis();
 
-    } catch(Exception e) {
-        e.printStackTrace();
-        e.printStackTrace(log_);
+    } catch (Exception e) {
+      logStackTrace(e);
     }
   }
 
-  void setStreamJobDetails(JobConf job)
-  {
+  void setStreamJobDetails(JobConf job) {
     jobLog_ = job.get("stream.jobLog_");
     String s = job.get("stream.minRecWrittenToEnableSkip_");
-    if(s != null) {
+    if (s != null) {
       minRecWrittenToEnableSkip_ = Long.parseLong(s);
       logprintln("JobConf set minRecWrittenToEnableSkip_ =" + minRecWrittenToEnableSkip_);
     }
+    taskId_ = StreamUtil.getTaskInfo(job_);
+    debugFailEarly_ = isDebugFail("early");
+    debugFailDuring_ = isDebugFail("during");
+    debugFailLate_ = isDebugFail("late");
+
+    sideOutputURI_ = job_.get("stream.sideoutput.uri");
+    useSingleSideOutputURI_ = (sideOutputURI_ != null);
   }
 
-  void setStreamProperties()
-  {
-    taskid_ = System.getProperty("stream.taskid");
-    if(taskid_ == null) {
-      taskid_ = "noid" + System.currentTimeMillis();
+  boolean isDebugFail(String kind) {
+    String execidlist = job_.get("stream.debugfail.reexec." + kind);
+    if (execidlist == null) {
+      return false;
+    }
+    String[] e = execidlist.split(",");
+    for (int i = 0; i < e.length; i++) {
+      int ei = Integer.parseInt(e[i]);
+      if (taskId_.execid == ei) {
+        return true;
+      }
     }
+    return false;
+  }
+
+  void setStreamProperties() {
     String s = System.getProperty("stream.port");
-    if(s != null) {
+    if (s != null) {
       reportPortPlusOne_ = Integer.parseInt(s);
     }
+  }
 
+  void logStackTrace(Exception e) {
+    if (e == null) return;
+    e.printStackTrace();
+    if (log_ != null) {
+      e.printStackTrace(log_);
+    }
   }
 
-  void logprintln(String s)
-  {
-    if(log_ != null) {
+  void logprintln(String s) {
+    if (log_ != null) {
       log_.println(s);
     } else {
       LOG.info(s); // or LOG.info()
     }
   }
 
-  void logflush()
-  {
-    if(log_ != null) {
+  void logflush() {
+    if (log_ != null) {
       log_.flush();
     }
   }
 
-  void addJobConfToEnvironment(JobConf conf, Properties env)
-  {
-    logprintln("addJobConfToEnvironment: begin");
+  void addJobConfToEnvironment(JobConf conf, Properties env) {
+    if (debug_) {
+      logprintln("addJobConfToEnvironment: begin");
+    }
     Iterator it = conf.entries();
-    while(it.hasNext()) {
-        Map.Entry en = (Map.Entry)it.next();
-        String name = (String)en.getKey();
-        String value = (String)en.getValue();
-        name = safeEnvVarName(name);
-        envPut(env, name, value);
-    }
-    logprintln("addJobConfToEnvironment: end");
+    while (it.hasNext()) {
+      Map.Entry en = (Map.Entry) it.next();
+      String name = (String) en.getKey();
+      //String value = (String)en.getValue(); // does not apply variable expansion
+      String value = conf.get(name); // does variable expansion 
+      name = safeEnvVarName(name);
+      envPut(env, name, value);
+    }
+    if (debug_) {
+      logprintln("addJobConfToEnvironment: end");
+    }
   }
-  
-  String safeEnvVarName(String var)
-  {
+
+  String safeEnvVarName(String var) {
     StringBuffer safe = new StringBuffer();
     int len = var.length();
-    for(int i=0; i<len; i++) {
-        char c = var.charAt(i);
-        char s;
-        if((c >= '0' && c <= '9') || (c >= 'A' && c <= 'Z') || (c >= 'a' && c <= 'z')) {
-          s = c;
-        } else {
-          s = '_';
-        }
-        safe.append(s);
+    for (int i = 0; i < len; i++) {
+      char c = var.charAt(i);
+      char s;
+      if ((c >= '0' && c <= '9') || (c >= 'A' && c <= 'Z') || (c >= 'a' && c <= 'z')) {
+        s = c;
+      } else {
+        s = '_';
+      }
+      safe.append(s);
     }
     return safe.toString();
   }
 
-  void addEnvironment(Properties env, String nameVals)
-  {
+  void addEnvironment(Properties env, String nameVals) {
     // encoding "a=b c=d" from StreamJob
-    if(nameVals == null) return;
+    if (nameVals == null) return;
     String[] nv = nameVals.split(" ");
-    for(int i=0; i<nv.length; i++) {
+    for (int i = 0; i < nv.length; i++) {
       String[] pair = nv[i].split("=", 2);
-      if(pair.length != 2) {
+      if (pair.length != 2) {
         logprintln("Skip ev entry:" + nv[i]);
       } else {
         envPut(env, pair[0], pair[1]);
@@ -319,43 +411,43 @@ public abstract class PipeMapRed {
     }
   }
 
-  void envPut(Properties env, String name, String value)
-  {
-    logprintln("Add  ev entry:" + name + "=" + value);
+  void envPut(Properties env, String name, String value) {
+    if (debug_) {
+      logprintln("Add  ev entry:" + name + "=" + value);
+    }
     env.put(name, value);
   }
-  
+
   /** .. and if successful: delete the task log */
-  void appendLogToJobLog(String status)
-  {
-    if(jobLog_ == null) {
+  void appendLogToJobLog(String status) {
+    if (jobLog_ == null) {
       return; // not using a common joblog
     }
-    StreamUtil.exec("/bin/rm " + LOGNAME, log_);
+    if (log_ != null) {
+      StreamUtil.exec("/bin/rm " + LOGNAME, log_);
+    }
     // TODO socket-based aggregator (in JobTrackerInfoServer)
   }
 
-
-  void startOutputThreads(OutputCollector output, Reporter reporter)
-  {
-      outThread_ = new MROutputThread(output, reporter);
-      outThread_.start();
-      errThread_ = new MRErrorThread(reporter);
-      errThread_.start();
+  void startOutputThreads(OutputCollector output, Reporter reporter) {
+    outThread_ = new MROutputThread(output, reporter);
+    outThread_.start();
+    errThread_ = new MRErrorThread(reporter);
+    errThread_.start();
   }
-  
+
   void waitOutputThreads() {
-      try {
-          sim.waitFor();
-          if(outThread_ != null) {
-              outThread_.join(joinDelay_);
-          }
-          if(errThread_ != null) {
-              errThread_.join(joinDelay_);
-          }
-      } catch(InterruptedException e) {
-          //ignore
+    try {
+      sim.waitFor();
+      if (outThread_ != null) {
+        outThread_.join(joinDelay_);
       }
+      if (errThread_ != null) {
+        errThread_.join(joinDelay_);
+      }
+    } catch (InterruptedException e) {
+      //ignore
+    }
   }
 
   /**
@@ -365,124 +457,137 @@ public abstract class PipeMapRed {
    * @param val: value of a record
    * @throws IOException
    */
-  void splitKeyVal(byte [] line, Text key, Text val) throws IOException
-  {
-    int pos=-1;
-    if(keyCols_ != ALL_COLS) {
-        pos = UTF8ByteArrayUtils.findTab(line);
+  void splitKeyVal(byte[] line, Text key, Text val) throws IOException {
+    int pos = -1;
+    if (keyCols_ != ALL_COLS) {
+      pos = UTF8ByteArrayUtils.findTab(line);
     }
     try {
-        if(pos == -1) {
-            key.set(line);
-            val.set("");
-        } else {
-            UTF8ByteArrayUtils.splitKeyVal(line, key, val, pos);
-        }
+      if (pos == -1) {
+        key.set(line);
+        val.set("");
+      } else {
+        UTF8ByteArrayUtils.splitKeyVal(line, key, val, pos);
+      }
     } catch (CharacterCodingException e) {
-        LOG.warn(e);
-        StringUtils.stringifyException(e);
+      LOG.warn(StringUtils.stringifyException(e));
     }
   }
-  
-  class MROutputThread extends Thread
-  {
-    MROutputThread(OutputCollector output, Reporter reporter)
-    {
+
+  class MROutputThread extends Thread {
+
+    MROutputThread(OutputCollector output, Reporter reporter) {
       setDaemon(true);
       this.output = output;
       this.reporter = reporter;
     }
+
     public void run() {
-        try {
-            Text key = new Text();
-            Text val = new Text();
-            // 3/4 Tool to Hadoop
-            while((answer=UTF8ByteArrayUtils.readLine((InputStream)clientIn_))!= null) {
-                // 4/4 Hadoop out
-                if(optSideEffect_) {
-                    sideEffectOut_.write(answer);
-                    sideEffectOut_.write('\n');
-                } else {
-                    splitKeyVal(answer, key, val);
-                    output.collect(key, val);
-                }
-                numRecWritten_++;
-                if(numRecWritten_ % 100 == 0) {
-                    logprintln(numRecRead_+"/"+numRecWritten_);
-                    logflush();
-                }
-            }
-        } catch(IOException io) {
-            io.printStackTrace(log_);
+      try {
+        Text key = new Text();
+        Text val = new Text();
+        // 3/4 Tool to Hadoop
+        while ((answer = UTF8ByteArrayUtils.readLine((InputStream) clientIn_)) != null) {
+          // 4/4 Hadoop out
+          if (optSideEffect_) {
+            sideEffectOut_.write(answer);
+            sideEffectOut_.write('\n');
+          } else {
+            splitKeyVal(answer, key, val);
+            output.collect(key, val);
+          }
+          numRecWritten_++;
+          if (numRecWritten_ % 100 == 0) {
+            logprintln(numRecRead_ + "/" + numRecWritten_);
+            logflush();
+          }
         }
-        logprintln("MROutputThread done");
+      } catch (IOException io) {
+        io.printStackTrace(log_);
+      }
+      logprintln("MROutputThread done");
     }
+
     OutputCollector output;
     Reporter reporter;
-    byte [] answer;
+    byte[] answer;
   }
 
-  class MRErrorThread extends Thread
-  {
-    public MRErrorThread(Reporter reporter)
-    {
+  class MRErrorThread extends Thread {
+
+    public MRErrorThread(Reporter reporter) {
       this.reporter = reporter;
       setDaemon(true);
     }
-    public void run()
-    {
-      byte [] line;
+
+    public void run() {
+      byte[] line;
       try {
         long num = 0;
-        while((line=UTF8ByteArrayUtils.readLine((InputStream)clientErr_)) != null) {
+        while ((line = UTF8ByteArrayUtils.readLine((InputStream) clientErr_)) != null) {
           num++;
-          String lineStr = new String(line, "UTF-8"); 
+          String lineStr = new String(line, "UTF-8");
           logprintln(lineStr);
-          if(num < 10) {
+          long now = System.currentTimeMillis(); 
+          if (num < 10 || (now-lastStderrReport > 10*1000)) {
             String hline = "MRErr: " + lineStr;
             System.err.println(hline);
             reporter.setStatus(hline);
+            lastStderrReport = now;
           }
         }
-      } catch(IOException io) {
-        io.printStackTrace(log_);
+      } catch (IOException io) {
+        logStackTrace(io);
       }
     }
+    long lastStderrReport = 0;
     Reporter reporter;
   }
 
-  public void mapRedFinished()
-  {
-      logprintln("mapRedFinished");
-      if(!doPipe_) return;
-
+  public void mapRedFinished() {
+    logprintln("mapRedFinished");
+    try {
+      if (!doPipe_) return;
+      try {
+        if (clientOut_ != null) {
+          clientOut_.close();
+        }
+      } catch (IOException io) {
+      }
+      waitOutputThreads();
       try {
-          try {
-              if(clientOut_ != null) {
-                  clientOut_.close();
-              }
-          } catch(IOException io) {
+        if (optSideEffect_) {
+          logprintln("closing " + sideEffectURI_);
+          if (sideEffectOut_ != null) sideEffectOut_.close();
+          logprintln("closed  " + sideEffectURI_);
+          if (sideEffectURI_.getScheme().equals("file")) {
+            logprintln("size  " + new File(sideEffectURI_).length());
           }
-          waitOutputThreads();
-          try {
-              if(optSideEffect_) {
-                  logprintln("closing " + sideEffectPath_);
-                  sideEffectOut_.close();
-                  logprintln("closed  " + sideEffectPath_);
-              }
-          } catch(IOException io) {
-              io.printStackTrace();
+          if (useSingleSideOutputURI_) {
+            // With sideEffectPath_ we wrote in-place. 
+            // Possibly a named pipe set up by user or a socket.
+          } else {
+            boolean del = sideFs_.delete(sideEffectPathFinal_);
+            logprintln("deleted  (" + del + ") " + sideEffectPathFinal_);
+            sideFs_.rename(new Path(sideEffectURI_.getSchemeSpecificPart()), sideEffectPathFinal_);
+            logprintln("renamed  " + sideEffectPathFinal_);
           }
-          sim.destroy();
-      } catch(RuntimeException e) {
-          e.printStackTrace(log_);
-          throw e;
+        }
+      } catch (IOException io) {
+        io.printStackTrace();
       }
+      if (sim != null) sim.destroy();
+    } catch (RuntimeException e) {
+      logStackTrace(e);
+      throw e;
+    }
+    if (debugFailLate_) {
+      throw new RuntimeException("debugFailLate_");
+    }
   }
 
-  void maybeLogRecord()
-  {
-    if(numRecRead_ >= nextRecReadLog_) {
+  void maybeLogRecord() {
+    if (numRecRead_ >= nextRecReadLog_) {
       String info = numRecInfo();
       logprintln(info);
       logflush();
@@ -492,8 +597,7 @@ public abstract class PipeMapRed {
     }
   }
 
-  public String getContext()
-  {
+  public String getContext() {
 
     String s = numRecInfo() + "\n";
     s += "minRecWrittenToEnableSkip_=" + minRecWrittenToEnableSkip_ + " ";
@@ -503,38 +607,38 @@ public abstract class PipeMapRed {
     s += envline("HADOOP_USER");
     //s += envline("PWD"); // =/home/crawler/hadoop/trunk
     s += "last Hadoop input: |" + mapredKey_ + "|\n";
-    s += "last tool output: |" + outThread_.answer + "|\n";
+    if (outThread_ != null) {
+      s += "last tool output: |" + outThread_.answer + "|\n";
+    }
     s += "Date: " + new Date() + "\n";
     // s += envline("HADOOP_HOME");
     // s += envline("REMOTE_HOST");
     return s;
   }
 
-  String envline(String var)
-  {
+  String envline(String var) {
     return var + "=" + StreamUtil.env().get(var) + "\n";
   }
 
-  String numRecInfo()
-  {
-    long elapsed = (System.currentTimeMillis() - startTime_)/1000;
-    long total = numRecRead_+numRecWritten_+numRecSkipped_;
-    return "R/W/S=" + numRecRead_+"/"+numRecWritten_+"/"+numRecSkipped_
-     + " in:"  + safeDiv(numRecRead_, elapsed) + " [rec/s]"
-     + " out:" + safeDiv(numRecWritten_, elapsed) + " [rec/s]";
+  String numRecInfo() {
+    long elapsed = (System.currentTimeMillis() - startTime_) / 1000;
+    long total = numRecRead_ + numRecWritten_ + numRecSkipped_;
+    return "R/W/S=" + numRecRead_ + "/" + numRecWritten_ + "/" + numRecSkipped_ + " in:"
+        + safeDiv(numRecRead_, elapsed) + " [rec/s]" + " out:" + safeDiv(numRecWritten_, elapsed)
+        + " [rec/s]";
   }
-  String safeDiv(long n, long d)
-  {
-    return (d==0) ? "NA" : ""+n/d + "=" + n + "/" + d;
+
+  String safeDiv(long n, long d) {
+    return (d == 0) ? "NA" : "" + n / d + "=" + n + "/" + d;
   }
-  String logFailure(Exception e)
-  {
-      StringWriter sw = new StringWriter();
-      PrintWriter pw = new PrintWriter(sw);
-      e.printStackTrace(pw);
-      String msg = "log:" + jobLog_ + "\n" + getContext() + sw + "\n";
-      logprintln(msg);
-      return msg;
+
+  String logFailure(Exception e) {
+    StringWriter sw = new StringWriter();
+    PrintWriter pw = new PrintWriter(sw);
+    e.printStackTrace(pw);
+    String msg = "log:" + jobLog_ + "\n" + getContext() + sw + "\n";
+    logprintln(msg);
+    return msg;
   }
 
   /**
@@ -543,24 +647,24 @@ public abstract class PipeMapRed {
    * @throws IOException
    */
   void write(Writable value) throws IOException {
-      byte[] bval;
-      int valSize;
-      if(value instanceof BytesWritable) {
-          BytesWritable val = (BytesWritable)value;
-          bval = val.get();
-          valSize = val.getSize();
-      } else if(value instanceof Text){
-          Text val = (Text)value;
-          bval = val.getBytes();
-          valSize = val.getLength();
-      } else  {
-          String sval = value.toString();
-          bval = sval.getBytes("UTF-8");
-          valSize = bval.length;
-      }
-      clientOut_.write(bval, 0, valSize);
+    byte[] bval;
+    int valSize;
+    if (value instanceof BytesWritable) {
+      BytesWritable val = (BytesWritable) value;
+      bval = val.get();
+      valSize = val.getSize();
+    } else if (value instanceof Text) {
+      Text val = (Text) value;
+      bval = val.getBytes();
+      valSize = val.getLength();
+    } else {
+      String sval = value.toString();
+      bval = sval.getBytes("UTF-8");
+      valSize = bval.length;
+    }
+    clientOut_.write(bval, 0, valSize);
   }
-  
+
   long startTime_;
   long numRecRead_ = 0;
   long numRecWritten_ = 0;
@@ -574,52 +678,43 @@ public abstract class PipeMapRed {
 
   long joinDelay_;
   JobConf job_;
+  FileSystem fs_;
+  FileSystem sideFs_;
 
   // generic MapRed parameters passed on by hadoopStreaming
-  String taskid_;
   int reportPortPlusOne_;
 
   boolean doPipe_;
   boolean debug_;
+  boolean debugFailEarly_;
+  boolean debugFailDuring_;
+  boolean debugFailLate_;
 
   Process sim;
   MROutputThread outThread_;
   String jobLog_;
   MRErrorThread errThread_;
   DataOutputStream clientOut_;
-  DataInputStream  clientErr_;
-  DataInputStream   clientIn_;
+  DataInputStream clientErr_;
+  DataInputStream clientIn_;
 
   // set in PipeMapper/PipeReducer subclasses
   String mapredKey_;
   int numExceptions_;
+  StreamUtil.TaskId taskId_;
 
   boolean optUseKey_ = true;
 
-  boolean optSideEffect_;
-  Path sideEffectPath_;
-  FSDataOutputStream sideEffectOut_;
+  private boolean optSideEffect_;
+  private URI sideEffectURI_;
+  private Path sideEffectPathFinal_;
+
+  private boolean useSingleSideOutputURI_;
+  private String sideOutputURI_;
+
+  private OutputStream sideEffectOut_;
 
   String LOGNAME;
   PrintStream log_;
 
-  /* curr. going to stderr so that it is preserved
-  { // instance initializer
-    try {
-      int id = (int)((System.currentTimeMillis()/2000) % 10);
-      String sid = id+ "." + StreamUtil.env().get("USER");
-      LOGNAME = "/tmp/PipeMapRed." + sid + ".log";
-      log_ = new PrintStream(new FileOutputStream(LOGNAME));
-      logprintln(new java.util.Date());
-      logflush();
-    } catch(IOException io) {
-      System.err.println("LOGNAME=" + LOGNAME);
-      io.printStackTrace();
-    } finally {
-      if(log_ == null) {
-        log_ = System.err;
-      }
-    }
-  }
-  */
 }

+ 29 - 35
src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapper.java

@@ -30,60 +30,55 @@ import org.apache.hadoop.io.Writable;
  *  It delegates operations to an external program via stdin and stdout.
  *  @author Michel Tourn
  */
-public class PipeMapper extends PipeMapRed implements Mapper
-{
+public class PipeMapper extends PipeMapRed implements Mapper {
 
-  String getPipeCommand(JobConf job)
-  {
+  String getPipeCommand(JobConf job) {
     return job.get("stream.map.streamprocessor");
   }
 
-  String getKeyColPropName()
-  {
+  String getKeyColPropName() {
     return "mapKeyCols";
-  }  
+  }
 
-  boolean getUseSideEffect()
-  {
-    String reduce = job_.get("stream.reduce.streamprocessor");
-    if(StreamJob.REDUCE_NONE.equals(reduce)) {
-      return true;  
-    }
-    return false;
+  boolean getUseSideEffect() {
+    return StreamUtil.getUseMapSideEffect(job_);
+  }
+
+  boolean getDoPipe() {
+    return true;
   }
-  
 
   // Do NOT declare default constructor
   // (MapRed creates it reflectively)
 
-  public void map(WritableComparable key, Writable value,
-                  OutputCollector output, Reporter reporter)
-    throws IOException
-  {
+  public void map(WritableComparable key, Writable value, OutputCollector output, Reporter reporter) throws IOException {
     // init
-    if(outThread_ == null) {
+    if (outThread_ == null) {
       startOutputThreads(output, reporter);
     }
     try {
       // 1/4 Hadoop in
       numRecRead_++;
       maybeLogRecord();
+      if (debugFailDuring_ && numRecRead_ == 3) {
+        throw new IOException("debugFailDuring_");
+      }
 
       // 2/4 Hadoop to Tool
-      if(numExceptions_==0) {
-          if(optUseKey_) {
-              write(key);
-              clientOut_.write('\t');
-          }
-          write(value);
-          clientOut_.write('\n');
-          clientOut_.flush();
+      if (numExceptions_ == 0) {
+        if (optUseKey_) {
+          write(key);
+          clientOut_.write('\t');
+        }
+        write(value);
+        clientOut_.write('\n');
+        clientOut_.flush();
       } else {
-          numRecSkipped_++;
+        numRecSkipped_++;
       }
-    } catch(IOException io) {
+    } catch (IOException io) {
       numExceptions_++;
-      if(numExceptions_ > 1 || numRecWritten_ < minRecWrittenToEnableSkip_) {
+      if (numExceptions_ > 1 || numRecWritten_ < minRecWrittenToEnableSkip_) {
         // terminate with failure
         String msg = logFailure(io);
         appendLogToJobLog("failure");
@@ -95,11 +90,10 @@ public class PipeMapper extends PipeMapRed implements Mapper
       }
     }
   }
-  
-  public void close()
-  {
+
+  public void close() {
     appendLogToJobLog("success");
     mapRedFinished();
   }
-  
+
 }

+ 19 - 18
src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeReducer.java

@@ -31,33 +31,35 @@ import org.apache.hadoop.io.Writable;
  *  It delegates operations to an external program via stdin and stdout.
  *  @author Michel Tourn
  */
-public class PipeReducer extends PipeMapRed implements Reducer
-{
+public class PipeReducer extends PipeMapRed implements Reducer {
 
-  String getPipeCommand(JobConf job)
-  {
+  String getPipeCommand(JobConf job) {
     return job.get("stream.reduce.streamprocessor");
   }
 
-  String getKeyColPropName()
-  {
+  boolean getDoPipe() {
+    String argv = getPipeCommand(job_);
+    // Currently: null is identity reduce. REDUCE_NONE is no-map-outputs.
+    return (argv != null) && !StreamJob.REDUCE_NONE.equals(argv);
+  }
+
+  String getKeyColPropName() {
     return "reduceKeyCols";
-  }  
-  
-  public void reduce(WritableComparable key, Iterator values,
-                     OutputCollector output, Reporter reporter)
-    throws IOException {
+  }
+
+  public void reduce(WritableComparable key, Iterator values, OutputCollector output,
+      Reporter reporter) throws IOException {
 
     // init
-    if(doPipe_ && outThread_ == null) {
+    if (doPipe_ && outThread_ == null) {
       startOutputThreads(output, reporter);
     }
     try {
       while (values.hasNext()) {
-        Writable val = (Writable)values.next();
+        Writable val = (Writable) values.next();
         numRecRead_++;
         maybeLogRecord();
-        if(doPipe_) {
+        if (doPipe_) {
           write(key);
           clientOut_.write('\t');
           write(val);
@@ -68,15 +70,14 @@ public class PipeReducer extends PipeMapRed implements Reducer
           output.collect(key, val);
         }
       }
-    } catch(IOException io) {
+    } catch (IOException io) {
       appendLogToJobLog("failure");
       mapRedFinished();
-      throw new IOException(getContext() + io.getMessage());    
+      throw new IOException(getContext() + io.getMessage());
     }
   }
 
-  public void close()
-  {
+  public void close() {
     appendLogToJobLog("success");
     mapRedFinished();
   }

+ 41 - 54
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamBaseRecordReader.java

@@ -30,7 +30,6 @@ import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.commons.logging.*;
 
-
 /** 
  * Shared functionality for hadoopStreaming formats.
  * A custom reader can be defined to be a RecordReader with the constructor below
@@ -39,18 +38,15 @@ import org.apache.commons.logging.*;
  * @see StreamXmlRecordReader 
  * @author Michel Tourn
  */
-public abstract class StreamBaseRecordReader implements RecordReader
-{
-    
+public abstract class StreamBaseRecordReader implements RecordReader {
+
   protected static final Log LOG = LogFactory.getLog(StreamBaseRecordReader.class.getName());
-  
+
   // custom JobConf properties for this class are prefixed with this namespace
   final static String CONF_NS = "stream.recordreader.";
 
-  public StreamBaseRecordReader(
-    FSDataInputStream in, FileSplit split, Reporter reporter, JobConf job, FileSystem fs)
-    throws IOException
-  {
+  public StreamBaseRecordReader(FSDataInputStream in, FileSplit split, Reporter reporter,
+      JobConf job, FileSystem fs) throws IOException {
     in_ = in;
     split_ = split;
     start_ = split_.getStart();
@@ -60,99 +56,90 @@ public abstract class StreamBaseRecordReader implements RecordReader
     reporter_ = reporter;
     job_ = job;
     fs_ = fs;
-    
+
     statusMaxRecordChars_ = job_.getInt(CONF_NS + "statuschars", 200);
   }
 
   /// RecordReader API
-  
+
   /** Read a record. Implementation should call numRecStats at the end
-   */  
+   */
   public abstract boolean next(Writable key, Writable value) throws IOException;
 
   /** This implementation always returns true. */
-  public boolean[] areValidInputDirectories(FileSystem fileSys,
-                                     Path[] inputDirs) throws IOException
-  {
+  public boolean[] areValidInputDirectories(FileSystem fileSys, Path[] inputDirs) throws IOException {
     int n = inputDirs.length;
     boolean[] b = new boolean[n];
-    for(int i=0; i<n; i++) {
+    for (int i = 0; i < n; i++) {
       b[i] = true;
     }
     return b;
   }
 
   /** Returns the current position in the input. */
-  public synchronized long getPos() throws IOException 
-  { 
-    return in_.getPos(); 
+  public synchronized long getPos() throws IOException {
+    return in_.getPos();
   }
 
   /** Close this to future operations.*/
-  public synchronized void close() throws IOException 
-  { 
-    in_.close(); 
+  public synchronized void close() throws IOException {
+    in_.close();
   }
 
   public WritableComparable createKey() {
     return new Text();
   }
-  
+
   public Writable createValue() {
     return new Text();
   }
-  
+
   /// StreamBaseRecordReader API
 
-  public void init() throws IOException
-  {
-    LOG.info("StreamBaseRecordReader.init: " +
-    " start_=" + start_ + " end_=" + end_ + " length_=" + length_ +
-    " start_ > in_.getPos() =" 
-        + (start_ > in_.getPos()) + " " + start_ 
-        + " > " + in_.getPos() );
+  public void init() throws IOException {
+    LOG.info("StreamBaseRecordReader.init: " + " start_=" + start_ + " end_=" + end_ + " length_="
+        + length_ + " start_ > in_.getPos() =" + (start_ > in_.getPos()) + " " + start_ + " > "
+        + in_.getPos());
     if (start_ > in_.getPos()) {
       in_.seek(start_);
-    }  
+    }
     seekNextRecordBoundary();
   }
-  
+
   /** Implementation should seek forward in_ to the first byte of the next record.
    *  The initial byte offset in the stream is arbitrary.
    */
   public abstract void seekNextRecordBoundary() throws IOException;
-  
-    
-  void numRecStats(byte[] record, int start, int len) throws IOException
-  {
-    numRec_++;          
-    if(numRec_ == nextStatusRec_) {
-      String recordStr = new String(record, start, 
-                Math.min(len, statusMaxRecordChars_), "UTF-8");    
-      nextStatusRec_ +=100;//*= 10;
+
+  void numRecStats(byte[] record, int start, int len) throws IOException {
+    numRec_++;
+    if (numRec_ == nextStatusRec_) {
+      String recordStr = new String(record, start, Math.min(len, statusMaxRecordChars_), "UTF-8");
+      nextStatusRec_ += 100;//*= 10;
       String status = getStatus(recordStr);
       LOG.info(status);
       reporter_.setStatus(status);
     }
   }
 
- long lastMem =0;
- String getStatus(CharSequence record)
- {
+  long lastMem = 0;
+
+  String getStatus(CharSequence record) {
     long pos = -1;
-    try { 
+    try {
       pos = getPos();
-    } catch(IOException io) {
+    } catch (IOException io) {
     }
     String recStr;
-    if(record.length() > statusMaxRecordChars_) {
-        recStr = record.subSequence(0, statusMaxRecordChars_) + "...";
+    if (record.length() > statusMaxRecordChars_) {
+      recStr = record.subSequence(0, statusMaxRecordChars_) + "...";
     } else {
-    	recStr = record.toString();
+      recStr = record.toString();
     }
-    String unqualSplit = split_.getFile().getName() + ":" + split_.getStart() + "+" + split_.getLength();
-    String status = "HSTR " + StreamUtil.HOST + " " + numRec_ + ". pos=" + pos
-     + " " + unqualSplit + " Processing record=" + recStr;
+    String unqualSplit = split_.getFile().getName() + ":" + split_.getStart() + "+"
+        + split_.getLength();
+    String status = "HSTR " + StreamUtil.HOST + " " + numRec_ + ". pos=" + pos + " " + unqualSplit
+        + " Processing record=" + recStr;
     status += " " + splitName_;
     return status;
   }
@@ -169,5 +156,5 @@ public abstract class StreamBaseRecordReader implements RecordReader
   int numRec_ = 0;
   int nextStatusRec_ = 1;
   int statusMaxRecordChars_;
-  
+
 }

+ 38 - 57
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamInputFormat.java

@@ -39,49 +39,38 @@ import org.apache.hadoop.mapred.*;
  * selects a RecordReader based on a JobConf property.
  * @author Michel Tourn
  */
-public class StreamInputFormat extends InputFormatBase
-{
+public class StreamInputFormat extends InputFormatBase {
 
   // an InputFormat should be public with the synthetic public default constructor
   // JobTracker's JobInProgress will instantiate with clazz.newInstance() (and a custom ClassLoader)
 
   protected static final Log LOG = LogFactory.getLog(StreamInputFormat.class.getName());
 
-  static {
-    //LOG.setLevel(Level.FINE);
-  }
-
   /** This implementation always returns true. */
-  public boolean[] areValidInputDirectories(FileSystem fileSys,
-                                            Path[] inputDirs
-                                            ) throws IOException {
+  public boolean[] areValidInputDirectories(FileSystem fileSys, Path[] inputDirs) throws IOException {
     boolean[] b = new boolean[inputDirs.length];
-    for(int i=0; i < inputDirs.length; ++i) {
+    for (int i = 0; i < inputDirs.length; ++i) {
       b[i] = true;
     }
     return b;
   }
 
-  static boolean isGzippedInput(JobConf job)
-  {
+  static boolean isGzippedInput(JobConf job) {
     String val = job.get(StreamBaseRecordReader.CONF_NS + "compression");
     return "gzip".equals(val);
   }
 
-  public FileSplit[] getSplits(FileSystem fs, JobConf job, int numSplits)
-    throws IOException {
-      
-    if(isGzippedInput(job)) {
+  public FileSplit[] getSplits(FileSystem fs, JobConf job, int numSplits) throws IOException {
+
+    if (isGzippedInput(job)) {
       return getFullFileSplits(fs, job);
     } else {
       return super.getSplits(fs, job, numSplits);
-    }   
+    }
   }
-  
+
   /** For the compressed-files case: override InputFormatBase to produce one split. */
-  FileSplit[] getFullFileSplits(FileSystem fs, JobConf job)
-    throws IOException
-  {
+  FileSplit[] getFullFileSplits(FileSystem fs, JobConf job) throws IOException {
     Path[] files = listPaths(fs, job);
     int numSplits = files.length;
     ArrayList splits = new ArrayList(numSplits);
@@ -90,37 +79,35 @@ public class StreamInputFormat extends InputFormatBase
       long splitSize = fs.getLength(file);
       splits.add(new FileSplit(file, 0, splitSize));
     }
-    return (FileSplit[])splits.toArray(new FileSplit[splits.size()]);
+    return (FileSplit[]) splits.toArray(new FileSplit[splits.size()]);
   }
 
-  protected Path[] listPaths(FileSystem fs, JobConf job)
-    throws IOException
-  {
+  protected Path[] listPaths(FileSystem fs, JobConf job) throws IOException {
     Path[] globs = job.getInputPaths();
     ArrayList list = new ArrayList();
     int dsup = globs.length;
-    for(int d=0; d<dsup; d++) {
+    for (int d = 0; d < dsup; d++) {
       String leafName = globs[d].getName();
       LOG.info("StreamInputFormat: globs[" + d + "] leafName = " + leafName);
-      Path[] paths; Path dir;
+      Path[] paths;
+      Path dir;
       PathFilter filter = new GlobFilter(fs, leafName);
       dir = new Path(globs[d].getParent().toString());
-      if(dir == null) dir = new Path(".");
+      if (dir == null) dir = new Path(".");
       paths = fs.listPaths(dir, filter);
       list.addAll(Arrays.asList(paths));
     }
-    return (Path[])list.toArray(new Path[]{});
+    return (Path[]) list.toArray(new Path[] {});
   }
 
-  class GlobFilter implements PathFilter
-  {
-    public GlobFilter(FileSystem fs, String glob)
-    {
+  class GlobFilter implements PathFilter {
+
+    public GlobFilter(FileSystem fs, String glob) {
       fs_ = fs;
       pat_ = Pattern.compile(globToRegexp(glob));
     }
-    String globToRegexp(String glob)
-    {
+
+    String globToRegexp(String glob) {
       String re = glob;
       re = re.replaceAll("\\.", "\\\\.");
       re = re.replaceAll("\\+", "\\\\+");
@@ -130,11 +117,10 @@ public class StreamInputFormat extends InputFormatBase
       return re;
     }
 
-    public boolean accept(Path pathname)
-    {
+    public boolean accept(Path pathname) {
       boolean acc = !fs_.isChecksumFile(pathname);
-      if(acc) {
-          acc = pat_.matcher(pathname.getName()).matches();
+      if (acc) {
+        acc = pat_.matcher(pathname.getName()).matches();
       }
       LOG.info("matches " + pat_ + ", " + pathname + " = " + acc);
       return acc;
@@ -144,10 +130,9 @@ public class StreamInputFormat extends InputFormatBase
     FileSystem fs_;
   }
 
-  public RecordReader getRecordReader(FileSystem fs, final FileSplit split,
-                                      JobConf job, Reporter reporter)
-    throws IOException {
-    LOG.info("getRecordReader start.....");
+  public RecordReader getRecordReader(FileSystem fs, final FileSplit split, JobConf job,
+      Reporter reporter) throws IOException {
+    LOG.info("getRecordReader start.....split=" + split);
     reporter.setStatus(split.toString());
 
     final long start = split.getStart();
@@ -160,45 +145,41 @@ public class StreamInputFormat extends InputFormatBase
     // Factory dispatch based on available params..
     Class readerClass;
     String c = job.get("stream.recordreader.class");
-    if(c == null) {
+    if (c == null) {
       readerClass = StreamLineRecordReader.class;
     } else {
       readerClass = StreamUtil.goodClassOrNull(c, null);
-      if(readerClass == null) {
+      if (readerClass == null) {
         throw new RuntimeException("Class not found: " + c);
       }
     }
 
     Constructor ctor;
     try {
-      ctor = readerClass.getConstructor(new Class[]{
-        FSDataInputStream.class, FileSplit.class, Reporter.class, JobConf.class, FileSystem.class});
-    } catch(NoSuchMethodException nsm) {
+      ctor = readerClass.getConstructor(new Class[] { FSDataInputStream.class, FileSplit.class,
+          Reporter.class, JobConf.class, FileSystem.class });
+    } catch (NoSuchMethodException nsm) {
       throw new RuntimeException(nsm);
     }
 
-
     StreamBaseRecordReader reader;
     try {
-        reader = (StreamBaseRecordReader) ctor.newInstance(new Object[]{
-            in, split, reporter, job, fs});
-    } catch(Exception nsm) {
+      reader = (StreamBaseRecordReader) ctor.newInstance(new Object[] { in, split, reporter, job,
+          fs });
+    } catch (Exception nsm) {
       throw new RuntimeException(nsm);
     }
 
     reader.init();
 
-
-    if(reader instanceof StreamSequenceRecordReader) {
+    if (reader instanceof StreamSequenceRecordReader) {
       // override k/v class types with types stored in SequenceFile
-      StreamSequenceRecordReader ss = (StreamSequenceRecordReader)reader;
+      StreamSequenceRecordReader ss = (StreamSequenceRecordReader) reader;
       job.setInputKeyClass(ss.rin_.getKeyClass());
       job.setInputValueClass(ss.rin_.getValueClass());
     }
 
-
     return reader;
   }
 
-  
 }

+ 370 - 206
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java

@@ -18,11 +18,14 @@ package org.apache.hadoop.streaming;
 
 import java.io.File;
 import java.io.IOException;
-import java.net.URL;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Iterator;
 import java.util.Map;
+import java.util.TreeMap;
+import java.util.TreeSet;
 
 import org.apache.commons.logging.*;
 
@@ -39,20 +42,19 @@ import org.apache.hadoop.mapred.RunningJob;
  * (Jar packaging, MapRed job submission and monitoring)
  * @author Michel Tourn
  */
-public class StreamJob
-{
-  protected static final Log LOG = LogFactory.getLog(StreamJob.class.getName());
+public class StreamJob {
 
+  protected static final Log LOG = LogFactory.getLog(StreamJob.class.getName());
   final static String REDUCE_NONE = "NONE";
 
-  public StreamJob(String[] argv, boolean mayExit)
-  {
+  private boolean reducerNone_;
+
+  public StreamJob(String[] argv, boolean mayExit) {
     argv_ = argv;
     mayExit_ = mayExit;
   }
 
-  public void go() throws IOException
-  {
+  public void go() throws IOException {
     init();
 
     preProcessArgs();
@@ -63,40 +65,37 @@ public class StreamJob
     submitAndMonitorJob();
   }
 
-  protected void init()
-  {
-     try {
-        env_ = new Environment();
-     } catch(IOException io) {
-        throw new RuntimeException(io);
-     }
+  protected void init() {
+    try {
+      env_ = new Environment();
+    } catch (IOException io) {
+      throw new RuntimeException(io);
+    }
   }
 
-  void preProcessArgs()
-  {
+  void preProcessArgs() {
     verbose_ = false;
     addTaskEnvironment_ = "";
   }
 
-  void postProcessArgs() throws IOException
-  {
-    if(cluster_ == null) {
-        // hadoop-default.xml is standard, hadoop-local.xml is not.
-        cluster_ = "default";
+  void postProcessArgs() throws IOException {
+    if (cluster_ == null) {
+      // hadoop-default.xml is standard, hadoop-local.xml is not.
+      cluster_ = "default";
     }
     hadoopAliasConf_ = "hadoop-" + getClusterNick() + ".xml";
-    if(inputGlobs_.size() == 0) {
-        fail("Required argument: -input <name>");
+    if (inputSpecs_.size() == 0) {
+      fail("Required argument: -input <name>");
     }
-    if(output_ == null) {
-        fail("Required argument: -output ");
+    if (output_ == null) {
+      fail("Required argument: -output ");
     }
     msg("addTaskEnvironment=" + addTaskEnvironment_);
 
     Iterator it = packageFiles_.iterator();
-    while(it.hasNext()) {
-      File f = new File((String)it.next());
-      if(f.isFile()) {
+    while (it.hasNext()) {
+      File f = new File((String) it.next());
+      if (f.isFile()) {
         shippedCanonFiles_.add(f.getCanonicalPath());
       }
     }
@@ -108,37 +107,40 @@ public class StreamJob
     redCmd_ = unqualifyIfLocalPath(redCmd_);
   }
 
-  void validateNameEqValue(String neqv)
-  {
+  void validateNameEqValue(String neqv) {
     String[] nv = neqv.split("=", 2);
-    if(nv.length < 2) {
-        fail("Invalid name=value spec: " + neqv);
+    if (nv.length < 2) {
+      fail("Invalid name=value spec: " + neqv);
     }
     msg("Recording name=value: name=" + nv[0] + " value=" + nv[1]);
   }
 
-  String unqualifyIfLocalPath(String cmd) throws IOException
-  {
-    if(cmd == null) {
+  String unqualifyIfLocalPath(String cmd) throws IOException {
+    if (cmd == null) {
       //
     } else {
       String prog = cmd;
       String args = "";
       int s = cmd.indexOf(" ");
-      if(s != -1) {
+      if (s != -1) {
         prog = cmd.substring(0, s);
-        args = cmd.substring(s+1);
+        args = cmd.substring(s + 1);
+      }
+      String progCanon;
+      try {
+        progCanon = new File(prog).getCanonicalPath();
+      } catch (IOException io) {
+        progCanon = prog;
       }
-      String progCanon = new File(prog).getCanonicalPath();
       boolean shipped = shippedCanonFiles_.contains(progCanon);
       msg("shipped: " + shipped + " " + progCanon);
-      if(shipped) {
+      if (shipped) {
         // Change path to simple filename.
         // That way when PipeMapRed calls Runtime.exec(),
         // it will look for the excutable in Task's working dir.
         // And this is where TaskRunner unjars our job jar.
         prog = new File(prog).getName();
-        if(args.length() > 0) {
+        if (args.length() > 0) {
           cmd = prog + " " + args;
         } else {
           cmd = prog;
@@ -149,68 +151,70 @@ public class StreamJob
     return cmd;
   }
 
-  String getHadoopAliasConfFile()
-  {
+  String getHadoopAliasConfFile() {
     return new File(getHadoopClientHome() + "/conf", hadoopAliasConf_).getAbsolutePath();
   }
 
-
-  void parseArgv()
-  {
-    if(argv_.length==0) {
+  void parseArgv() {
+    if (argv_.length == 0) {
       exitUsage(false);
     }
-    int i=0;
-    while(i < argv_.length) {
+    int i = 0;
+    while (i < argv_.length) {
       String s;
-      if(argv_[i].equals("-verbose")) {
+      if (argv_[i].equals("-verbose")) {
         verbose_ = true;
-      } else if(argv_[i].equals("-info")) {
+      } else if (argv_[i].equals("-info")) {
         detailedUsage_ = true;
-      } else if(argv_[i].equals("-debug")) {
+      } else if (argv_[i].equals("-debug")) {
         debug_++;
-      } else if((s = optionArg(argv_, i, "-input", false)) != null) {
+      } else if ((s = optionArg(argv_, i, "-input", false)) != null) {
         i++;
-        inputGlobs_.add(s);
-      } else if((s = optionArg(argv_, i, "-output", output_ != null)) != null) {
+        inputSpecs_.add(s);
+      } else if (argv_[i].equals("-inputtagged")) {
+        inputTagged_ = true;
+      } else if ((s = optionArg(argv_, i, "-output", output_ != null)) != null) {
         i++;
         output_ = s;
-      } else if((s = optionArg(argv_, i, "-mapper", mapCmd_ != null)) != null) {
+      } else if ((s = optionArg(argv_, i, "-mapsideoutput", mapsideoutURI_ != null)) != null) {
+        i++;
+        mapsideoutURI_ = s;
+      } else if ((s = optionArg(argv_, i, "-mapper", mapCmd_ != null)) != null) {
         i++;
         mapCmd_ = s;
-      } else if((s = optionArg(argv_, i, "-combiner", comCmd_ != null)) != null) {
+      } else if ((s = optionArg(argv_, i, "-combiner", comCmd_ != null)) != null) {
         i++;
         comCmd_ = s;
-      } else if((s = optionArg(argv_, i, "-reducer", redCmd_ != null)) != null) {
+      } else if ((s = optionArg(argv_, i, "-reducer", redCmd_ != null)) != null) {
         i++;
         redCmd_ = s;
-      } else if((s = optionArg(argv_, i, "-file", false)) != null) {
+      } else if ((s = optionArg(argv_, i, "-file", false)) != null) {
         i++;
         packageFiles_.add(s);
-      } else if((s = optionArg(argv_, i, "-cluster", cluster_ != null)) != null) {
+      } else if ((s = optionArg(argv_, i, "-cluster", cluster_ != null)) != null) {
         i++;
         cluster_ = s;
-      } else if((s = optionArg(argv_, i, "-config", false)) != null) {
+      } else if ((s = optionArg(argv_, i, "-config", false)) != null) {
         i++;
         configPath_.add(s);
-      } else if((s = optionArg(argv_, i, "-dfs", false)) != null) {
+      } else if ((s = optionArg(argv_, i, "-dfs", false)) != null) {
         i++;
-        userJobConfProps_.add("fs.default.name="+s);
-      } else if((s = optionArg(argv_, i, "-jt", false)) != null) {
+        userJobConfProps_.add("fs.default.name=" + s);
+      } else if ((s = optionArg(argv_, i, "-jt", false)) != null) {
         i++;
-        userJobConfProps_.add("mapred.job.tracker="+s);
-      } else if((s = optionArg(argv_, i, "-jobconf", false)) != null) {
+        userJobConfProps_.add("mapred.job.tracker=" + s);
+      } else if ((s = optionArg(argv_, i, "-jobconf", false)) != null) {
         i++;
         validateNameEqValue(s);
         userJobConfProps_.add(s);
-      } else if((s = optionArg(argv_, i, "-cmdenv", false)) != null) {
+      } else if ((s = optionArg(argv_, i, "-cmdenv", false)) != null) {
         i++;
         validateNameEqValue(s);
-        if(addTaskEnvironment_.length() > 0) {
-            addTaskEnvironment_ += " ";
+        if (addTaskEnvironment_.length() > 0) {
+          addTaskEnvironment_ += " ";
         }
         addTaskEnvironment_ += s;
-      } else if((s = optionArg(argv_, i, "-inputreader", inReaderSpec_ != null)) != null) {
+      } else if ((s = optionArg(argv_, i, "-inputreader", inReaderSpec_ != null)) != null) {
         i++;
         inReaderSpec_ = s;
       } else {
@@ -219,37 +223,35 @@ public class StreamJob
       }
       i++;
     }
-    if(detailedUsage_) {
-        exitUsage(true);
+    if (detailedUsage_) {
+      exitUsage(true);
     }
   }
 
-  String optionArg(String[] args, int index, String arg, boolean argSet)
-  {
-    if(index >= args.length || ! args[index].equals(arg)) {
+  String optionArg(String[] args, int index, String arg, boolean argSet) {
+    if (index >= args.length || !args[index].equals(arg)) {
       return null;
     }
-    if(argSet) {
+    if (argSet) {
       throw new IllegalArgumentException("Can only have one " + arg + " option");
     }
-    if(index >= args.length-1) {
+    if (index >= args.length - 1) {
       throw new IllegalArgumentException("Expected argument after option " + args[index]);
     }
-    return args[index+1];
+    return args[index + 1];
   }
 
-  protected void msg(String msg)
-  {
-    if(verbose_) {
+  protected void msg(String msg) {
+    if (verbose_) {
       System.out.println("STREAM: " + msg);
     }
   }
 
-  public void exitUsage(boolean detailed)
-  {
-                      //         1         2         3         4         5         6         7
-                      //1234567890123456789012345678901234567890123456789012345678901234567890123456789
-    System.out.println("Usage: $HADOOP_HOME/bin/hadoop jar build/hadoop-streaming.jar [options]");
+  public void exitUsage(boolean detailed) {
+    //         1         2         3         4         5         6         7
+    //1234567890123456789012345678901234567890123456789012345678901234567890123456789
+    System.out.println("Usage: $HADOOP_HOME/bin/hadoop [--config dir] jar \\");
+    System.out.println("          $HADOOP_HOME/hadoop-streaming.jar [options]");
     System.out.println("Options:");
     System.out.println("  -input    <path>     DFS input file(s) for the Map step");
     System.out.println("  -output   <path>     DFS output directory for the Reduce step");
@@ -257,58 +259,82 @@ public class StreamJob
     System.out.println("  -combiner <cmd>      The streaming command to run");
     System.out.println("  -reducer  <cmd>      The streaming command to run");
     System.out.println("  -file     <file>     File/dir to be shipped in the Job jar file");
-    System.out.println("  -cluster  <name>     Default uses hadoop-default.xml and hadoop-site.xml");
-    System.out.println("  -config   <file>     Optional. One or more paths to xml config files");
-    System.out.println("  -dfs      <h:p>      Optional. Override DFS configuration");
-    System.out.println("  -jt       <h:p>      Optional. Override JobTracker configuration");
+    //Only advertise the standard way: [--config dir] in our launcher 
+    //System.out.println("  -cluster  <name>     Default uses hadoop-default.xml and hadoop-site.xml");
+    //System.out.println("  -config   <file>     Optional. One or more paths to xml config files");
+    System.out.println("  -dfs    <h:p>|local  Optional. Override DFS configuration");
+    System.out.println("  -jt     <h:p>|local  Optional. Override JobTracker configuration");
     System.out.println("  -inputreader <spec>  Optional.");
-    System.out.println("  -jobconf  <n>=<v>    Optional.");
+    System.out.println("  -jobconf  <n>=<v>    Optional. Add or override a JobConf property");
     System.out.println("  -cmdenv   <n>=<v>    Optional. Pass env.var to streaming commands");
     System.out.println("  -verbose");
     System.out.println();
-    if(!detailed) {
-    System.out.println("For more details about these options:");
-    System.out.println("Use $HADOOP_HOME/bin/hadoop jar build/hadoop-streaming.jar -info");
-        fail("");
+    if (!detailed) {
+      System.out.println("For more details about these options:");
+      System.out.println("Use $HADOOP_HOME/bin/hadoop jar build/hadoop-streaming.jar -info");
+      fail("");
     }
     System.out.println("In -input: globbing on <path> is supported and can have multiple -input");
     System.out.println("Default Map input format: a line is a record in UTF-8");
     System.out.println("  the key part ends at first TAB, the rest of the line is the value");
     System.out.println("Custom Map input format: -inputreader package.MyRecordReader,n=v,n=v ");
-    System.out.println("  comma-separated name-values can be specified to configure the InputFormat");
+    System.out
+        .println("  comma-separated name-values can be specified to configure the InputFormat");
     System.out.println("  Ex: -inputreader 'StreamXmlRecordReader,begin=<doc>,end=</doc>'");
     System.out.println("Map output format, reduce input/output format:");
-    System.out.println("  Format defined by what mapper command outputs. Line-oriented");
+    System.out.println("  Format defined by what the mapper command outputs. Line-oriented");
     System.out.println();
-    System.out.println("Use -cluster <name> to switch between \"local\" Hadoop and one or more remote ");
-    System.out.println("  Hadoop clusters. ");
-    System.out.println("  The default is to use the normal hadoop-default.xml and hadoop-site.xml");
-    System.out.println("  Else configuration will use $HADOOP_HOME/conf/hadoop-<name>.xml");
+    System.out.println("The files or directories named in the -file argument[s] end up in the");
+    System.out.println("  working directory when the mapper and reducer are run.");
+    System.out.println("  The location of this working directory is unspecified.");
     System.out.println();
-    System.out.println("To skip the shuffle/sort/reduce step:" );
+    //System.out.println("Use -cluster <name> to switch between \"local\" Hadoop and one or more remote ");
+    //System.out.println("  Hadoop clusters. ");
+    //System.out.println("  The default is to use the normal hadoop-default.xml and hadoop-site.xml");
+    //System.out.println("  Else configuration will use $HADOOP_HOME/conf/hadoop-<name>.xml");
+    //System.out.println();
+    System.out.println("To skip the sort/combine/shuffle/sort/reduce step:");
     System.out.println("  Use -reducer " + REDUCE_NONE);
-    System.out.println("  This preserves the map input order and speeds up processing");
+    System.out
+        .println("  A Task's Map output then becomes a 'side-effect output' rather than a reduce input");
+    System.out
+        .println("  This speeds up processing, This also feels more like \"in-place\" processing");
+    System.out.println("  because the input filename and the map input order are preserved");
+    System.out.println("To specify a single side-effect output file");
+    System.out.println("    -mapsideoutput [file:/C:/win|file:/unix/|socket://host:port]");//-output for side-effects will be soon deprecated
+    System.out.println("  If the jobtracker is local this is a local file");
+    System.out.println("  This currently requires -reducer NONE");
     System.out.println();
     System.out.println("To set the number of reduce tasks (num. of output files):");
     System.out.println("  -jobconf mapred.reduce.tasks=10");
-    System.out.println("To name the job (appears in the JobTrack Web UI):");
+    System.out.println("To speed up the last reduces:");
+    System.out.println("  -jobconf mapred.speculative.execution=true");
+    System.out.println("  Do not use this along -reducer " + REDUCE_NONE);
+    System.out.println("To name the job (appears in the JobTracker Web UI):");
     System.out.println("  -jobconf mapred.job.name='My Job' ");
     System.out.println("To specify that line-oriented input is in gzip format:");
-    System.out.println("(at this time ALL input files must be gzipped and this is not recognized based on file extension)");
+    System.out
+        .println("(at this time ALL input files must be gzipped and this is not recognized based on file extension)");
     System.out.println("   -jobconf stream.recordreader.compression=gzip ");
     System.out.println("To change the local temp directory:");
-    System.out.println("  -jobconf dfs.data.dir=/tmp");
+    System.out.println("  -jobconf dfs.data.dir=/tmp/dfs");
+    System.out.println("  -jobconf stream.tmpdir=/tmp/streaming");
     System.out.println("Additional local temp directories with -cluster local:");
     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("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
+        .println("    [...] -jobconf stream.shipped.hadoopstreaming=/path/my-hadoop-streaming.jar");
     System.out.println("For more details about jobconf parameters see:");
     System.out.println("  http://wiki.apache.org/lucene-hadoop/JobConfFile");
     System.out.println("To set an environement variable in a streaming command:");
     System.out.println("   -cmdenv EXAMPLE_DIR=/home/example/dictionaries/");
     System.out.println();
-    System.out.println("Shortcut to run from any directory:");
-    System.out.println("   setenv HSTREAMING \"$HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/build/hadoop-streaming.jar\"");
+    System.out.println("Shortcut:");
+    System.out
+        .println("   setenv HSTREAMING \"$HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/hadoop-streaming.jar\"");
     System.out.println();
     System.out.println("Example: $HSTREAMING -mapper \"/usr/local/bin/perl5 filter.pl\"");
     System.out.println("           -file /local/filter.pl -input \"/logs/0604*/*\" [...]");
@@ -318,81 +344,87 @@ public class StreamJob
     fail("");
   }
 
-  public void fail(String message)
-  {
-    if(mayExit_) {
-        System.err.println(message);
-        System.exit(1);
+  public void fail(String message) {
+    if (mayExit_) {
+      System.err.println(message);
+      System.exit(1);
     } else {
-       throw new IllegalArgumentException(message);
+      throw new IllegalArgumentException(message);
     }
   }
 
   // --------------------------------------------
 
-
-  protected String getHadoopClientHome()
-  {
+  protected String getHadoopClientHome() {
     String h = env_.getProperty("HADOOP_HOME"); // standard Hadoop
-    if(h == null) {
+    if (h == null) {
       //fail("Missing required environment variable: HADOOP_HOME");
       h = "UNDEF";
     }
     return h;
   }
 
-
-  protected boolean isLocalHadoop()
-  {
+  protected boolean isLocalHadoop() {
     boolean local;
-    if(jobConf_ == null) {
-        local = getClusterNick().equals("local");
+    if (jobConf_ == null) {
+      local = getClusterNick().equals("local");
     } else {
-        local = jobConf_.get("mapred.job.tracker", "").equals("local");
+      local = StreamUtil.isLocalJobTracker(jobConf_);
     }
     return local;
   }
-  protected String getClusterNick()
-  {
+
+  protected String getClusterNick() {
     return cluster_;
   }
 
   /** @return path to the created Jar file or null if no files are necessary.
-  */
-  protected String packageJobJar() throws IOException
-  {
+   */
+  protected String packageJobJar() throws IOException {
     ArrayList unjarFiles = new ArrayList();
 
     // Runtime code: ship same version of code as self (job submitter code)
     // usually found in: build/contrib or build/hadoop-<version>-dev-streaming.jar
-    String runtimeClasses = StreamUtil.findInClasspath(StreamJob.class.getName());
-    if(runtimeClasses == null) {
-        throw new IOException("runtime classes not found: " + getClass().getPackage());
+
+    // First try an explicit spec: it's too hard to find our own location in this case:
+    // $HADOOP_HOME/bin/hadoop jar /not/first/on/classpath/custom-hadoop-streaming.jar
+    // where findInClasspath() would find the version of hadoop-streaming.jar in $HADOOP_HOME
+    String runtimeClasses = jobConf_.get("stream.shipped.hadoopstreaming"); // jar or class dir
+
+    if (runtimeClasses == null) {
+      runtimeClasses = StreamUtil.findInClasspath(StreamJob.class.getName());
+    }
+    if (runtimeClasses == null) {
+      throw new IOException("runtime classes not found: " + getClass().getPackage());
     } else {
-        msg("Found runtime classes in: " + runtimeClasses);
+      msg("Found runtime classes in: " + runtimeClasses);
     }
-    if(isLocalHadoop()) {
+    if (isLocalHadoop()) {
       // don't package class files (they might get unpackaged in "." and then
       //  hide the intended CLASSPATH entry)
       // we still package everything else (so that scripts and executable are found in
       //  Task workdir like distributed Hadoop)
     } else {
-      if(new File(runtimeClasses).isDirectory()) {
-          packageFiles_.add(runtimeClasses);
+      if (new File(runtimeClasses).isDirectory()) {
+        packageFiles_.add(runtimeClasses);
       } else {
-          unjarFiles.add(runtimeClasses);
+        unjarFiles.add(runtimeClasses);
       }
     }
-    if(packageFiles_.size() + unjarFiles.size()==0) {
+    if (packageFiles_.size() + unjarFiles.size() == 0) {
       return null;
     }
-    File jobJar = File.createTempFile("streamjob", ".jar");
-    System.out.println("packageJobJar: " + packageFiles_ + " " + unjarFiles + " " + jobJar);
-    if(debug_ == 0) {
+    String tmp = jobConf_.get("stream.tmpdir"); //, "/tmp/${user.name}/"
+    File tmpDir = (tmp == null) ? null : new File(tmp);
+    // tmpDir=null means OS default tmp dir
+    File jobJar = File.createTempFile("streamjob", ".jar", tmpDir);
+    System.out.println("packageJobJar: " + packageFiles_ + " " + unjarFiles + " " + jobJar
+        + " tmpDir=" + tmpDir);
+    if (debug_ == 0) {
       jobJar.deleteOnExit();
     }
     JarBuilder builder = new JarBuilder();
-    if(verbose_) {
+    if (verbose_) {
       builder.setVerbose(true);
     }
     String jobJarName = jobJar.getAbsolutePath();
@@ -400,53 +432,81 @@ public class StreamJob
     return jobJarName;
   }
 
-  protected void setJobConf() throws IOException
-  {
+  protected void setUserJobConfProps(boolean doEarlyProps) {
+    Iterator it = userJobConfProps_.iterator();
+    while (it.hasNext()) {
+      String prop = (String) it.next();
+      String[] nv = prop.split("=", 2);
+      if (doEarlyProps == nv[0].equals("fs.default.name")) {
+        msg("xxxJobConf: set(" + nv[0] + ", " + nv[1] + ") early=" + doEarlyProps);
+        jobConf_.set(nv[0], nv[1]);
+      }
+    }
+  }
+
+  protected void setJobConf() throws IOException {
     msg("hadoopAliasConf_ = " + hadoopAliasConf_);
     config_ = new Configuration();
-    if(!cluster_.equals("default")) {
-        config_.addFinalResource(new Path(getHadoopAliasConfFile()));
+    if (!cluster_.equals("default")) {
+      config_.addFinalResource(new Path(getHadoopAliasConfFile()));
     } else {
       // use only defaults: hadoop-default.xml and hadoop-site.xml
     }
     Iterator it = configPath_.iterator();
-    while(it.hasNext()) {
-        String pathName = (String)it.next();
-        config_.addFinalResource(new Path(pathName));
+    while (it.hasNext()) {
+      String pathName = (String) it.next();
+      config_.addFinalResource(new Path(pathName));
     }
+
+    testMerge_ = (-1 != userJobConfProps_.toString().indexOf("stream.testmerge"));
+
     // general MapRed job properties
     jobConf_ = new JobConf(config_);
-    for(int i=0; i<inputGlobs_.size(); i++) {
-      jobConf_.addInputPath(new Path((String)inputGlobs_.get(i)));
+
+    setUserJobConfProps(true);
+
+    // The correct FS must be set before this is called!
+    // (to resolve local vs. dfs drive letter differences) 
+    // (mapred.working.dir will be lazily initialized ONCE and depends on FS)
+    for (int i = 0; i < inputSpecs_.size(); i++) {
+      addInputSpec((String) inputSpecs_.get(i), i);
+    }
+    jobConf_.setBoolean("stream.inputtagged", inputTagged_);
+    jobConf_.set("stream.numinputspecs", "" + inputSpecs_.size());
+
+    Class fmt;
+    if (testMerge_ && false == hasSimpleInputSpecs_) {
+      // this ignores -inputreader
+      fmt = MergerInputFormat.class;
+    } else {
+      // need to keep this case to support custom -inputreader 
+      // and their parameters ,n=v,n=v
+      fmt = StreamInputFormat.class;
     }
+    jobConf_.setInputFormat(fmt);
 
-    jobConf_.setInputFormat(StreamInputFormat.class);
     // for SequenceFile, input classes may be overriden in getRecordReader
     jobConf_.setInputKeyClass(Text.class);
     jobConf_.setInputValueClass(Text.class);
 
     jobConf_.setOutputKeyClass(Text.class);
     jobConf_.setOutputValueClass(Text.class);
-    //jobConf_.setCombinerClass();
-
-    jobConf_.setOutputPath(new Path(output_));
-    jobConf_.setOutputFormat(StreamOutputFormat.class);
 
     jobConf_.set("stream.addenvironment", addTaskEnvironment_);
 
     String defaultPackage = this.getClass().getPackage().getName();
 
     Class c = StreamUtil.goodClassOrNull(mapCmd_, defaultPackage);
-    if(c != null) {
+    if (c != null) {
       jobConf_.setMapperClass(c);
     } else {
       jobConf_.setMapperClass(PipeMapper.class);
       jobConf_.set("stream.map.streamprocessor", mapCmd_);
     }
 
-    if(comCmd_ != null) {
+    if (comCmd_ != null) {
       c = StreamUtil.goodClassOrNull(comCmd_, defaultPackage);
-      if(c != null) {
+      if (c != null) {
         jobConf_.setCombinerClass(c);
       } else {
         jobConf_.setCombinerClass(PipeCombiner.class);
@@ -454,9 +514,11 @@ public class StreamJob
       }
     }
 
-    if(redCmd_ != null) {
+    reducerNone_ = false;
+    if (redCmd_ != null) {
+      reducerNone_ = redCmd_.equals(REDUCE_NONE);
       c = StreamUtil.goodClassOrNull(redCmd_, defaultPackage);
-      if(c != null) {
+      if (c != null) {
         jobConf_.setReducerClass(c);
       } else {
         jobConf_.setReducerClass(PipeReducer.class);
@@ -464,66 +526,165 @@ public class StreamJob
       }
     }
 
-    if(inReaderSpec_ != null) {
-        String[] args = inReaderSpec_.split(",");
-        String readerClass = args[0];
-        // this argument can only be a Java class
-        c = StreamUtil.goodClassOrNull(readerClass, defaultPackage);
-        if(c != null) {
-            jobConf_.set("stream.recordreader.class", c.getName());
-        } else {
-            fail("-inputreader: class not found: " + readerClass);
-        }
-        for(int i=1; i<args.length; i++) {
-            String[] nv = args[i].split("=", 2);
-            String k = "stream.recordreader." + nv[0];
-            String v = (nv.length>1) ? nv[1] : "";
-            jobConf_.set(k, v);
-        }
+    if (inReaderSpec_ != null) {
+      String[] args = inReaderSpec_.split(",");
+      String readerClass = args[0];
+      // this argument can only be a Java class
+      c = StreamUtil.goodClassOrNull(readerClass, defaultPackage);
+      if (c != null) {
+        jobConf_.set("stream.recordreader.class", c.getName());
+      } else {
+        fail("-inputreader: class not found: " + readerClass);
+      }
+      for (int i = 1; i < args.length; i++) {
+        String[] nv = args[i].split("=", 2);
+        String k = "stream.recordreader." + nv[0];
+        String v = (nv.length > 1) ? nv[1] : "";
+        jobConf_.set(k, v);
+      }
     }
 
-    jar_ = packageJobJar();
-    if(jar_ != null) {
-        jobConf_.setJar(jar_);
+    // output setup is done late so we can customize for reducerNone_
+    //jobConf_.setOutputDir(new File(output_));
+    setOutputSpec();
+    if (testMerge_) {
+      fmt = MuxOutputFormat.class;
+    } else {
+      fmt = StreamOutputFormat.class;
     }
+    jobConf_.setOutputFormat(fmt);
 
     // last, allow user to override anything
     // (although typically used with properties we didn't touch)
-    it = userJobConfProps_.iterator();
-    while(it.hasNext()) {
-        String prop = (String)it.next();
-        String[] nv = prop.split("=", 2);
-        msg("xxxJobConf: set(" + nv[0] + ", " + nv[1]+")");
-        jobConf_.set(nv[0], nv[1]);
+    setUserJobConfProps(false);
+
+    jar_ = packageJobJar();
+    if (jar_ != null) {
+      jobConf_.setJar(jar_);
     }
+
+    if(verbose_) {
+      listJobConfProperties();
+    }
+    
     msg("submitting to jobconf: " + getJobTrackerHostPort());
   }
 
-  protected String getJobTrackerHostPort()
+  protected void listJobConfProperties()
   {
+    msg("==== JobConf properties:");
+    Iterator it = jobConf_.entries();
+    TreeMap sorted = new TreeMap();
+    while(it.hasNext()) {
+      Map.Entry en = (Map.Entry)it.next();
+      sorted.put(en.getKey(), en.getValue());
+    }
+    it = sorted.entrySet().iterator();
+    while(it.hasNext()) {
+      Map.Entry en = (Map.Entry)it.next();
+      msg(en.getKey() + "=" + en.getValue());
+    }
+    msg("====");
+  }
+  
+  /** InputSpec-s encode: a glob pattern x additional column files x additional joins */
+  protected void addInputSpec(String inSpec, int index) {
+    if (!testMerge_) {
+      jobConf_.addInputPath(new Path(inSpec));
+    } else {
+      CompoundDirSpec spec = new CompoundDirSpec(inSpec, true);
+      msg("Parsed -input:\n" + spec.toTableString());
+      if (index == 0) {
+        hasSimpleInputSpecs_ = (spec.paths_.length == 0);
+        msg("hasSimpleInputSpecs_=" + hasSimpleInputSpecs_);
+      }
+      String primary = spec.primarySpec();
+      if (!seenPrimary_.add(primary)) {
+        // this won't detect glob overlaps and noncanonical path variations
+        fail("Primary used in multiple -input spec: " + primary);
+      }
+      jobConf_.addInputPath(new Path(primary));
+      // during Job execution, will reparse into a CompoundDirSpec 
+      jobConf_.set("stream.inputspecs." + index, inSpec);
+    }
+  }
+
+  /** uses output_ and mapsideoutURI_ */
+  protected void setOutputSpec() throws IOException {
+    CompoundDirSpec spec = new CompoundDirSpec(output_, false);
+    msg("Parsed -output:\n" + spec.toTableString());
+    String primary = spec.primarySpec();
+    String channel0;
+    // TODO simplify cases, encapsulate in a StreamJobConf
+    if (!reducerNone_) {
+      channel0 = primary;
+    } else {
+      if (mapsideoutURI_ != null) {
+        // user can override in case this is in a difft filesystem..
+        try {
+          URI uri = new URI(mapsideoutURI_);
+          if (uri.getScheme() == null || uri.getScheme().equals("file")) { // || uri.getScheme().equals("hdfs")
+            if (!new Path(uri.getSchemeSpecificPart()).isAbsolute()) {
+              fail("Must be absolute: " + mapsideoutURI_);
+            }
+          } else if (uri.getScheme().equals("socket")) {
+            // ok
+          } else {
+            fail("Invalid scheme: " + uri.getScheme() + " for -mapsideoutput " + mapsideoutURI_);
+          }
+        } catch (URISyntaxException e) {
+          throw (IOException) new IOException().initCause(e);
+        }
+      } else {
+        mapsideoutURI_ = primary;
+      }
+      // an empty reduce output named "part-00002" will go here and not collide.
+      channel0 = primary + ".NONE";
+      // the side-effect of the first split of an input named "part-00002" 
+      // will go in this directory
+      jobConf_.set("stream.sideoutput.dir", primary);
+      // oops if user overrides low-level this isn't set yet :-(
+      boolean localjt = StreamUtil.isLocalJobTracker(jobConf_);
+      // just a guess user may prefer remote..
+      jobConf_.setBoolean("stream.sideoutput.localfs", localjt);
+    }
+    // a path in fs.name.default filesystem
+    System.out.println(channel0);
+    System.out.println(new Path(channel0));
+    jobConf_.setOutputPath(new Path(channel0));
+    // will reparse remotely
+    jobConf_.set("stream.outputspec", output_);
+    if (null != mapsideoutURI_) {
+      // a path in "jobtracker's filesystem"
+      // overrides sideoutput.dir
+      jobConf_.set("stream.sideoutput.uri", mapsideoutURI_);
+    }
+  }
+
+  protected String getJobTrackerHostPort() {
     return jobConf_.get("mapred.job.tracker");
   }
 
-  protected void jobInfo()
-  {
-    if(isLocalHadoop()) {
+  protected void jobInfo() {
+    if (isLocalHadoop()) {
       LOG.info("Job running in-process (local Hadoop)");
     } else {
       String hp = getJobTrackerHostPort();
       LOG.info("To kill this job, run:");
-      LOG.info(getHadoopClientHome() + "/bin/hadoop job  -Dmapred.job.tracker=" + hp + " -kill " + jobId_);
+      LOG.info(getHadoopClientHome() + "/bin/hadoop job  -Dmapred.job.tracker=" + hp + " -kill "
+          + jobId_);
       //LOG.info("Job file: " + running_.getJobFile() );
-      LOG.info("Tracking URL: "  + StreamUtil.qualifyHost(running_.getTrackingURL()));
+      LOG.info("Tracking URL: " + StreamUtil.qualifyHost(running_.getTrackingURL()));
     }
   }
 
   // Based on JobClient
   public void submitAndMonitorJob() throws IOException {
 
-    if(jar_ != null && isLocalHadoop()) {
-        // getAbs became required when shell and subvm have different working dirs...
-        File wd = new File(".").getAbsoluteFile();
-        StreamUtil.unJar(new File(jar_), wd);
+    if (jar_ != null && isLocalHadoop()) {
+      // getAbs became required when shell and subvm have different working dirs...
+      File wd = new File(".").getAbsoluteFile();
+      StreamUtil.unJar(new File(jar_), wd);
     }
 
     // if jobConf_ changes must recreate a JobClient
@@ -542,11 +703,12 @@ public class StreamJob
       while (!running_.isComplete()) {
         try {
           Thread.sleep(1000);
-        } catch (InterruptedException e) {}
+        } catch (InterruptedException e) {
+        }
         running_ = jc_.getJob(jobId_);
         String report = null;
-        report = " map "+Math.round(running_.mapProgress()*100)
-        +"%  reduce " + Math.round(running_.reduceProgress()*100)+"%";
+        report = " map " + Math.round(running_.mapProgress() * 100) + "%  reduce "
+            + Math.round(running_.reduceProgress() * 100) + "%";
 
         if (!report.equals(lastReport)) {
           LOG.info(report);
@@ -569,7 +731,6 @@ public class StreamJob
     }
   }
 
-
   protected boolean mayExit_;
   protected String[] argv_;
   protected boolean verbose_;
@@ -585,11 +746,15 @@ public class StreamJob
   protected JobClient jc_;
 
   // command-line arguments
-  protected ArrayList inputGlobs_       = new ArrayList(); // <String>
-  protected ArrayList packageFiles_     = new ArrayList(); // <String>
-  protected ArrayList shippedCanonFiles_= new ArrayList(); // <String>
-  protected ArrayList userJobConfProps_ = new ArrayList(); // <String>
+  protected ArrayList inputSpecs_ = new ArrayList(); // <String>
+  protected boolean inputTagged_ = false;
+  protected TreeSet seenPrimary_ = new TreeSet(); // <String>
+  protected boolean hasSimpleInputSpecs_;
+  protected ArrayList packageFiles_ = new ArrayList(); // <String>
+  protected ArrayList shippedCanonFiles_ = new ArrayList(); // <String>
+  protected ArrayList userJobConfProps_ = new ArrayList(); // <String> name=value
   protected String output_;
+  protected String mapsideoutURI_;
   protected String mapCmd_;
   protected String comCmd_;
   protected String redCmd_;
@@ -598,6 +763,7 @@ public class StreamJob
   protected String hadoopAliasConf_;
   protected String inReaderSpec_;
 
+  protected boolean testMerge_;
 
   // Use to communicate config to the external processes (ex env.var.HADOOP_USER)
   // encoding "a=b c=d"
@@ -609,6 +775,4 @@ public class StreamJob
   protected RunningJob running_;
   protected String jobId_;
 
-
 }
-

+ 48 - 48
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamLineRecordReader.java

@@ -18,7 +18,8 @@ package org.apache.hadoop.streaming;
 
 import java.io.*;
 import java.nio.charset.MalformedInputException;
-import java.util.zip.GZIPInputStream; 
+import java.util.Arrays;
+import java.util.zip.GZIPInputStream;
 
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Text;
@@ -35,25 +36,21 @@ import org.apache.hadoop.util.StringUtils;
  * but delimits key and value with a TAB.
  * @author Michel Tourn
  */
-public class StreamLineRecordReader extends StreamBaseRecordReader 
-{
+public class StreamLineRecordReader extends StreamBaseRecordReader {
 
-  public StreamLineRecordReader(
-    FSDataInputStream in, FileSplit split, Reporter reporter, JobConf job, FileSystem fs)
-    throws IOException
-  {
+  public StreamLineRecordReader(FSDataInputStream in, FileSplit split, Reporter reporter,
+      JobConf job, FileSystem fs) throws IOException {
     super(in, split, reporter, job, fs);
     gzipped_ = StreamInputFormat.isGzippedInput(job);
-    if(gzipped_) {
+    if (gzipped_) {
       din_ = new DataInputStream(new GZIPInputStream(in_));
     } else {
       din_ = in_;
     }
   }
 
-  public void seekNextRecordBoundary() throws IOException
-  {
-    if(gzipped_) {
+  public void seekNextRecordBoundary() throws IOException {
+    if (gzipped_) {
       // no skipping: use din_ as-is 
       // assumes splitter created only one split per file
       return;
@@ -63,7 +60,7 @@ public class StreamLineRecordReader extends StreamBaseRecordReader
         in_.seek(start_ - 1);
         // scan to the next newline in the file
         while (in_.getPos() < end_) {
-          char c = (char)in_.read();
+          char c = (char) in_.read();
           bytesSkipped++;
           if (c == '\r' || c == '\n') {
             break;
@@ -75,51 +72,54 @@ public class StreamLineRecordReader extends StreamBaseRecordReader
     }
   }
 
-  public synchronized boolean next(Writable key, Writable value)
-    throws IOException {  
-    if(!(key instanceof Text)) {
-        throw new IllegalArgumentException(
-                "Key should be of type Text but: "+key.getClass().getName());
+  public synchronized boolean next(Writable key, Writable value) throws IOException {
+    if (!(key instanceof Text)) {
+      throw new IllegalArgumentException("Key should be of type Text but: "
+          + key.getClass().getName());
     }
-    if(!(value instanceof Text)) {
-        throw new IllegalArgumentException(
-                "Value should be of type Text but: "+value.getClass().getName());
+    if (!(value instanceof Text)) {
+      throw new IllegalArgumentException("Value should be of type Text but: "
+          + value.getClass().getName());
     }
 
-    Text tKey = (Text)key;
-    Text tValue = (Text)value;
-    byte [] line;
-    
+    Text tKey = (Text) key;
+    Text tValue = (Text) value;
+    byte[] line;
+
     while (true) {
-        if(gzipped_) {
-            // figure EOS from readLine
+      if (gzipped_) {
+        // figure EOS from readLine
+      } else {
+        long pos = in_.getPos();
+        if (pos >= end_) return false;
+      }
+
+      line = UTF8ByteArrayUtils.readLine((InputStream) in_);
+      try {
+        Text.validateUTF8(line);
+      } catch (MalformedInputException m) {
+        System.err.println("line=" + line + "|" + new Text(line));
+        System.out.flush();
+      }
+      if (line == null) return false;
+      try {
+        int tab = UTF8ByteArrayUtils.findTab(line);
+        if (tab == -1) {
+          tKey.set(line);
+          tValue.set("");
         } else {
-            long pos = in_.getPos();
-            if (pos >= end_)
-                return false;
-        }
-        
-        line = UTF8ByteArrayUtils.readLine((InputStream)in_);
-        if(line==null)
-            return false;
-        try {
-            int tab=UTF8ByteArrayUtils.findTab(line);
-            if(tab == -1) {
-                tKey.set(line);
-                tValue.set("");
-            } else {
-                UTF8ByteArrayUtils.splitKeyVal(line, tKey, tValue, tab);
-            }
-            break;
-        } catch (MalformedInputException e) {
-            LOG.warn(e);
-            StringUtils.stringifyException(e);
+          UTF8ByteArrayUtils.splitKeyVal(line, tKey, tValue, tab);
         }
+        break;
+      } catch (MalformedInputException e) {
+        LOG.warn(StringUtils.stringifyException(e));
+      }
     }
-    numRecStats( line, 0, line.length );
+    numRecStats(line, 0, line.length);
     return true;
   }
+
   boolean gzipped_;
   GZIPInputStream zin_;
-  DataInputStream din_; // GZIP or plain
+  DataInputStream din_; // GZIP or plain  
 }

+ 17 - 19
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamOutputFormat.java

@@ -35,28 +35,27 @@ import org.apache.hadoop.util.Progressable;
  */
 public class StreamOutputFormat implements OutputFormat {
 
-  public RecordWriter getRecordWriter(FileSystem fs, JobConf job,
-                                      String name, Progressable progr) throws IOException {
- 
+  public RecordWriter getRecordWriter(FileSystem fs, JobConf job, String name, Progressable progr) throws IOException {
+
     File file = new File(job.getOutputDir(), name);
 
     final FSDataOutputStream out = fs.create(file);
 
     return new RecordWriter() {
-        public synchronized void write(WritableComparable key, Writable value)
-          throws IOException {
-          out.write(key.toString().getBytes("UTF-8"));
-          out.writeByte('\t');
-          out.write(value.toString().getBytes("UTF-8"));
-          out.writeByte('\n');
-        }
-        public synchronized void close(Reporter reporter) throws IOException {
-          out.close();
-        }
-      };
+
+      public synchronized void write(WritableComparable key, Writable value) throws IOException {
+        out.write(key.toString().getBytes("UTF-8"));
+        out.writeByte('\t');
+        out.write(value.toString().getBytes("UTF-8"));
+        out.writeByte('\n');
+      }
+
+      public synchronized void close(Reporter reporter) throws IOException {
+        out.close();
+      }
+    };
   }
-  
-  
+
   /** Check whether the output specification for a job is appropriate.  Called
    * when a job is submitted.  Typically checks that it does not already exist,
    * throwing an exception when it already exists, so that output is not
@@ -65,9 +64,8 @@ public class StreamOutputFormat implements OutputFormat {
    * @param job the job whose output will be written
    * @throws IOException when output should not be attempted
    */
-  public void checkOutputSpecs(FileSystem fs, JobConf job) throws IOException
-  {
+  public void checkOutputSpecs(FileSystem fs, JobConf job) throws IOException {
     // allow existing data (for app-level restartability)
   }
-  
+
 }

+ 15 - 25
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamSequenceRecordReader.java

@@ -29,22 +29,16 @@ import org.apache.hadoop.mapred.JobConf;
 
 import org.apache.hadoop.util.ReflectionUtils;
 
-public class StreamSequenceRecordReader extends StreamBaseRecordReader
-{
+public class StreamSequenceRecordReader extends StreamBaseRecordReader {
 
-  public StreamSequenceRecordReader (
-    FSDataInputStream in, FileSplit split, Reporter reporter, JobConf job, FileSystem fs)
-    throws IOException
-  {
+  public StreamSequenceRecordReader(FSDataInputStream in, FileSplit split, Reporter reporter,
+      JobConf job, FileSystem fs) throws IOException {
     super(in, split, reporter, job, fs);
     numFailed_ = 0;
     // super.in_ ignored, using rin_ instead
   }
 
-
-  public synchronized boolean next(Writable key, Writable value)
-   throws IOException
-  {
+  public synchronized boolean next(Writable key, Writable value) throws IOException {
     boolean success;
     do {
       if (!more_) return false;
@@ -58,29 +52,26 @@ public class StreamSequenceRecordReader extends StreamBaseRecordReader
           more_ = eof;
         }
         success = true;
-      } catch(IOException io) {
+      } catch (IOException io) {
         numFailed_++;
-        if(numFailed_ < 100 || numFailed_ % 100 == 0) {
-          err_.println("StreamSequenceRecordReader: numFailed_/numRec_="
-            + numFailed_+ "/" + numRec_);
+        if (numFailed_ < 100 || numFailed_ % 100 == 0) {
+          err_.println("StreamSequenceRecordReader: numFailed_/numRec_=" + numFailed_ + "/"
+              + numRec_);
         }
         io.printStackTrace(err_);
         success = false;
       }
-    } while(!success);
-    
+    } while (!success);
+
     numRecStats(new byte[0], 0, 0);
     return more_;
   }
 
-
-  public void seekNextRecordBoundary() throws IOException
-  {
+  public void seekNextRecordBoundary() throws IOException {
     rin_ = new SequenceFile.Reader(fs_, split_.getPath(), job_);
     end_ = split_.getStart() + split_.getLength();
 
-    if (split_.getStart() > rin_.getPosition())
-      rin_.sync(split_.getStart());                  // sync to start
+    if (split_.getStart() > rin_.getPosition()) rin_.sync(split_.getStart()); // sync to start
 
     more_ = rin_.getPosition() < end_;
 
@@ -90,14 +81,13 @@ public class StreamSequenceRecordReader extends StreamBaseRecordReader
   }
 
   public WritableComparable createKey() {
-    return (WritableComparable) 
-           ReflectionUtils.newInstance(rin_.getKeyClass(), null);
+    return (WritableComparable) ReflectionUtils.newInstance(rin_.getKeyClass(), null);
   }
-  
+
   public Writable createValue() {
     return (Writable) ReflectionUtils.newInstance(rin_.getValueClass(), null);
   }
-  
+
   boolean more_;
   SequenceFile.Reader rin_;
   int numFailed_;

+ 316 - 139
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamUtil.java

@@ -19,69 +19,81 @@ package org.apache.hadoop.streaming;
 import java.text.DecimalFormat;
 import java.io.*;
 import java.net.*;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Enumeration;
+import java.util.Iterator;
+import java.util.List;
 import java.util.jar.*;
 
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.JobConf;
+
 /** Utilities not available elsewhere in Hadoop.
  *  
  */
-public class StreamUtil
-{
+public class StreamUtil {
 
-  public static Class goodClassOrNull(String className, String defaultPackage)
-  {
-    if(className.indexOf('.') == -1 && defaultPackage != null) {
-        className = defaultPackage + "." + className;
+  /** It may seem strange to silently switch behaviour when a String
+   * is not a classname; the reason is simplified Usage:<pre>
+   * -mapper [classname | program ]
+   * instead of the explicit Usage:
+   * [-mapper program | -javamapper classname], -mapper and -javamapper are mutually exclusive.
+   * (repeat for -reducer, -combiner) </pre>
+   */
+  public static Class goodClassOrNull(String className, String defaultPackage) {
+    if (className.indexOf('.') == -1 && defaultPackage != null) {
+      className = defaultPackage + "." + className;
     }
     Class clazz = null;
     try {
-        clazz = Class.forName(className);
-    } catch(ClassNotFoundException cnf) {
-    } catch(LinkageError cnf) {
+      clazz = Class.forName(className);
+    } catch (ClassNotFoundException cnf) {
+    } catch (LinkageError cnf) {
     }
     return clazz;
   }
-  
-   /** @return a jar file path or a base directory or null if not found.
+
+  public static String findInClasspath(String className) {
+    return findInClasspath(className, StreamUtil.class.getClassLoader());
+  }
+
+  /** @return a jar file path or a base directory or null if not found.
    */
-   public static String findInClasspath(String className) 
-   {
+  public static String findInClasspath(String className, ClassLoader loader) {
 
     String relPath = className;
-    if (!relPath.startsWith("/")) {
-      relPath = "/" + relPath;
-    }
     relPath = relPath.replace('.', '/');
     relPath += ".class";
-
-    java.net.URL classUrl = StreamUtil.class.getResource(relPath);
+    java.net.URL classUrl = loader.getResource(relPath);
 
     String codePath;
     if (classUrl != null) {
-        boolean inJar = classUrl.getProtocol().equals("jar");
-        codePath = classUrl.toString();
-        if(codePath.startsWith("jar:")) {
-            codePath = codePath.substring("jar:".length());
-        }
-        if(codePath.startsWith("file:")) { // can have both
-            codePath = codePath.substring("file:".length());
-        }
-        if(inJar) {          
-          // A jar spec: remove class suffix in /path/my.jar!/package/Class
-          int bang = codePath.lastIndexOf('!');
-          codePath = codePath.substring(0, bang);
-        } else {
-          // A class spec: remove the /my/package/Class.class portion
-          int pos = codePath.lastIndexOf(relPath);
-          if(pos == -1) {
-            throw new IllegalArgumentException(
-              "invalid codePath: className=" + className + " codePath=" + codePath);
-          }
-          codePath = codePath.substring(0, pos);
+      boolean inJar = classUrl.getProtocol().equals("jar");
+      codePath = classUrl.toString();
+      if (codePath.startsWith("jar:")) {
+        codePath = codePath.substring("jar:".length());
+      }
+      if (codePath.startsWith("file:")) { // can have both
+        codePath = codePath.substring("file:".length());
+      }
+      if (inJar) {
+        // A jar spec: remove class suffix in /path/my.jar!/package/Class
+        int bang = codePath.lastIndexOf('!');
+        codePath = codePath.substring(0, bang);
+      } else {
+        // A class spec: remove the /my/package/Class.class portion
+        int pos = codePath.lastIndexOf(relPath);
+        if (pos == -1) {
+          throw new IllegalArgumentException("invalid codePath: className=" + className
+              + " codePath=" + codePath);
         }
+        codePath = codePath.substring(0, pos);
+      }
     } else {
-        codePath = null;
+      codePath = null;
     }
     return codePath;
   }
@@ -92,7 +104,7 @@ public class StreamUtil
     try {
       Enumeration entries = jar.entries();
       while (entries.hasMoreElements()) {
-        JarEntry entry = (JarEntry)entries.nextElement();
+        JarEntry entry = (JarEntry) entries.nextElement();
         if (!entry.isDirectory()) {
           InputStream in = jar.getInputStream(entry);
           try {
@@ -117,9 +129,7 @@ public class StreamUtil
       jar.close();
     }
   }
-  
 
-  
   final static long KB = 1024L * 1;
   final static long MB = 1024L * KB;
   final static long GB = 1024L * MB;
@@ -128,64 +138,61 @@ public class StreamUtil
 
   static DecimalFormat dfm = new DecimalFormat("####.000");
   static DecimalFormat ifm = new DecimalFormat("###,###,###,###,###");
-  
-  public static String dfmt(double d)
-  {
+
+  public static String dfmt(double d) {
     return dfm.format(d);
   }
-  public static String ifmt(double d)
-  {
+
+  public static String ifmt(double d) {
     return ifm.format(d);
   }
-  
-  public static String formatBytes(long numBytes)
-  {
+
+  public static String formatBytes(long numBytes) {
     StringBuffer buf = new StringBuffer();
-    boolean bDetails = true;    
+    boolean bDetails = true;
     double num = numBytes;
-    
-    if(numBytes < KB) {
+
+    if (numBytes < KB) {
       buf.append(numBytes + " B");
       bDetails = false;
-    } else if(numBytes < MB) {
-      buf.append(dfmt(num/KB) + " KB");
-    } else if(numBytes < GB) {
-      buf.append(dfmt(num/MB) + " MB");
-    } else if(numBytes < TB) {
-      buf.append(dfmt(num/GB) + " GB");
-    } else if(numBytes < PB) {
-      buf.append(dfmt(num/TB) + " TB");
+    } else if (numBytes < MB) {
+      buf.append(dfmt(num / KB) + " KB");
+    } else if (numBytes < GB) {
+      buf.append(dfmt(num / MB) + " MB");
+    } else if (numBytes < TB) {
+      buf.append(dfmt(num / GB) + " GB");
+    } else if (numBytes < PB) {
+      buf.append(dfmt(num / TB) + " TB");
     } else {
-      buf.append(dfmt(num/PB) + " PB");
+      buf.append(dfmt(num / PB) + " PB");
     }
-    if(bDetails) {
+    if (bDetails) {
       buf.append(" (" + ifmt(numBytes) + " bytes)");
     }
     return buf.toString();
   }
 
-  public static String formatBytes2(long numBytes)
-  {
+  public static String formatBytes2(long numBytes) {
     StringBuffer buf = new StringBuffer();
     long u = 0;
-    if(numBytes >= TB) {
-      u = numBytes/TB;
-      numBytes -= u*TB;
+    if (numBytes >= TB) {
+      u = numBytes / TB;
+      numBytes -= u * TB;
       buf.append(u + " TB ");
     }
-    if(numBytes >= GB) {
-      u = numBytes/GB;
-      numBytes -= u*GB;
+    if (numBytes >= GB) {
+      u = numBytes / GB;
+      numBytes -= u * GB;
       buf.append(u + " GB ");
     }
-    if(numBytes >= MB) {
-      u = numBytes/MB;
-      numBytes -= u*MB;
+    if (numBytes >= MB) {
+      u = numBytes / MB;
+      numBytes -= u * MB;
       buf.append(u + " MB ");
     }
-    if(numBytes >= KB) {
-      u = numBytes/KB;
-      numBytes -= u*KB;
+    if (numBytes >= KB) {
+      u = numBytes / KB;
+      numBytes -= u * KB;
       buf.append(u + " KB ");
     }
     buf.append(u + " B"); //even if zero
@@ -194,125 +201,295 @@ public class StreamUtil
 
   static Environment env;
   static String HOST;
-  
+
   static {
     try {
       env = new Environment();
       HOST = env.getHost();
-    } catch(IOException io) {
+    } catch (IOException io) {
       io.printStackTrace();
     }
   }
 
-  static class StreamConsumer extends Thread
-  {
-    StreamConsumer(InputStream in, OutputStream out)
-    {
-      this.bin = new LineNumberReader(
-        new BufferedReader(new InputStreamReader(in)));
-      if(out != null) {
+  static class StreamConsumer extends Thread {
+
+    StreamConsumer(InputStream in, OutputStream out) {
+      this.bin = new LineNumberReader(new BufferedReader(new InputStreamReader(in)));
+      if (out != null) {
         this.bout = new DataOutputStream(out);
       }
     }
-    public void run()
-    {
+
+    public void run() {
       try {
         String line;
-        while((line=bin.readLine()) != null) {
-          if(bout != null) {
+        while ((line = bin.readLine()) != null) {
+          if (bout != null) {
             bout.writeUTF(line); //writeChars
             bout.writeChar('\n');
           }
         }
         bout.flush();
-      } catch(IOException io) {        
+      } catch (IOException io) {
       }
     }
+
     LineNumberReader bin;
     DataOutputStream bout;
   }
 
-  static void exec(String arg, PrintStream log)
-  {
-    exec( new String[] {arg}, log );
+  static void exec(String arg, PrintStream log) {
+    exec(new String[] { arg }, log);
   }
-  
-  static void exec(String[] args, PrintStream log)
-  {
-      try {
-        log.println("Exec: start: " + Arrays.asList(args));
-        Process proc = Runtime.getRuntime().exec(args);
-        new StreamConsumer(proc.getErrorStream(), log).start();
-        new StreamConsumer(proc.getInputStream(), log).start();
-        int status = proc.waitFor();
-        //if status != 0
-        log.println("Exec: status=" + status + ": " + Arrays.asList(args));
-      } catch(InterruptedException in) {
-        in.printStackTrace();
-      } catch(IOException io) {
-        io.printStackTrace();
-      }
+
+  static void exec(String[] args, PrintStream log) {
+    try {
+      log.println("Exec: start: " + Arrays.asList(args));
+      Process proc = Runtime.getRuntime().exec(args);
+      new StreamConsumer(proc.getErrorStream(), log).start();
+      new StreamConsumer(proc.getInputStream(), log).start();
+      int status = proc.waitFor();
+      //if status != 0
+      log.println("Exec: status=" + status + ": " + Arrays.asList(args));
+    } catch (InterruptedException in) {
+      in.printStackTrace();
+    } catch (IOException io) {
+      io.printStackTrace();
+    }
   }
-  
-  static String qualifyHost(String url)
-  {
+
+  static String qualifyHost(String url) {
     try {
-        return qualifyHost(new URL(url)).toString();
-    } catch(IOException io) {
-        return url;
+      return qualifyHost(new URL(url)).toString();
+    } catch (IOException io) {
+      return url;
     }
   }
-  
-  static URL qualifyHost(URL url)
-  {    
+
+  static URL qualifyHost(URL url) {
     try {
       InetAddress a = InetAddress.getByName(url.getHost());
       String qualHost = a.getCanonicalHostName();
       URL q = new URL(url.getProtocol(), qualHost, url.getPort(), url.getFile());
       return q;
-    } catch(IOException io) {
+    } catch (IOException io) {
       return url;
     }
   }
-  
+
   static final String regexpSpecials = "[]()?*+|.!^-\\~@";
-  
-  public static String regexpEscape(String plain)
-  {
+
+  public static String regexpEscape(String plain) {
     StringBuffer buf = new StringBuffer();
     char[] ch = plain.toCharArray();
     int csup = ch.length;
-    for(int c=0; c<csup; c++) {
-      if(regexpSpecials.indexOf(ch[c]) != -1) {
-        buf.append("\\");    
+    for (int c = 0; c < csup; c++) {
+      if (regexpSpecials.indexOf(ch[c]) != -1) {
+        buf.append("\\");
       }
       buf.append(ch[c]);
     }
     return buf.toString();
   }
-  
-  static String slurp(File f) throws IOException
-  {
+
+  public static String safeGetCanonicalPath(File f) {
+    try {
+      String s = f.getCanonicalPath();
+      return (s == null) ? f.toString() : s;
+    } catch (IOException io) {
+      return f.toString();
+    }
+  }
+
+  static String slurp(File f) throws IOException {
+    int len = (int) f.length();
+    byte[] buf = new byte[len];
     FileInputStream in = new FileInputStream(f);
-    int len = (int)f.length();
+    String contents = null;
+    try {
+      in.read(buf, 0, len);
+      contents = new String(buf, "UTF-8");
+    } finally {
+      in.close();
+    }
+    return contents;
+  }
+
+  static String slurpHadoop(Path p, FileSystem fs) throws IOException {
+    int len = (int) fs.getLength(p);
     byte[] buf = new byte[len];
-    in.read(buf, 0, len);
-    return new String(buf);
+    InputStream in = fs.open(p);
+    String contents = null;
+    try {
+      in.read(buf, 0, len);
+      contents = new String(buf, "UTF-8");
+    } finally {
+      in.close();
+    }
+    return contents;
   }
-  
+
+  public static String rjustify(String s, int width) {
+    if (s == null) s = "null";
+    if (width > s.length()) {
+      s = getSpace(width - s.length()) + s;
+    }
+    return s;
+  }
+
+  public static String ljustify(String s, int width) {
+    if (s == null) s = "null";
+    if (width > s.length()) {
+      s = s + getSpace(width - s.length());
+    }
+    return s;
+  }
+
+  static char[] space;
+  static {
+    space = new char[300];
+    Arrays.fill(space, '\u0020');
+  }
+
+  public static String getSpace(int len) {
+    if (len > space.length) {
+      space = new char[Math.max(len, 2 * space.length)];
+      Arrays.fill(space, '\u0020');
+    }
+    return new String(space, 0, len);
+  }
+
   static private Environment env_;
-  
-  static Environment env()
-  {
-    if(env_ != null) {
+
+  static Environment env() {
+    if (env_ != null) {
       return env_;
     }
     try {
       env_ = new Environment();
-    } catch(IOException io) {      
+    } catch (IOException io) {
       io.printStackTrace();
     }
     return env_;
   }
+
+  public static String makeJavaCommand(Class main, String[] argv) {
+    ArrayList vargs = new ArrayList();
+    File javaHomeBin = new File(System.getProperty("java.home"), "bin");
+    File jvm = new File(javaHomeBin, "java");
+    vargs.add(jvm.toString());
+    // copy parent classpath
+    vargs.add("-classpath");
+    vargs.add("\"" + System.getProperty("java.class.path") + "\"");
+
+    // Add main class and its arguments
+    vargs.add(main.getName());
+    for (int i = 0; i < argv.length; i++) {
+      vargs.add(argv[i]);
+    }
+    return collate(vargs, " ");
+  }
+
+  public static String collate(Object[] args, String sep) {
+    return collate(Arrays.asList(args), sep);
+  }
+
+  public static String collate(List args, String sep) {
+    StringBuffer buf = new StringBuffer();
+    Iterator it = args.iterator();
+    while (it.hasNext()) {
+      if (buf.length() > 0) {
+        buf.append(" ");
+      }
+      buf.append(it.next());
+    }
+    return buf.toString();
+  }
+
+  // JobConf helpers
+
+  public static FileSplit getCurrentSplit(JobConf job) {
+    String path = job.get("map.input.file");
+    if (path == null) {
+      return null;
+    }
+    Path p = new Path(path);
+    long start = Long.parseLong(job.get("map.input.start"));
+    long length = Long.parseLong(job.get("map.input.length"));
+    return new FileSplit(p, start, length);
+  }
+
+  static class TaskId {
+
+    boolean mapTask;
+    String jobid;
+    int taskid;
+    int execid;
+  }
+
+  public static boolean isLocalJobTracker(JobConf job) {
+    return job.get("mapred.job.tracker", "local").equals("local");
+  }
+
+  public static TaskId getTaskInfo(JobConf job) {
+    TaskId res = new TaskId();
+
+    String id = job.get("mapred.task.id");
+    if (isLocalJobTracker(job)) {
+      // it uses difft naming 
+      res.mapTask = job.getBoolean("mapred.task.is.map", true);
+      res.jobid = "0";
+      res.taskid = 0;
+      res.execid = 0;
+    } else {
+      String[] e = id.split("_");
+      res.mapTask = e[2].equals("m");
+      res.jobid = e[1];
+      res.taskid = Integer.parseInt(e[3]);
+      res.execid = Integer.parseInt(e[4]);
+    }
+    return res;
+  }
+
+  static boolean getUseMapSideEffect(JobConf job) {
+    String reduce = job.get("stream.reduce.streamprocessor");
+    return StreamJob.REDUCE_NONE.equals(reduce);
+  }
+
+  public static void touch(File file) throws IOException {
+    file = file.getAbsoluteFile();
+    FileOutputStream out = new FileOutputStream(file);
+    out.close();
+    if (!file.exists()) {
+      throw new IOException("touch failed: " + file);
+    }
+  }
+
+  public static boolean isCygwin() {
+    String OS = System.getProperty("os.name");
+    return (OS.indexOf("Windows") > -1);
+  }
+
+  public static String localizeBin(String path) {
+    if (isCygwin()) {
+      path = "C:/cygwin/" + path;
+    }
+    return path;
+  }
   
+  /** @param name foo where &lt;junit>&lt;sysproperty key="foo" value="${foo}"/> 
+   * If foo is undefined then Ant sets the unevaluated value. 
+   * Take this into account when setting defaultVal. */
+  public static String getBoundAntProperty(String name, String defaultVal)
+  {
+    String val = System.getProperty(name);
+    if(val != null && val.indexOf("${") >= 0) {
+      val = null;
+    }
+    if(val == null) {
+      val = defaultVal;
+    }
+    return val;
+  }
+
 }

+ 114 - 135
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamXmlRecordReader.java

@@ -42,91 +42,82 @@ import org.apache.hadoop.mapred.JobConf;
  *
  *  @author Michel Tourn
  */
-public class StreamXmlRecordReader extends StreamBaseRecordReader 
-{
-  public StreamXmlRecordReader(
-    FSDataInputStream in, FileSplit split, Reporter reporter, JobConf job, FileSystem fs)
-    throws IOException
-  {
+public class StreamXmlRecordReader extends StreamBaseRecordReader {
+
+  public StreamXmlRecordReader(FSDataInputStream in, FileSplit split, Reporter reporter,
+      JobConf job, FileSystem fs) throws IOException {
     super(in, split, reporter, job, fs);
-    
+
     beginMark_ = checkJobGet(CONF_NS + "begin");
-    endMark_   = checkJobGet(CONF_NS + "end");
+    endMark_ = checkJobGet(CONF_NS + "end");
 
-    maxRecSize_= job_.getInt(CONF_NS + "maxrec", 50*1000);
-    lookAhead_ = job_.getInt(CONF_NS + "lookahead", 2*maxRecSize_);
+    maxRecSize_ = job_.getInt(CONF_NS + "maxrec", 50 * 1000);
+    lookAhead_ = job_.getInt(CONF_NS + "lookahead", 2 * maxRecSize_);
     synched_ = false;
-    
+
     slowMatch_ = job_.getBoolean(CONF_NS + "slowmatch", false);
-    if(slowMatch_) {
-      beginPat_  = makePatternCDataOrMark(beginMark_);
-      endPat_    = makePatternCDataOrMark(endMark_);
+    if (slowMatch_) {
+      beginPat_ = makePatternCDataOrMark(beginMark_);
+      endPat_ = makePatternCDataOrMark(endMark_);
     }
   }
-  
+
   int numNext = 0;
-  public synchronized boolean next(Writable key, Writable value)
-   throws IOException
-  {
+
+  public synchronized boolean next(Writable key, Writable value) throws IOException {
     long pos = in_.getPos();
     numNext++;
     if (pos >= end_) {
       return false;
     }
-    
+
     DataOutputBuffer buf = new DataOutputBuffer();
-    if(!readUntilMatchBegin()) {
-        return false;
+    if (!readUntilMatchBegin()) {
+      return false;
     }
-    if(!readUntilMatchEnd(buf)) {
-        return false;
+    if (!readUntilMatchEnd(buf)) {
+      return false;
     }
-    
+
     // There is only one elem..key/value splitting is not done here.
-    byte [] record = new byte[buf.getLength()];
+    byte[] record = new byte[buf.getLength()];
     System.arraycopy(buf.getData(), 0, record, 0, record.length);
-    
-    numRecStats(record, 0, record.length); 
 
-    ((Text)key).set(record);
-    ((Text)value).set("");
-    
+    numRecStats(record, 0, record.length);
+
+    ((Text) key).set(record);
+    ((Text) value).set("");
+
     /*if(numNext < 5) {
-        System.out.println("@@@ " + numNext + ". true next k=|" + key.toString().replaceAll("[\\r\\n]", " ")
-        + "|, len=" + buf.length() + " v=|" + value.toString().replaceAll("[\\r\\n]", " ") + "|");
-    }*/
+     System.out.println("@@@ " + numNext + ". true next k=|" + key.toString().replaceAll("[\\r\\n]", " ")
+     + "|, len=" + buf.length() + " v=|" + value.toString().replaceAll("[\\r\\n]", " ") + "|");
+     }*/
 
     return true;
   }
-  
-  public void seekNextRecordBoundary() throws IOException
-  {
+
+  public void seekNextRecordBoundary() throws IOException {
     readUntilMatchBegin();
   }
-  
-  boolean readUntilMatchBegin() throws IOException
-  {
-    if(slowMatch_) {
-        return slowReadUntilMatch(beginPat_, false, null);
+
+  boolean readUntilMatchBegin() throws IOException {
+    if (slowMatch_) {
+      return slowReadUntilMatch(beginPat_, false, null);
     } else {
-        return fastReadUntilMatch(beginMark_, false, null);
+      return fastReadUntilMatch(beginMark_, false, null);
     }
   }
-  
-  private boolean readUntilMatchEnd(DataOutputBuffer buf) throws IOException
-  {
-    if(slowMatch_) {
+
+  private boolean readUntilMatchEnd(DataOutputBuffer buf) throws IOException {
+    if (slowMatch_) {
       return slowReadUntilMatch(endPat_, true, buf);
     } else {
       return fastReadUntilMatch(endMark_, true, buf);
     }
   }
-  
-  
-  private boolean slowReadUntilMatch(Pattern markPattern, boolean includePat, 
-          DataOutputBuffer outBufOrNull) 
-    throws IOException   
-  {
+
+  private boolean slowReadUntilMatch(Pattern markPattern, boolean includePat,
+      DataOutputBuffer outBufOrNull) throws IOException {
     try {
       long inStart = in_.getPos();
       byte[] buf = new byte[Math.max(lookAhead_, maxRecSize_)];
@@ -134,34 +125,33 @@ public class StreamXmlRecordReader extends StreamBaseRecordReader
       boolean success = true;
       in_.mark(lookAhead_ + 2);
       read = in_.read(buf);
-      if( read == -1 )
-          return false;
-      
-      String sbuf = new String(buf, 0, read, "UTF-8");        
+      if (read == -1) return false;
+
+      String sbuf = new String(buf, 0, read, "UTF-8");
       Matcher match = markPattern.matcher(sbuf);
 
       firstMatchStart_ = NA;
       firstMatchEnd_ = NA;
       int bufPos = 0;
       int state = synched_ ? CDATA_OUT : CDATA_UNK;
-      int s=0;
+      int s = 0;
       int matchLen = 0;
-      while(match.find(bufPos)) {
+      while (match.find(bufPos)) {
         int input;
         matchLen = match.group(0).length();
-        if(match.group(1) != null) {
+        if (match.group(1) != null) {
           input = CDATA_BEGIN;
-        } else if(match.group(2) != null) {
+        } else if (match.group(2) != null) {
           input = CDATA_END;
           firstMatchStart_ = NA; // |<DOC CDATA[ </DOC> ]]> should keep it
         } else {
           input = RECORD_MAYBE;
         }
-        if(input == RECORD_MAYBE) {
-            if(firstMatchStart_ == NA) {
-              firstMatchStart_ = match.start();
-              firstMatchEnd_   = match.end();
-            }
+        if (input == RECORD_MAYBE) {
+          if (firstMatchStart_ == NA) {
+            firstMatchStart_ = match.start();
+            firstMatchEnd_ = match.end();
+          }
         }
         state = nextState(state, input, match.start());
         /*System.out.println("@@@" +
@@ -169,164 +159,153 @@ public class StreamXmlRecordReader extends StreamBaseRecordReader
          " state=" + state + " input=" + input + 
          " firstMatchStart_=" + firstMatchStart_ + " startinstream=" + (inStart+firstMatchStart_) + 
          " match=" + match.group(0) + " in=" + in_.getPos());*/
-        if(state == RECORD_ACCEPT) {
+        if (state == RECORD_ACCEPT) {
           break;
         }
         bufPos = match.end();
         s++;
       }
-      if(state != CDATA_UNK) {
+      if (state != CDATA_UNK) {
         synched_ = true;
       }
-      boolean matched = (firstMatchStart_ != NA) && (state == RECORD_ACCEPT || state == CDATA_UNK); 
-      if(matched) {
+      boolean matched = (firstMatchStart_ != NA) && (state == RECORD_ACCEPT || state == CDATA_UNK);
+      if (matched) {
         int endPos = includePat ? firstMatchEnd_ : firstMatchStart_;
         //System.out.println("firstMatchStart_=" + firstMatchStart_ + " firstMatchEnd_=" + firstMatchEnd_);
         //String snip = sbuf.substring(firstMatchStart_, firstMatchEnd_);
         //System.out.println(" match snip=|" + snip + "| markPattern=" + markPattern);
-        if(outBufOrNull != null) {
-          in_.reset();      
-          outBufOrNull.write(in_,endPos);  
+        if (outBufOrNull != null) {
+          in_.reset();
+          outBufOrNull.write(in_, endPos);
         } else {
           //System.out.println("Skip to " + (inStart + endPos));
           in_.seek(inStart + endPos);
         }
       }
       return matched;
-    } catch(Exception e) {
+    } catch (Exception e) {
       e.printStackTrace();
     } finally {
       // in_ ?
     }
     return false;
-  }  
-  
+  }
+
   // states
-  final static int CDATA_IN  = 10;
+  final static int CDATA_IN = 10;
   final static int CDATA_OUT = 11;
   final static int CDATA_UNK = 12;
   final static int RECORD_ACCEPT = 13;
   // inputs
   final static int CDATA_BEGIN = 20;
-  final static int CDATA_END   = 21;
-  final static int RECORD_MAYBE= 22;
-  
+  final static int CDATA_END = 21;
+  final static int RECORD_MAYBE = 22;
+
   /* also updates firstMatchStart_;*/
-  int nextState(int state, int input, int bufPos)
-  {
-    switch(state) {
-      case CDATA_UNK:
-      case CDATA_OUT:
-        switch(input) {
-          case CDATA_BEGIN:
-            return CDATA_IN;
-          case CDATA_END:
-            if(state==CDATA_OUT) {
-              //System.out.println("buggy XML " + bufPos);
-            }
-            return CDATA_OUT;
-          case RECORD_MAYBE:
-            return (state==CDATA_UNK) ? CDATA_UNK : RECORD_ACCEPT;
+  int nextState(int state, int input, int bufPos) {
+    switch (state) {
+    case CDATA_UNK:
+    case CDATA_OUT:
+      switch (input) {
+      case CDATA_BEGIN:
+        return CDATA_IN;
+      case CDATA_END:
+        if (state == CDATA_OUT) {
+          //System.out.println("buggy XML " + bufPos);
         }
-      break;
-      case CDATA_IN:
-       return (input==CDATA_END) ? CDATA_OUT : CDATA_IN;
+        return CDATA_OUT;
+      case RECORD_MAYBE:
+        return (state == CDATA_UNK) ? CDATA_UNK : RECORD_ACCEPT;
+      }
+    break;
+    case CDATA_IN:
+      return (input == CDATA_END) ? CDATA_OUT : CDATA_IN;
     }
     throw new IllegalStateException(state + " " + input + " " + bufPos + " " + splitName_);
   }
-  
-    
-  Pattern makePatternCDataOrMark(String escapedMark)
-  {
+
+  Pattern makePatternCDataOrMark(String escapedMark) {
     StringBuffer pat = new StringBuffer();
-    addGroup(pat, StreamUtil.regexpEscape("CDATA["));   // CDATA_BEGIN
-    addGroup(pat, StreamUtil.regexpEscape("]]>"));      // CDATA_END
-    addGroup(pat, escapedMark);                         // RECORD_MAYBE
+    addGroup(pat, StreamUtil.regexpEscape("CDATA[")); // CDATA_BEGIN
+    addGroup(pat, StreamUtil.regexpEscape("]]>")); // CDATA_END
+    addGroup(pat, escapedMark); // RECORD_MAYBE
     return Pattern.compile(pat.toString());
   }
-  void addGroup(StringBuffer pat, String escapedGroup)
-  {
-    if(pat.length() > 0) {
-        pat.append("|");
+
+  void addGroup(StringBuffer pat, String escapedGroup) {
+    if (pat.length() > 0) {
+      pat.append("|");
     }
     pat.append("(");
     pat.append(escapedGroup);
     pat.append(")");
   }
-  
-  
-  
-  boolean fastReadUntilMatch(String textPat, 
-          boolean includePat, 
-          DataOutputBuffer outBufOrNull) throws IOException 
-  {
+
+  boolean fastReadUntilMatch(String textPat, boolean includePat, DataOutputBuffer outBufOrNull) throws IOException {
     //System.out.println("@@@BEGIN readUntilMatch inPos=" + in_.getPos());  
     byte[] cpat = textPat.getBytes("UTF-8");
     int m = 0;
     boolean match = false;
     long markPos = -1;
     int msup = cpat.length;
-    if(!includePat) {
+    if (!includePat) {
       int LL = 120000 * 10;
       markPos = in_.getPos();
       in_.mark(LL); // lookAhead_
     }
     while (true) {
       int b = in_.read();
-      if (b == -1)
-        break;
+      if (b == -1) break;
 
-      byte c = (byte)b; // this assumes eight-bit matching. OK with UTF-8
+      byte c = (byte) b; // this assumes eight-bit matching. OK with UTF-8
       if (c == cpat[m]) {
         m++;
-        if(m==msup) {
+        if (m == msup) {
           match = true;
           break;
         }
       } else {
-        if(outBufOrNull != null) {
+        if (outBufOrNull != null) {
           outBufOrNull.write(cpat, 0, m);
           outBufOrNull.write(c);
         }
-        
+
         m = 0;
       }
     }
-    if(!includePat && match) {
+    if (!includePat && match) {
       long pos = in_.getPos() - textPat.length();
       in_.reset();
       in_.seek(pos);
-    } else if(outBufOrNull != null){
+    } else if (outBufOrNull != null) {
       outBufOrNull.write(cpat);
     }
     //System.out.println("@@@DONE  readUntilMatch inPos=" + in_.getPos() + " includePat=" + includePat + " pat=" + textPat + ", buf=|" + outBufOrNull + "|");
     return match;
   }
-  
-  String checkJobGet(String prop) throws IOException
-  {
+
+  String checkJobGet(String prop) throws IOException {
     String val = job_.get(prop);
-    if(val == null) {
-        throw new IOException("JobConf: missing required property: " + prop);
+    if (val == null) {
+      throw new IOException("JobConf: missing required property: " + prop);
     }
     return val;
   }
-  
-  
+
   String beginMark_;
   String endMark_;
-  
+
   Pattern beginPat_;
   Pattern endPat_;
 
-  boolean slowMatch_;  
+  boolean slowMatch_;
   int lookAhead_; // bytes to read to try to synch CDATA/non-CDATA. Should be more than max record size
   int maxRecSize_;
 
-  final static int NA = -1;  
+  final static int NA = -1;
   int firstMatchStart_ = 0; // candidate record boundary. Might just be CDATA.
   int firstMatchEnd_ = 0;
-  
+
   boolean isRecordMatch_;
   boolean synched_;
 }

+ 86 - 0
src/contrib/streaming/src/java/org/apache/hadoop/streaming/TupleInputFormat.java

@@ -0,0 +1,86 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.IOException;
+import java.util.ArrayList;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapred.*;
+
+/**
+ @author Michel Tourn
+ */
+public class TupleInputFormat extends InputFormatBase {
+
+  public TupleInputFormat() {
+    fmts_ = new ArrayList();
+  }
+
+  public void setPrimary(InputFormat fmt) {
+    if (fmts_.size() == 0) {
+      fmts_.add(fmt);
+    } else {
+      fmts_.set(0, fmt);
+    }
+  }
+
+  public void addSecondary(InputFormat fmt) {
+    if (fmts_.size() == 0) {
+      throw new IllegalStateException("this.setPrimary() has not been called");
+    }
+    fmts_.add(fmt);
+  }
+
+  /**
+   */
+  public RecordReader getRecordReader(FileSystem fs, FileSplit split, JobConf job, Reporter reporter) throws IOException {
+
+    reporter.setStatus(split.toString());
+
+    return new MultiRecordReader();
+  }
+
+  class MultiRecordReader implements RecordReader {
+
+    MultiRecordReader() {
+    }
+
+    public boolean next(Writable key, Writable value) throws IOException {
+      return false;
+    }
+
+    public long getPos() throws IOException {
+      return 0;
+    }
+
+    public void close() throws IOException {
+    }
+
+    public WritableComparable createKey() {
+      return new UTF8();
+    }
+
+    public Writable createValue() {
+      return new UTF8();
+    }
+
+  }
+
+  ArrayList/*<InputFormat>*/fmts_;
+}

+ 317 - 0
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamedMerge.java

@@ -0,0 +1,317 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.LineNumberInputStream;
+import java.io.OutputStream;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.dfs.DFSShell;
+import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * This JUnit test is not pure-Java and is not run as 
+ * part of the standard ant test* targets.   
+ * Two ways to run this:<pre>
+ * 1. main(), a Java application.
+ * 2. cd src/contrib/streaming/ 
+ *    ant \
+ *     [-Dfs.default.name=h:p] \ 
+ *     [-Dhadoop.test.localoutputfile=/tmp/fifo] \ 
+ *     test-unix 
+ * </pre>
+ * @author michel
+ */
+public class TestStreamedMerge extends TestCase {
+
+  public TestStreamedMerge() throws IOException {
+    UtilTest utilTest = new UtilTest(getClass().getName());
+    utilTest.checkUserDir();
+    //  utilTest.redirectIfAntJunit();
+  }
+
+  final static int NAME_PORT = 8200;
+  final static int SOC_PORT = 1888;
+
+  void addInput(String path, String contents) throws IOException {
+    OutputStream out = fs_.create(new Path(path));
+    DataOutputStream dout = new DataOutputStream(out);
+    dout.write(contents.getBytes("UTF-8"));
+    dout.close();
+    System.err.println("addInput done: " + path);
+  }
+
+  String createInputs(boolean tag) throws IOException {
+    fs_.delete(new Path("/input/"));
+
+    // i18n() replaces some ASCII with multibyte UTF-8 chars
+    addInput("/input/part-00", i18n("k1\tv1\n" + "k3\tv5\n"));
+    addInput("/input/part-01", i18n("k1\tv2\n" + "k2\tv4\n"));
+    addInput("/input/part-02", i18n("k1\tv3\n"));
+    addInput("/input/part-03", "");
+    
+    // tags are one-based: ">1" corresponds to part-00, etc.
+    // Expected result it the merge-sort order of the records.
+    // keys are compared as Strings and ties are broken by stream index
+    // For example (k1; stream 2) < (k1; stream 3)
+    String expect = i18n(
+        unt(">1\tk1\tv1\n", tag) + 
+        unt(">2\tk1\tv2\n", tag) + 
+        unt(">3\tk1\tv3\n", tag) + 
+        unt(">2\tk2\tv4\n", tag) +
+        unt(">1\tk3\tv5\n", tag)
+    );
+    return expect;
+  }
+  
+  String unt(String line, boolean keepTag)
+  {
+    return keepTag ? line : line.substring(line.indexOf('\t')+1);
+  }
+  String i18n(String c) {
+    c = c.replace('k', '\u20ac'); // Euro sign, in UTF-8: E282AC
+    c = c.replace('v', '\u00a2'); // Cent sign, in UTF-8: C2A2 ; UTF-16 contains null
+    // "\ud800\udc00" // A surrogate pair, U+10000. OK also works
+    return c;
+  }
+
+  void lsr() {
+    try {
+      System.out.println("lsr /");
+      DFSShell shell = new DFSShell();
+      shell.setConf(conf_);
+      shell.init();
+      shell.ls("/", true);
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
+  void printSampleInput() {
+    try {
+      System.out.println("cat /input/part-00");
+      String content = StreamUtil.slurpHadoop(new Path("/input/part-00"), fs_);
+      System.out.println(content);
+      System.out.println("cat done.");
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
+  void callStreaming(String argSideOutput, boolean inputTagged) throws IOException {
+    String[] testargs = new String[] {
+        //"-jobconf", "stream.debug=1",
+        "-verbose", 
+        "-jobconf", "stream.testmerge=1", 
+        "-input", "+/input/part-00 | /input/part-01 | /input/part-02", 
+        "-mapper", StreamUtil.localizeBin("/bin/cat"), 
+        "-reducer", "NONE", 
+        "-output", "/my.output",
+        "-mapsideoutput", argSideOutput, 
+        "-dfs", conf_.get("fs.default.name"), 
+        "-jt", "local",
+        "-jobconf", "stream.sideoutput.localfs=true", 
+    };
+    ArrayList argList = new ArrayList();
+    argList.addAll(Arrays.asList(testargs));
+    if (inputTagged) {
+      argList.add("-inputtagged");
+    }
+    testargs = (String[])argList.toArray(new String[0]);
+    String sss = StreamUtil.collate(argList, " ");
+    System.out.println("bin/hadoop jar build/hadoop-streaming.jar " + sss);
+    //HadoopStreaming.main(testargs);
+    StreamJob job = new StreamJob(testargs, false);
+    job.go();
+  }
+
+  SideEffectConsumer startSideEffectConsumer(StringBuffer outBuf) {
+    SideEffectConsumer t = new SideEffectConsumer(outBuf) {
+      ServerSocket listen;
+      Socket client;
+      InputStream in;
+      
+      InputStream connectInputStream() throws IOException {
+        listen = new ServerSocket(SOC_PORT);
+        client = listen.accept();
+        in = client.getInputStream();
+        return in;
+      }
+      
+      void close() throws IOException
+      {
+        listen.close();
+        System.out.println("@@@listen closed");
+      }
+    };
+    t.start();
+    return t;
+  }
+
+  abstract class SideEffectConsumer extends Thread {
+
+    SideEffectConsumer(StringBuffer buf) {
+      buf_ = buf;
+      setDaemon(true);
+    }
+
+    abstract InputStream connectInputStream() throws IOException;
+    
+    abstract void close() throws IOException;
+    
+    public void run() {
+      try {
+        in_ = connectInputStream();
+        while (true) {
+          byte[] b = UTF8ByteArrayUtils.readLine(in_);
+          if (b == null) {
+            break;
+          }
+          buf_.append(new String(b, "UTF-8"));
+          buf_.append('\n');
+        }
+        in_.close();
+      } catch (IOException io) {
+        throw new RuntimeException(io);
+      }
+    }
+    
+    InputStream in_;
+    StringBuffer buf_;
+  }
+
+  public void testMain() throws IOException {
+    boolean success = false;
+    String base = new File(".").getAbsolutePath();
+    System.setProperty("hadoop.log.dir", base + "/logs");
+    conf_ = new Configuration();
+    String overrideFS = StreamUtil.getBoundAntProperty("fs.default.name", null);
+    MiniDFSCluster cluster = null;
+    try {
+      if(overrideFS == null) {
+        cluster = new MiniDFSCluster(NAME_PORT, conf_, false);
+        fs_ = cluster.getFileSystem();
+      } else {
+        System.out.println("overrideFS: " + overrideFS);
+        conf_.set("fs.default.name", overrideFS);
+        fs_ = FileSystem.get(conf_);
+      }
+      doAllTestJobs();
+      success = true;
+    } catch (IOException io) {
+      io.printStackTrace();
+    } finally {
+      try {
+        fs_.close();
+      } catch (IOException io) {
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+        System.out.println("cluster.shutdown(); DONE");
+      }
+      System.out.println(getClass().getName() + ": success=" + success);
+    }
+  }
+
+  void doAllTestJobs() throws IOException
+  {
+    goSocketTagged(true, false);
+    goSocketTagged(false, false);
+    goSocketTagged(true, true);
+  }
+  
+  void goSocketTagged(boolean socket, boolean inputTagged) throws IOException {
+    System.out.println("***** goSocketTagged: " + socket + ", " + inputTagged);
+    String expect = createInputs(inputTagged);
+    lsr();
+    printSampleInput();
+
+    StringBuffer outputBuf = new StringBuffer();
+    String sideOutput = null;
+    File f = null;
+    SideEffectConsumer consumer = null;
+    if (socket) {
+      consumer = startSideEffectConsumer(outputBuf);
+      sideOutput = "socket://localhost:" + SOC_PORT + "/";
+    } else {
+      String userOut = StreamUtil.getBoundAntProperty(
+          "hadoop.test.localoutputfile", null);
+      if(userOut != null) {
+        f = new File(userOut);
+        // don't delete so they can mkfifo
+        maybeFifoOutput_ = true;
+      } else {
+        f = new File("localoutputfile");
+        f.delete();
+        maybeFifoOutput_ = false;
+      }
+      String s = new Path(f.getAbsolutePath()).toString();
+      if(! s.startsWith("/")) {
+        s = "/" + s; // Windows "file:/C:/"
+      }
+      sideOutput = "file:" + s;
+    }
+    System.out.println("sideOutput=" + sideOutput);
+    callStreaming(sideOutput, inputTagged);
+    String output;
+    if (socket) {
+      try {
+        consumer.join();
+        consumer.close();
+      } catch (InterruptedException e) {
+        throw (IOException) new IOException().initCause(e);
+      }
+      output = outputBuf.toString();
+    } else {
+      if(maybeFifoOutput_) {
+        System.out.println("assertEquals will fail.");
+        output = "potential FIFO: not retrieving to avoid blocking on open() "
+          + f.getAbsoluteFile();
+      } else {
+        output = StreamUtil.slurp(f.getAbsoluteFile());
+      }
+    }
+
+    lsr();
+    
+    System.out.println("output=|" + output + "|");
+    System.out.println("expect=|" + expect + "|");
+    assertEquals(expect, output);
+  }
+
+  Configuration conf_;
+  FileSystem fs_;
+  boolean maybeFifoOutput_;
+
+  public static void main(String[] args) throws Throwable {
+    TestStreamedMerge test = new TestStreamedMerge();
+    test.testMain();
+  }
+  
+}

+ 14 - 62
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreaming.java

@@ -35,74 +35,29 @@ public class TestStreaming extends TestCase
   String OUTPUT_DIR = "out";
   String input = "roses.are.red\nviolets.are.blue\nbunnies.are.pink\n";
   // map behaves like "/usr/bin/tr . \\n"; (split words into lines)
-  String map = makeJavaCommand(TrApp.class, new String[]{".", "\\n"});
+  String map = StreamUtil.makeJavaCommand(TrApp.class, new String[]{".", "\\n"});
   // combine, reduce behave like /usr/bin/uniq. But also prepend lines with C, R.
-  String combine  = makeJavaCommand(UniqApp.class, new String[]{"C"});
-  String reduce = makeJavaCommand(UniqApp.class, new String[]{"R"});
+  String combine  = StreamUtil.makeJavaCommand(UniqApp.class, new String[]{"C"});
+  String reduce = StreamUtil.makeJavaCommand(UniqApp.class, new String[]{"R"});
   String outputExpect = "RCare\t\nRCblue\t\nRCbunnies\t\nRCpink\t\nRCred\t\nRCroses\t\nRCviolets\t\n";
 
   StreamJob job;
 
   public TestStreaming() throws IOException
   {
-    // trunk/src/contrib/streaming --> trunk/build/contrib/streaming/test/data
-    String userDir = System.getProperty("user.dir");
-    String antTestDir = System.getProperty("test.build.data", userDir);
-    if(! userDir.equals(antTestDir)) {
-        // because changes to user.dir are ignored by File
-        throw new IllegalStateException("user.dir != test.build.data. The junit Ant task must be forked.");
-    }
-
-    boolean fromAntJunit = System.getProperty("test.build.data") != null;
-    if(fromAntJunit) {
-      new File(antTestDir).mkdirs();
-      File outFile = new File(antTestDir, getClass().getName()+".log");
-      PrintStream out = new PrintStream(new FileOutputStream(outFile));
-      System.setOut(out);
-      System.setErr(out);
-    }
-    System.out.println("test.build.data=" + antTestDir);
+    UtilTest utilTest = new UtilTest(getClass().getName());
+    utilTest.checkUserDir();
+    utilTest.redirectIfAntJunit();
   }
 
   void createInput() throws IOException
   {
     String path = new File(".", INPUT_FILE).getAbsolutePath();// needed from junit forked vm
     DataOutputStream out = new DataOutputStream(new FileOutputStream(path));
-    out.writeBytes(input);
+    out.write(input.getBytes("UTF-8"));
     out.close();
   }
 
-  public String makeJavaCommand(Class main, String[] argv)
-  {
-    ArrayList vargs = new ArrayList();
-    File javaHomeBin = new File(System.getProperty("java.home"), "bin");
-    File jvm = new File(javaHomeBin, "java");
-    vargs.add(jvm.toString());
-    // copy parent classpath
-    vargs.add("-classpath");
-    vargs.add("\"" + System.getProperty("java.class.path") + "\"");
-
-    // Add main class and its arguments
-    vargs.add(main.getName());
-    for(int i=0; i<argv.length; i++) {
-      vargs.add(argv[i]);
-    }
-    return collate(vargs, " ");
-  }
-
-  String collate(ArrayList args, String sep)
-  {
-    StringBuffer buf = new StringBuffer();
-    Iterator it = args.iterator();
-    while(it.hasNext()) {
-      if(buf.length() > 0) {
-        buf.append(" ");
-      }
-      buf.append(it.next());
-    }
-    return buf.toString();
-  }
-
   public void testCommandLine()
   {
     try {
@@ -117,22 +72,21 @@ public class TestStreaming extends TestCase
           "-mapper", map,
           "-combiner", combine,
           "-reducer", reduce,
-          /*"-debug",*/
-          "-verbose"
+          //"-verbose",
+          //"-jobconf", "stream.debug=set"
+          "-jobconf", "keep.failed.task.files=true",
       };
-      job = new StreamJob(argv, mayExit);
+      job = new StreamJob(argv, mayExit);      
       job.go();
       File outFile = new File(".", OUTPUT_DIR + "/part-00000").getAbsoluteFile();
       String output = StreamUtil.slurp(outFile);
-      System.out.println("outEx=" + outputExpect);
-      System.out.println("  out=" + output);
+      System.err.println("outEx1=" + outputExpect);
+      System.err.println("  out1=" + output);
       assertEquals(outputExpect, output);
 
     } catch(Exception e) {
-        failTrace(e);
+      failTrace(e);
     }
-
-
   }
 
   void failTrace(Exception e)
@@ -142,8 +96,6 @@ public class TestStreaming extends TestCase
     fail(sw.toString());
   }
 
-
-
   public static void main(String[]args) throws Exception
   {
     new TestStreaming().testCommandLine();

+ 10 - 18
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TrApp.java

@@ -17,12 +17,13 @@
 package org.apache.hadoop.streaming;
 
 import java.io.*;
+
 import org.apache.hadoop.streaming.Environment;
 
 /** A minimal Java implementation of /usr/bin/tr.
     Used to test the usage of external applications without adding
     platform-specific dependencies.
-*/
+ */
 public class TrApp
 {
 
@@ -35,16 +36,6 @@ public class TrApp
   void testParentJobConfToEnvVars() throws IOException
   {
     env = new Environment();
-
-    /* To get some ideas of stable env.vars:
-    Enumeration it = env.keys();
-     while(it.hasMoreElements()) {
-        String key = (String)it.nextElement();
-        String val = (String)env.get(key);
-        System.out.println("@@@" + key + "=" + val);
-     }
-     */
-
     // test that some JobConf properties are exposed as expected     
     // Note the dots translated to underscore: 
     // property names have been escaped in PipeMapRed.safeEnvVarName()
@@ -52,7 +43,8 @@ public class TrApp
     expect("mapred_job_tracker", "local");
     expect("mapred_input_key_class", "org.apache.hadoop.io.Text");
     expect("mapred_input_value_class", "org.apache.hadoop.io.Text");
-    expect("mapred_local_dir", "build/test/mapred/local");
+    //expect("mapred_local_dir", "build/test/mapred/local");
+    expectDefined("mapred_local_dir");
     expect("mapred_output_format_class", "org.apache.hadoop.streaming.StreamOutputFormat");
     expect("mapred_output_key_class", "org.apache.hadoop.io.Text");
     expect("mapred_output_value_class", "org.apache.hadoop.io.Text");
@@ -64,12 +56,12 @@ public class TrApp
     expectDefined("map_input_file");
     expect("map_input_start", "0");
     expectDefined("map_input_length");
-    
+
     expectDefined("io_sort_factor");
 
     // the FileSplit context properties are not available in local hadoop..
     // so can't check them in this test.
-    
+
   }
 
   // this runs in a subprocess; won't use JUnit's assertTrue()    
@@ -81,7 +73,7 @@ public class TrApp
       throw new IOException(msg);
     }
   }
-  
+
   void expectDefined(String evName) throws IOException
   {
     String got = env.getProperty(evName);
@@ -90,7 +82,7 @@ public class TrApp
       throw new IOException(msg);
     }
   }
-  
+
   public void go() throws IOException
   {
     testParentJobConfToEnvVars();
@@ -98,8 +90,8 @@ public class TrApp
     String line;
 
     while ((line = in.readLine()) != null) {
-        String out = line.replace(find, replace);
-        System.out.println(out);
+      String out = line.replace(find, replace);
+      System.out.println(out);
     }
   }
 

+ 6 - 5
src/contrib/streaming/src/test/org/apache/hadoop/streaming/UniqApp.java

@@ -17,12 +17,13 @@
 package org.apache.hadoop.streaming;
 
 import java.io.*;
+import java.util.Date;
 
 /** A minimal Java implementation of /usr/bin/uniq
     Used to test the usage of external applications without adding
     platform-specific dependencies.
     Uniques lines and prepends a header on the line.
-*/
+ */
 public class UniqApp
 {
 
@@ -36,10 +37,10 @@ public class UniqApp
     String line;
     String prevLine = null;
     while ((line = in.readLine()) != null) {
-        if(! line.equals(prevLine)) {
-          System.out.println(header + line);
-        }
-        prevLine = line;
+      if(! line.equals(prevLine)) {
+        System.out.println(header + line);
+      }
+      prevLine = line;
     }
   }
 

+ 56 - 0
src/contrib/streaming/src/test/org/apache/hadoop/streaming/UtilTest.java

@@ -0,0 +1,56 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+class UtilTest {
+
+  public UtilTest(String testName) {
+    testName_ = testName;
+    userDir_ = System.getProperty("user.dir");
+    antTestDir_ = System.getProperty("test.build.data", userDir_);
+    System.out.println("test.build.data-or-user.dir=" + antTestDir_);
+  }
+
+  void checkUserDir() {
+    // trunk/src/contrib/streaming --> trunk/build/contrib/streaming/test/data
+    if(! userDir_.equals(antTestDir_)) {
+      // because changes to user.dir are ignored by File static methods.
+      throw new IllegalStateException("user.dir != test.build.data. The junit Ant task must be forked.");
+    }
+  }
+
+  void redirectIfAntJunit() throws IOException
+  {
+    boolean fromAntJunit = System.getProperty("test.build.data") != null;
+    if(fromAntJunit) {
+      new File(antTestDir_).mkdirs();
+      File outFile = new File(antTestDir_, testName_+".log");
+      PrintStream out = new PrintStream(new FileOutputStream(outFile));
+      System.setOut(out);
+      System.setErr(out);
+    }
+  }
+
+  private String userDir_;
+  private String antTestDir_;
+  private String testName_;
+}