瀏覽代碼

Merge -c 1299136 from branch-1 to branch-1.0 to fix HADOOP-1722. Allow hadoop streaming to handle non-utf8 byte array. Contributed by Klaas Bosteels and Matthias Lehmann.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-1.0@1299138 13f79535-47bb-0310-9956-ffa450edef68
Arun Murthy 13 年之前
父節點
當前提交
9bc66fc4bb
共有 38 個文件被更改,包括 4114 次插入121 次删除
  1. 3 0
      CHANGES.txt
  2. 74 0
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/AutoInputFormat.java
  3. 133 0
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/DumpTypedBytes.java
  4. 22 3
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/HadoopStreaming.java
  5. 92 0
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/LoadTypedBytes.java
  6. 108 97
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapRed.java
  7. 24 10
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapper.java
  8. 18 7
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeReducer.java
  9. 42 4
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java
  10. 125 0
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/io/IdentifierResolver.java
  11. 48 0
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/io/InputWriter.java
  12. 59 0
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/io/OutputReader.java
  13. 73 0
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/io/RawBytesInputWriter.java
  14. 88 0
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/io/RawBytesOutputReader.java
  15. 75 0
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/io/TextInputWriter.java
  16. 115 0
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/io/TextOutputReader.java
  17. 63 0
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/io/TypedBytesInputWriter.java
  18. 76 0
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/io/TypedBytesOutputReader.java
  19. 44 0
      src/contrib/streaming/src/java/org/apache/hadoop/typedbytes/Type.java
  20. 490 0
      src/contrib/streaming/src/java/org/apache/hadoop/typedbytes/TypedBytesInput.java
  21. 298 0
      src/contrib/streaming/src/java/org/apache/hadoop/typedbytes/TypedBytesOutput.java
  22. 159 0
      src/contrib/streaming/src/java/org/apache/hadoop/typedbytes/TypedBytesRecordInput.java
  23. 137 0
      src/contrib/streaming/src/java/org/apache/hadoop/typedbytes/TypedBytesRecordOutput.java
  24. 88 0
      src/contrib/streaming/src/java/org/apache/hadoop/typedbytes/TypedBytesWritable.java
  25. 335 0
      src/contrib/streaming/src/java/org/apache/hadoop/typedbytes/TypedBytesWritableInput.java
  26. 221 0
      src/contrib/streaming/src/java/org/apache/hadoop/typedbytes/TypedBytesWritableOutput.java
  27. 66 0
      src/contrib/streaming/src/java/org/apache/hadoop/typedbytes/package.html
  28. 65 0
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/RawBytesMapApp.java
  29. 74 0
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/RawBytesReduceApp.java
  30. 111 0
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestAutoInputFormat.java
  31. 91 0
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestDumpTypedBytes.java
  32. 88 0
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestLoadTypedBytes.java
  33. 95 0
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestRawBytesStreaming.java
  34. 93 0
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestTypedBytesStreaming.java
  35. 59 0
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TypedBytesMapApp.java
  36. 58 0
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TypedBytesReduceApp.java
  37. 242 0
      src/contrib/streaming/src/test/org/apache/hadoop/typedbytes/TestIO.java
  38. 62 0
      src/contrib/streaming/src/test/org/apache/hadoop/typedbytes/TestTypedBytesWritable.java

+ 3 - 0
CHANGES.txt

@@ -12,6 +12,9 @@ Release 1.0.2 - unreleased
     MAPREDUCE-3773. Add queue metrics with buckets for job run times. (omalley
     via acmurthy)
 
+    HADOOP-1722. Allow hadoop streaming to handle non-utf8 byte array. (Klaas
+    Bosteels and Matthias Lehmann via acmurthy)
+
   BUG FIXES
 
     HADOOP-8050. Deadlock in metrics. (Kihwal Lee via mattf)

+ 74 - 0
src/contrib/streaming/src/java/org/apache/hadoop/streaming/AutoInputFormat.java

@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming;
+
+import java.io.EOFException;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.SequenceFileInputFormat;
+import org.apache.hadoop.mapred.TextInputFormat;
+
+/**
+ * An {@link InputFormat} that tries to deduce the types of the input files
+ * automatically. It can currently handle text and sequence files.
+ */
+public class AutoInputFormat extends FileInputFormat {
+
+  private TextInputFormat textInputFormat = new TextInputFormat();
+
+  private SequenceFileInputFormat seqFileInputFormat = 
+    new SequenceFileInputFormat();
+
+  public void configure(JobConf job) {
+    textInputFormat.configure(job);
+    // SequenceFileInputFormat has no configure() method
+  }
+
+  public RecordReader getRecordReader(InputSplit split, JobConf job,
+    Reporter reporter) throws IOException {
+    FileSplit fileSplit = (FileSplit) split;
+    FileSystem fs = FileSystem.get(job);
+    FSDataInputStream is = fs.open(fileSplit.getPath());
+    byte[] header = new byte[3];
+    RecordReader reader = null;
+    try {
+      is.readFully(header);
+    } catch (EOFException eof) {
+      reader = textInputFormat.getRecordReader(split, job, reporter);
+    } finally {
+      is.close();
+    }
+    if (header[0] == 'S' && header[1] == 'E' && header[2] == 'Q') {
+      reader = seqFileInputFormat.getRecordReader(split, job, reporter);
+    } else {
+      reader = textInputFormat.getRecordReader(split, job, reporter);
+    }
+    return reader;
+  }
+
+}

+ 133 - 0
src/contrib/streaming/src/java/org/apache/hadoop/streaming/DumpTypedBytes.java

@@ -0,0 +1,133 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.typedbytes.TypedBytesOutput;
+import org.apache.hadoop.typedbytes.TypedBytesWritableOutput;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * Utility program that fetches all files that match a given pattern and dumps
+ * their content to stdout as typed bytes. This works for all files that can be
+ * handled by {@link org.apache.hadoop.streaming.AutoInputFormat}.
+ */
+public class DumpTypedBytes implements Tool {
+
+  private Configuration conf;
+
+  public DumpTypedBytes(Configuration conf) {
+    this.conf = conf;
+  }
+  
+  public DumpTypedBytes() {
+    this(new Configuration());
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  /**
+   * The main driver for <code>DumpTypedBytes</code>.
+   */
+  public int run(String[] args) throws Exception {
+    Path pattern = new Path(args[0]);
+    FileSystem fs = pattern.getFileSystem(getConf());
+    fs.setVerifyChecksum(true);
+    for (Path p : FileUtil.stat2Paths(fs.globStatus(pattern), pattern)) {
+      List<FileStatus> inputFiles = new ArrayList<FileStatus>();
+      FileStatus status = fs.getFileStatus(p);
+      if (status.isDir()) {
+        FileStatus[] files = fs.listStatus(p);
+        Collections.addAll(inputFiles, files);
+      } else {
+        inputFiles.add(status);
+      }
+      return dumpTypedBytes(inputFiles);
+    }
+    return -1;
+  }
+
+  /**
+   * Dump given list of files to standard output as typed bytes.
+   */
+  @SuppressWarnings("unchecked")
+  private int dumpTypedBytes(List<FileStatus> files) throws IOException {
+    JobConf job = new JobConf(getConf()); 
+    DataOutputStream dout = new DataOutputStream(System.out);
+    AutoInputFormat autoInputFormat = new AutoInputFormat();
+    for (FileStatus fileStatus : files) {
+      FileSplit split = new FileSplit(fileStatus.getPath(), 0,
+        fileStatus.getLen() * fileStatus.getBlockSize(),
+        (String[]) null);
+      RecordReader recReader = null;
+      try {
+        recReader = autoInputFormat.getRecordReader(split, job, Reporter.NULL);
+        Object key = recReader.createKey();
+        Object value = recReader.createValue();
+        while (recReader.next(key, value)) {
+          if (key instanceof Writable) {
+            TypedBytesWritableOutput.get(dout).write((Writable) key);
+          } else {
+            TypedBytesOutput.get(dout).write(key);
+          }
+          if (value instanceof Writable) {
+            TypedBytesWritableOutput.get(dout).write((Writable) value);
+          } else {
+            TypedBytesOutput.get(dout).write(value);
+          }
+        }
+      } finally {
+        if (recReader != null) {
+          recReader.close();
+        }
+      }
+    }
+    dout.flush();
+    return 0;
+  }
+
+  public static void main(String[] args) throws Exception {
+    DumpTypedBytes dumptb = new DumpTypedBytes();
+    int res = ToolRunner.run(dumptb, args);
+    System.exit(res);
+  }
+  
+}

+ 22 - 3
src/contrib/streaming/src/java/org/apache/hadoop/streaming/HadoopStreaming.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.streaming;
 
+import java.util.Arrays;
+
 import org.apache.hadoop.util.ToolRunner;
 
 /** The main entrypoint. Usually invoked with the script bin/hadoopStreaming
@@ -27,11 +29,28 @@ import org.apache.hadoop.util.ToolRunner;
 public class HadoopStreaming {
 
   public static void main(String[] args) throws Exception {
+    if (args.length < 1) {
+      System.err.println("No Arguments Given!");
+      System.exit(1);
+    }
     int returnStatus = 0;
-    StreamJob job = new StreamJob();
-    returnStatus = ToolRunner.run(job, args);
+    String cmd = args[0];
+    String[] remainingArgs = Arrays.copyOfRange(args, 1, args.length);
+    if (cmd.equalsIgnoreCase("dumptb")) {
+      DumpTypedBytes dumptb = new DumpTypedBytes();
+      returnStatus = ToolRunner.run(dumptb, remainingArgs);
+    } else if (cmd.equalsIgnoreCase("loadtb")) {
+      LoadTypedBytes loadtb = new LoadTypedBytes();
+      returnStatus = ToolRunner.run(loadtb, remainingArgs);
+    } else if (cmd.equalsIgnoreCase("streamjob")) {
+      StreamJob job = new StreamJob();
+      returnStatus = ToolRunner.run(job, remainingArgs);
+    } else { // for backward compatibility
+      StreamJob job = new StreamJob();
+      returnStatus = ToolRunner.run(job, args);
+    }
     if (returnStatus != 0) {
-      System.err.println("Streaming Job Failed!");
+      System.err.println("Streaming Command Failed!");
       System.exit(returnStatus);
     }
   }

+ 92 - 0
src/contrib/streaming/src/java/org/apache/hadoop/streaming/LoadTypedBytes.java

@@ -0,0 +1,92 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming;
+
+import java.io.DataInputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.typedbytes.TypedBytesInput;
+import org.apache.hadoop.typedbytes.TypedBytesWritable;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * Utility program that reads typed bytes from standard input and stores them in
+ * a sequence file for which the path is given as an argument.
+ */
+public class LoadTypedBytes implements Tool {
+
+  private Configuration conf;
+
+  public LoadTypedBytes(Configuration conf) {
+    this.conf = conf;
+  }
+  
+  public LoadTypedBytes() {
+    this(new Configuration());
+  }
+  
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+  
+  /**
+   * The main driver for <code>LoadTypedBytes</code>.
+   */
+  public int run(String[] args) throws Exception {
+    Path path = new Path(args[0]);
+    FileSystem fs = path.getFileSystem(getConf());
+    if (fs.exists(path)) {
+      System.err.println("given path exists already!");
+      return -1;
+    }
+    TypedBytesInput tbinput = new TypedBytesInput(new DataInputStream(System.in));
+    SequenceFile.Writer writer = SequenceFile.createWriter(fs, conf, path,
+      TypedBytesWritable.class, TypedBytesWritable.class);
+    try {
+      TypedBytesWritable key = new TypedBytesWritable();
+      TypedBytesWritable value = new TypedBytesWritable();
+      byte[] rawKey = tbinput.readRaw();
+      while (rawKey != null) {
+        byte[] rawValue = tbinput.readRaw();
+        key.set(rawKey, 0, rawKey.length);
+        value.set(rawValue, 0, rawValue.length);
+        writer.append(key, value);
+        rawKey = tbinput.readRaw();
+      }
+    } finally {
+      writer.close();
+    }
+    return 0;
+  }
+
+  public static void main(String[] args) throws Exception {
+    LoadTypedBytes loadtb = new LoadTypedBytes();
+    int res = ToolRunner.run(loadtb, args);
+    System.exit(res);
+  }
+
+}

+ 108 - 97
src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapRed.java

@@ -19,8 +19,6 @@
 package org.apache.hadoop.streaming;
 
 import java.io.*;
-import java.nio.charset.CharacterCodingException;
-import java.io.IOException;
 import java.util.Date;
 import java.util.Map;
 import java.util.Iterator;
@@ -30,16 +28,20 @@ import java.util.Properties;
 
 import org.apache.commons.logging.*;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.streaming.io.InputWriter;
+import org.apache.hadoop.streaming.io.OutputReader;
+import org.apache.hadoop.streaming.io.TextInputWriter;
+import org.apache.hadoop.streaming.io.TextOutputReader;
 import org.apache.hadoop.util.LineReader;
+import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.UTF8ByteArrayUtils;
 
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.BytesWritable;
 
 import org.apache.hadoop.fs.FileSystem;
 
@@ -49,15 +51,49 @@ public abstract class PipeMapRed {
 
   protected static final Log LOG = LogFactory.getLog(PipeMapRed.class.getName());
 
-  /** The command to be spawned as a subprocess.
-   * Mapper/Reducer operations will delegate to it
+  /**
+   * Returns the Configuration.
    */
-  abstract String getPipeCommand(JobConf job);
-
-  abstract byte[] getFieldSeparator();
+  public Configuration getConfiguration() {
+    return job_;
+  }
+  
+  /**
+   * Returns the DataOutput to which the client input is written.
+   */
+  public DataOutput getClientOutput() {
+    return clientOut_;
+  }
+  
+  /**
+   * Returns the DataInput from which the client output is read.
+   */
+  public DataInput getClientInput() {
+    return clientIn_;
+  }
+  
+  /**
+   * Returns the input separator to be used.
+   */
+  public abstract byte[] getInputSeparator();
+  
+  /**
+   * Returns the field separator to be used.
+   */
+  public abstract byte[] getFieldSeparator();
 
-  abstract int getNumOfKeyFields();
+  /**
+   * Returns the number of key fields.
+   */
+  public abstract int getNumOfKeyFields();
 
+  
+  /** 
+   * Returns the command to be spawned as a subprocess.
+   * Mapper/Reducer operations will delegate to it
+   */
+  abstract String getPipeCommand(JobConf job);
+  
   abstract boolean getDoPipe();
 
   final static int OUTSIDE = 1;
@@ -120,7 +156,19 @@ public abstract class PipeMapRed {
 
       job_ = job;
       fs_ = FileSystem.get(job_);
-
+      
+      mapInputWriterClass_ = 
+        job_.getClass("stream.map.input.writer.class", 
+          TextInputWriter.class, InputWriter.class);
+      mapOutputReaderClass_ = 
+        job_.getClass("stream.map.output.reader.class",
+          TextOutputReader.class, OutputReader.class);
+      reduceInputWriterClass_ = 
+        job_.getClass("stream.reduce.input.writer.class",
+          TextInputWriter.class, InputWriter.class);
+      reduceOutputReaderClass_ = 
+        job_.getClass("stream.reduce.output.reader.class",
+          TextOutputReader.class, OutputReader.class);
       nonZeroExitIsFailure_ = job_.getBoolean("stream.non.zero.exit.is.failure", true);
       
       doPipe_ = getDoPipe();
@@ -280,13 +328,16 @@ public abstract class PipeMapRed {
     }
   }
 
-  void startOutputThreads(OutputCollector output, Reporter reporter) {
-    outThread_ = new MROutputThread(output, reporter);
+  void startOutputThreads(OutputCollector output, Reporter reporter) 
+    throws IOException {
+    inWriter_ = createInputWriter();
+    outReader_ = createOutputReader();
+    outThread_ = new MROutputThread(outReader_, output, reporter);
     outThread_.start();
     errThread_.setReporter(reporter);
   }
-
-  void waitOutputThreads() {
+  
+  void waitOutputThreads() throws IOException {
     try {
       if (outThread_ == null) {
         // This happens only when reducer has empty input(So reduce() is not
@@ -328,58 +379,46 @@ public abstract class PipeMapRed {
       //ignore
     }
   }
-
-  /**
-   * Split a line into key and value.
-   * @param line: a byte array of line containing UTF-8 bytes
-   * @param key: key of a record
-   * @param val: value of a record
-   * @throws IOException
-   */
-  void splitKeyVal(byte[] line, int length, Text key, Text val)
-  throws IOException {
-    int numKeyFields = getNumOfKeyFields();
-    byte[] separator = getFieldSeparator();
-    
-    // Need to find numKeyFields separators
-    int pos = UTF8ByteArrayUtils.findBytes(line, 0, length, separator);
-    for(int k=1; k<numKeyFields && pos!=-1; k++) {
-      pos = UTF8ByteArrayUtils.findBytes(line, pos + separator.length, 
-          length, separator);
-    }
-    try {
-      if (pos == -1) {
-        key.set(line, 0, length);
-        val.set("");
-      } else {
-        StreamKeyValUtil.splitKeyVal(line, 0, length, key, val, pos, separator.length);
-      }
-    } catch (CharacterCodingException e) {
-      LOG.warn(StringUtils.stringifyException(e));
-    }
+  
+  
+  abstract InputWriter createInputWriter() throws IOException;
+  
+  InputWriter createInputWriter(Class<? extends InputWriter> inputWriterClass) 
+    throws IOException {
+    InputWriter inputWriter =
+      ReflectionUtils.newInstance(inputWriterClass, job_);
+    inputWriter.initialize(this);
+    return inputWriter;
   }
 
+  abstract OutputReader createOutputReader() throws IOException;
+
+  OutputReader createOutputReader(Class<? extends OutputReader> outputReaderClass) 
+    throws IOException {
+    OutputReader outputReader =
+      ReflectionUtils.newInstance(outputReaderClass, job_);
+    outputReader.initialize(this);
+    return outputReader;
+  }
+  
+  
   class MROutputThread extends Thread {
 
-    MROutputThread(OutputCollector output, Reporter reporter) {
+    MROutputThread(OutputReader outReader, OutputCollector outCollector,
+      Reporter reporter) {
       setDaemon(true);
-      this.output = output;
+      this.outReader = outReader;
+      this.outCollector = outCollector;
       this.reporter = reporter;
     }
 
     public void run() {
-      LineReader lineReader = null;
       try {
-        Text key = new Text();
-        Text val = new Text();
-        Text line = new Text();
-        lineReader = new LineReader((InputStream)clientIn_, job_);
         // 3/4 Tool to Hadoop
-        while (lineReader.readLine(line) > 0) {
-          answer = line.getBytes();
-          splitKeyVal(answer, line.getLength(), key, val);
-          output.collect(key, val);
-          line.clear();
+        while (outReader.readKeyValue()) {
+          Object key = outReader.getCurrentKey();
+          Object value = outReader.getCurrentValue();
+          outCollector.collect(key, value);
           numRecWritten_++;
           long now = System.currentTimeMillis();
           if (now-lastStdoutReport > reporterOutDelay_) {
@@ -394,21 +433,11 @@ public abstract class PipeMapRed {
             logflush();
           }
         }
-        if (lineReader != null) {
-          lineReader.close();
-        }
-        if (clientIn_ != null) {
-          clientIn_.close();
-          clientIn_ = null;
-          LOG.info("MROutputThread done");
-        }
       } catch (Throwable th) {
         outerrThreadsThrowable = th;
         LOG.warn(StringUtils.stringifyException(th));
+      } finally {
         try {
-          if (lineReader != null) {
-            lineReader.close();
-          }
           if (clientIn_ != null) {
             clientIn_.close();
             clientIn_ = null;
@@ -419,9 +448,9 @@ public abstract class PipeMapRed {
       }
     }
 
-    OutputCollector output;
-    Reporter reporter;
-    byte[] answer;
+    OutputReader outReader = null;
+    OutputCollector outCollector = null;
+    Reporter reporter = null;
     long lastStdoutReport = 0;
     
   }
@@ -540,9 +569,10 @@ public abstract class PipeMapRed {
           clientOut_.flush();
           clientOut_.close();
         }
+        waitOutputThreads();
       } catch (IOException io) {
+        LOG.warn(StringUtils.stringifyException(io));
       }
-      waitOutputThreads();
       if (sim != null) sim.destroy();
       logprintln("mapRedFinished");
     } catch (RuntimeException e) {
@@ -579,7 +609,7 @@ public abstract class PipeMapRed {
     //s += envline("PWD"); // =/home/crawler/hadoop/trunk
     s += "last Hadoop input: |" + mapredKey_ + "|\n";
     if (outThread_ != null) {
-      s += "last tool output: |" + outThread_.answer + "|\n";
+      s += "last tool output: |" + outReader_.getLastOutput() + "|\n";
     }
     s += "Date: " + new Date() + "\n";
     // s += envline("HADOOP_HOME");
@@ -611,37 +641,12 @@ public abstract class PipeMapRed {
     return msg;
   }
 
-  /**
-   * Write a value to the output stream using UTF-8 encoding
-   * @param value output value
-   * @throws IOException
-   */
-  void write(Object value) throws IOException {
-    byte[] bval;
-    int valSize;
-    if (value instanceof BytesWritable) {
-      BytesWritable val = (BytesWritable) value;
-      bval = val.getBytes();
-      valSize = val.getLength();
-    } 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;
   long numRecSkipped_ = 0;
   long nextRecReadLog_ = 1;
 
-  
   long minRecWrittenToEnableSkip_ = Long.MAX_VALUE;
 
   long reporterOutDelay_ = 10*1000L; 
@@ -656,9 +661,15 @@ public abstract class PipeMapRed {
   boolean debugFailDuring_;
   boolean debugFailLate_;
 
+  Class<? extends InputWriter> mapInputWriterClass_;
+  Class<? extends OutputReader> mapOutputReaderClass_;
+  Class<? extends InputWriter> reduceInputWriterClass_;
+  Class<? extends OutputReader> reduceOutputReaderClass_;
   boolean nonZeroExitIsFailure_;
   
   Process sim;
+  InputWriter inWriter_;
+  OutputReader outReader_;
   MROutputThread outThread_;
   String jobLog_;
   MRErrorThread errThread_;

+ 24 - 10
src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapper.java

@@ -27,6 +27,9 @@ import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.SkipBadRecords;
 import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.streaming.io.InputWriter;
+import org.apache.hadoop.streaming.io.OutputReader;
+import org.apache.hadoop.streaming.io.TextInputWriter;
 import org.apache.hadoop.util.StringUtils;
 
 /** A generic Mapper bridge.
@@ -66,9 +69,11 @@ public class PipeMapper extends PipeMapRed implements Mapper {
     //records input.
     SkipBadRecords.setAutoIncrMapperProcCount(job, false);
     skipping = job.getBoolean("mapred.skip.on", false);
-    String inputFormatClassName = job.getClass("mapred.input.format.class", TextInputFormat.class).getCanonicalName();
-    ignoreKey = inputFormatClassName.equals(TextInputFormat.class.getCanonicalName());
-
+    if (mapInputWriterClass_.getCanonicalName().equals(TextInputWriter.class.getCanonicalName())) {
+      String inputFormatClassName = job.getClass("mapred.input.format.class", TextInputFormat.class).getCanonicalName();
+      ignoreKey = inputFormatClassName.equals(TextInputFormat.class.getCanonicalName());
+    }
+    
     try {
       mapOutputFieldSeparator = job.get("stream.map.output.field.separator", "\t").getBytes("UTF-8");
       mapInputFieldSeparator = job.get("stream.map.input.field.separator", "\t").getBytes("UTF-8");
@@ -99,11 +104,9 @@ public class PipeMapper extends PipeMapRed implements Mapper {
       // 2/4 Hadoop to Tool
       if (numExceptions_ == 0) {
         if (!this.ignoreKey) {
-          write(key);
-          clientOut_.write(getInputSeparator());
+          inWriter_.writeKey(key);
         }
-        write(value);
-        clientOut_.write('\n');
+        inWriter_.writeValue(value);
         if(skipping) {
           //flush the streams on every record input if running in skip mode
           //so that we don't buffer other records surrounding a bad record. 
@@ -132,18 +135,29 @@ public class PipeMapper extends PipeMapRed implements Mapper {
     mapRedFinished();
   }
 
-  byte[] getInputSeparator() {
+  @Override
+  public byte[] getInputSeparator() {
     return mapInputFieldSeparator;
   }
 
   @Override
-  byte[] getFieldSeparator() {
+  public byte[] getFieldSeparator() {
     return mapOutputFieldSeparator;
   }
 
   @Override
-  int getNumOfKeyFields() {
+  public int getNumOfKeyFields() {
     return numOfMapOutputKeyFields;
   }
 
+  @Override
+  InputWriter createInputWriter() throws IOException {
+    return super.createInputWriter(mapInputWriterClass_);
+  }
+
+  @Override
+  OutputReader createOutputReader() throws IOException {
+    return super.createOutputReader(mapOutputReaderClass_);
+  }
+
 }

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

@@ -28,6 +28,8 @@ import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.SkipBadRecords;
+import org.apache.hadoop.streaming.io.InputWriter;
+import org.apache.hadoop.streaming.io.OutputReader;
 import org.apache.hadoop.util.StringUtils;
 
 import org.apache.hadoop.io.Writable;
@@ -97,10 +99,8 @@ public class PipeReducer extends PipeMapRed implements Reducer {
                                    + StringUtils.stringifyException(
                                                                     outerrThreadsThrowable));
           }
-          write(key);
-          clientOut_.write(getInputSeparator());
-          write(val);
-          clientOut_.write('\n');
+          inWriter_.writeKey(key);
+          inWriter_.writeValue(val);
         } else {
           // "identity reduce"
           output.collect(key, val);
@@ -137,18 +137,29 @@ public class PipeReducer extends PipeMapRed implements Reducer {
     mapRedFinished();
   }
 
-  byte[] getInputSeparator() {
+  @Override
+  public byte[] getInputSeparator() {
     return reduceInputFieldSeparator;
   }
 
   @Override
-  byte[] getFieldSeparator() {
+  public byte[] getFieldSeparator() {
     return reduceOutFieldSeparator;
   }
   
   @Override
-  int getNumOfKeyFields() {
+  public int getNumOfKeyFields() {
     return numOfReduceOutputKeyFields;
   }
+  
+  @Override
+  InputWriter createInputWriter() throws IOException {
+    return super.createInputWriter(reduceInputWriterClass_);
+  }
+
+  @Override
+  OutputReader createOutputReader() throws IOException {
+    return super.createOutputReader(reduceOutputReaderClass_);
+  }
 
 }

+ 42 - 4
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java

@@ -47,7 +47,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.FileAlreadyExistsException;
 import org.apache.hadoop.mapred.FileInputFormat;
 import org.apache.hadoop.mapred.FileOutputFormat;
@@ -63,7 +62,11 @@ import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.hadoop.mapred.lib.aggregate.ValueAggregatorCombiner;
 import org.apache.hadoop.mapred.lib.aggregate.ValueAggregatorReducer;
+import org.apache.hadoop.streaming.io.IdentifierResolver;
+import org.apache.hadoop.streaming.io.InputWriter;
+import org.apache.hadoop.streaming.io.OutputReader;
 import org.apache.hadoop.util.GenericOptionsParser;
+import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 
@@ -284,6 +287,7 @@ public class StreamJob implements Tool {
       inReaderSpec_ = (String)cmdLine.getOptionValue("inputreader"); 
       mapDebugSpec_ = (String)cmdLine.getOptionValue("mapdebug");    
       reduceDebugSpec_ = (String)cmdLine.getOptionValue("reducedebug");
+      ioSpec_ = (String)cmdLine.getOptionValue("io");
       
       String[] car = cmdLine.getOptionValues("cacheArchive");
       if (null != car && car.length > 0){
@@ -408,6 +412,8 @@ public class StreamJob implements Tool {
                                     "File name URI", "fileNameURI", Integer.MAX_VALUE, false);
     Option cacheArchive = createOption("cacheArchive", 
                                        "File name URI", "fileNameURI", Integer.MAX_VALUE, false);
+    Option io = createOption("io",
+                             "Optional.", "spec", 1, false);
     
     // boolean properties
     
@@ -437,6 +443,7 @@ public class StreamJob implements Tool {
       addOption(cmdenv).
       addOption(cacheFile).
       addOption(cacheArchive).
+      addOption(io).
       addOption(verbose).
       addOption(info).
       addOption(debug).
@@ -467,6 +474,7 @@ public class StreamJob implements Tool {
     "To run this script when a map task fails ");
     System.out.println("  -reducedebug <path>  Optional." +
     " To run this script when a reduce task fails ");
+    System.out.println("  -io <identifier>  Optional.");
     System.out.println("  -verbose");
     System.out.println();
     GenericOptionsParser.printGenericCommandUsage(System.out);
@@ -689,9 +697,38 @@ public class StreamJob implements Tool {
 
     jobConf_.setInputFormat(fmt);
 
-    jobConf_.setOutputKeyClass(Text.class);
-    jobConf_.setOutputValueClass(Text.class);
-
+    if (ioSpec_ != null) {
+      jobConf_.set("stream.map.input", ioSpec_);
+      jobConf_.set("stream.map.output", ioSpec_);
+      jobConf_.set("stream.reduce.input", ioSpec_);
+      jobConf_.set("stream.reduce.output", ioSpec_);
+    }
+    
+    Class<? extends IdentifierResolver> idResolverClass = 
+      jobConf_.getClass("stream.io.identifier.resolver.class",
+        IdentifierResolver.class, IdentifierResolver.class);
+    IdentifierResolver idResolver = ReflectionUtils.newInstance(idResolverClass, jobConf_);
+    
+    idResolver.resolve(jobConf_.get("stream.map.input", IdentifierResolver.TEXT_ID));
+    jobConf_.setClass("stream.map.input.writer.class",
+      idResolver.getInputWriterClass(), InputWriter.class);
+    
+    idResolver.resolve(jobConf_.get("stream.reduce.input", IdentifierResolver.TEXT_ID));
+    jobConf_.setClass("stream.reduce.input.writer.class",
+      idResolver.getInputWriterClass(), InputWriter.class);
+    
+    idResolver.resolve(jobConf_.get("stream.map.output", IdentifierResolver.TEXT_ID));
+    jobConf_.setClass("stream.map.output.reader.class",
+      idResolver.getOutputReaderClass(), OutputReader.class);
+    jobConf_.setMapOutputKeyClass(idResolver.getOutputKeyClass());
+    jobConf_.setMapOutputValueClass(idResolver.getOutputValueClass());
+    
+    idResolver.resolve(jobConf_.get("stream.reduce.output", IdentifierResolver.TEXT_ID));
+    jobConf_.setClass("stream.reduce.output.reader.class",
+      idResolver.getOutputReaderClass(), OutputReader.class);
+    jobConf_.setOutputKeyClass(idResolver.getOutputKeyClass());
+    jobConf_.setOutputValueClass(idResolver.getOutputValueClass());
+    
     jobConf_.set("stream.addenvironment", addTaskEnvironment_);
 
     if (mapCmd_ != null) {
@@ -962,6 +999,7 @@ public class StreamJob implements Tool {
   protected String additionalConfSpec_;
   protected String mapDebugSpec_;
   protected String reduceDebugSpec_;
+  protected String ioSpec_;
 
   // Use to communicate config to the external processes (ex env.var.HADOOP_USER)
   // encoding "a=b c=d"

+ 125 - 0
src/contrib/streaming/src/java/org/apache/hadoop/streaming/io/IdentifierResolver.java

@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming.io;
+
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.typedbytes.TypedBytesWritable;
+
+/**
+ * This class is used to resolve a string identifier into the required IO
+ * classes. By extending this class and pointing the property
+ * <tt>stream.io.identifier.resolver.class</tt> to this extension, additional
+ * IO classes can be added by external code.
+ */
+public class IdentifierResolver {
+
+  // note that the identifiers are case insensitive
+  public static final String TEXT_ID = "text";
+  public static final String RAW_BYTES_ID = "rawbytes";
+  public static final String TYPED_BYTES_ID = "typedbytes";
+  
+  private Class<? extends InputWriter> inputWriterClass = null;
+  private Class<? extends OutputReader> outputReaderClass = null;
+  private Class outputKeyClass = null;
+  private Class outputValueClass = null;
+  
+  /**
+   * Resolves a given identifier. This method has to be called before calling
+   * any of the getters.
+   */
+  public void resolve(String identifier) {
+    if (identifier.equalsIgnoreCase(RAW_BYTES_ID)) {
+      setInputWriterClass(RawBytesInputWriter.class);
+      setOutputReaderClass(RawBytesOutputReader.class);
+      setOutputKeyClass(BytesWritable.class);
+      setOutputValueClass(BytesWritable.class);
+    } else if (identifier.equalsIgnoreCase(TYPED_BYTES_ID)) {
+      setInputWriterClass(TypedBytesInputWriter.class);
+      setOutputReaderClass(TypedBytesOutputReader.class);
+      setOutputKeyClass(TypedBytesWritable.class);
+      setOutputValueClass(TypedBytesWritable.class);
+    } else { // assume TEXT_ID
+      setInputWriterClass(TextInputWriter.class);
+      setOutputReaderClass(TextOutputReader.class);
+      setOutputKeyClass(Text.class);
+      setOutputValueClass(Text.class);
+    }
+  }
+  
+  /**
+   * Returns the resolved {@link InputWriter} class.
+   */
+  public Class<? extends InputWriter> getInputWriterClass() {
+    return inputWriterClass;
+  }
+
+  /**
+   * Returns the resolved {@link OutputReader} class.
+   */
+  public Class<? extends OutputReader> getOutputReaderClass() {
+    return outputReaderClass;
+  }
+  
+  /**
+   * Returns the resolved output key class.
+   */
+  public Class getOutputKeyClass() {
+    return outputKeyClass;
+  }
+
+  /**
+   * Returns the resolved output value class.
+   */
+  public Class getOutputValueClass() {
+    return outputValueClass;
+  }
+  
+  
+  /**
+   * Sets the {@link InputWriter} class.
+   */
+  protected void setInputWriterClass(Class<? extends InputWriter>
+    inputWriterClass) {
+    this.inputWriterClass = inputWriterClass;
+  }
+  
+  /**
+   * Sets the {@link OutputReader} class.
+   */
+  protected void setOutputReaderClass(Class<? extends OutputReader>
+    outputReaderClass) {
+    this.outputReaderClass = outputReaderClass;
+  }
+  
+  /**
+   * Sets the output key class class.
+   */
+  protected void setOutputKeyClass(Class outputKeyClass) {
+    this.outputKeyClass = outputKeyClass;
+  }
+  
+  /**
+   * Sets the output value class.
+   */
+  protected void setOutputValueClass(Class outputValueClass) {
+    this.outputValueClass = outputValueClass;
+  }
+
+}

+ 48 - 0
src/contrib/streaming/src/java/org/apache/hadoop/streaming/io/InputWriter.java

@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming.io;
+
+import java.io.IOException;
+
+import org.apache.hadoop.streaming.PipeMapRed;
+
+/**
+ * Abstract base for classes that write the client's input.
+ */
+public abstract class InputWriter<K, V> {
+  
+  /**
+   * Initializes the InputWriter. This method has to be called before calling
+   * any of the other methods.
+   */
+  public void initialize(PipeMapRed pipeMapRed) throws IOException {
+    // nothing here yet, but that might change in the future
+  }
+  
+  /**
+   * Writes an input key.
+   */
+  public abstract void writeKey(K key) throws IOException;
+
+  /**
+   * Writes an input value.
+   */
+  public abstract void writeValue(V value) throws IOException;
+
+}

+ 59 - 0
src/contrib/streaming/src/java/org/apache/hadoop/streaming/io/OutputReader.java

@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming.io;
+
+import java.io.IOException;
+
+import org.apache.hadoop.streaming.PipeMapRed;
+
+/**
+ * Abstract base for classes that read the client's output.
+ */
+public abstract class OutputReader<K, V> {
+  
+  /**
+   * Initializes the OutputReader. This method has to be called before
+   * calling any of the other methods.
+   */
+  public void initialize(PipeMapRed pipeMapRed) throws IOException {
+    // nothing here yet, but that might change in the future
+  }
+  
+  /**
+   * Read the next key/value pair outputted by the client.
+   * @return true iff a key/value pair was read
+   */
+  public abstract boolean readKeyValue() throws IOException;
+  
+  /**
+   * Returns the current key.
+   */
+  public abstract K getCurrentKey() throws IOException;
+  
+  /**
+   * Returns the current value.
+   */
+  public abstract V getCurrentValue() throws IOException;
+  
+  /**
+   * Returns the last output from the client as a String.
+   */
+  public abstract String getLastOutput();
+  
+}

+ 73 - 0
src/contrib/streaming/src/java/org/apache/hadoop/streaming/io/RawBytesInputWriter.java

@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming.io;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.streaming.PipeMapRed;
+
+/**
+ * InputWriter that writes the client's input as raw bytes.
+ */
+public class RawBytesInputWriter extends InputWriter<Writable, Writable> {
+
+  private DataOutput clientOut;
+  private ByteArrayOutputStream bufferOut;
+  private DataOutputStream bufferDataOut;
+
+  @Override
+  public void initialize(PipeMapRed pipeMapRed) throws IOException {
+    super.initialize(pipeMapRed);
+    clientOut = pipeMapRed.getClientOutput();
+    bufferOut = new ByteArrayOutputStream();
+    bufferDataOut = new DataOutputStream(bufferOut);
+  }
+  
+  @Override
+  public void writeKey(Writable key) throws IOException {
+    writeRawBytes(key);
+  }
+
+  @Override
+  public void writeValue(Writable value) throws IOException {
+    writeRawBytes(value);
+  }
+
+  private void writeRawBytes(Writable writable) throws IOException {
+    if (writable instanceof BytesWritable) {
+      BytesWritable bw = (BytesWritable) writable;
+      byte[] bytes = bw.getBytes();
+      int length = bw.getLength();
+      clientOut.writeInt(length);
+      clientOut.write(bytes, 0, length);
+    } else {
+      bufferOut.reset();
+      writable.write(bufferDataOut);
+      byte[] bytes = bufferOut.toByteArray();
+      clientOut.writeInt(bytes.length);
+      clientOut.write(bytes);
+    }
+  }
+  
+}

+ 88 - 0
src/contrib/streaming/src/java/org/apache/hadoop/streaming/io/RawBytesOutputReader.java

@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming.io;
+
+import java.io.DataInput;
+import java.io.EOFException;
+import java.io.IOException;
+
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.streaming.PipeMapRed;
+
+/**
+ * OutputReader that reads the client's output as raw bytes.
+ */
+public class RawBytesOutputReader 
+  extends OutputReader<BytesWritable, BytesWritable> {
+
+  private DataInput clientIn;
+  private byte[] bytes;
+  private BytesWritable key;
+  private BytesWritable value;
+
+  @Override
+  public void initialize(PipeMapRed pipeMapRed) throws IOException {
+    super.initialize(pipeMapRed);
+    clientIn = pipeMapRed.getClientInput();
+    key = new BytesWritable();
+    value = new BytesWritable();
+  }
+  
+  @Override
+  public boolean readKeyValue() throws IOException {
+    int length = readLength();
+    if (length < 0) {
+      return false;
+    }
+    key.set(readBytes(length), 0, length);
+    length = readLength();
+    value.set(readBytes(length), 0, length);
+    return true;
+  }
+  
+  @Override
+  public BytesWritable getCurrentKey() throws IOException {
+    return key;
+  }
+  
+  @Override
+  public BytesWritable getCurrentValue() throws IOException {
+    return value;
+  }
+
+  @Override
+  public String getLastOutput() {
+    return new BytesWritable(bytes).toString();
+  }
+
+  private int readLength() throws IOException {
+    try {
+      return clientIn.readInt();
+    } catch (EOFException eof) {
+      return -1;
+    }
+  }
+  
+  private byte[] readBytes(int length) throws IOException {
+    bytes = new byte[length];
+    clientIn.readFully(bytes);
+    return bytes;
+  }
+  
+}

+ 75 - 0
src/contrib/streaming/src/java/org/apache/hadoop/streaming/io/TextInputWriter.java

@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming.io;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.streaming.PipeMapRed;
+
+/**
+ * InputWriter that writes the client's input as text.
+ */
+public class TextInputWriter extends InputWriter<Object, Object> {
+  
+  private DataOutput clientOut;
+  private byte[] inputSeparator;
+  
+  @Override
+  public void initialize(PipeMapRed pipeMapRed) throws IOException {
+    super.initialize(pipeMapRed);
+    clientOut = pipeMapRed.getClientOutput();
+    inputSeparator = pipeMapRed.getInputSeparator();
+  }
+  
+  @Override
+  public void writeKey(Object key) throws IOException {
+    writeUTF8(key);
+    clientOut.write(inputSeparator);
+  }
+
+  @Override
+  public void writeValue(Object value) throws IOException {
+    writeUTF8(value);
+    clientOut.write('\n');
+  }
+  
+  // Write an object to the output stream using UTF-8 encoding
+  private void writeUTF8(Object object) throws IOException {
+    byte[] bval;
+    int valSize;
+    if (object instanceof BytesWritable) {
+      BytesWritable val = (BytesWritable) object;
+      bval = val.getBytes();
+      valSize = val.getLength();
+    } else if (object instanceof Text) {
+      Text val = (Text) object;
+      bval = val.getBytes();
+      valSize = val.getLength();
+    } else {
+      String sval = object.toString();
+      bval = sval.getBytes("UTF-8");
+      valSize = bval.length;
+    }
+    clientOut.write(bval, 0, valSize);
+  }
+  
+}

+ 115 - 0
src/contrib/streaming/src/java/org/apache/hadoop/streaming/io/TextOutputReader.java

@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming.io;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.UnsupportedEncodingException;
+import java.nio.charset.CharacterCodingException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.streaming.PipeMapRed;
+import org.apache.hadoop.streaming.StreamKeyValUtil;
+import org.apache.hadoop.util.LineReader;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.UTF8ByteArrayUtils;
+
+/**
+ * OutputReader that reads the client's output as text.
+ */
+public class TextOutputReader extends OutputReader<Text, Text> {
+
+  private LineReader lineReader;
+  private byte[] bytes;
+  private DataInput clientIn;
+  private Configuration conf;
+  private int numKeyFields;
+  private byte[] separator;
+  private Text key;
+  private Text value;
+  private Text line;
+  
+  @Override
+  public void initialize(PipeMapRed pipeMapRed) throws IOException {
+    super.initialize(pipeMapRed);
+    clientIn = pipeMapRed.getClientInput();
+    conf = pipeMapRed.getConfiguration();
+    numKeyFields = pipeMapRed.getNumOfKeyFields();
+    separator = pipeMapRed.getFieldSeparator();
+    lineReader = new LineReader((InputStream)clientIn, conf);
+    key = new Text();
+    value = new Text();
+    line = new Text();
+  }
+  
+  @Override
+  public boolean readKeyValue() throws IOException {
+    if (lineReader.readLine(line) <= 0) {
+      return false;
+    }
+    bytes = line.getBytes();
+    splitKeyVal(bytes, line.getLength(), key, value);
+    line.clear();
+    return true;
+  }
+  
+  @Override
+  public Text getCurrentKey() throws IOException {
+    return key;
+  }
+  
+  @Override
+  public Text getCurrentValue() throws IOException {
+    return value;
+  }
+
+  @Override
+  public String getLastOutput() {
+    try {
+      return new String(bytes, "UTF-8");
+    } catch (UnsupportedEncodingException e) {
+      return "<undecodable>";
+    }
+  }
+
+  // split a UTF-8 line into key and value
+  private void splitKeyVal(byte[] line, int length, Text key, Text val)
+    throws IOException {
+    // Need to find numKeyFields separators
+    int pos = UTF8ByteArrayUtils.findBytes(line, 0, length, separator);
+    for(int k=1; k<numKeyFields && pos!=-1; k++) {
+      pos = UTF8ByteArrayUtils.findBytes(line, pos + separator.length, 
+        length, separator);
+    }
+    try {
+      if (pos == -1) {
+        key.set(line, 0, length);
+        val.set("");
+      } else {
+        StreamKeyValUtil.splitKeyVal(line, 0, length, key, val, pos,
+          separator.length);
+      }
+    } catch (CharacterCodingException e) {
+      throw new IOException(StringUtils.stringifyException(e));
+    }
+  }
+  
+}

+ 63 - 0
src/contrib/streaming/src/java/org/apache/hadoop/streaming/io/TypedBytesInputWriter.java

@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming.io;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.streaming.PipeMapRed;
+import org.apache.hadoop.typedbytes.TypedBytesOutput;
+import org.apache.hadoop.typedbytes.TypedBytesWritableOutput;
+
+/**
+ * InputWriter that writes the client's input as typed bytes.
+ */
+public class TypedBytesInputWriter extends InputWriter<Object, Object> {
+
+  private TypedBytesOutput tbOut;
+  private TypedBytesWritableOutput tbwOut;
+
+  @Override
+  public void initialize(PipeMapRed pipeMapRed) throws IOException {
+    super.initialize(pipeMapRed);
+    DataOutput clientOut = pipeMapRed.getClientOutput();
+    tbOut = new TypedBytesOutput(clientOut);
+    tbwOut = new TypedBytesWritableOutput(clientOut);
+  }
+
+  @Override
+  public void writeKey(Object key) throws IOException {
+    writeTypedBytes(key);
+  }
+
+  @Override
+  public void writeValue(Object value) throws IOException {
+    writeTypedBytes(value);
+  }
+  
+  private void writeTypedBytes(Object value) throws IOException {
+    if (value instanceof Writable) {
+      tbwOut.write((Writable) value);
+    } else {
+      tbOut.write(value);
+    }
+  }
+  
+}

+ 76 - 0
src/contrib/streaming/src/java/org/apache/hadoop/streaming/io/TypedBytesOutputReader.java

@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming.io;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.hadoop.streaming.PipeMapRed;
+import org.apache.hadoop.typedbytes.TypedBytesInput;
+import org.apache.hadoop.typedbytes.TypedBytesWritable;
+
+/**
+ * OutputReader that reads the client's output as typed bytes.
+ */
+public class TypedBytesOutputReader extends 
+  OutputReader<TypedBytesWritable, TypedBytesWritable> {
+
+  private byte[] bytes;
+  private DataInput clientIn;
+  private TypedBytesWritable key;
+  private TypedBytesWritable value;
+  private TypedBytesInput in;
+  
+  @Override
+  public void initialize(PipeMapRed pipeMapRed) throws IOException {
+    super.initialize(pipeMapRed);
+    clientIn = pipeMapRed.getClientInput();
+    key = new TypedBytesWritable();
+    value = new TypedBytesWritable();
+    in = new TypedBytesInput(clientIn);
+  }
+  
+  @Override
+  public boolean readKeyValue() throws IOException {
+    bytes = in.readRaw();
+    if (bytes == null) {
+      return false;
+    }
+    key.set(bytes, 0, bytes.length);
+    bytes = in.readRaw();
+    value.set(bytes, 0, bytes.length);
+    return true;
+  }
+  
+  @Override
+  public TypedBytesWritable getCurrentKey() throws IOException {
+    return key;
+  }
+  
+  @Override
+  public TypedBytesWritable getCurrentValue() throws IOException {
+    return value;
+  }
+
+  @Override
+  public String getLastOutput() {
+    return new TypedBytesWritable(bytes).toString();
+  }
+
+}

+ 44 - 0
src/contrib/streaming/src/java/org/apache/hadoop/typedbytes/Type.java

@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.typedbytes;
+
+/**
+ * The possible type codes.
+ */
+public enum Type {
+
+  BYTES(0),
+  BYTE(1),
+  BOOL(2),
+  INT(3),
+  LONG(4),
+  FLOAT(5),
+  DOUBLE(6),
+  STRING(7),
+  VECTOR(8),
+  LIST(9),
+  MAP(10),
+  MARKER(255);
+
+  final int code;
+
+  Type(int code) {
+    this.code = code;
+  }
+}

+ 490 - 0
src/contrib/streaming/src/java/org/apache/hadoop/typedbytes/TypedBytesInput.java

@@ -0,0 +1,490 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.typedbytes;
+
+import java.io.DataInput;
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.TreeMap;
+
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.record.Buffer;
+
+/**
+ * Provides functionality for reading typed bytes.
+ */
+public class TypedBytesInput {
+
+  private DataInput in;
+
+  private TypedBytesInput() {}
+
+  private void setDataInput(DataInput in) {
+    this.in = in;
+  }
+
+  private static ThreadLocal tbIn = new ThreadLocal() {
+    protected synchronized Object initialValue() {
+      return new TypedBytesInput();
+    }
+  };
+
+  /**
+   * Get a thread-local typed bytes input for the supplied {@link DataInput}.
+   * @param in data input object
+   * @return typed bytes input corresponding to the supplied {@link DataInput}.
+   */
+  public static TypedBytesInput get(DataInput in) {
+    TypedBytesInput bin = (TypedBytesInput) tbIn.get();
+    bin.setDataInput(in);
+    return bin;
+  }
+
+  /** Creates a new instance of TypedBytesInput. */
+  public TypedBytesInput(DataInput in) {
+    this.in = in;
+  }
+
+  /**
+   * Reads a typed bytes sequence and converts it to a Java object. The first 
+   * byte is interpreted as a type code, and then the right number of 
+   * subsequent bytes are read depending on the obtained type.
+   * @return the obtained object or null when the end of the file is reached
+   * @throws IOException
+   */
+  public Object read() throws IOException {
+    int code = 1;
+    try {
+      code = in.readUnsignedByte();
+    } catch (EOFException eof) {
+      return null;
+    }
+    if (code == Type.BYTES.code) {
+      return new Buffer(readBytes());
+    } else if (code == Type.BYTE.code) {
+      return readByte();
+    } else if (code == Type.BOOL.code) {
+      return readBool();
+    } else if (code == Type.INT.code) {
+      return readInt();
+    } else if (code == Type.LONG.code) {
+      return readLong();
+    } else if (code == Type.FLOAT.code) {
+      return readFloat();
+    } else if (code == Type.DOUBLE.code) {
+      return readDouble();
+    } else if (code == Type.STRING.code) {
+      return readString();
+    } else if (code == Type.VECTOR.code) {
+      return readVector();
+    } else if (code == Type.LIST.code) {
+      return readList();
+    } else if (code == Type.MAP.code) {
+      return readMap();
+    } else if (code == Type.MARKER.code) {
+      return null;
+    } else {
+      throw new RuntimeException("unknown type");
+    }
+  }
+
+  /**
+   * Reads a typed bytes sequence. The first byte is interpreted as a type code,
+   * and then the right number of subsequent bytes are read depending on the
+   * obtained type.
+   * 
+   * @return the obtained typed bytes sequence or null when the end of the file
+   *         is reached
+   * @throws IOException
+   */
+  public byte[] readRaw() throws IOException {
+    int code = -1;
+    try {
+      code = in.readUnsignedByte();
+    } catch (EOFException eof) {
+      return null;
+    }
+    if (code == Type.BYTES.code) {
+      return readRawBytes();
+    } else if (code == Type.BYTE.code) {
+      return readRawByte();
+    } else if (code == Type.BOOL.code) {
+      return readRawBool();
+    } else if (code == Type.INT.code) {
+      return readRawInt();
+    } else if (code == Type.LONG.code) {
+      return readRawLong();
+    } else if (code == Type.FLOAT.code) {
+      return readRawFloat();
+    } else if (code == Type.DOUBLE.code) {
+      return readRawDouble();
+    } else if (code == Type.STRING.code) {
+      return readRawString();
+    } else if (code == Type.VECTOR.code) {
+      return readRawVector();
+    } else if (code == Type.LIST.code) {
+      return readRawList();
+    } else if (code == Type.MAP.code) {
+      return readRawMap();
+    } else if (code == Type.MARKER.code) {
+      return null;
+    } else {
+      throw new RuntimeException("unknown type");
+    }
+  }
+
+  /**
+   * Reads a type byte and returns the corresponding {@link Type}.
+   * @return the obtained Type or null when the end of the file is reached
+   * @throws IOException
+   */
+  public Type readType() throws IOException {
+    int code = -1;
+    try {
+      code = in.readUnsignedByte();
+    } catch (EOFException eof) {
+      return null;
+    }
+    for (Type type : Type.values()) {
+      if (type.code == code) {
+        return type;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Skips a type byte.
+   * @return true iff the end of the file was not reached
+   * @throws IOException
+   */
+  public boolean skipType() throws IOException {
+    try {
+      in.readByte();
+      return true;
+    } catch (EOFException eof) {
+      return false;
+    }
+  }
+
+  /**
+   * Reads the bytes following a <code>Type.BYTES</code> code.
+   * @return the obtained bytes sequence
+   * @throws IOException
+   */
+  public byte[] readBytes() throws IOException {
+    int length = in.readInt();
+    byte[] bytes = new byte[length];
+    in.readFully(bytes);
+    return bytes;
+  }
+
+  /**
+   * Reads the raw bytes following a <code>Type.BYTES</code> code.
+   * @return the obtained bytes sequence
+   * @throws IOException
+   */
+  public byte[] readRawBytes() throws IOException {
+    int length = in.readInt();
+    byte[] bytes = new byte[5 + length];
+    bytes[0] = (byte) Type.BYTES.code;
+    bytes[1] = (byte) (0xff & (length >> 24));
+    bytes[2] = (byte) (0xff & (length >> 16));
+    bytes[3] = (byte) (0xff & (length >> 8));
+    bytes[4] = (byte) (0xff & length);
+    in.readFully(bytes, 5, length);
+    return bytes;
+  }
+
+  /**
+   * Reads the byte following a <code>Type.BYTE</code> code.
+   * @return the obtained byte
+   * @throws IOException
+   */
+  public byte readByte() throws IOException {
+    return in.readByte();
+  }
+
+  /**
+   * Reads the raw byte following a <code>Type.BYTE</code> code.
+   * @return the obtained byte
+   * @throws IOException
+   */
+  public byte[] readRawByte() throws IOException {
+    byte[] bytes = new byte[2];
+    bytes[0] = (byte) Type.BYTE.code;
+    in.readFully(bytes, 1, 1);
+    return bytes;
+  }
+
+  /**
+   * Reads the boolean following a <code>Type.BOOL</code> code.
+   * @return the obtained boolean
+   * @throws IOException
+   */
+  public boolean readBool() throws IOException {
+    return in.readBoolean();
+  }
+
+  /**
+   * Reads the raw bytes following a <code>Type.BOOL</code> code.
+   * @return the obtained bytes sequence
+   * @throws IOException
+   */
+  public byte[] readRawBool() throws IOException {
+    byte[] bytes = new byte[2];
+    bytes[0] = (byte) Type.BOOL.code;
+    in.readFully(bytes, 1, 1);
+    return bytes;
+  }
+
+  /**
+   * Reads the integer following a <code>Type.INT</code> code.
+   * @return the obtained integer
+   * @throws IOException
+   */
+  public int readInt() throws IOException {
+    return in.readInt();
+  }
+
+  /**
+   * Reads the raw bytes following a <code>Type.INT</code> code.
+   * @return the obtained bytes sequence
+   * @throws IOException
+   */
+  public byte[] readRawInt() throws IOException {
+    byte[] bytes = new byte[5];
+    bytes[0] = (byte) Type.INT.code;
+    in.readFully(bytes, 1, 4);
+    return bytes;
+  }
+
+  /**
+   * Reads the long following a <code>Type.LONG</code> code.
+   * @return the obtained long
+   * @throws IOException
+   */
+  public long readLong() throws IOException {
+    return in.readLong();
+  }
+
+  /**
+   * Reads the raw bytes following a <code>Type.LONG</code> code.
+   * @return the obtained bytes sequence
+   * @throws IOException
+   */
+  public byte[] readRawLong() throws IOException {
+    byte[] bytes = new byte[9];
+    bytes[0] = (byte) Type.LONG.code;
+    in.readFully(bytes, 1, 8);
+    return bytes;
+  }
+
+  /**
+   * Reads the float following a <code>Type.FLOAT</code> code.
+   * @return the obtained float
+   * @throws IOException
+   */
+  public float readFloat() throws IOException {
+    return in.readFloat();
+  }
+
+  /**
+   * Reads the raw bytes following a <code>Type.FLOAT</code> code.
+   * @return the obtained bytes sequence
+   * @throws IOException
+   */
+  public byte[] readRawFloat() throws IOException {
+    byte[] bytes = new byte[5];
+    bytes[0] = (byte) Type.FLOAT.code;
+    in.readFully(bytes, 1, 4);
+    return bytes;
+  }
+
+  /**
+   * Reads the double following a <code>Type.DOUBLE</code> code.
+   * @return the obtained double
+   * @throws IOException
+   */
+  public double readDouble() throws IOException {
+    return in.readDouble();
+  }
+
+  /**
+   * Reads the raw bytes following a <code>Type.DOUBLE</code> code.
+   * @return the obtained bytes sequence
+   * @throws IOException
+   */
+  public byte[] readRawDouble() throws IOException {
+    byte[] bytes = new byte[9];
+    bytes[0] = (byte) Type.DOUBLE.code;
+    in.readFully(bytes, 1, 8);
+    return bytes;
+  }
+
+  /**
+   * Reads the string following a <code>Type.STRING</code> code.
+   * @return the obtained string
+   * @throws IOException
+   */
+  public String readString() throws IOException {
+    return WritableUtils.readString(in);
+  }
+
+  /**
+   * Reads the raw bytes following a <code>Type.STRING</code> code.
+   * @return the obtained bytes sequence
+   * @throws IOException
+   */
+  public byte[] readRawString() throws IOException {
+    int length = in.readInt();
+    byte[] bytes = new byte[5 + length];
+    bytes[0] = (byte) Type.STRING.code;
+    bytes[1] = (byte) (0xff & (length >> 24));
+    bytes[2] = (byte) (0xff & (length >> 16));
+    bytes[3] = (byte) (0xff & (length >> 8));
+    bytes[4] = (byte) (0xff & length);
+    in.readFully(bytes, 5, length);
+    return bytes;
+  }
+
+  /**
+   * Reads the vector following a <code>Type.VECTOR</code> code.
+   * @return the obtained vector
+   * @throws IOException
+   */
+  @SuppressWarnings("unchecked")
+  public ArrayList readVector() throws IOException {
+    int length = readVectorHeader();
+    ArrayList result = new ArrayList(length);
+    for (int i = 0; i < length; i++) {
+      result.add(read());
+    }
+    return result;
+  }
+
+  /**
+   * Reads the raw bytes following a <code>Type.VECTOR</code> code.
+   * @return the obtained bytes sequence
+   * @throws IOException
+   */
+  public byte[] readRawVector() throws IOException {
+    Buffer buffer = new Buffer();
+    int length = readVectorHeader();
+    buffer.append(new byte[] {
+      (byte) Type.VECTOR.code,
+      (byte) (0xff & (length >> 24)), (byte) (0xff & (length >> 16)),
+      (byte) (0xff & (length >> 8)), (byte) (0xff & length)
+    });
+    for (int i = 0; i < length; i++) {
+      buffer.append(readRaw());
+    }
+    return buffer.get();
+  }
+
+  /**
+   * Reads the header following a <code>Type.VECTOR</code> code.
+   * @return the number of elements in the vector
+   * @throws IOException
+   */
+  public int readVectorHeader() throws IOException {
+    return in.readInt();
+  }
+
+  /**
+   * Reads the list following a <code>Type.LIST</code> code.
+   * @return the obtained list
+   * @throws IOException
+   */
+  @SuppressWarnings("unchecked")
+  public List readList() throws IOException {
+    List list = new ArrayList();
+    Object obj = read();
+    while (obj != null) {
+      list.add(obj);
+      obj = read();
+    }
+    return list;
+  }
+
+  /**
+   * Reads the raw bytes following a <code>Type.LIST</code> code.
+   * @return the obtained bytes sequence
+   * @throws IOException
+   */
+  public byte[] readRawList() throws IOException {
+    Buffer buffer = new Buffer(new byte[] { (byte) Type.LIST.code });
+    byte[] bytes = readRaw();
+    while (bytes != null) {
+      buffer.append(bytes);
+      bytes = readRaw();
+    }
+    buffer.append(new byte[] { (byte) Type.MARKER.code });
+    return buffer.get();
+  }
+
+  /**
+   * Reads the map following a <code>Type.MAP</code> code.
+   * @return the obtained map
+   * @throws IOException
+   */
+  @SuppressWarnings("unchecked")
+  public TreeMap readMap() throws IOException {
+    int length = readMapHeader();
+    TreeMap result = new TreeMap();
+    for (int i = 0; i < length; i++) {
+      Object key = read();
+      Object value = read();
+      result.put(key, value);
+    }
+    return result;
+  }
+
+  /**
+   * Reads the raw bytes following a <code>Type.MAP</code> code.
+   * @return the obtained bytes sequence
+   * @throws IOException
+   */
+  public byte[] readRawMap() throws IOException {
+    Buffer buffer = new Buffer();
+    int length = readMapHeader();
+    buffer.append(new byte[] {
+      (byte) Type.MAP.code,
+      (byte) (0xff & (length >> 24)), (byte) (0xff & (length >> 16)),
+      (byte) (0xff & (length >> 8)), (byte) (0xff & length)
+    });
+    for (int i = 0; i < length; i++) {
+      buffer.append(readRaw());
+      buffer.append(readRaw());
+    }
+    return buffer.get();
+  }
+
+  /**
+   * Reads the header following a <code>Type.MAP</code> code.
+   * @return the number of key-value pairs in the map
+   * @throws IOException
+   */
+  public int readMapHeader() throws IOException {
+    return in.readInt();
+  }
+
+}

+ 298 - 0
src/contrib/streaming/src/java/org/apache/hadoop/typedbytes/TypedBytesOutput.java

@@ -0,0 +1,298 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.typedbytes;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.record.Buffer;
+
+/**
+ * Provides functionality for writing typed bytes.
+ */
+public class TypedBytesOutput {
+
+  private DataOutput out;
+
+  private TypedBytesOutput() {}
+
+  private void setDataOutput(DataOutput out) {
+    this.out = out;
+  }
+
+  private static ThreadLocal tbOut = new ThreadLocal() {
+    protected synchronized Object initialValue() {
+      return new TypedBytesOutput();
+    }
+  };
+
+  /**
+   * Get a thread-local typed bytes output for the supplied {@link DataOutput}.
+   * 
+   * @param out data output object
+   * @return typed bytes output corresponding to the supplied 
+   * {@link DataOutput}.
+   */
+  public static TypedBytesOutput get(DataOutput out) {
+    TypedBytesOutput bout = (TypedBytesOutput) tbOut.get();
+    bout.setDataOutput(out);
+    return bout;
+  }
+
+  /** Creates a new instance of TypedBytesOutput. */
+  public TypedBytesOutput(DataOutput out) {
+    this.out = out;
+  }
+  
+  /**
+   * Writes a Java object as a typed bytes sequence.
+   * 
+   * @param obj the object to be written
+   * @throws IOException
+   */
+  public void write(Object obj) throws IOException {
+    if (obj instanceof Buffer) {
+      writeBytes(((Buffer) obj).get());
+    } else if (obj instanceof Byte) {
+      writeByte((Byte) obj);
+    } else if (obj instanceof Boolean) {
+      writeBool((Boolean) obj);
+    } else if (obj instanceof Integer) {
+      writeInt((Integer) obj);
+    } else if (obj instanceof Long) {
+      writeLong((Long) obj);
+    } else if (obj instanceof Float) {
+      writeFloat((Float) obj);
+    } else if (obj instanceof Double) {
+      writeDouble((Double) obj);
+    } else if (obj instanceof String) {
+      writeString((String) obj);
+    } else if (obj instanceof ArrayList) {
+      writeVector((ArrayList) obj);
+    } else if (obj instanceof List) {
+      writeList((List) obj);
+    } else if (obj instanceof Map) {
+      writeMap((Map) obj);
+    } else {
+      throw new RuntimeException("cannot write objects of this type");
+    }
+  }
+
+  /**
+   * Writes a raw sequence of typed bytes.
+   * 
+   * @param bytes the bytes to be written
+   * @throws IOException
+   */
+  public void writeRaw(byte[] bytes) throws IOException {
+    out.write(bytes);
+  }
+
+  /**
+   * Writes a raw sequence of typed bytes.
+   * 
+   * @param bytes the bytes to be written
+   * @param offset an offset in the given array
+   * @param length number of bytes from the given array to write
+   * @throws IOException
+   */
+  public void writeRaw(byte[] bytes, int offset, int length)
+    throws IOException {
+    out.write(bytes, offset, length);
+  }
+
+  /**
+   * Writes a bytes array as a typed bytes sequence.
+   * 
+   * @param bytes the bytes array to be written
+   * @throws IOException
+   */
+  public void writeBytes(byte[] bytes) throws IOException {
+    out.write(Type.BYTES.code);
+    out.writeInt(bytes.length);
+    out.write(bytes);
+  }
+
+  /**
+   * Writes a byte as a typed bytes sequence.
+   * 
+   * @param b the byte to be written
+   * @throws IOException
+   */
+  public void writeByte(byte b) throws IOException {
+    out.write(Type.BYTE.code);
+    out.write(b);
+  }
+
+  /**
+   * Writes a boolean as a typed bytes sequence.
+   * 
+   * @param b the boolean to be written
+   * @throws IOException
+   */
+  public void writeBool(boolean b) throws IOException {
+    out.write(Type.BOOL.code);
+    out.writeBoolean(b);
+  }
+
+  /**
+   * Writes an integer as a typed bytes sequence.
+   * 
+   * @param i the integer to be written
+   * @throws IOException
+   */
+  public void writeInt(int i) throws IOException {
+    out.write(Type.INT.code);
+    out.writeInt(i);
+  }
+
+  /**
+   * Writes a long as a typed bytes sequence.
+   * 
+   * @param l the long to be written
+   * @throws IOException
+   */
+  public void writeLong(long l) throws IOException {
+    out.write(Type.LONG.code);
+    out.writeLong(l);
+  }
+
+  /**
+   * Writes a float as a typed bytes sequence.
+   * 
+   * @param f the float to be written
+   * @throws IOException
+   */
+  public void writeFloat(float f) throws IOException {
+    out.write(Type.FLOAT.code);
+    out.writeFloat(f);
+  }
+
+  /**
+   * Writes a double as a typed bytes sequence.
+   * 
+   * @param d the double to be written
+   * @throws IOException
+   */
+  public void writeDouble(double d) throws IOException {
+    out.write(Type.DOUBLE.code);
+    out.writeDouble(d);
+  }
+
+  /**
+   * Writes a string as a typed bytes sequence.
+   * 
+   * @param s the string to be written
+   * @throws IOException
+   */
+  public void writeString(String s) throws IOException {
+    out.write(Type.STRING.code);
+    WritableUtils.writeString(out, s);
+  }
+
+  /**
+   * Writes a vector as a typed bytes sequence.
+   * 
+   * @param vector the vector to be written
+   * @throws IOException
+   */
+  public void writeVector(ArrayList vector) throws IOException {
+    writeVectorHeader(vector.size());
+    for (Object obj : vector) {
+      write(obj);
+    }
+  }
+
+  /**
+   * Writes a vector header.
+   * 
+   * @param length the number of elements in the vector
+   * @throws IOException
+   */
+  public void writeVectorHeader(int length) throws IOException {
+    out.write(Type.VECTOR.code);
+    out.writeInt(length);
+  }
+
+  /**
+   * Writes a list as a typed bytes sequence.
+   * 
+   * @param list the list to be written
+   * @throws IOException
+   */
+  public void writeList(List list) throws IOException {
+    writeListHeader();
+    for (Object obj : list) {
+      write(obj);
+    }
+    writeListFooter();
+  }
+
+  /**
+   * Writes a list header.
+   * 
+   * @throws IOException
+   */
+  public void writeListHeader() throws IOException {
+    out.write(Type.LIST.code);
+  }
+
+  /**
+   * Writes a list footer.
+   * 
+   * @throws IOException
+   */
+  public void writeListFooter() throws IOException {
+    out.write(Type.MARKER.code);
+  }
+
+  /**
+   * Writes a map as a typed bytes sequence.
+   * 
+   * @param map the map to be written
+   * @throws IOException
+   */
+  @SuppressWarnings("unchecked")
+  public void writeMap(Map map) throws IOException {
+    writeMapHeader(map.size());
+    Set<Entry> entries = map.entrySet();
+    for (Entry entry : entries) {
+      write(entry.getKey());
+      write(entry.getValue());
+    }
+  }
+
+  /**
+   * Writes a map header.
+   * 
+   * @param length the number of key-value pairs in the map
+   * @throws IOException
+   */
+  public void writeMapHeader(int length) throws IOException {
+    out.write(Type.MAP.code);
+    out.writeInt(length);
+  }
+
+}

+ 159 - 0
src/contrib/streaming/src/java/org/apache/hadoop/typedbytes/TypedBytesRecordInput.java

@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.typedbytes;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.hadoop.record.Buffer;
+import org.apache.hadoop.record.Index;
+import org.apache.hadoop.record.RecordInput;
+
+/**
+ * Serializer for records that writes typed bytes.
+ */
+public class TypedBytesRecordInput implements RecordInput {
+
+  private TypedBytesInput in;
+
+  private TypedBytesRecordInput() {}
+
+  private void setTypedBytesInput(TypedBytesInput in) {
+    this.in = in;
+  }
+
+  private static ThreadLocal tbIn = new ThreadLocal() {
+    protected synchronized Object initialValue() {
+      return new TypedBytesRecordInput();
+    }
+  };
+
+  /**
+   * Get a thread-local typed bytes record input for the supplied
+   * {@link TypedBytesInput}.
+   * 
+   * @param in typed bytes input object
+   * @return typed bytes record input corresponding to the supplied
+   *         {@link TypedBytesInput}.
+   */
+  public static TypedBytesRecordInput get(TypedBytesInput in) {
+    TypedBytesRecordInput bin = (TypedBytesRecordInput) tbIn.get();
+    bin.setTypedBytesInput(in);
+    return bin;
+  }
+
+  /**
+   * Get a thread-local typed bytes record input for the supplied
+   * {@link DataInput}.
+   * 
+   * @param in data input object
+   * @return typed bytes record input corresponding to the supplied
+   *         {@link DataInput}.
+   */
+  public static TypedBytesRecordInput get(DataInput in) {
+    return get(TypedBytesInput.get(in));
+  }
+
+  /** Creates a new instance of TypedBytesRecordInput. */
+  public TypedBytesRecordInput(TypedBytesInput in) {
+    this.in = in;
+  }
+
+  /** Creates a new instance of TypedBytesRecordInput. */
+  public TypedBytesRecordInput(DataInput in) {
+    this(new TypedBytesInput(in));
+  }
+
+  public boolean readBool(String tag) throws IOException {
+    in.skipType();
+    return in.readBool();
+  }
+
+  public Buffer readBuffer(String tag) throws IOException {
+    in.skipType();
+    return new Buffer(in.readBytes());
+  }
+
+  public byte readByte(String tag) throws IOException {
+    in.skipType();
+    return in.readByte();
+  }
+
+  public double readDouble(String tag) throws IOException {
+    in.skipType();
+    return in.readDouble();
+  }
+
+  public float readFloat(String tag) throws IOException {
+    in.skipType();
+    return in.readFloat();
+  }
+
+  public int readInt(String tag) throws IOException {
+    in.skipType();
+    return in.readInt();
+  }
+
+  public long readLong(String tag) throws IOException {
+    in.skipType();
+    return in.readLong();
+  }
+
+  public String readString(String tag) throws IOException {
+    in.skipType();
+    return in.readString();
+  }
+
+  public void startRecord(String tag) throws IOException {
+    in.skipType();
+  }
+
+  public Index startVector(String tag) throws IOException {
+    in.skipType();
+    return new TypedBytesIndex(in.readVectorHeader());
+  }
+
+  public Index startMap(String tag) throws IOException {
+    in.skipType();
+    return new TypedBytesIndex(in.readMapHeader());
+  }
+
+  public void endRecord(String tag) throws IOException {}
+
+  public void endVector(String tag) throws IOException {}
+
+  public void endMap(String tag) throws IOException {}
+
+  private static  final class TypedBytesIndex implements Index {
+    private int nelems;
+
+    private TypedBytesIndex(int nelems) {
+      this.nelems = nelems;
+    }
+
+    public boolean done() {
+      return (nelems <= 0);
+    }
+
+    public void incr() {
+      nelems--;
+    }
+  }
+
+}

+ 137 - 0
src/contrib/streaming/src/java/org/apache/hadoop/typedbytes/TypedBytesRecordOutput.java

@@ -0,0 +1,137 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.typedbytes;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.TreeMap;
+
+import org.apache.hadoop.record.Buffer;
+import org.apache.hadoop.record.Record;
+import org.apache.hadoop.record.RecordOutput;
+
+/**
+ * Deserialized for records that reads typed bytes.
+ */
+public class TypedBytesRecordOutput implements RecordOutput {
+
+  private TypedBytesOutput out;
+
+  private TypedBytesRecordOutput() {}
+
+  private void setTypedBytesOutput(TypedBytesOutput out) {
+    this.out = out;
+  }
+
+  private static ThreadLocal tbOut = new ThreadLocal() {
+    protected synchronized Object initialValue() {
+      return new TypedBytesRecordOutput();
+    }
+  };
+
+  /**
+   * Get a thread-local typed bytes record input for the supplied
+   * {@link TypedBytesOutput}.
+   * 
+   * @param out typed bytes output object
+   * @return typed bytes record output corresponding to the supplied
+   *         {@link TypedBytesOutput}.
+   */
+  public static TypedBytesRecordOutput get(TypedBytesOutput out) {
+    TypedBytesRecordOutput bout = (TypedBytesRecordOutput) tbOut.get();
+    bout.setTypedBytesOutput(out);
+    return bout;
+  }
+
+  /**
+   * Get a thread-local typed bytes record output for the supplied
+   * {@link DataOutput}.
+   * 
+   * @param out data output object
+   * @return typed bytes record output corresponding to the supplied
+   *         {@link DataOutput}.
+   */
+  public static TypedBytesRecordOutput get(DataOutput out) {
+    return get(TypedBytesOutput.get(out));
+  }
+
+  /** Creates a new instance of TypedBytesRecordOutput. */
+  public TypedBytesRecordOutput(TypedBytesOutput out) {
+    this.out = out;
+  }
+
+  /** Creates a new instance of TypedBytesRecordOutput. */
+  public TypedBytesRecordOutput(DataOutput out) {
+    this(new TypedBytesOutput(out));
+  }
+
+  public void writeBool(boolean b, String tag) throws IOException {
+    out.writeBool(b);
+  }
+
+  public void writeBuffer(Buffer buf, String tag) throws IOException {
+    out.writeBytes(buf.get());
+  }
+
+  public void writeByte(byte b, String tag) throws IOException {
+    out.writeByte(b);
+  }
+
+  public void writeDouble(double d, String tag) throws IOException {
+    out.writeDouble(d);
+  }
+
+  public void writeFloat(float f, String tag) throws IOException {
+    out.writeFloat(f);
+  }
+
+  public void writeInt(int i, String tag) throws IOException {
+    out.writeInt(i);
+  }
+
+  public void writeLong(long l, String tag) throws IOException {
+    out.writeLong(l);
+  }
+
+  public void writeString(String s, String tag) throws IOException {
+    out.writeString(s);
+  }
+
+  public void startRecord(Record r, String tag) throws IOException {
+    out.writeListHeader();
+  }
+
+  public void startVector(ArrayList v, String tag) throws IOException {
+    out.writeVectorHeader(v.size());
+  }
+
+  public void startMap(TreeMap m, String tag) throws IOException {
+    out.writeMapHeader(m.size());
+  }
+
+  public void endRecord(Record r, String tag) throws IOException {
+    out.writeListFooter();
+  }
+
+  public void endVector(ArrayList v, String tag) throws IOException {}
+
+  public void endMap(TreeMap m, String tag) throws IOException {}
+
+}

+ 88 - 0
src/contrib/streaming/src/java/org/apache/hadoop/typedbytes/TypedBytesWritable.java

@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.typedbytes;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.io.BytesWritable;
+
+/**
+ * Writable for typed bytes.
+ */
+public class TypedBytesWritable extends BytesWritable {
+
+  /** Create a TypedBytesWritable. */
+  public TypedBytesWritable() {
+    super();
+  }
+
+  /** Create a TypedBytesWritable with a given byte array as initial value. */
+  public TypedBytesWritable(byte[] bytes) {
+    super(bytes);
+  }
+
+  /** Set the typed bytes from a given Java object. */
+  public void setValue(Object obj) {
+    try {
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      TypedBytesOutput tbo = TypedBytesOutput.get(new DataOutputStream(baos));
+      tbo.write(obj);
+      byte[] bytes = baos.toByteArray();
+      set(bytes, 0, bytes.length);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /** Get the typed bytes as a Java object. */
+  public Object getValue() {
+    try {
+      ByteArrayInputStream bais = new ByteArrayInputStream(getBytes());
+      TypedBytesInput tbi = TypedBytesInput.get(new DataInputStream(bais));
+      Object obj = tbi.read();
+      return obj;
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /** Get the type code embedded in the first byte. */
+  public Type getType() {
+    byte[] bytes = getBytes();
+    if (bytes == null || bytes.length == 0) {
+      return null;
+    }
+    for (Type type : Type.values()) {
+      if (type.code == (int) bytes[0]) {
+        return type;
+      }
+    }
+    return null;
+  }
+
+  /** Generate a suitable string representation. */
+  public String toString() {
+    return getValue().toString();
+  }
+
+}

+ 335 - 0
src/contrib/streaming/src/java/org/apache/hadoop/typedbytes/TypedBytesWritableInput.java

@@ -0,0 +1,335 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.typedbytes;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.ByteWritable;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.MapWritable;
+import org.apache.hadoop.io.SortedMapWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.VIntWritable;
+import org.apache.hadoop.io.VLongWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Provides functionality for reading typed bytes as Writable objects.
+ * 
+ * @see TypedBytesInput
+ */
+public class TypedBytesWritableInput {
+
+  private TypedBytesInput in;
+
+  private TypedBytesWritableInput() {}
+
+  private void setTypedBytesInput(TypedBytesInput in) {
+    this.in = in;
+  }
+
+  private static ThreadLocal tbIn = new ThreadLocal() {
+    protected synchronized Object initialValue() {
+      return new TypedBytesWritableInput();
+    }
+  };
+
+  /**
+   * Get a thread-local typed bytes writable input for the supplied
+   * {@link TypedBytesInput}.
+   * 
+   * @param in typed bytes input object
+   * @return typed bytes writable input corresponding to the supplied
+   *         {@link TypedBytesInput}.
+   */
+  public static TypedBytesWritableInput get(TypedBytesInput in) {
+    TypedBytesWritableInput bin = (TypedBytesWritableInput) tbIn.get();
+    bin.setTypedBytesInput(in);
+    return bin;
+  }
+
+  /**
+   * Get a thread-local typed bytes writable input for the supplied
+   * {@link DataInput}.
+   * 
+   * @param in data input object
+   * @return typed bytes writable input corresponding to the supplied
+   *         {@link DataInput}.
+   */
+  public static TypedBytesWritableInput get(DataInput in) {
+    return get(TypedBytesInput.get(in));
+  }
+
+  /** Creates a new instance of TypedBytesWritableInput. */
+  public TypedBytesWritableInput(TypedBytesInput in) {
+    this.in = in;
+  }
+
+  /** Creates a new instance of TypedBytesWritableInput. */
+  public TypedBytesWritableInput(DataInput din) {
+    this(new TypedBytesInput(din));
+  }
+
+  public Writable read() throws IOException {
+    Type type = in.readType();
+    if (type == null) {
+      return null;
+    }
+    switch (type) {
+    case BYTES:
+      return readBytes();
+    case BYTE:
+      return readByte();
+    case BOOL:
+      return readBoolean();
+    case INT:
+      return readVInt();
+    case LONG:
+      return readVLong();
+    case FLOAT:
+      return readFloat();
+    case DOUBLE:
+      return readDouble();
+    case STRING:
+      return readText();
+    case VECTOR:
+      return readArray();
+    case MAP:
+      return readMap();
+    default:
+      throw new RuntimeException("unknown type");
+    }
+  }
+
+  public Class<? extends Writable> readType() throws IOException {
+    Type type = in.readType();
+    if (type == null) {
+      return null;
+    }
+    switch (type) {
+    case BYTES:
+      return BytesWritable.class;
+    case BYTE:
+      return ByteWritable.class;
+    case BOOL:
+      return BooleanWritable.class;
+    case INT:
+      return VIntWritable.class;
+    case LONG:
+      return VLongWritable.class;
+    case FLOAT:
+      return FloatWritable.class;
+    case DOUBLE:
+      return DoubleWritable.class;
+    case STRING:
+      return Text.class;
+    case VECTOR:
+      return ArrayWritable.class;
+    case MAP:
+      return MapWritable.class;
+    default:
+      throw new RuntimeException("unknown type");
+    }
+  }
+
+  public BytesWritable readBytes(BytesWritable bw) throws IOException {
+    byte[] bytes = in.readBytes();
+    if (bw == null) {
+      bw = new BytesWritable(bytes);
+    } else {
+      bw.set(bytes, 0, bytes.length);
+    }
+    return bw;
+  }
+
+  public BytesWritable readBytes() throws IOException {
+    return readBytes(null);
+  }
+
+  public ByteWritable readByte(ByteWritable bw) throws IOException {
+    if (bw == null) {
+      bw = new ByteWritable();
+    }
+    bw.set(in.readByte());
+    return bw;
+  }
+
+  public ByteWritable readByte() throws IOException {
+    return readByte(null);
+  }
+
+  public BooleanWritable readBoolean(BooleanWritable bw) throws IOException {
+    if (bw == null) {
+      bw = new BooleanWritable();
+    }
+    bw.set(in.readBool());
+    return bw;
+  }
+
+  public BooleanWritable readBoolean() throws IOException {
+    return readBoolean(null);
+  }
+
+  public IntWritable readInt(IntWritable iw) throws IOException {
+    if (iw == null) {
+      iw = new IntWritable();
+    }
+    iw.set(in.readInt());
+    return iw;
+  }
+
+  public IntWritable readInt() throws IOException {
+    return readInt(null);
+  }
+
+  public VIntWritable readVInt(VIntWritable iw) throws IOException {
+    if (iw == null) {
+      iw = new VIntWritable();
+    }
+    iw.set(in.readInt());
+    return iw;
+  }
+
+  public VIntWritable readVInt() throws IOException {
+    return readVInt(null);
+  }
+
+  public LongWritable readLong(LongWritable lw) throws IOException {
+    if (lw == null) {
+      lw = new LongWritable();
+    }
+    lw.set(in.readLong());
+    return lw;
+  }
+
+  public LongWritable readLong() throws IOException {
+    return readLong(null);
+  }
+
+  public VLongWritable readVLong(VLongWritable lw) throws IOException {
+    if (lw == null) {
+      lw = new VLongWritable();
+    }
+    lw.set(in.readLong());
+    return lw;
+  }
+
+  public VLongWritable readVLong() throws IOException {
+    return readVLong(null);
+  }
+
+  public FloatWritable readFloat(FloatWritable fw) throws IOException {
+    if (fw == null) {
+      fw = new FloatWritable();
+    }
+    fw.set(in.readFloat());
+    return fw;
+  }
+
+  public FloatWritable readFloat() throws IOException {
+    return readFloat(null);
+  }
+
+  public DoubleWritable readDouble(DoubleWritable dw) throws IOException {
+    if (dw == null) {
+      dw = new DoubleWritable();
+    }
+    dw.set(in.readDouble());
+    return dw;
+  }
+
+  public DoubleWritable readDouble() throws IOException {
+    return readDouble(null);
+  }
+
+  public Text readText(Text t) throws IOException {
+    if (t == null) {
+      t = new Text();
+    }
+    t.set(in.readString());
+    return t;
+  }
+
+  public Text readText() throws IOException {
+    return readText(null);
+  }
+
+  public ArrayWritable readArray(ArrayWritable aw) throws IOException {
+    if (aw == null) {
+      aw = new ArrayWritable(TypedBytesWritable.class);
+    } else if (!aw.getValueClass().equals(TypedBytesWritable.class)) {
+      throw new RuntimeException("value class has to be TypedBytesWritable");
+    }
+    int length = in.readVectorHeader();
+    Writable[] writables = new Writable[length];
+    for (int i = 0; i < length; i++) {
+      writables[i] = new TypedBytesWritable(in.readRaw());
+    }
+    aw.set(writables);
+    return aw;
+  }
+
+  public ArrayWritable readArray() throws IOException {
+    return readArray(null);
+  }
+
+  public MapWritable readMap(MapWritable mw) throws IOException {
+    if (mw == null) {
+      mw = new MapWritable();
+    }
+    int length = in.readMapHeader();
+    for (int i = 0; i < length; i++) {
+      Writable key = read();
+      Writable value = read();
+      mw.put(key, value);
+    }
+    return mw;
+  }
+
+  public MapWritable readMap() throws IOException {
+    return readMap(null);
+  }
+
+  public SortedMapWritable readSortedMap(SortedMapWritable mw)
+    throws IOException {
+    if (mw == null) {
+      mw = new SortedMapWritable();
+    }
+    int length = in.readMapHeader();
+    for (int i = 0; i < length; i++) {
+      WritableComparable key = (WritableComparable) read();
+      Writable value = read();
+      mw.put(key, value);
+    }
+    return mw;
+  }
+
+  public SortedMapWritable readSortedMap() throws IOException {
+    return readSortedMap(null);
+  }
+
+}

+ 221 - 0
src/contrib/streaming/src/java/org/apache/hadoop/typedbytes/TypedBytesWritableOutput.java

@@ -0,0 +1,221 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.typedbytes;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Map;
+
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.ByteWritable;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.MapWritable;
+import org.apache.hadoop.io.SortedMapWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.VIntWritable;
+import org.apache.hadoop.io.VLongWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.record.Record;
+
+/**
+ * Provides functionality for writing Writable objects as typed bytes.
+ * 
+ * @see TypedBytesOutput
+ */
+public class TypedBytesWritableOutput {
+
+  private TypedBytesOutput out;
+
+  private TypedBytesWritableOutput() {}
+
+  private void setTypedBytesOutput(TypedBytesOutput out) {
+    this.out = out;
+  }
+
+  private static ThreadLocal tbOut = new ThreadLocal() {
+    protected synchronized Object initialValue() {
+      return new TypedBytesWritableOutput();
+    }
+  };
+
+  /**
+   * Get a thread-local typed bytes writable input for the supplied
+   * {@link TypedBytesOutput}.
+   * 
+   * @param out typed bytes output object
+   * @return typed bytes writable output corresponding to the supplied
+   *         {@link TypedBytesOutput}.
+   */
+  public static TypedBytesWritableOutput get(TypedBytesOutput out) {
+    TypedBytesWritableOutput bout = (TypedBytesWritableOutput) tbOut.get();
+    bout.setTypedBytesOutput(out);
+    return bout;
+  }
+
+  /**
+   * Get a thread-local typed bytes writable output for the supplied
+   * {@link DataOutput}.
+   * 
+   * @param out data output object
+   * @return typed bytes writable output corresponding to the supplied
+   *         {@link DataOutput}.
+   */
+  public static TypedBytesWritableOutput get(DataOutput out) {
+    return get(TypedBytesOutput.get(out));
+  }
+
+  /** Creates a new instance of TypedBytesWritableOutput. */
+  public TypedBytesWritableOutput(TypedBytesOutput out) {
+    this.out = out;
+  }
+
+  /** Creates a new instance of TypedBytesWritableOutput. */
+  public TypedBytesWritableOutput(DataOutput dout) {
+    this(new TypedBytesOutput(dout));
+  }
+
+  public void write(Writable w) throws IOException {
+    if (w instanceof TypedBytesWritable) {
+      writeTypedBytes((TypedBytesWritable) w);
+    } else if (w instanceof BytesWritable) {
+      writeBytes((BytesWritable) w);
+    } else if (w instanceof ByteWritable) {
+      writeByte((ByteWritable) w);
+    } else if (w instanceof BooleanWritable) {
+      writeBoolean((BooleanWritable) w);
+    } else if (w instanceof IntWritable) {
+      writeInt((IntWritable) w);
+    } else if (w instanceof VIntWritable) {
+      writeVInt((VIntWritable) w);
+    } else if (w instanceof LongWritable) {
+      writeLong((LongWritable) w);
+    } else if (w instanceof VLongWritable) {
+      writeVLong((VLongWritable) w);
+    } else if (w instanceof FloatWritable) {
+      writeFloat((FloatWritable) w);
+    } else if (w instanceof DoubleWritable) {
+      writeDouble((DoubleWritable) w);
+    } else if (w instanceof Text) {
+      writeText((Text) w);
+    } else if (w instanceof ArrayWritable) {
+      writeArray((ArrayWritable) w);
+    } else if (w instanceof MapWritable) {
+      writeMap((MapWritable) w);
+    } else if (w instanceof SortedMapWritable) {
+      writeSortedMap((SortedMapWritable) w);
+    } else if (w instanceof Record) {
+      writeRecord((Record) w);
+    } else {
+      writeWritable(w); // last resort
+    }
+  }
+
+  public void writeTypedBytes(TypedBytesWritable tbw) throws IOException {
+    out.writeRaw(tbw.getBytes(), 0, tbw.getLength());
+  }
+
+  public void writeBytes(BytesWritable bw) throws IOException {
+    byte[] bytes = Arrays.copyOfRange(bw.getBytes(), 0, bw.getLength());
+    out.writeBytes(bytes);
+  }
+
+  public void writeByte(ByteWritable bw) throws IOException {
+    out.writeByte(bw.get());
+  }
+
+  public void writeBoolean(BooleanWritable bw) throws IOException {
+    out.writeBool(bw.get());
+  }
+
+  public void writeInt(IntWritable iw) throws IOException {
+    out.writeInt(iw.get());
+  }
+
+  public void writeVInt(VIntWritable viw) throws IOException {
+    out.writeInt(viw.get());
+  }
+
+  public void writeLong(LongWritable lw) throws IOException {
+    out.writeLong(lw.get());
+  }
+
+  public void writeVLong(VLongWritable vlw) throws IOException {
+    out.writeLong(vlw.get());
+  }
+
+  public void writeFloat(FloatWritable fw) throws IOException {
+    out.writeFloat(fw.get());
+  }
+
+  public void writeDouble(DoubleWritable dw) throws IOException {
+    out.writeDouble(dw.get());
+  }
+
+  public void writeText(Text t) throws IOException {
+    out.writeString(t.toString());
+  }
+
+  public void writeArray(ArrayWritable aw) throws IOException {
+    Writable[] writables = aw.get();
+    out.writeVectorHeader(writables.length);
+    for (Writable writable : writables) {
+      write(writable);
+    }
+  }
+
+  public void writeMap(MapWritable mw) throws IOException {
+    out.writeMapHeader(mw.size());
+    for (Map.Entry<Writable, Writable> entry : mw.entrySet()) {
+      write(entry.getKey());
+      write(entry.getValue());
+    }
+  }
+
+  public void writeSortedMap(SortedMapWritable smw) throws IOException {
+    out.writeMapHeader(smw.size());
+    for (Map.Entry<WritableComparable, Writable> entry : smw.entrySet()) {
+      write(entry.getKey());
+      write(entry.getValue());
+    }
+  }
+
+  public void writeRecord(Record r) throws IOException {
+    r.serialize(TypedBytesRecordOutput.get(out));
+  }
+
+  public void writeWritable(Writable w) throws IOException {
+    out.writeVectorHeader(2);
+    out.writeString(w.getClass().getName());
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(baos);
+    w.write(dos);
+    dos.close();
+    out.writeBytes(baos.toByteArray());
+  }
+
+}

+ 66 - 0
src/contrib/streaming/src/java/org/apache/hadoop/typedbytes/package.html

@@ -0,0 +1,66 @@
+<html>
+
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<body>
+
+Typed bytes are sequences of bytes in which the first byte is a type code. They are especially useful as a 
+(simple and very straightforward) binary format for transferring data to and from Hadoop Streaming programs.
+
+<h3>Type Codes</h3>
+
+Each typed bytes sequence starts with an unsigned byte that contains the type code. Possible values are:
+<p>
+<table border="1" cellpadding="2">
+<tr><th>Code</th><th>Type</th></tr>
+<tr><td><i>0</i></td><td>A sequence of bytes.</td></tr>
+<tr><td><i>1</i></td><td>A byte.</td></tr>
+<tr><td><i>2</i></td><td>A boolean.</td></tr>
+<tr><td><i>3</i></td><td>An integer.</td></tr>
+<tr><td><i>4</i></td><td>A long.</td></tr>
+<tr><td><i>5</i></td><td>A float.</td></tr>
+<tr><td><i>6</i></td><td>A double.</td></tr>
+<tr><td><i>7</i></td><td>A string.</td></tr>
+<tr><td><i>8</i></td><td>A vector.</td></tr>
+<tr><td><i>9</i></td><td>A list.</td></tr>
+<tr><td><i>10</i></td><td>A map.</td></tr>
+</table>
+</p>
+
+<h3>Subsequent Bytes</h3>
+
+These are the subsequent bytes for the different type codes (everything is big-endian and unpadded):
+<p>
+<table border="1" cellpadding="2">
+<tr><th>Code</th><th>Subsequent Bytes</th></tr>
+<tr><td><i>0</i></td><td>&lt;32-bit signed integer&gt; &lt;as many bytes as indicated by the integer&gt;</td></tr>
+<tr><td><i>1</i></td><td>&lt;signed byte&gt;</td></tr>
+<tr><td><i>2</i></td><td>&lt;signed byte (<i>0 = <i>false</i> and <i>1</i> = <i>true</i>)&gt;</td></tr>
+<tr><td><i>3</i></td><td>&lt;32-bit signed integer&gt;</td></tr>
+<tr><td><i>4</i></td><td>&lt;64-bit signed integer&gt;</td></tr>
+<tr><td><i>5</i></td><td>&lt;32-bit IEEE floating point number&gt;</td></tr>
+<tr><td><i>6</i></td><td>&lt;64-bit IEEE floating point number&gt;</td></tr>
+<tr><td><i>7</i></td><td>&lt;32-bit signed integer&gt; &lt;as many UTF-8 bytes as indicated by the integer&gt;</td></tr>
+<tr><td><i>8</i></td><td>&lt;32-bit signed integer&gt; &lt;as many typed bytes sequences as indicated by the integer&gt;</td></tr>
+<tr><td><i>9</i></td><td>&lt;variable number of typed bytes sequences&gt; &lt;<i>255</i> written as an unsigned byte&gt;</td></tr>
+<tr><td><i>10</i></td><td>&lt;32-bit signed integer&gt; &lt;as many (key-value) pairs of typed bytes sequences as indicated by the integer&gt;</td></tr>
+</table>
+</p>
+
+</body>
+</html>

+ 65 - 0
src/contrib/streaming/src/test/org/apache/hadoop/streaming/RawBytesMapApp.java

@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming;
+
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+
+import org.apache.hadoop.io.IntWritable;
+
+public class RawBytesMapApp {
+  private String find;
+  private DataOutputStream dos;
+
+  public RawBytesMapApp(String find) {
+    this.find = find;
+    dos = new DataOutputStream(System.out);
+  }
+
+  public void go() throws IOException {
+    BufferedReader in = new BufferedReader(new InputStreamReader(System.in));
+    String line;
+    while ((line = in.readLine()) != null) {
+      for (String part : line.split(find)) {
+        writeString(part);  // write key
+        writeInt(1);        // write value
+      }
+    }
+    System.out.flush();
+  }
+  
+  public static void main(String[] args) throws IOException {
+    RawBytesMapApp app = new RawBytesMapApp(args[0].replace(".","\\."));
+    app.go();
+  }
+  
+  private void writeString(String str) throws IOException {
+    byte[] bytes = str.getBytes("UTF-8");
+    dos.writeInt(bytes.length);
+    dos.write(bytes);
+  }
+  
+  private void writeInt(int i) throws IOException {
+    dos.writeInt(4);
+    IntWritable iw = new IntWritable(i);
+    iw.write(dos);
+  }
+}

+ 74 - 0
src/contrib/streaming/src/test/org/apache/hadoop/streaming/RawBytesReduceApp.java

@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming;
+
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+
+import org.apache.hadoop.io.IntWritable;
+
+public class RawBytesReduceApp {
+  private DataInputStream dis;
+
+  public RawBytesReduceApp() {
+    dis = new DataInputStream(System.in);
+  }
+  
+  public void go() throws IOException {
+    String prevKey = null;
+    int sum = 0;
+    String key = readString();
+    while (key != null) {
+      if (prevKey != null && !key.equals(prevKey)) {
+        System.out.println(prevKey + "\t" + sum);
+        sum = 0;
+      }
+      sum += readInt();
+      prevKey = key;
+      key = readString();
+    }
+    System.out.println(prevKey + "\t" + sum);
+    System.out.flush();
+  }
+
+  public static void main(String[] args) throws IOException {
+    RawBytesReduceApp app = new RawBytesReduceApp();
+    app.go();
+  }
+  
+  private String readString() throws IOException {
+    int length;
+    try {
+      length = dis.readInt();
+    } catch (EOFException eof) {
+      return null;
+    }
+    byte[] bytes = new byte[length];
+    dis.readFully(bytes);
+    return new String(bytes, "UTF-8");
+  }
+  
+  private int readInt() throws IOException {
+    dis.readInt(); // ignore (we know it's 4)
+    IntWritable iw = new IntWritable();
+    iw.readFields(dis);
+    return iw.get();
+  }
+}

+ 111 - 0
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestAutoInputFormat.java

@@ -0,0 +1,111 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming;
+
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.streaming.AutoInputFormat;
+
+import junit.framework.TestCase;
+
+public class TestAutoInputFormat extends TestCase {
+
+  private static Configuration conf = new Configuration();
+
+  private static final int LINES_COUNT = 3;
+
+  private static final int RECORDS_COUNT = 3;
+
+  private static final int SPLITS_COUNT = 2;
+
+  @SuppressWarnings( { "unchecked", "deprecation" })
+  public void testFormat() throws IOException {
+    JobConf job = new JobConf(conf);
+    FileSystem fs = FileSystem.getLocal(conf);
+    Path dir = new Path(System.getProperty("test.build.data", ".") + "/mapred");
+    Path txtFile = new Path(dir, "auto.txt");
+    Path seqFile = new Path(dir, "auto.seq");
+
+    fs.delete(dir, true);
+
+    FileInputFormat.setInputPaths(job, dir);
+
+    Writer txtWriter = new OutputStreamWriter(fs.create(txtFile));
+    try {
+      for (int i = 0; i < LINES_COUNT; i++) {
+        txtWriter.write("" + (10 * i));
+        txtWriter.write("\n");
+      }
+    } finally {
+      txtWriter.close();
+    }
+
+    SequenceFile.Writer seqWriter = SequenceFile.createWriter(fs, conf,
+      seqFile, IntWritable.class, LongWritable.class);
+    try {
+      for (int i = 0; i < RECORDS_COUNT; i++) {
+        IntWritable key = new IntWritable(11 * i);
+        LongWritable value = new LongWritable(12 * i);
+        seqWriter.append(key, value);
+      }
+    } finally {
+      seqWriter.close();
+    }
+
+    AutoInputFormat format = new AutoInputFormat();
+    InputSplit[] splits = format.getSplits(job, SPLITS_COUNT);
+    for (InputSplit split : splits) {
+      RecordReader reader = format.getRecordReader(split, job, Reporter.NULL);
+      Object key = reader.createKey();
+      Object value = reader.createValue();
+      try {
+        while (reader.next(key, value)) {
+          if (key instanceof LongWritable) {
+            assertEquals("Wrong value class.", Text.class, value.getClass());
+            assertTrue("Invalid value", Integer.parseInt(((Text) value)
+              .toString()) % 10 == 0);
+          } else {
+            assertEquals("Wrong key class.", IntWritable.class, key.getClass());
+            assertEquals("Wrong value class.", LongWritable.class, value
+              .getClass());
+            assertTrue("Invalid key.", ((IntWritable) key).get() % 11 == 0);
+            assertTrue("Invalid value.", ((LongWritable) value).get() % 12 == 0);
+          }
+        }
+      } finally {
+        reader.close();
+      }
+    }
+  }
+
+}

+ 91 - 0
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestDumpTypedBytes.java

@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.streaming.DumpTypedBytes;
+import org.apache.hadoop.typedbytes.TypedBytesInput;
+
+import junit.framework.TestCase;
+
+public class TestDumpTypedBytes extends TestCase {
+
+  public void testDumping() throws Exception {
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    FileSystem fs = cluster.getFileSystem();
+    PrintStream psBackup = System.out;
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    PrintStream psOut = new PrintStream(out);
+    System.setOut(psOut);
+    DumpTypedBytes dumptb = new DumpTypedBytes(conf);
+
+    try {
+      Path root = new Path("/typedbytestest");
+      assertTrue(fs.mkdirs(root));
+      assertTrue(fs.exists(root));
+      OutputStreamWriter writer = new OutputStreamWriter(fs.create(new Path(
+        root, "test.txt")));
+      try {
+        for (int i = 0; i < 100; i++) {
+          writer.write("" + (10 * i) + "\n");
+        }
+      } finally {
+        writer.close();
+      }
+
+      String[] args = new String[1];
+      args[0] = "/typedbytestest";
+      int ret = dumptb.run(args);
+      assertEquals("Return value != 0.", 0, ret);
+
+      ByteArrayInputStream in = new ByteArrayInputStream(out.toByteArray());
+      TypedBytesInput tbinput = new TypedBytesInput(new DataInputStream(in));
+      int counter = 0;
+      Object key = tbinput.read();
+      while (key != null) {
+        assertEquals(Long.class, key.getClass()); // offset
+        Object value = tbinput.read();
+        assertEquals(String.class, value.getClass());
+        assertTrue("Invalid output.",
+          Integer.parseInt(value.toString()) % 10 == 0);
+        counter++;
+        key = tbinput.read();
+      }
+      assertEquals("Wrong number of outputs.", 100, counter);
+    } finally {
+      try {
+        fs.close();
+      } catch (Exception e) {
+      }
+      System.setOut(psBackup);
+      cluster.shutdown();
+    }
+  }
+
+}

+ 88 - 0
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestLoadTypedBytes.java

@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.InputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.typedbytes.TypedBytesOutput;
+import org.apache.hadoop.typedbytes.TypedBytesWritable;
+
+import junit.framework.TestCase;
+
+public class TestLoadTypedBytes extends TestCase {
+
+  public void testLoading() throws Exception {
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    FileSystem fs = cluster.getFileSystem();
+    
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    TypedBytesOutput tboutput = new TypedBytesOutput(new DataOutputStream(out));
+    for (int i = 0; i < 100; i++) {
+      tboutput.write(new Long(i)); // key
+      tboutput.write("" + (10 * i)); // value
+    }
+    InputStream isBackup = System.in;
+    ByteArrayInputStream in = new ByteArrayInputStream(out.toByteArray());
+    System.setIn(in);
+    LoadTypedBytes loadtb = new LoadTypedBytes(conf);
+
+    try {
+      Path root = new Path("/typedbytestest");
+      assertTrue(fs.mkdirs(root));
+      assertTrue(fs.exists(root));
+      
+      String[] args = new String[1];
+      args[0] = "/typedbytestest/test.seq";
+      int ret = loadtb.run(args);
+      assertEquals("Return value != 0.", 0, ret);
+
+      Path file = new Path(root, "test.seq");
+      assertTrue(fs.exists(file));
+      SequenceFile.Reader reader = new SequenceFile.Reader(fs, file, conf);
+      int counter = 0;
+      TypedBytesWritable key = new TypedBytesWritable();
+      TypedBytesWritable value = new TypedBytesWritable();
+      while (reader.next(key, value)) {
+        assertEquals(Long.class, key.getValue().getClass());
+        assertEquals(String.class, value.getValue().getClass());
+        assertTrue("Invalid record.",
+          Integer.parseInt(value.toString()) % 10 == 0);
+        counter++;
+      }
+      assertEquals("Wrong number of records.", 100, counter);
+    } finally {
+      try {
+        fs.close();
+      } catch (Exception e) {
+      }
+      System.setIn(isBackup);
+      cluster.shutdown();
+    }
+  }
+  
+}

+ 95 - 0
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestRawBytesStreaming.java

@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+
+import junit.framework.TestCase;
+
+public class TestRawBytesStreaming extends TestCase {
+
+  protected File INPUT_FILE = new File("input.txt");
+  protected File OUTPUT_DIR = new File("out");
+  protected String input = "roses.are.red\nviolets.are.blue\nbunnies.are.pink\n";
+  protected String map = StreamUtil.makeJavaCommand(RawBytesMapApp.class, new String[]{"."});
+  protected String reduce = StreamUtil.makeJavaCommand(RawBytesReduceApp.class, new String[0]);
+  protected String outputExpect = "are\t3\nblue\t1\nbunnies\t1\npink\t1\nred\t1\nroses\t1\nviolets\t1\n";
+  
+  public TestRawBytesStreaming() throws IOException {
+    UtilTest utilTest = new UtilTest(getClass().getName());
+    utilTest.checkUserDir();
+    utilTest.redirectIfAntJunit();
+  }
+
+  protected void createInput() throws IOException {
+    DataOutputStream out = new DataOutputStream(new FileOutputStream(INPUT_FILE.getAbsoluteFile()));
+    out.write(input.getBytes("UTF-8"));
+    out.close();
+  }
+
+  protected String[] genArgs() {
+    return new String[] {
+      "-input", INPUT_FILE.getAbsolutePath(),
+      "-output", OUTPUT_DIR.getAbsolutePath(),
+      "-mapper", map,
+      "-reducer", reduce,
+      "-jobconf", "keep.failed.task.files=true",
+      "-jobconf", "stream.tmpdir="+System.getProperty("test.build.data","/tmp"),
+      "-jobconf", "stream.map.output=rawbytes",
+      "-jobconf", "stream.reduce.input=rawbytes",
+      "-verbose"
+    };
+  }
+  
+  public void testCommandLine() throws Exception {
+    try {
+      try {
+        OUTPUT_DIR.getAbsoluteFile().delete();
+      } catch (Exception e) {
+      }
+
+      createInput();
+      OUTPUT_DIR.delete();
+
+      // During tests, the default Configuration will use a local mapred
+      // So don't specify -config or -cluster
+      StreamJob job = new StreamJob();
+      job.setConf(new Configuration());
+      job.run(genArgs());
+      File outFile = new File(OUTPUT_DIR, "part-00000").getAbsoluteFile();
+      String output = StreamUtil.slurp(outFile);
+      outFile.delete();
+      System.out.println("   map=" + map);
+      System.out.println("reduce=" + reduce);
+      System.err.println("outEx1=" + outputExpect);
+      System.err.println("  out1=" + output);
+      assertEquals(outputExpect, output);
+    } finally {
+      File outFileCRC = new File(OUTPUT_DIR, ".part-00000.crc").getAbsoluteFile();
+      INPUT_FILE.delete();
+      outFileCRC.delete();
+      OUTPUT_DIR.getAbsoluteFile().delete();
+    }
+  }
+}

+ 93 - 0
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestTypedBytesStreaming.java

@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+
+import junit.framework.TestCase;
+
+public class TestTypedBytesStreaming extends TestCase {
+
+  protected File INPUT_FILE = new File("input.txt");
+  protected File OUTPUT_DIR = new File("out");
+  protected String input = "roses.are.red\nviolets.are.blue\nbunnies.are.pink\n";
+  protected String map = StreamUtil.makeJavaCommand(TypedBytesMapApp.class, new String[]{"."});
+  protected String reduce = StreamUtil.makeJavaCommand(TypedBytesReduceApp.class, new String[0]);
+  protected String outputExpect = "are\t3\nred\t1\nblue\t1\npink\t1\nroses\t1\nbunnies\t1\nviolets\t1\n";
+  
+  public TestTypedBytesStreaming() throws IOException {
+    UtilTest utilTest = new UtilTest(getClass().getName());
+    utilTest.checkUserDir();
+    utilTest.redirectIfAntJunit();
+  }
+
+  protected void createInput() throws IOException {
+    DataOutputStream out = new DataOutputStream(new FileOutputStream(INPUT_FILE.getAbsoluteFile()));
+    out.write(input.getBytes("UTF-8"));
+    out.close();
+  }
+
+  protected String[] genArgs() {
+    return new String[] {
+      "-input", INPUT_FILE.getAbsolutePath(),
+      "-output", OUTPUT_DIR.getAbsolutePath(),
+      "-mapper", map,
+      "-reducer", reduce,
+      "-jobconf", "keep.failed.task.files=true",
+      "-jobconf", "stream.tmpdir="+System.getProperty("test.build.data","/tmp"),
+      "-io", "typedbytes"
+    };
+  }
+  
+  public void testCommandLine() throws Exception {
+    try {
+      try {
+        OUTPUT_DIR.getAbsoluteFile().delete();
+      } catch (Exception e) {
+      }
+
+      createInput();
+      OUTPUT_DIR.delete();
+
+      // During tests, the default Configuration will use a local mapred
+      // So don't specify -config or -cluster
+      StreamJob job = new StreamJob();
+      job.setConf(new Configuration());
+      job.run(genArgs());
+      File outFile = new File(OUTPUT_DIR, "part-00000").getAbsoluteFile();
+      String output = StreamUtil.slurp(outFile);
+      outFile.delete();
+      System.out.println("   map=" + map);
+      System.out.println("reduce=" + reduce);
+      System.err.println("outEx1=" + outputExpect);
+      System.err.println("  out1=" + output);
+      assertEquals(outputExpect, output);
+    } finally {
+      File outFileCRC = new File(OUTPUT_DIR, ".part-00000.crc").getAbsoluteFile();
+      INPUT_FILE.delete();
+      outFileCRC.delete();
+      OUTPUT_DIR.getAbsoluteFile().delete();
+    }
+  }
+}

+ 59 - 0
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TypedBytesMapApp.java

@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.typedbytes.TypedBytesInput;
+import org.apache.hadoop.typedbytes.TypedBytesOutput;
+
+public class TypedBytesMapApp {
+
+  private String find;
+
+  public TypedBytesMapApp(String find) {
+    this.find = find;
+  }
+
+  public void go() throws IOException {
+    TypedBytesInput tbinput = new TypedBytesInput(new DataInputStream(System.in));
+    TypedBytesOutput tboutput = new TypedBytesOutput(new DataOutputStream(System.out));
+
+    Object key = tbinput.readRaw();
+    while (key != null) {
+      Object value = tbinput.read();
+      for (String part : value.toString().split(find)) {
+        tboutput.write(part);  // write key
+        tboutput.write(1);     // write value
+      }
+      System.err.println("reporter:counter:UserCounters,InputLines,1");
+      key = tbinput.readRaw();
+    }
+    
+    System.out.flush();
+  }
+  
+  public static void main(String[] args) throws IOException {
+    TypedBytesMapApp app = new TypedBytesMapApp(args[0].replace(".","\\."));
+    app.go();
+  }
+  
+}

+ 58 - 0
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TypedBytesReduceApp.java

@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.typedbytes.TypedBytesInput;
+import org.apache.hadoop.typedbytes.TypedBytesOutput;
+
+public class TypedBytesReduceApp {
+
+  public void go() throws IOException {
+    TypedBytesInput tbinput = new TypedBytesInput(new DataInputStream(System.in));
+    TypedBytesOutput tboutput = new TypedBytesOutput(new DataOutputStream(System.out));
+    
+    Object prevKey = null;
+    int sum = 0;
+    Object key = tbinput.read();
+    while (key != null) {
+      if (prevKey != null && !key.equals(prevKey)) {
+        tboutput.write(prevKey);  // write key
+        tboutput.write(sum);      // write value
+        sum = 0;
+      }
+      sum += (Integer) tbinput.read();
+      prevKey = key;
+      key = tbinput.read();
+    }
+    tboutput.write(prevKey);
+    tboutput.write(sum);
+    
+    System.out.flush();
+  }
+
+  public static void main(String[] args) throws IOException {
+    TypedBytesReduceApp app = new TypedBytesReduceApp();
+    app.go();
+  }
+  
+}

+ 242 - 0
src/contrib/streaming/src/test/org/apache/hadoop/typedbytes/TestIO.java

@@ -0,0 +1,242 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.typedbytes;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.ByteWritable;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.MapWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.VIntWritable;
+import org.apache.hadoop.io.VLongWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.record.Buffer;
+import org.apache.hadoop.record.RecRecord0;
+import org.apache.hadoop.record.RecRecord1;
+
+import junit.framework.TestCase;
+
+public class TestIO extends TestCase {
+
+  private File tmpfile;
+
+  protected void setUp() throws Exception {
+    this.tmpfile = new File(System.getProperty("test.build.data", "/tmp"),
+      "typedbytes.bin");
+  }
+
+  protected void tearDown() throws Exception {
+    tmpfile.delete();
+  }
+
+  public void testIO() throws IOException {
+    ArrayList<Object> vector = new ArrayList<Object>();
+    vector.add("test");
+    vector.add(false);
+    vector.add(12345);
+    List<Object> list = new LinkedList<Object>();
+    list.add("another test");
+    list.add(true);
+    list.add(123456789L);
+    Map<Object, Object> map = new HashMap<Object, Object>();
+    map.put("one", 1);
+    map.put("vector", vector);
+    Object[] objects = new Object[] {
+      new Buffer(new byte[] { 1, 2, 3, 4 }),
+      (byte) 123, true, 12345, 123456789L, (float) 1.2, 1.234,
+      "random string", vector, list, map 
+    };
+
+    FileOutputStream ostream = new FileOutputStream(tmpfile);
+    DataOutputStream dostream = new DataOutputStream(ostream);
+    TypedBytesOutput out = new TypedBytesOutput(dostream);
+    for (Object obj : objects) {
+      out.write(obj);
+    }
+    dostream.close();
+    ostream.close();
+
+    FileInputStream istream = new FileInputStream(tmpfile);
+    DataInputStream distream = new DataInputStream(istream);
+    TypedBytesInput in = new TypedBytesInput(distream);
+    for (Object obj : objects) {
+      assertEquals(obj, in.read());
+    }
+    distream.close();
+    istream.close();
+
+    istream = new FileInputStream(tmpfile);
+    distream = new DataInputStream(istream);
+    in = new TypedBytesInput(distream);
+    for (Object obj : objects) {
+      byte[] bytes = in.readRaw();
+      ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+      DataInputStream dis = new DataInputStream(bais);
+      assertEquals(obj, (new TypedBytesInput(dis)).read());
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      TypedBytesOutput tbout = new TypedBytesOutput(new DataOutputStream(baos));
+      tbout.writeRaw(bytes);
+      bais = new ByteArrayInputStream(bytes);
+      dis = new DataInputStream(bais);
+      assertEquals(obj, (new TypedBytesInput(dis)).read());
+    }
+    distream.close();
+    istream.close();
+  }
+
+  public void testRecordIO() throws IOException {
+    RecRecord1 r1 = new RecRecord1();
+    r1.setBoolVal(true);
+    r1.setByteVal((byte) 0x66);
+    r1.setFloatVal(3.145F);
+    r1.setDoubleVal(1.5234);
+    r1.setIntVal(-4567);
+    r1.setLongVal(-2367L);
+    r1.setStringVal("random text");
+    r1.setBufferVal(new Buffer());
+    r1.setVectorVal(new ArrayList<String>());
+    r1.setMapVal(new TreeMap<String, String>());
+    RecRecord0 r0 = new RecRecord0();
+    r0.setStringVal("other random text");
+    r1.setRecordVal(r0);
+
+    FileOutputStream ostream = new FileOutputStream(tmpfile);
+    DataOutputStream dostream = new DataOutputStream(ostream);
+    TypedBytesRecordOutput out = TypedBytesRecordOutput.get(dostream);
+    r1.serialize(out, "");
+    dostream.close();
+    ostream.close();
+
+    FileInputStream istream = new FileInputStream(tmpfile);
+    DataInputStream distream = new DataInputStream(istream);
+    TypedBytesRecordInput in = TypedBytesRecordInput.get(distream);
+    RecRecord1 r2 = new RecRecord1();
+    r2.deserialize(in, "");
+    distream.close();
+    istream.close();
+    assertEquals(r1, r2);
+  }
+
+  public void testWritableIO() throws IOException {
+    Writable[] vectorValues = new Writable[] {
+      new Text("test1"), new Text("test2"), new Text("test3")
+    };
+    ArrayWritable vector = new ArrayWritable(Text.class, vectorValues);
+    MapWritable map = new MapWritable();
+    map.put(new Text("one"), new VIntWritable(1));
+    map.put(new Text("two"), new VLongWritable(2));
+    Writable[] writables = new Writable[] {
+      new BytesWritable(new byte[] { 1, 2, 3, 4 }),
+      new ByteWritable((byte) 123), new BooleanWritable(true),
+      new VIntWritable(12345), new VLongWritable(123456789L),
+      new FloatWritable((float) 1.2), new DoubleWritable(1.234),
+      new Text("random string")
+    };
+    TypedBytesWritable tbw = new TypedBytesWritable();
+    tbw.setValue("typed bytes text");
+    RecRecord1 r1 = new RecRecord1();
+    r1.setBoolVal(true);
+    r1.setByteVal((byte) 0x66);
+    r1.setFloatVal(3.145F);
+    r1.setDoubleVal(1.5234);
+    r1.setIntVal(-4567);
+    r1.setLongVal(-2367L);
+    r1.setStringVal("random text");
+    r1.setBufferVal(new Buffer());
+    r1.setVectorVal(new ArrayList<String>());
+    r1.setMapVal(new TreeMap<String, String>());
+    RecRecord0 r0 = new RecRecord0();
+    r0.setStringVal("other random text");
+    r1.setRecordVal(r0);
+
+    FileOutputStream ostream = new FileOutputStream(tmpfile);
+    DataOutputStream dostream = new DataOutputStream(ostream);
+    TypedBytesWritableOutput out = new TypedBytesWritableOutput(dostream);
+    for (Writable w : writables) {
+      out.write(w);
+    }
+    out.write(tbw);
+    out.write(vector);
+    out.write(map);
+    out.write(r1);
+    dostream.close();
+    ostream.close();
+
+    FileInputStream istream = new FileInputStream(tmpfile);
+    DataInputStream distream = new DataInputStream(istream);
+
+    TypedBytesWritableInput in = new TypedBytesWritableInput(distream);
+    for (Writable w : writables) {
+      assertEquals(w, in.read());
+    }
+
+    assertEquals(tbw.getValue().toString(), in.read().toString());
+
+    assertEquals(ArrayWritable.class, in.readType());
+    ArrayWritable aw = in.readArray();
+    Writable[] writables1 = vector.get(), writables2 = aw.get();
+    assertEquals(writables1.length, writables2.length);
+    for (int i = 0; i < writables1.length; i++) {
+      assertEquals(((Text) writables1[i]).toString(),
+        ((TypedBytesWritable) writables2[i]).getValue());
+    }
+    assertEquals(MapWritable.class, in.readType());
+
+    MapWritable mw = in.readMap();
+    assertEquals(map.entrySet(), mw.entrySet());
+
+    assertEquals(Type.LIST, TypedBytesInput.get(distream).readType());
+    assertEquals(r1.getBoolVal(), TypedBytesInput.get(distream).read());
+    assertEquals(r1.getByteVal(), TypedBytesInput.get(distream).read());
+    assertEquals(r1.getIntVal(), TypedBytesInput.get(distream).read());
+    assertEquals(r1.getLongVal(), TypedBytesInput.get(distream).read());
+    assertEquals(r1.getFloatVal(), TypedBytesInput.get(distream).read());
+    assertEquals(r1.getDoubleVal(), TypedBytesInput.get(distream).read());
+    assertEquals(r1.getStringVal(), TypedBytesInput.get(distream).read());
+    Object prevObj = null, obj = TypedBytesInput.get(distream).read();
+    while (obj != null) {
+      prevObj = obj;
+      obj = TypedBytesInput.get(distream).read();
+    }
+    List recList = (List) prevObj;
+    assertEquals(r0.getStringVal(), recList.get(0));
+
+    distream.close();
+    istream.close();
+  }
+
+}

+ 62 - 0
src/contrib/streaming/src/test/org/apache/hadoop/typedbytes/TestTypedBytesWritable.java

@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.typedbytes;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import junit.framework.TestCase;
+
+public class TestTypedBytesWritable extends TestCase {
+
+  public void testToString() {
+    TypedBytesWritable tbw = new TypedBytesWritable();
+    tbw.setValue(true);
+    assertEquals("true", tbw.toString());
+    tbw.setValue(12345);
+    assertEquals("12345", tbw.toString());
+    tbw.setValue(123456789L);
+    assertEquals("123456789", tbw.toString());
+    tbw.setValue((float) 1.23);
+    assertEquals("1.23", tbw.toString());
+    tbw.setValue(1.23456789);
+    assertEquals("1.23456789", tbw.toString());
+    tbw.setValue("random text");
+    assertEquals("random text", tbw.toString());
+  }
+
+  public void testIO() throws IOException {
+    TypedBytesWritable tbw = new TypedBytesWritable();
+    tbw.setValue(12345);
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutput dout = new DataOutputStream(baos);
+    tbw.write(dout);
+    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+    DataInput din = new DataInputStream(bais);
+    TypedBytesWritable readTbw = new TypedBytesWritable();
+    readTbw.readFields(din);
+    assertEquals(tbw, readTbw);
+  }
+
+}