Browse Source

HADOOP-65. Initial version of multi-language record system. Contributed by Milind Bhandarkar.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@399509 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting 19 years ago
parent
commit
b783ae9ecf
78 changed files with 11147 additions and 1 deletions
  1. 6 0
      CHANGES.txt
  2. 97 0
      bin/rcc
  3. 9 1
      build.xml
  4. 52 0
      src/c++/librecordio/Makefile
  5. 118 0
      src/c++/librecordio/archive.hh
  6. 349 0
      src/c++/librecordio/binarchive.cc
  7. 80 0
      src/c++/librecordio/binarchive.hh
  8. 355 0
      src/c++/librecordio/csvarchive.cc
  9. 126 0
      src/c++/librecordio/csvarchive.hh
  10. 140 0
      src/c++/librecordio/exception.cc
  11. 125 0
      src/c++/librecordio/exception.hh
  12. 96 0
      src/c++/librecordio/filestream.cc
  13. 53 0
      src/c++/librecordio/filestream.hh
  14. 73 0
      src/c++/librecordio/recordio.cc
  15. 78 0
      src/c++/librecordio/recordio.hh
  16. 46 0
      src/c++/librecordio/test/Makefile
  17. 109 0
      src/c++/librecordio/test/test.cc
  18. 24 0
      src/c++/librecordio/test/test.hh
  19. 35 0
      src/c++/librecordio/test/test.jr
  20. 72 0
      src/c++/librecordio/test/testFromJava.cc
  21. 24 0
      src/c++/librecordio/test/testFromJava.hh
  22. 423 0
      src/c++/librecordio/xmlarchive.cc
  23. 263 0
      src/c++/librecordio/xmlarchive.hh
  24. 115 0
      src/java/org/apache/hadoop/record/BinaryInputArchive.java
  25. 101 0
      src/java/org/apache/hadoop/record/BinaryOutputArchive.java
  26. 209 0
      src/java/org/apache/hadoop/record/CsvInputArchive.java
  27. 147 0
      src/java/org/apache/hadoop/record/CsvOutputArchive.java
  28. 37 0
      src/java/org/apache/hadoop/record/Index.java
  29. 45 0
      src/java/org/apache/hadoop/record/InputArchive.java
  30. 47 0
      src/java/org/apache/hadoop/record/OutputArchive.java
  31. 31 0
      src/java/org/apache/hadoop/record/Record.java
  32. 96 0
      src/java/org/apache/hadoop/record/RecordReader.java
  33. 115 0
      src/java/org/apache/hadoop/record/RecordWriter.java
  34. 366 0
      src/java/org/apache/hadoop/record/Utils.java
  35. 250 0
      src/java/org/apache/hadoop/record/XmlInputArchive.java
  36. 251 0
      src/java/org/apache/hadoop/record/XmlOutputArchive.java
  37. 64 0
      src/java/org/apache/hadoop/record/compiler/CppGenerator.java
  38. 41 0
      src/java/org/apache/hadoop/record/compiler/JBoolean.java
  39. 67 0
      src/java/org/apache/hadoop/record/compiler/JBuffer.java
  40. 33 0
      src/java/org/apache/hadoop/record/compiler/JByte.java
  41. 51 0
      src/java/org/apache/hadoop/record/compiler/JCompType.java
  42. 38 0
      src/java/org/apache/hadoop/record/compiler/JDouble.java
  43. 93 0
      src/java/org/apache/hadoop/record/compiler/JField.java
  44. 56 0
      src/java/org/apache/hadoop/record/compiler/JFile.java
  45. 37 0
      src/java/org/apache/hadoop/record/compiler/JFloat.java
  46. 33 0
      src/java/org/apache/hadoop/record/compiler/JInt.java
  47. 37 0
      src/java/org/apache/hadoop/record/compiler/JLong.java
  48. 99 0
      src/java/org/apache/hadoop/record/compiler/JMap.java
  49. 384 0
      src/java/org/apache/hadoop/record/compiler/JRecord.java
  50. 47 0
      src/java/org/apache/hadoop/record/compiler/JString.java
  51. 127 0
      src/java/org/apache/hadoop/record/compiler/JType.java
  52. 92 0
      src/java/org/apache/hadoop/record/compiler/JVector.java
  53. 47 0
      src/java/org/apache/hadoop/record/compiler/JavaGenerator.java
  54. 208 0
      src/java/org/apache/hadoop/record/compiler/generated/ParseException.java
  55. 517 0
      src/java/org/apache/hadoop/record/compiler/generated/Rcc.java
  56. 86 0
      src/java/org/apache/hadoop/record/compiler/generated/RccConstants.java
  57. 841 0
      src/java/org/apache/hadoop/record/compiler/generated/RccTokenManager.java
  58. 455 0
      src/java/org/apache/hadoop/record/compiler/generated/SimpleCharStream.java
  59. 97 0
      src/java/org/apache/hadoop/record/compiler/generated/Token.java
  60. 149 0
      src/java/org/apache/hadoop/record/compiler/generated/TokenMgrError.java
  61. 366 0
      src/java/org/apache/hadoop/record/compiler/generated/rcc.jj
  62. 785 0
      src/java/org/apache/hadoop/record/package.html
  63. 6 0
      src/test/ddl/buffer.jr
  64. 6 0
      src/test/ddl/int.jr
  65. 18 0
      src/test/ddl/links.jr
  66. 8 0
      src/test/ddl/location.jr
  67. 6 0
      src/test/ddl/string.jr
  68. 19 0
      src/test/ddl/test.jr
  69. 124 0
      src/test/org/apache/hadoop/record/test/FromCpp.java
  70. 95 0
      src/test/org/apache/hadoop/record/test/RecBuffer.java
  71. 96 0
      src/test/org/apache/hadoop/record/test/RecInt.java
  72. 96 0
      src/test/org/apache/hadoop/record/test/RecRecord0.java
  73. 331 0
      src/test/org/apache/hadoop/record/test/RecRecord1.java
  74. 96 0
      src/test/org/apache/hadoop/record/test/RecString.java
  75. 452 0
      src/test/org/apache/hadoop/record/test/TestMapRed.java
  76. 138 0
      src/test/org/apache/hadoop/record/test/TestRecordIO.java
  77. 126 0
      src/test/org/apache/hadoop/record/test/TestWritable.java
  78. 117 0
      src/test/org/apache/hadoop/record/test/ToCpp.java

+ 6 - 0
CHANGES.txt

@@ -171,6 +171,12 @@ Trunk (unreleased)
     correctly handle job jar files that contain a lib directory with
     nested jar files.  (cutting)
 
+45. HADOOP-65.  Initial version of record I/O framework that enables
+    the specification of record types and generates marshalling code
+    in both Java and C++.  Generated Java code implements
+    WritableComparable, but is not yet otherwise used by
+    Hadoop. (Milind Bhandarkar via cutting)
+
 
 Release 0.1.1 - 2006-04-08
 

+ 97 - 0
bin/rcc

@@ -0,0 +1,97 @@
+#!/bin/bash
+# 
+# The Hadoop record compiler
+#
+# Environment Variables
+#
+#   JAVA_HOME        The java implementation to use.  Overrides JAVA_HOME.
+#
+#   HADOOP_OPTS      Extra Java runtime options.
+#
+#   HADOOP_CONF_DIR  Alternate conf dir. Default is ${HADOOP_HOME}/conf.
+#
+
+# resolve links - $0 may be a softlink
+THIS="$0"
+while [ -h "$THIS" ]; do
+  ls=`ls -ld "$THIS"`
+  link=`expr "$ls" : '.*-> \(.*\)$'`
+  if expr "$link" : '.*/.*' > /dev/null; then
+    THIS="$link"
+  else
+    THIS=`dirname "$THIS"`/"$link"
+  fi
+done
+
+# some directories
+THIS_DIR=`dirname "$THIS"`
+HADOOP_HOME=`cd "$THIS_DIR/.." ; pwd`
+
+# Allow alternate conf dir location.
+HADOOP_CONF_DIR="${HADOOP_CONF_DIR:-$HADOOP_HOME/conf}"
+
+if [ -f "${HADOOP_CONF_DIR}/hadoop-env.sh" ]; then
+  source "${HADOOP_CONF_DIR}/hadoop-env.sh"
+fi
+
+# some Java parameters
+if [ "$JAVA_HOME" != "" ]; then
+  #echo "run java in $JAVA_HOME"
+  JAVA_HOME=$JAVA_HOME
+fi
+  
+if [ "$JAVA_HOME" = "" ]; then
+  echo "Error: JAVA_HOME is not set."
+  exit 1
+fi
+
+JAVA=$JAVA_HOME/bin/java
+JAVA_HEAP_MAX=-Xmx1000m 
+
+# CLASSPATH initially contains $HADOOP_CONF_DIR
+CLASSPATH="${HADOOP_CONF_DIR}"
+CLASSPATH=${CLASSPATH}:$JAVA_HOME/lib/tools.jar
+
+# for developers, add Hadoop classes to CLASSPATH
+if [ -d "$HADOOP_HOME/build/classes" ]; then
+  CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build/classes
+fi
+if [ -d "$HADOOP_HOME/build/webapps" ]; then
+  CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build
+fi
+if [ -d "$HADOOP_HOME/build/test/classes" ]; then
+  CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build/test/classes
+fi
+
+# so that filenames w/ spaces are handled correctly in loops below
+IFS=
+
+# for releases, add hadoop jars & webapps to CLASSPATH
+if [ -d "$HADOOP_HOME/webapps" ]; then
+  CLASSPATH=${CLASSPATH}:$HADOOP_HOME
+fi
+for f in $HADOOP_HOME/hadoop-*.jar; do
+  CLASSPATH=${CLASSPATH}:$f;
+done
+
+# add libs to CLASSPATH
+for f in $HADOOP_HOME/lib/*.jar; do
+  CLASSPATH=${CLASSPATH}:$f;
+done
+
+for f in $HADOOP_HOME/lib/jetty-ext/*.jar; do
+  CLASSPATH=${CLASSPATH}:$f;
+done
+
+# restore ordinary behaviour
+unset IFS
+
+CLASS='org.apache.hadoop.record.compiler.generated.Rcc'
+
+# cygwin path translation
+if expr `uname` : 'CYGWIN*' > /dev/null; then
+  CLASSPATH=`cygpath -p -w "$CLASSPATH"`
+fi
+
+# run it
+exec "$JAVA" $HADOOP_OPTS -classpath "$CLASSPATH" $CLASS "$@"

+ 9 - 1
build.xml

@@ -108,7 +108,15 @@
       <classpath refid="classpath"/>
   </taskdef>
 
-  <target name="compile" depends="init">
+  <target name="record-parser" depends="init" if="javacc.home">
+      <javacc
+          target="${src.dir}/org/apache/hadoop/record/compiler/generated/rcc.jj"
+          outputdirectory="${src.dir}/org/apache/hadoop/record/compiler/generated"
+          javacchome="${javacc.home}"
+      />
+  </target>
+
+  <target name="compile" depends="init, record-parser">
 
     <jsp-compile
      uriroot="${src.webapps}/mapred"

+ 52 - 0
src/c++/librecordio/Makefile

@@ -0,0 +1,52 @@
+#
+# Copyright 2005 The Apache Software Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+all: librecordio.a test
+
+librecordio.a: recordio.o filestream.o binarchive.o csvarchive.o xmlarchive.o exception.o
+	ar cru librecordio.a recordio.o filestream.o binarchive.o csvarchive.o xmlarchive.o exception.o
+
+recordio.o: recordio.cc
+	g++ -g3 -O0 -c -I${XERCESCROOT}/include -o recordio.o recordio.cc
+	
+filestream.o: filestream.cc
+	g++ -g3 -O0 -c -o filestream.o filestream.cc
+	
+binarchive.o: binarchive.cc
+	g++ -g3 -O0 -c -o binarchive.o binarchive.cc
+
+csvarchive.o: csvarchive.cc
+	g++ -g3 -O0 -c -o csvarchive.o csvarchive.cc
+
+xmlarchive.o: xmlarchive.cc
+	g++ -g3 -O0 -c -I${XERCESCROOT}/include -o xmlarchive.o xmlarchive.cc
+		
+exception.o: exception.cc
+	g++ -g3 -O0 -c -o exception.o exception.cc
+	
+recordio.cc: recordio.hh archive.hh exception.hh
+filestream.cc: recordio.hh filestream.hh 
+binarchive.cc: recordio.hh binarchive.hh 
+csvarchive.cc: recordio.hh csvarchive.hh 
+xmlarchive.cc: recordio.hh xmlarchive.hh 
+exception.cc: exception.hh 
+
+test: librecordio.a
+	make -C test all
+	
+clean:
+	rm -f *~ *.o *.a
+	make -C test clean

+ 118 - 0
src/c++/librecordio/archive.hh

@@ -0,0 +1,118 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef ARCHIVE_HH_
+#define ARCHIVE_HH_
+#include "recordio.hh"
+
+namespace hadoop {
+
+class Index {
+public:
+  virtual bool done() = 0;
+  virtual void incr() = 0;
+  virtual ~Index() {}
+};
+
+class IArchive {
+public:
+  virtual void deserialize(int8_t& t, const char* tag) = 0;
+  virtual void deserialize(bool& t, const char* tag) = 0;
+  virtual void deserialize(int32_t& t, const char* tag) = 0;
+  virtual void deserialize(int64_t& t, const char* tag) = 0;
+  virtual void deserialize(float& t, const char* tag) = 0;
+  virtual void deserialize(double& t, const char* tag) = 0;
+  virtual void deserialize(std::string& t, const char* tag) = 0;
+  virtual void deserialize(std::string& t, size_t& len, const char* tag) = 0;
+  virtual void startRecord(hadoop::Record& s, const char* tag) = 0;
+  virtual void endRecord(hadoop::Record& s, const char* tag) = 0;
+  virtual Index* startVector(const char* tag) = 0;
+  virtual void endVector(Index* idx, const char* tag) = 0;
+  virtual Index* startMap(const char* tag) = 0;
+  virtual void endMap(Index* idx, const char* tag) = 0;
+  virtual void deserialize(hadoop::Record& s, const char* tag) {
+    s.deserialize(*this, tag);
+  }
+  template <typename T>
+  void deserialize(std::vector<T>& v, const char* tag) {
+    Index* idx = startVector(tag);
+    while (!idx->done()) {
+      T t;
+      deserialize(t, tag);
+      v.push_back(t);
+      idx->incr();
+    }
+    endVector(idx, tag);
+  }
+  template <typename K, typename V>
+  void deserialize(std::map<K,V>& v, const char* tag) {
+    Index* idx = startMap(tag);
+    while (!idx->done()) {
+      K key;
+      deserialize(key, tag);
+      V value;
+      deserialize(value, tag);
+      v[key] = value;
+      idx->incr();
+    }
+    endMap(idx, tag);
+  }
+};
+
+class OArchive {
+public:
+  virtual void serialize(int8_t t, const char* tag) = 0;
+  virtual void serialize(bool t, const char* tag) = 0;
+  virtual void serialize(int32_t t, const char* tag) = 0;
+  virtual void serialize(int64_t t, const char* tag) = 0;
+  virtual void serialize(float t, const char* tag) = 0;
+  virtual void serialize(double t, const char* tag) = 0;
+  virtual void serialize(const std::string& t, const char* tag) = 0;
+  virtual void serialize(const std::string& t, size_t len, const char* tag) = 0;
+  virtual void startRecord(const hadoop::Record& s, const char* tag) = 0;
+  virtual void endRecord(const hadoop::Record& s, const char* tag) = 0;
+  virtual void startVector(size_t len, const char* tag) = 0;
+  virtual void endVector(size_t len, const char* tag) = 0;
+  virtual void startMap(size_t len, const char* tag) = 0;
+  virtual void endMap(size_t len, const char* tag) = 0;
+  virtual void serialize(hadoop::Record& s, const char* tag) {
+    s.serialize(*this, tag);
+  }
+  template <typename T>
+  void serialize(const std::vector<T>& v, const char* tag) {
+    startVector(v.size(), tag);
+    if (v.size()>0) {
+      for (size_t cur = 0; cur<v.size(); cur++) {
+        serialize(v[cur], tag);
+      }
+    }
+    endVector(v.size(), tag);
+  }
+  template <typename K, typename V>
+  void serialize(const std::map<K,V>& v, const char* tag) {
+    startMap(v.size(), tag);
+    if (v.size()>0) {
+      typedef typename std::map<K,V>::const_iterator CI;
+      for (CI cur = v.begin(); cur!=v.end(); cur++) {
+        serialize(cur->first, tag);
+        serialize(cur->second, tag);
+      }
+    }
+    endMap(v.size(), tag);
+ }
+};
+}; // end namespace hadoop
+#endif /*ARCHIVE_HH_*/

+ 349 - 0
src/c++/librecordio/binarchive.cc

@@ -0,0 +1,349 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "binarchive.hh"
+
+using namespace hadoop;
+
+template <typename T>
+static void serialize(T t, OutStream& stream)
+{
+  if (sizeof(T) != stream.write((const void *) &t, sizeof(T))) {
+    throw new IOException("Error serializing data.");
+  }
+}
+
+template <typename T>
+static void deserialize(T& t, InStream& stream)
+{
+  if (sizeof(T) != stream.read((void *) &t, sizeof(T))) {
+    throw new IOException("Error deserializing data.");
+  }
+}
+
+static void serializeInt(int32_t t, OutStream& stream)
+{
+  if (t >= -120 && t <= 127) {
+    int8_t b = t;
+    stream.write(&b, 1);
+    return;
+  }
+        
+  int8_t len = -120;
+  if (t < 0) {
+    t &= 0x7FFFFFFF; // reset the sign bit
+    len = -124;
+  }
+        
+  uint32_t tmp = t;
+  while (tmp != 0) {
+    tmp = tmp >> 8;
+    len--;
+  }
+  
+  stream.write(&len, 1);      
+  len = (len < -124) ? -(len + 124) : -(len + 120);
+        
+  for (uint32_t idx = len; idx != 0; idx--) {
+    uint32_t shiftbits = (idx - 1) * 8;
+    uint32_t mask = 0xFF << shiftbits;
+    uint8_t b = (t & mask) >> shiftbits;
+    stream.write(&b, 1);
+  }
+}
+
+static void deserializeInt(int32_t& t, InStream& stream)
+{
+  int8_t b;
+  if (1 != stream.read(&b, 1)) {
+    throw new IOException("Error deserializing int");
+  }
+  if (b >= -120) {
+    t = b;
+    return;
+  }
+  b = (b < -124) ? -(b + 124) : -(b + 120);
+  uint8_t barr[b];
+  if (b != stream.read(barr, b)) {
+    throw new IOException("Error deserializing long");
+  }
+  t = 0;
+  for (int idx = 0; idx < b; idx++) {
+    t = t << 8;
+    t |= (barr[idx] & 0xFF);
+  }
+}
+
+static void serializeLong(int64_t t, OutStream& stream)
+{
+  if (t >= -112 && t <= 127) {
+    int8_t b = t;
+    stream.write(&b, 1);
+    return;
+  }
+        
+  int8_t len = -112;
+  if (t < 0) {
+    t &= 0x7FFFFFFFFFFFFFFFLL; // reset the sign bit
+    len = -120;
+  }
+        
+  uint64_t tmp = t;
+  while (tmp != 0) {
+    tmp = tmp >> 8;
+    len--;
+  }
+  
+  stream.write(&len, 1);
+        
+  len = (len < -120) ? -(len + 120) : -(len + 112);
+        
+  for (uint32_t idx = len; idx != 0; idx--) {
+    uint32_t shiftbits = (idx - 1) * 8;
+    uint64_t mask = 0xFFLL << shiftbits;
+    uint8_t b = (t & mask) >> shiftbits;
+    stream.write(&b, 1);
+  }
+}
+
+static void deserializeLong(int64_t& t, InStream& stream)
+{
+  int8_t b;
+  if (1 != stream.read(&b, 1)) {
+    throw new IOException("Error deserializing long.");
+  }
+  if (b >= -112) {
+    t = b;
+    return;
+  }
+  b = (b < -120) ? -(b + 120) : -(b + 112);
+  uint8_t barr[b];
+  if (b != stream.read(barr, b)) {
+    throw new IOException("Error deserializing long.");
+  }
+  t = 0;
+  for (int idx = 0; idx < b; idx++) {
+    t = t << 8;
+    t |= (barr[idx] & 0xFF);
+  }
+}
+
+static void serializeFloat(float t, OutStream& stream)
+{
+  char buf[sizeof(float)];
+  XDR xdrs;
+  xdrmem_create(&xdrs, buf, sizeof(float), XDR_ENCODE);
+  xdr_float(&xdrs, &t);
+  stream.write(buf, sizeof(float));
+}
+
+static void deserializeFloat(float& t, InStream& stream)
+{
+  char buf[sizeof(float)];
+  if (sizeof(float) != stream.read(buf, sizeof(float))) {
+    throw new IOException("Error deserializing float.");
+  }
+  XDR xdrs;
+  xdrmem_create(&xdrs, buf, sizeof(float), XDR_DECODE);
+  xdr_float(&xdrs, &t);
+}
+
+static void serializeDouble(double t, OutStream& stream)
+{
+  char buf[sizeof(double)];
+  XDR xdrs;
+  xdrmem_create(&xdrs, buf, sizeof(double), XDR_ENCODE);
+  xdr_double(&xdrs, &t);
+  stream.write(buf, sizeof(double));
+}
+
+static void deserializeDouble(double& t, InStream& stream)
+{
+  char buf[sizeof(double)];
+  stream.read(buf, sizeof(double));
+  XDR xdrs;
+  xdrmem_create(&xdrs, buf, sizeof(double), XDR_DECODE);
+  xdr_double(&xdrs, &t);
+}
+
+static void serializeString(const std::string& t, OutStream& stream)
+{
+  ::serializeInt(t.length(), stream);
+  if (t.length() > 0) {
+    stream.write(t.data(), t.length());
+  }
+}
+
+static void deserializeString(std::string& t, InStream& stream)
+{
+  int32_t len = 0;
+  ::deserializeInt(len, stream);
+  if (len > 0) {
+    char buf[len];
+    stream.read((void*) buf, len);
+    std::string s(buf, len);
+    t = s;
+  }
+}
+
+void hadoop::IBinArchive::deserialize(int8_t& t, const char* tag)
+{
+  ::deserialize(t, stream);
+}
+
+void hadoop::IBinArchive::deserialize(bool& t, const char* tag)
+{
+  ::deserialize(t, stream);
+}
+
+void hadoop::IBinArchive::deserialize(int32_t& t, const char* tag)
+{
+  ::deserializeInt(t, stream);
+}
+
+void hadoop::IBinArchive::deserialize(int64_t& t, const char* tag)
+{
+  ::deserializeLong(t, stream);
+}
+
+void hadoop::IBinArchive::deserialize(float& t, const char* tag)
+{
+  ::deserializeFloat(t, stream);
+}
+
+void hadoop::IBinArchive::deserialize(double& t, const char* tag)
+{
+  ::deserializeDouble(t, stream);
+}
+
+void hadoop::IBinArchive::deserialize(std::string& t, const char* tag)
+{
+  ::deserializeString(t, stream);
+}
+
+void hadoop::IBinArchive::deserialize(std::string& t, size_t& len, const char* tag)
+{
+  ::deserializeString(t, stream);
+  len = t.length();
+}
+
+void hadoop::IBinArchive::startRecord(Record& s, const char* tag)
+{
+}
+
+void hadoop::IBinArchive::endRecord(Record& s, const char* tag)
+{
+}
+
+Index* hadoop::IBinArchive::startVector(const char* tag)
+{
+  int32_t len;
+  ::deserializeInt(len, stream);
+  BinIndex *idx = new BinIndex((size_t) len);
+  return idx;
+}
+
+void hadoop::IBinArchive::endVector(Index* idx, const char* tag)
+{
+  delete idx;
+}
+
+Index* hadoop::IBinArchive::startMap(const char* tag)
+{
+  int32_t len;
+  ::deserializeInt(len, stream);
+  BinIndex *idx = new BinIndex((size_t) len);
+  return idx;
+}
+
+void hadoop::IBinArchive::endMap(Index* idx, const char* tag)
+{
+  delete idx;
+}
+
+hadoop::IBinArchive::~IBinArchive()
+{
+}
+
+void hadoop::OBinArchive::serialize(int8_t t, const char* tag)
+{
+  ::serialize(t, stream);
+}
+
+void hadoop::OBinArchive::serialize(bool t, const char* tag)
+{
+  ::serialize(t, stream);
+}
+
+void hadoop::OBinArchive::serialize(int32_t t, const char* tag)
+{
+  ::serializeInt(t, stream);
+}
+
+void hadoop::OBinArchive::serialize(int64_t t, const char* tag)
+{
+  ::serializeLong(t, stream);
+}
+
+void hadoop::OBinArchive::serialize(float t, const char* tag)
+{
+  ::serializeFloat(t, stream);
+}
+
+void hadoop::OBinArchive::serialize(double t, const char* tag)
+{
+  ::serializeDouble(t, stream);
+}
+
+void hadoop::OBinArchive::serialize(const std::string& t, const char* tag)
+{
+  ::serializeString(t, stream);
+}
+
+void hadoop::OBinArchive::serialize(const std::string& t, size_t len, const char* tag)
+{
+  ::serializeString(t, stream);
+}
+
+void hadoop::OBinArchive::startRecord(const Record& s, const char* tag)
+{
+}
+
+void hadoop::OBinArchive::endRecord(const Record& s, const char* tag)
+{
+}
+
+void hadoop::OBinArchive::startVector(size_t len, const char* tag)
+{
+  ::serializeInt(len, stream);
+}
+
+void hadoop::OBinArchive::endVector(size_t len, const char* tag)
+{
+}
+
+void hadoop::OBinArchive::startMap(size_t len, const char* tag)
+{
+  ::serializeInt(len, stream);
+}
+
+void hadoop::OBinArchive::endMap(size_t len, const char* tag)
+{
+}
+
+hadoop::OBinArchive::~OBinArchive()
+{
+}

+ 80 - 0
src/c++/librecordio/binarchive.hh

@@ -0,0 +1,80 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef BINARCHIVE_HH_
+#define BINARCHIVE_HH_
+
+#include "recordio.hh"
+#include <rpc/xdr.h>
+
+namespace hadoop {
+
+class BinIndex : public Index {
+private:
+  size_t size;
+public:
+  BinIndex(size_t size_) { size = size_; }
+  bool done() { return (size==0); }
+  void incr() { size--; }
+  ~BinIndex() {}
+};
+  
+class IBinArchive : public IArchive {
+private:
+  InStream& stream;
+public:
+  IBinArchive(InStream& _stream) : stream(_stream) {}
+  virtual void deserialize(int8_t& t, const char* tag);
+  virtual void deserialize(bool& t, const char* tag);
+  virtual void deserialize(int32_t& t, const char* tag);
+  virtual void deserialize(int64_t& t, const char* tag);
+  virtual void deserialize(float& t, const char* tag);
+  virtual void deserialize(double& t, const char* tag);
+  virtual void deserialize(std::string& t, const char* tag);
+  virtual void deserialize(std::string& t, size_t& len, const char* tag);
+  virtual void startRecord(Record& s, const char* tag);
+  virtual void endRecord(Record& s, const char* tag);
+  virtual Index* startVector(const char* tag);
+  virtual void endVector(Index* idx, const char* tag);
+  virtual Index* startMap(const char* tag);
+  virtual void endMap(Index* idx, const char* tag);
+  virtual ~IBinArchive();
+};
+
+class OBinArchive : public OArchive {
+private:
+  OutStream& stream;
+public:
+  OBinArchive(OutStream& _stream) : stream(_stream) {}
+  virtual void serialize(int8_t t, const char* tag);
+  virtual void serialize(bool t, const char* tag);
+  virtual void serialize(int32_t t, const char* tag);
+  virtual void serialize(int64_t t, const char* tag);
+  virtual void serialize(float t, const char* tag);
+  virtual void serialize(double t, const char* tag);
+  virtual void serialize(const std::string& t, const char* tag);
+  virtual void serialize(const std::string& t, size_t len, const char* tag);
+  virtual void startRecord(const Record& s, const char* tag);
+  virtual void endRecord(const Record& s, const char* tag);
+  virtual void startVector(size_t len, const char* tag);
+  virtual void endVector(size_t len, const char* tag);
+  virtual void startMap(size_t len, const char* tag);
+  virtual void endMap(size_t len, const char* tag);
+  virtual ~OBinArchive();
+};
+
+}
+#endif /*BINARCHIVE_HH_*/

+ 355 - 0
src/c++/librecordio/csvarchive.cc

@@ -0,0 +1,355 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "csvarchive.hh"
+#include <stdlib.h>
+
+using namespace hadoop;
+
+static std::string readUptoTerminator(PushBackInStream& stream)
+{
+  std::string s;
+  while (1) {
+    char c;
+    if (1 != stream.read(&c, 1)) {
+      throw new IOException("Error in deserialization.");
+    }
+    if (c == ',' || c == '\n' || c == '}') {
+      if (c != ',') {
+        stream.pushBack(c);
+      }
+      break;
+    }
+    s.push_back(c);
+  }
+  return s;
+}
+
+void hadoop::ICsvArchive::deserialize(int8_t& t, const char* tag)
+{
+  std::string s = readUptoTerminator(stream);
+  t = (int8_t) strtol(s.c_str(), NULL, 10);
+}
+
+void hadoop::ICsvArchive::deserialize(bool& t, const char* tag)
+{
+  std::string s = readUptoTerminator(stream);
+  t = (s == "T") ? true : false;
+}
+
+void hadoop::ICsvArchive::deserialize(int32_t& t, const char* tag)
+{
+  std::string s = readUptoTerminator(stream);
+  t = strtol(s.c_str(), NULL, 10);
+}
+
+void hadoop::ICsvArchive::deserialize(int64_t& t, const char* tag)
+{
+  std::string s = readUptoTerminator(stream);
+  t = strtoll(s.c_str(), NULL, 10);
+}
+
+void hadoop::ICsvArchive::deserialize(float& t, const char* tag)
+{
+  std::string s = readUptoTerminator(stream);
+  t = strtof(s.c_str(), NULL);
+}
+
+void hadoop::ICsvArchive::deserialize(double& t, const char* tag)
+{
+  std::string s = readUptoTerminator(stream);
+  t = strtod(s.c_str(), NULL);
+}
+
+static void replaceAll(std::string s, const char *src, char c)
+{
+  std::string::size_type pos = 0;
+  while (pos != std::string::npos) {
+    pos = s.find(src);
+    if (pos != std::string::npos) {
+      s.replace(pos, strlen(src), 1, c);
+    }
+  }
+}
+
+void hadoop::ICsvArchive::deserialize(std::string& t, const char* tag)
+{
+  t = readUptoTerminator(stream);
+  if (t[0] != '\'') {
+    throw new IOException("Errror deserializing string.");
+  }
+  t.erase(0, 1); /// erase first character
+  replaceAll(t, "%0D", 0x0D);
+  replaceAll(t, "%0A", 0x0A);
+  replaceAll(t, "%7D", 0x7D);
+  replaceAll(t, "%00", 0x00);
+  replaceAll(t, "%2C", 0x2C);
+  replaceAll(t, "%25", 0x25);
+
+}
+
+void hadoop::ICsvArchive::deserialize(std::string& t, size_t& len, const char* tag)
+{
+  std::string s = readUptoTerminator(stream);
+  if (s[0] != '#') {
+    throw new IOException("Errror deserializing buffer.");
+  }
+  s.erase(0, 1); /// erase first character
+  len = s.length();
+  if (len%2 == 1) { // len is guaranteed to be even
+    throw new IOException("Errror deserializing buffer.");
+  }
+  len >> 1;
+  for (size_t idx = 0; idx < len; idx++) {
+    char buf[3];
+    buf[0] = s[2*idx];
+    buf[1] = s[2*idx+1];
+    buf[2] = '\0';
+    int i;
+    if (1 != sscanf(buf, "%2x", &i)) {
+      throw new IOException("Errror deserializing buffer.");
+    }
+    t.push_back((char) i);
+  }
+  len = t.length();
+}
+
+void hadoop::ICsvArchive::startRecord(Record& s, const char* tag)
+{
+  if (tag != NULL) {
+    char mark[2];
+    if (2 != stream.read(mark, 2)) {
+      throw new IOException("Error deserializing record.");
+    }
+    if (mark[0] != 's' || mark[1] != '{') {
+      throw new IOException("Error deserializing record.");
+    }
+  }
+}
+
+void hadoop::ICsvArchive::endRecord(Record& s, const char* tag)
+{
+  char mark;
+  if (1 != stream.read(&mark, 1)) {
+    throw new IOException("Error deserializing record.");
+  }
+  if (tag == NULL) {
+    if (mark != '\n') {
+      throw new IOException("Error deserializing record.");
+    }
+  } else if (mark != '}') {
+    throw new IOException("Error deserializing record.");
+  } else {
+    readUptoTerminator(stream);
+  }
+}
+
+Index* hadoop::ICsvArchive::startVector(const char* tag)
+{
+  char mark[2];
+  if (2 != stream.read(mark, 2)) {
+    throw new IOException("Error deserializing vector.");
+  }
+  if (mark[0] != 'v' || mark[1] != '{') {
+    throw new IOException("Error deserializing vector.");
+  }
+  return new CsvIndex(stream);
+}
+
+void hadoop::ICsvArchive::endVector(Index* idx, const char* tag)
+{
+  delete idx;
+  char mark;
+  if (1 != stream.read(&mark, 1)) {
+    throw new IOException("Error deserializing vector.");
+  }
+  if (mark != '}') {
+    throw new IOException("Error deserializing vector.");
+  }
+  readUptoTerminator(stream);
+}
+
+Index* hadoop::ICsvArchive::startMap(const char* tag)
+{
+  char mark[2];
+  if (2 != stream.read(mark, 2)) {
+    throw new IOException("Error deserializing map.");
+  }
+  if (mark[0] != 'm' || mark[1] != '{') {
+    throw new IOException("Error deserializing map.");
+  }
+
+  return new CsvIndex(stream);
+}
+
+void hadoop::ICsvArchive::endMap(Index* idx, const char* tag)
+{
+  delete idx;
+  char mark;
+  if (1 != stream.read(&mark, 1)) {
+    throw new IOException("Error deserializing map.");
+  }
+  if (mark != '}') {
+    throw new IOException("Error deserializing map.");
+  }
+  readUptoTerminator(stream);
+}
+
+hadoop::ICsvArchive::~ICsvArchive()
+{
+}
+
+void hadoop::OCsvArchive::serialize(int8_t t, const char* tag)
+{
+  printCommaUnlessFirst();
+  char sval[5];
+  sprintf(sval, "%d", t);
+  stream.write(sval, strlen(sval));
+}
+
+void hadoop::OCsvArchive::serialize(bool t, const char* tag)
+{
+  printCommaUnlessFirst();
+  const char *sval = t ? "T" : "F";
+  stream.write(sval,1);  
+}
+
+void hadoop::OCsvArchive::serialize(int32_t t, const char* tag)
+{
+  printCommaUnlessFirst();
+  char sval[128];
+  sprintf(sval, "%d", t);
+  stream.write(sval, strlen(sval));
+}
+
+void hadoop::OCsvArchive::serialize(int64_t t, const char* tag)
+{
+  printCommaUnlessFirst();
+  char sval[128];
+  sprintf(sval, "%lld", t);
+  stream.write(sval, strlen(sval));
+}
+
+void hadoop::OCsvArchive::serialize(float t, const char* tag)
+{
+  printCommaUnlessFirst();
+  char sval[128];
+  sprintf(sval, "%f", t);
+  stream.write(sval, strlen(sval));
+}
+
+void hadoop::OCsvArchive::serialize(double t, const char* tag)
+{
+  printCommaUnlessFirst();
+  char sval[128];
+  sprintf(sval, "%lf", t);
+  stream.write(sval, strlen(sval));
+}
+
+void hadoop::OCsvArchive::serialize(const std::string& t, const char* tag)
+{
+  printCommaUnlessFirst();
+  stream.write("'",1);
+  int len = t.length();
+  for (int idx = 0; idx < len; idx++) {
+    char c = t[idx];
+    switch(c) {
+      case '\0':
+        stream.write("%00",3);
+        break;
+      case 0x0A:
+        stream.write("%0A",3);
+        break;
+      case 0x0D:
+        stream.write("%0D",3);
+        break;
+      case 0x25:
+        stream.write("%25",3);
+        break;
+      case 0x2C:
+        stream.write("%2C",3);
+        break;
+      case 0x7D:
+        stream.write("%7D",3);
+        break;
+      default:
+        stream.write(&c,1);
+        break;
+    }
+  }
+}
+
+void hadoop::OCsvArchive::serialize(const std::string& t, size_t len, const char* tag)
+{
+  printCommaUnlessFirst();
+  stream.write("#",1);
+  for(int idx = 0; idx < len; idx++) {
+    uint8_t b = t[idx];
+    char sval[3];
+    sprintf(sval,"%2x",b);
+    stream.write(sval, 2);
+  }
+}
+
+void hadoop::OCsvArchive::startRecord(const Record& s, const char* tag)
+{
+  printCommaUnlessFirst();
+  if (tag != NULL && strlen(tag) != 0) {
+    stream.write("s{",2);
+  }
+  isFirst = true;
+}
+
+void hadoop::OCsvArchive::endRecord(const Record& s, const char* tag)
+{
+  if (tag == NULL || strlen(tag) == 0) {
+    stream.write("\n",1);
+    isFirst = true;
+  } else {
+    stream.write("}",1);
+    isFirst = false;
+  }
+}
+
+void hadoop::OCsvArchive::startVector(size_t len, const char* tag)
+{
+  printCommaUnlessFirst();
+  stream.write("v{",2);
+  isFirst = true;
+}
+
+void hadoop::OCsvArchive::endVector(size_t len, const char* tag)
+{
+  stream.write("}",1);
+  isFirst = false;
+}
+
+void hadoop::OCsvArchive::startMap(size_t len, const char* tag)
+{
+  printCommaUnlessFirst();
+  stream.write("m{",2);
+  isFirst = true;
+}
+
+void hadoop::OCsvArchive::endMap(size_t len, const char* tag)
+{
+  stream.write("}",1);
+  isFirst = false;
+}
+
+hadoop::OCsvArchive::~OCsvArchive()
+{
+}

+ 126 - 0
src/c++/librecordio/csvarchive.hh

@@ -0,0 +1,126 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef CSVARCHIVE_HH_
+#define CSVARCHIVE_HH_
+
+#include "recordio.hh"
+
+namespace hadoop {
+
+class PushBackInStream {
+private:
+  InStream* stream;
+  bool isAvail;
+  char pbchar;
+public:
+  void setStream(InStream* stream_) {
+    stream = stream_;
+    isAvail = false;
+    pbchar = 0;
+  }
+  ssize_t read(void* buf, size_t len) {
+    if (len > 0 && isAvail) {
+      char* p = (char*) buf;
+      *p = pbchar;
+      isAvail = false;
+      if (len > 1) {
+        ssize_t ret = stream->read((char*)buf + 1, len - 1);
+        return ret + 1;
+      } else {
+        return 1;
+      }
+    } else {
+      return stream->read(buf, len);
+    }
+  }
+  void pushBack(char c) {
+    pbchar = c;
+    isAvail = true;
+  }
+};
+
+class CsvIndex : public Index {
+private:
+  PushBackInStream& stream;
+public:
+  CsvIndex(PushBackInStream& _stream) : stream(_stream) {}
+  bool done() {
+    char c;
+    stream.read(&c, 1);
+    if (c != ',') {
+      stream.pushBack(c);
+    }
+    return (c == '}') ? true : false;
+  }
+  void incr() {}
+  ~CsvIndex() {} 
+};
+  
+class ICsvArchive : public IArchive {
+private:
+  PushBackInStream stream;
+public:
+  ICsvArchive(InStream& _stream) { stream.setStream(&_stream); }
+  virtual void deserialize(int8_t& t, const char* tag);
+  virtual void deserialize(bool& t, const char* tag);
+  virtual void deserialize(int32_t& t, const char* tag);
+  virtual void deserialize(int64_t& t, const char* tag);
+  virtual void deserialize(float& t, const char* tag);
+  virtual void deserialize(double& t, const char* tag);
+  virtual void deserialize(std::string& t, const char* tag);
+  virtual void deserialize(std::string& t, size_t& len, const char* tag);
+  virtual void startRecord(Record& s, const char* tag);
+  virtual void endRecord(Record& s, const char* tag);
+  virtual Index* startVector(const char* tag);
+  virtual void endVector(Index* idx, const char* tag);
+  virtual Index* startMap(const char* tag);
+  virtual void endMap(Index* idx, const char* tag);
+  virtual ~ICsvArchive();
+};
+
+class OCsvArchive : public OArchive {
+private:
+  OutStream& stream;
+  bool isFirst;
+  
+  void printCommaUnlessFirst() {
+    if (!isFirst) {
+      stream.write(",",1);
+    }
+    isFirst = false;
+  }
+public:
+  OCsvArchive(OutStream& _stream) : stream(_stream) {isFirst = true;}
+  virtual void serialize(int8_t t, const char* tag);
+  virtual void serialize(bool t, const char* tag);
+  virtual void serialize(int32_t t, const char* tag);
+  virtual void serialize(int64_t t, const char* tag);
+  virtual void serialize(float t, const char* tag);
+  virtual void serialize(double t, const char* tag);
+  virtual void serialize(const std::string& t, const char* tag);
+  virtual void serialize(const std::string& t, size_t len, const char* tag);
+  virtual void startRecord(const Record& s, const char* tag);
+  virtual void endRecord(const Record& s, const char* tag);
+  virtual void startVector(size_t len, const char* tag);
+  virtual void endVector(size_t len, const char* tag);
+  virtual void startMap(size_t len, const char* tag);
+  virtual void endMap(size_t len, const char* tag);
+  virtual ~OCsvArchive();
+};
+
+}
+#endif /*CSVARCHIVE_HH_*/

+ 140 - 0
src/c++/librecordio/exception.cc

@@ -0,0 +1,140 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "exception.hh"
+#include <execinfo.h>
+
+#include <errno.h>
+#include <sstream>
+#include <typeinfo>
+
+using std::string;
+
+namespace hadoop {
+
+  /**
+   * Create an exception.
+   * @param message The message to give to the user.
+   * @param reason The exception that caused the new exception.
+   */
+  Exception::Exception(const string& message,
+                       const string& component,
+                       const string& location,
+                       const Exception* reason
+                       ): mMessage(message),
+                          mComponent(component),
+                          mLocation(location),
+                          mReason(reason)
+                          
+  {
+    mCalls = backtrace(mCallStack, sMaxCallStackDepth);
+  }
+
+  /**
+   * Copy the exception.
+   * Clones the reason, if there is one.
+   */
+  Exception::Exception(const Exception& other
+                       ): mMessage(other.mMessage), 
+                          mComponent(other.mComponent),
+                          mLocation(other.mLocation),
+                          mCalls(other.mCalls)
+  {
+    for(int i=0; i < mCalls; ++i) {
+      mCallStack[i] = other.mCallStack[i];
+    }
+    if (other.mReason) {
+      mReason = other.mReason->clone();
+    } else {
+      mReason = NULL;
+    }
+  }
+
+  Exception::~Exception() throw () {
+    delete mReason;
+  }
+
+  /**
+   * Print all of the information about the exception.
+   */
+  void Exception::print(std::ostream& stream) const {
+    stream << "Exception " << getTypename();
+    if (mComponent.size() != 0) {
+      stream << " (" << mComponent << ")";
+    }
+    stream << ": " << mMessage << "\n";
+    if (mLocation.size() != 0) {
+      stream << "  thrown at " << mLocation << "\n";
+    }
+    printCallStack(stream);
+    if (mReason) {
+      stream << "caused by: ";
+      mReason->print(stream);
+    }
+    stream.flush();
+  }
+
+  /**
+   * Result of print() as a string.
+   */
+  string Exception::toString() const {
+    std::ostringstream stream;
+    print(stream);
+    return stream.str();
+}
+
+  /**
+   * Print the call stack where the exception was created.
+   */
+  void Exception::printCallStack(std::ostream& stream) const {
+      char ** symbols = backtrace_symbols(mCallStack, mCalls);
+      for(int i=0; i < mCalls; ++i) {
+        stream << "  ";
+        if (i == 0) {
+          stream << "at ";
+        } else {
+          stream << "from ";
+        }
+        stream << symbols[i] << "\n";
+      }
+      free(symbols);
+  }
+
+  const char* Exception::getTypename() const {
+    return "Exception";
+  }
+
+  Exception* Exception::clone() const {
+    return new Exception(*this);
+  }
+
+  IOException::IOException(const string& message,
+                         const string& component,
+                         const string& location,
+                         const Exception* reason
+                         ): Exception(message, component, location, reason) 
+  {
+  }
+
+  const char* IOException::getTypename() const {
+    return "IOException";
+  }
+
+  IOException* IOException::clone() const {
+    return new IOException(*this);
+  }
+
+}

+ 125 - 0
src/c++/librecordio/exception.hh

@@ -0,0 +1,125 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef EXCEPTION_HH
+#define EXCEPTION_HH
+
+#include <exception>
+#include <iostream>
+#include <string>
+
+namespace hadoop {
+
+  /**
+   * Parent-type for all exceptions in hadoop.
+   * Provides an application specified message to the user, a call stack from
+   * where the exception was created, and optionally an exception that caused 
+   * this one.
+   */
+  class Exception: public std::exception {
+  public:
+
+    /**
+     * Create an exception.
+     * @param message The message to give to the user.
+     * @param reason The exception that caused the new exception.
+     */
+    explicit Exception(const std::string& message,
+                       const std::string& component="",
+                       const std::string& location="",
+                       const Exception* reason=NULL);
+
+    /**
+     * Copy the exception.
+     * Clones the reason, if there is one.
+     */
+    Exception(const Exception&);
+
+    virtual ~Exception() throw ();
+
+    /**
+     * Make a new copy of the given exception by dynamically allocating
+     * memory.
+     */
+    virtual Exception* clone() const;
+
+    /**
+     * Print all of the information about the exception.
+     */
+    virtual void print(std::ostream& stream=std::cerr) const;
+
+    /**
+     * Result of print() as a string.
+     */
+    virtual std::string toString() const;
+
+    /**
+     * Print the call stack where the exception was created.
+     */
+    virtual void printCallStack(std::ostream& stream=std::cerr) const;
+
+    const std::string& getMessage() const {
+      return mMessage;
+    }
+
+    const std::string& getComponent() const {
+      return mComponent;
+    }
+
+    const std::string& getLocation() const {
+      return mLocation;
+    }
+
+    const Exception* getReason() const {
+      return mReason;
+    }
+
+    /**
+     * Provide a body for the virtual from std::exception.
+     */
+    virtual const char* what() const throw () {
+      return mMessage.c_str();
+    }
+
+    virtual const char* getTypename() const;
+
+  private:
+    const static int sMaxCallStackDepth = 10;
+    const std::string mMessage;
+    const std::string mComponent;
+    const std::string mLocation;
+    int mCalls;
+    void* mCallStack[sMaxCallStackDepth];
+    const Exception* mReason;
+
+    // NOT IMPLEMENTED
+    std::exception& operator=(const std::exception& right) throw ();
+  };
+
+  class IOException: public Exception {
+  public:
+    IOException(const std::string& message,
+                const std::string& component="",
+                const std::string& location="",
+                const Exception* reason = NULL);
+
+    virtual IOException* clone() const;
+    virtual const char* getTypename() const;
+
+  };
+
+}
+#endif

+ 96 - 0
src/c++/librecordio/filestream.cc

@@ -0,0 +1,96 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "filestream.hh"
+
+using namespace hadoop;
+
+hadoop::FileInStream::FileInStream()
+{
+  mFile = NULL;
+}
+
+bool hadoop::FileInStream::open(const std::string& name)
+{
+  mFile = fopen(name.c_str(), "rb");
+  return (mFile != NULL);
+}
+
+ssize_t hadoop::FileInStream::read(void *buf, size_t len)
+{
+  return fread(buf, 1, len, mFile);
+}
+
+bool hadoop::FileInStream::skip(size_t nbytes)
+{
+  return (0==fseek(mFile, nbytes, SEEK_CUR));
+}
+
+bool hadoop::FileInStream::close()
+{
+  int ret = fclose(mFile);
+  mFile = NULL;
+  return (ret==0);
+}
+
+hadoop::FileInStream::~FileInStream()
+{
+  if (mFile != NULL) {
+    close();
+  }
+}
+
+hadoop::FileOutStream::FileOutStream()
+{
+  mFile = NULL;
+}
+
+bool hadoop::FileOutStream::open(const std::string& name, bool overwrite)
+{
+  if (!overwrite) {
+    mFile = fopen(name.c_str(), "rb");
+    if (mFile != NULL) {
+      fclose(mFile);
+      return false;
+    }
+  }
+  mFile = fopen(name.c_str(), "wb");
+  return (mFile != NULL);
+}
+
+ssize_t hadoop::FileOutStream::write(const void* buf, size_t len)
+{
+  return fwrite(buf, 1, len, mFile);
+}
+
+bool hadoop::FileOutStream::advance(size_t nbytes)
+{
+  return (0==fseek(mFile, nbytes, SEEK_CUR));
+}
+
+bool hadoop::FileOutStream::close()
+{
+  int ret = fclose(mFile);
+  mFile = NULL;
+  return (ret == 0);
+}
+
+hadoop::FileOutStream::~FileOutStream()
+{
+  if (mFile != NULL) {
+    close();
+  }
+}

+ 53 - 0
src/c++/librecordio/filestream.hh

@@ -0,0 +1,53 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef FILESTREAM_HH_
+#define FILESTREAM_HH_
+
+#include <stdio.h>
+#include <stdint.h>
+#include <string>
+#include "recordio.hh"
+
+namespace hadoop {
+
+class FileInStream : public InStream {
+public:
+  FileInStream();
+  bool open(const std::string& name);
+  ssize_t read(void *buf, size_t buflen);
+  bool skip(size_t nbytes);
+  bool close();
+  virtual ~FileInStream();
+private:
+  FILE *mFile;
+};
+
+
+class FileOutStream: public OutStream {
+public:
+  FileOutStream();
+  bool open(const std::string& name, bool overwrite);
+  ssize_t write(const void* buf, size_t len);
+  bool advance(size_t nbytes);
+  bool close();
+  virtual ~FileOutStream();
+private:
+  FILE *mFile;
+};
+
+}; // end namespace
+#endif /*FILESTREAM_HH_*/

+ 73 - 0
src/c++/librecordio/recordio.cc

@@ -0,0 +1,73 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "recordio.hh"
+#include "binarchive.hh"
+#include "csvarchive.hh"
+#include "xmlarchive.hh"
+
+using namespace hadoop;
+
+hadoop::RecordReader::RecordReader(InStream& stream, RecFormat f)
+{
+  switch (f) {
+    case kBinary:
+      mpArchive = new IBinArchive(stream);
+      break;
+    case kCSV:
+      mpArchive = new ICsvArchive(stream);
+      break;
+    case kXML:
+      mpArchive = new IXmlArchive(stream);
+      break;
+  }
+}
+
+hadoop::RecordReader::~RecordReader()
+{
+  delete mpArchive;
+}
+
+void hadoop::RecordReader::read(Record& record)
+{
+  record.deserialize(*mpArchive, (const char*) NULL);
+}
+
+hadoop::RecordWriter::RecordWriter(OutStream& stream, RecFormat f)
+{
+  switch (f) {
+    case kBinary:
+      mpArchive = new OBinArchive(stream);
+      break;
+    case kCSV:
+      mpArchive = new OCsvArchive(stream);
+      break;
+    case kXML:
+      mpArchive = new OXmlArchive(stream);
+      break;
+  }
+}
+
+hadoop::RecordWriter::~RecordWriter()
+{
+  delete mpArchive;
+}
+
+void hadoop::RecordWriter::write(Record& record)
+{
+  record.serialize(*mpArchive, (const char*) NULL);
+}
+

+ 78 - 0
src/c++/librecordio/recordio.hh

@@ -0,0 +1,78 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef RECORDIO_HH_
+#define RECORDIO_HH_
+
+#include <stdio.h>
+#include <stdint.h>
+#include <iostream>
+#include <cstring>
+#include <string>
+#include <vector>
+#include <map>
+#include <bitset>
+
+namespace hadoop {
+  
+class InStream {
+public:
+  virtual ssize_t read(void *buf, size_t buflen) = 0;
+};
+
+class OutStream {
+public:
+  virtual ssize_t write(const void *buf, size_t len) = 0;
+};
+
+class IArchive;
+class OArchive;
+
+class Record {
+public:
+  virtual bool validate() const = 0;
+  virtual void serialize(OArchive& archive, const char* tag) = 0;
+  virtual void deserialize(IArchive& archive, const char* tag) = 0;
+  virtual const std::string& type() const = 0;
+  virtual const std::string& signature() const = 0;
+};
+
+enum RecFormat { kBinary, kXML, kCSV };
+
+class RecordReader {
+private:
+  IArchive* mpArchive;
+public:
+  RecordReader(InStream& stream, RecFormat f);
+  virtual void read(hadoop::Record& record);
+  virtual ~RecordReader();
+};
+
+class RecordWriter {
+private:
+  OArchive* mpArchive;
+public:
+  RecordWriter(OutStream& stream, RecFormat f);
+  virtual void write(hadoop::Record& record);
+  virtual ~RecordWriter();
+};
+}; // end namspace hadoop
+
+#include "archive.hh"
+#include "exception.hh"
+
+#endif /*RECORDIO_HH_*/
+

+ 46 - 0
src/c++/librecordio/test/Makefile

@@ -0,0 +1,46 @@
+#
+# Copyright 2005 The Apache Software Foundation
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+all: test testFromJava
+
+test: test.o test.jr.o
+	g++ -g3 -O0 -o test test.o test.jr.o -L.. -L${XERCESCROOT}/lib -lrecordio -lxerces-c
+	
+test.o: test.cc
+	g++ -g3 -O0 -c -I.. -o test.o test.cc
+
+testFromJava: testFromJava.o test.jr.o
+	g++ -g3 -O0 -o testFromJava testFromJava.o test.jr.o -L.. -L${XERCESCROOT}/lib -lrecordio -lxerces-c
+	
+testFromJava.o: testFromJava.cc
+	g++ -g3 -O0 -c -I.. -o testFromJava.o testFromJava.cc
+
+
+test.jr.o: test.jr.cc
+	g++ -g3 -O0 -c -I.. -o test.jr.o test.jr.cc
+
+%.jr.cc %.jr.hh: %.jr
+	${HADOOP_HOME}/bin/rcc --language c++ $<
+
+%: %.o
+%: %.cc
+
+test.cc: test.hh
+test.hh: test.jr.hh ../recordio.hh ../filestream.hh
+
+clean:
+	rm -f *~ *.o test testFromJava *.jr.*
+

+ 109 - 0
src/c++/librecordio/test/test.cc

@@ -0,0 +1,109 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "test.hh"
+
+int main()
+{
+  org::apache::hadoop::record::test::TestRecord1 r1;
+  org::apache::hadoop::record::test::TestRecord1 r2;
+  {
+    hadoop::FileOutStream ostream;
+    ostream.open("/tmp/hadooptmp.dat", true);
+    hadoop::RecordWriter writer(ostream, hadoop::kBinary);
+    r1.setBoolVal(true);
+    r1.setByteVal((int8_t)0x66);
+    r1.setFloatVal(3.145);
+    r1.setDoubleVal(1.5234);
+    r1.setIntVal(4567);
+    r1.setLongVal(0x5a5a5a5a5a5aLL);
+    std::string& s = r1.getStringVal();
+    s = "random text";
+    std::string& buf = r1.getBufferVal();
+    std::vector<std::string>& v = r1.getVectorVal();
+    std::map<std::string,std::string>& m = r1.getMapVal();
+    writer.write(r1);
+    ostream.close();
+    hadoop::FileInStream istream;
+    istream.open("/tmp/hadooptmp.dat");
+    hadoop::RecordReader reader(istream, hadoop::kBinary);
+    reader.read(r2);
+    if (r1 == r2) {
+      printf("Binary archive test passed.\n");
+    } else {
+      printf("Binary archive test failed.\n");
+    }
+    istream.close();
+  }
+  {
+    hadoop::FileOutStream ostream;
+    ostream.open("/tmp/hadooptmp.txt", true);
+    hadoop::RecordWriter writer(ostream, hadoop::kCSV);
+    r1.setBoolVal(true);
+    r1.setByteVal((int8_t)0x66);
+    r1.setFloatVal(3.145);
+    r1.setDoubleVal(1.5234);
+    r1.setIntVal(4567);
+    r1.setLongVal(0x5a5a5a5a5a5aLL);
+    std::string& s = r1.getStringVal();
+    s = "random text";
+    std::string& buf = r1.getBufferVal();
+    std::vector<std::string>& v = r1.getVectorVal();
+    std::map<std::string,std::string>& m = r1.getMapVal();
+    writer.write(r1);
+    ostream.close();
+    hadoop::FileInStream istream;
+    istream.open("/tmp/hadooptmp.txt");
+    hadoop::RecordReader reader(istream, hadoop::kCSV);
+    reader.read(r2);
+    if (r1 == r2) {
+      printf("CSV archive test passed.\n");
+    } else {
+      printf("CSV archive test failed.\n");
+    }
+    istream.close();
+  }
+  {
+    hadoop::FileOutStream ostream;
+    ostream.open("/tmp/hadooptmp.xml", true);
+    hadoop::RecordWriter writer(ostream, hadoop::kXML);
+    r1.setBoolVal(true);
+    r1.setByteVal((int8_t)0x66);
+    r1.setFloatVal(3.145);
+    r1.setDoubleVal(1.5234);
+    r1.setIntVal(4567);
+    r1.setLongVal(0x5a5a5a5a5a5aLL);
+    std::string& s = r1.getStringVal();
+    s = "random text";
+    std::string& buf = r1.getBufferVal();
+    std::vector<std::string>& v = r1.getVectorVal();
+    std::map<std::string,std::string>& m = r1.getMapVal();
+    writer.write(r1);
+    ostream.close();
+    hadoop::FileInStream istream;
+    istream.open("/tmp/hadooptmp.xml");
+    hadoop::RecordReader reader(istream, hadoop::kXML);
+    reader.read(r2);
+    if (r1 == r2) {
+      printf("XML archive test passed.\n");
+    } else {
+      printf("XML archive test failed.\n");
+    }
+    istream.close();
+  }
+  return 0;
+}
+

+ 24 - 0
src/c++/librecordio/test/test.hh

@@ -0,0 +1,24 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef TEST_HH_
+#define TEST_HH_
+
+#include "recordio.hh"
+#include "filestream.hh"
+#include "test.jr.hh"
+
+#endif /*TEST_HH_*/

+ 35 - 0
src/c++/librecordio/test/test.jr

@@ -0,0 +1,35 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+module org.apache.hadoop.record.test {
+    class TestRecord0 {
+        ustring     StringVal;
+    }
+
+    class TestRecord1 {
+        boolean         BoolVal;
+        byte            ByteVal;
+        int             IntVal;
+        long            LongVal;
+        float           FloatVal;
+        double          DoubleVal;
+        ustring         StringVal;
+        buffer          BufferVal;
+        vector<ustring> VectorVal;
+        map<ustring, ustring>   MapVal;
+    }
+}
+

+ 72 - 0
src/c++/librecordio/test/testFromJava.cc

@@ -0,0 +1,72 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "test.hh"
+
+int main()
+{
+  org::apache::hadoop::record::test::TestRecord1 r1;
+  org::apache::hadoop::record::test::TestRecord1 r2;
+  r1.setBoolVal(true);
+  r1.setByteVal((int8_t)0x66);
+  r1.setFloatVal(3.145);
+  r1.setDoubleVal(1.5234);
+  r1.setIntVal(4567);
+  r1.setLongVal(0x5a5a5a5a5a5aLL);
+  std::string& s = r1.getStringVal();
+  s = "random text";
+  std::string& buf = r1.getBufferVal();
+  std::vector<std::string>& v = r1.getVectorVal();
+  std::map<std::string,std::string>& m = r1.getMapVal();
+  {
+    hadoop::FileInStream istream;
+    istream.open("/tmp/hadooptemp.dat");
+    hadoop::RecordReader reader(istream, hadoop::kBinary);
+    reader.read(r2);
+    if (r1 == r2) {
+      printf("Binary archive test passed.\n");
+    } else {
+      printf("Binary archive test failed.\n");
+    }
+    istream.close();
+  }
+  {
+    hadoop::FileInStream istream;
+    istream.open("/tmp/hadooptemp.txt");
+    hadoop::RecordReader reader(istream, hadoop::kCSV);
+    reader.read(r2);
+    if (r1 == r2) {
+      printf("CSV archive test passed.\n");
+    } else {
+      printf("CSV archive test failed.\n");
+    }
+    istream.close();
+  }
+  {
+    hadoop::FileInStream istream;
+    istream.open("/tmp/hadooptemp.xml");
+    hadoop::RecordReader reader(istream, hadoop::kXML);
+    reader.read(r2);
+    if (r1 == r2) {
+      printf("XML archive test passed.\n");
+    } else {
+      printf("XML archive test failed.\n");
+    }
+    istream.close();
+  }
+  return 0;
+}
+

+ 24 - 0
src/c++/librecordio/test/testFromJava.hh

@@ -0,0 +1,24 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef TEST_HH_
+#define TEST_HH_
+
+#include "recordio.hh"
+#include "filestream.hh"
+#include "test.jr.hh"
+
+#endif /*TEST_HH_*/

+ 423 - 0
src/c++/librecordio/xmlarchive.cc

@@ -0,0 +1,423 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "xmlarchive.hh"
+#include <stdlib.h>
+
+using namespace hadoop;
+
+void hadoop::MySAXHandler::startElement(const XMLCh* const name, AttributeList& attr)
+{
+  charsValid = false;
+  char* qname = XMLString::transcode(name);
+  if(std::string("boolean") == qname ||
+    std::string("ex:i1") == qname ||
+    std::string("i4") == qname ||
+    std::string("int") == qname ||
+    std::string("ex:i8") == qname ||
+    std::string("ex:float") == qname ||
+    std::string("double") == qname ||
+    std::string("string") == qname) {
+    std::string s(qname);
+    Value v(s);
+    vlist.push_back(v);
+    charsValid = true;
+  } else if(std::string("struct") == qname ||
+    std::string("array") == qname) {
+    std::string s(qname);
+    Value v(s);
+    vlist.push_back(v);
+  }
+  XMLString::release(&qname);
+}
+
+void hadoop::MySAXHandler::endElement(const XMLCh* const name)
+{
+  charsValid = false;
+  char* qname = XMLString::transcode(name);
+  if(std::string("struct") == qname ||
+    std::string("array") == qname) {
+    std::string s = "/";
+    Value v(s + qname);
+    vlist.push_back(v);
+  }
+  XMLString::release(&qname);
+}
+
+void hadoop::MySAXHandler::characters(const XMLCh* const buf, const unsigned int len)
+{
+  if (charsValid) {
+    char *cstr = XMLString::transcode(buf);
+    Value& v = vlist.back();
+    v.addChars(cstr, strlen(cstr));
+    XMLString::release(&cstr);
+  }
+}
+
+static char hexchars[] = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'A', 'B', 'C', 'D', 'E', 'F' };
+
+static std::string toXMLString(std::string s)
+{
+  std::string r;
+  size_t len = s.length();
+  size_t i;
+  const char* data = s.data();
+  for (i=0; i<len; i++, data++) {
+    char ch = *data;
+    if (((ch >= 'A') && (ch <='Z')) ||
+        ((ch >= 'a') && (ch <='z')) ||
+        ((ch >= '0') && (ch <='9'))) {
+        r.push_back(ch);
+    } else if (ch == ' ') {
+      r.push_back('+');
+    } else {
+      uint8_t* pb = (uint8_t*) &ch;
+      char ch1 = hexchars[*pb/16];
+      char ch2 = hexchars[*pb%16];
+      r.push_back('%');
+      r.push_back(ch1);
+      r.push_back(ch2);
+    }
+  }
+  return r;
+}
+
+static uint8_t h2b(char ch) {
+  if ((ch >= 'A') || (ch <= 'F')) {
+    return ch - 'A';
+  }
+  if ((ch >= 'a') || (ch <= 'f')) {
+    return ch - 'a';
+  }
+  if ((ch >= '0') || (ch <= '9')) {
+    return ch - '0';
+  }
+  return 0;
+}
+
+static std::string fromXMLString(std::string s)
+{
+  std::string r;
+  size_t len = s.length();
+  size_t i;
+  uint8_t* pb = (uint8_t*) s.data();
+  for (i = 0; i < len; i++) {
+    uint8_t b = *pb;
+    if (b == '+') {
+      r.push_back(' ');
+    } else if (b == '%') {
+      char *pc = (char*) (pb+1);
+      char ch1 = *pc++;
+      char ch2 = *pc++;
+      pb += 2;
+      uint8_t cnv = h2b(ch1)*16 + h2b(ch2);
+      pc = (char*) &cnv;
+      r.push_back(*pc);
+    } else {
+      char *pc = (char*) pb;
+      r.push_back(*pc);
+    }
+    pb++;
+  }
+  return r;
+}
+
+static std::string toXMLBuffer(std::string s, size_t len)
+{
+  std::string r;
+  size_t i;
+  uint8_t* data = (uint8_t*) s.data();
+  for (i=0; i<len; i++, data++) {
+    uint8_t b = *data;
+    char ch1 = hexchars[b/16];
+    char ch2 = hexchars[b%16];
+    r.push_back(ch1);
+    r.push_back(ch2);
+  }
+  return r;
+}
+
+static std::string fromXMLBuffer(std::string s, size_t& len)
+{
+  len = s.length();
+  if (len%2 == 1) { // len is guaranteed to be even
+    throw new IOException("Errror deserializing buffer.");
+  }
+  len >> 1;
+  std::string t;
+  for (size_t idx = 0; idx < len; idx++) {
+    char buf[3];
+    buf[0] = s[2*idx];
+    buf[1] = s[2*idx+1];
+    buf[2] = '\0';
+    int i;
+    if (1 != sscanf(buf, "%2x", &i)) {
+      throw new IOException("Errror deserializing buffer.");
+    }
+    t.push_back((char) i);
+  }
+  len = t.length();
+  return t;
+}
+
+void hadoop::IXmlArchive::deserialize(int8_t& t, const char* tag)
+{
+  Value v = next();
+  if (v.getType() != "ex:i1") {
+    throw new IOException("Error deserializing byte");
+  }
+  t = (int8_t) strtol(v.getValue().c_str(), NULL, 10);
+}
+
+void hadoop::IXmlArchive::deserialize(bool& t, const char* tag)
+{
+  Value v = next();
+  if (v.getType() != "boolean") {
+    throw new IOException("Error deserializing boolean");
+  }
+  t = (v.getValue() == "1");
+}
+
+void hadoop::IXmlArchive::deserialize(int32_t& t, const char* tag)
+{
+  Value v = next();
+  if (v.getType() != "i4" && v.getType() != "int") {
+    throw new IOException("Error deserializing int");
+  }
+  t = (int32_t) strtol(v.getValue().c_str(), NULL, 10);
+}
+
+void hadoop::IXmlArchive::deserialize(int64_t& t, const char* tag)
+{
+  Value v = next();
+  if (v.getType() != "ex:i8") {
+    throw new IOException("Error deserializing long");
+  }
+  t = strtoll(v.getValue().c_str(), NULL, 10);
+}
+
+void hadoop::IXmlArchive::deserialize(float& t, const char* tag)
+{
+  Value v = next();
+  if (v.getType() != "ex:float") {
+    throw new IOException("Error deserializing float");
+  }
+  t = strtof(v.getValue().c_str(), NULL);
+}
+
+void hadoop::IXmlArchive::deserialize(double& t, const char* tag)
+{
+  Value v = next();
+  if (v.getType() != "double") {
+    throw new IOException("Error deserializing double");
+  }
+  t = strtod(v.getValue().c_str(), NULL);
+}
+
+void hadoop::IXmlArchive::deserialize(std::string& t, const char* tag)
+{
+  Value v = next();
+  if (v.getType() != "string") {
+    throw new IOException("Error deserializing string");
+  }
+  t = fromXMLString(v.getValue());
+}
+
+void hadoop::IXmlArchive::deserialize(std::string& t, size_t& len, const char* tag)
+{
+  Value v = next();
+  if (v.getType() != "string") {
+    throw new IOException("Error deserializing buffer");
+  }
+  t = fromXMLBuffer(v.getValue(), len);
+}
+
+void hadoop::IXmlArchive::startRecord(Record& s, const char* tag)
+{
+  Value v = next();
+  if (v.getType() != "struct") {
+    throw new IOException("Error deserializing record");
+  }
+}
+
+void hadoop::IXmlArchive::endRecord(Record& s, const char* tag)
+{
+  Value v = next();
+  if (v.getType() != "/struct") {
+    throw new IOException("Error deserializing record");
+  }
+}
+
+Index* hadoop::IXmlArchive::startVector(const char* tag)
+{
+  Value v = next();
+  if (v.getType() != "array") {
+    throw new IOException("Error deserializing vector");
+  }
+  return new XmlIndex(vlist, vidx);
+}
+
+void hadoop::IXmlArchive::endVector(Index* idx, const char* tag)
+{
+  Value v = next();
+  if (v.getType() != "/array") {
+    throw new IOException("Error deserializing vector");
+  }
+  delete idx;
+}
+
+Index* hadoop::IXmlArchive::startMap(const char* tag)
+{
+  Value v = next();
+  if (v.getType() != "array") {
+    throw new IOException("Error deserializing map");
+  }
+  return new XmlIndex(vlist, vidx);
+}
+
+void hadoop::IXmlArchive::endMap(Index* idx, const char* tag)
+{
+  Value v = next();
+  if (v.getType() != "/array") {
+    throw new IOException("Error deserializing map");
+  }
+  delete idx;
+}
+
+void hadoop::OXmlArchive::serialize(int8_t t, const char* tag)
+{
+  printBeginEnvelope(tag);
+  p("<ex:i1>");
+  char sval[5];
+  sprintf(sval, "%d", t);
+  p(sval);
+  p("</ex:i1>");
+  printEndEnvelope(tag);
+}
+
+void hadoop::OXmlArchive::serialize(bool t, const char* tag)
+{
+  printBeginEnvelope(tag);
+  p("<boolean>");
+  p(t ? "1" : "0");
+  p("</boolean>");
+  printEndEnvelope(tag);
+}
+
+void hadoop::OXmlArchive::serialize(int32_t t, const char* tag)
+{
+  printBeginEnvelope(tag);
+  p("<i4>");
+  char sval[128];
+  sprintf(sval, "%d", t);
+  p(sval);
+  p("</i4>");
+  printEndEnvelope(tag);
+}
+
+void hadoop::OXmlArchive::serialize(int64_t t, const char* tag)
+{
+  printBeginEnvelope(tag);
+  p("<ex:i8>");
+  char sval[128];
+  sprintf(sval, "%lld", t);
+  p(sval);
+  p("</ex:i8>");
+  printEndEnvelope(tag);
+
+}
+
+void hadoop::OXmlArchive::serialize(float t, const char* tag)
+{
+  printBeginEnvelope(tag);
+  p("<ex:float>");
+  char sval[128];
+  sprintf(sval, "%f", t);
+  p(sval);
+  p("</ex:float>");
+  printEndEnvelope(tag);
+}
+
+void hadoop::OXmlArchive::serialize(double t, const char* tag)
+{
+  printBeginEnvelope(tag);
+  p("<double>");
+  char sval[128];
+  sprintf(sval, "%lf", t);
+  p(sval);
+  p("</double>");
+  printEndEnvelope(tag);
+}
+
+void hadoop::OXmlArchive::serialize(const std::string& t, const char* tag)
+{
+  printBeginEnvelope(tag);
+  p("<string>");
+  std::string s = toXMLString(t);
+  stream.write(s.data(), s.length());
+  p("</string>");
+  printEndEnvelope(tag);
+}
+
+void hadoop::OXmlArchive::serialize(const std::string& t, size_t len, const char* tag)
+{
+  printBeginEnvelope(tag);
+  p("<string>");
+  std::string s = toXMLBuffer(t, len);
+  stream.write(s.data(), s.length());
+  p("</string>");
+  printEndEnvelope(tag);
+}
+
+void hadoop::OXmlArchive::startRecord(const Record& s, const char* tag)
+{
+  insideRecord(tag);
+  p("<struct>\n");
+}
+
+void hadoop::OXmlArchive::endRecord(const Record& s, const char* tag)
+{
+  p("</struct>\n");
+  outsideRecord(tag);
+}
+
+void hadoop::OXmlArchive::startVector(size_t len, const char* tag)
+{
+  insideVector(tag);
+  p("<array>\n");
+}
+
+void hadoop::OXmlArchive::endVector(size_t len, const char* tag)
+{
+  p("</array>\n");
+  outsideVector(tag);
+}
+
+void hadoop::OXmlArchive::startMap(size_t len, const char* tag)
+{
+  insideMap(tag);
+  p("<array>\n");
+}
+
+void hadoop::OXmlArchive::endMap(size_t len, const char* tag)
+{
+  p("</array>\n");
+  outsideMap(tag);
+}
+
+hadoop::OXmlArchive::~OXmlArchive()
+{
+}

+ 263 - 0
src/c++/librecordio/xmlarchive.hh

@@ -0,0 +1,263 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef XMLARCHIVE_HH_
+#define XMLARCHIVE_HH_
+
+#include <xercesc/parsers/SAXParser.hpp>
+#include <xercesc/util/PlatformUtils.hpp>
+#include <xercesc/util/BinInputStream.hpp>
+#include <xercesc/sax/HandlerBase.hpp>
+#include <xercesc/sax/InputSource.hpp>
+#include "recordio.hh"
+
+XERCES_CPP_NAMESPACE_USE
+
+namespace hadoop {
+
+class Value {
+private:
+  std::string type;
+  std::string value;
+public:
+  Value(const std::string& t) { type = t; }
+  void addChars(const char* buf, unsigned int len) {
+    value += std::string(buf, len);
+  }
+  const std::string& getType() const { return type; }
+  const std::string& getValue() const { return value; }
+};
+  
+class MySAXHandler : public HandlerBase {
+private:
+  std::vector<Value>& vlist;
+  bool charsValid;
+public:
+  MySAXHandler(std::vector<Value>& list) : vlist(list) {charsValid = false;}
+  void startElement(const XMLCh* const name, AttributeList& attr);
+  void endElement(const XMLCh* const name);
+  void characters(const XMLCh* const buf, unsigned int len);
+};
+
+class XmlIndex : public Index {
+private:
+  std::vector<Value>& vlist;
+  unsigned int& vidx;
+public:
+  XmlIndex(std::vector<Value>& list, unsigned int& idx) : vlist(list), vidx(idx) {}
+  bool done() {
+   Value v = vlist[vidx];
+   return (v.getType() == "/array") ? true : false;
+  }
+  void incr() {}
+  ~XmlIndex() {} 
+};
+
+class MyBinInputStream : public BinInputStream {
+private:
+  InStream& stream;
+  unsigned int pos;
+public:
+  MyBinInputStream(InStream& s) : stream(s) { pos = 0; }
+  virtual unsigned int curPos() const { return pos; }
+  virtual unsigned int readBytes(XMLByte* const toFill,
+      const unsigned int maxToRead) {
+    ssize_t nread = stream.read(toFill, maxToRead);
+    if (nread < 0) {
+      return 0;
+    } else {
+      pos += nread;
+      return nread;
+    }
+  }
+};
+
+
+class MyInputSource : public InputSource {
+private:
+  InStream& stream;
+public:
+  MyInputSource(InStream& s) : stream(s) {  }
+  virtual BinInputStream* makeStream() const {
+    return new MyBinInputStream(stream);
+  }
+  virtual const XMLCh* getEncoding() const {
+    return XMLString::transcode("UTF-8");
+  }
+  virtual ~MyInputSource() {}
+};
+  
+class IXmlArchive : public IArchive {
+private:
+  std::vector<Value> vlist;
+  unsigned int vidx;
+  MySAXHandler *docHandler;
+  SAXParser *parser;
+  MyInputSource* src;
+  Value next() {
+    Value v = vlist[vidx];
+    vidx++;
+    return v;
+  }
+public:
+  IXmlArchive(InStream& _stream) {
+    vidx = 0;
+    try {
+      XMLPlatformUtils::Initialize();
+    } catch (const XMLException& e) {
+      throw new IOException("Unable to initialize XML Parser.");
+    }
+    parser = new SAXParser();
+    docHandler = new MySAXHandler(vlist);
+    parser->setDocumentHandler(docHandler);
+    src = new MyInputSource(_stream);
+    try {
+      parser->parse(*src);
+    } catch (const XMLException& e) {
+      throw new IOException("Unable to parse XML stream.");
+    } catch (const SAXParseException& e) {
+      throw new IOException("Unable to parse XML stream.");
+    }
+    delete parser;
+    delete docHandler;
+  }
+  virtual void deserialize(int8_t& t, const char* tag);
+  virtual void deserialize(bool& t, const char* tag);
+  virtual void deserialize(int32_t& t, const char* tag);
+  virtual void deserialize(int64_t& t, const char* tag);
+  virtual void deserialize(float& t, const char* tag);
+  virtual void deserialize(double& t, const char* tag);
+  virtual void deserialize(std::string& t, const char* tag);
+  virtual void deserialize(std::string& t, size_t& len, const char* tag);
+  virtual void startRecord(Record& s, const char* tag);
+  virtual void endRecord(Record& s, const char* tag);
+  virtual Index* startVector(const char* tag);
+  virtual void endVector(Index* idx, const char* tag);
+  virtual Index* startMap(const char* tag);
+  virtual void endMap(Index* idx, const char* tag);
+  virtual ~IXmlArchive() {
+    XMLPlatformUtils::Terminate();
+  }
+};
+
+class OXmlArchive : public OArchive {
+private:
+  OutStream& stream;
+  
+  std::vector<std::string> cstack;
+  
+  void insideRecord(const char* tag) {
+    printBeginEnvelope(tag);
+    cstack.push_back("record");
+  }
+  
+  void outsideRecord(const char* tag) {
+    std::string s = cstack.back();
+    cstack.pop_back();
+    if (s != "record") {
+      throw new IOException("Error deserializing record.");
+    }
+    printEndEnvelope(tag);
+  }
+  
+  void insideVector(const char* tag) {
+    printBeginEnvelope(tag);
+    cstack.push_back("vector");
+  }
+  
+  void outsideVector(const char* tag) {
+    std::string s = cstack.back();
+    cstack.pop_back();
+    if (s != "vector") {
+      throw new IOException("Error deserializing vector.");
+    }
+    printEndEnvelope(tag);
+  }
+  
+  void insideMap(const char* tag) {
+    printBeginEnvelope(tag);
+    cstack.push_back("map");
+  }
+  
+  void outsideMap(const char* tag) {
+    std::string s = cstack.back();
+    cstack.pop_back();
+    if (s != "map") {
+      throw new IOException("Error deserializing map.");
+    }
+    printEndEnvelope(tag);
+  }
+  
+  void p(const char* cstr) {
+    stream.write(cstr, strlen(cstr));
+  }
+  
+  void printBeginEnvelope(const char* tag) {
+    if (cstack.size() != 0) {
+      std::string s = cstack.back();
+      if ("record" == s) {
+        p("<member>\n");
+        p("<name>");
+        p(tag);
+        p("</name>\n");
+        p("<value>");
+      } else if ("vector" == s) {
+        p("<value>");
+      } else if ("map" == s) {
+        p("<value>");
+      }
+    } else {
+      p("<value>");
+    }
+  }
+  
+  void printEndEnvelope(const char* tag) {
+    if (cstack.size() != 0) {
+      std::string s = cstack.back();
+      if ("record" == s) {
+        p("</value>\n");
+        p("</member>\n");
+      } else if ("vector" == s) {
+        p("</value>\n");
+      } else if ("map" == s) {
+        p("</value>\n");
+      }
+    } else {
+      p("</value>\n");
+    }
+  }
+  
+public:
+  OXmlArchive(OutStream& _stream) : stream(_stream) {}
+  virtual void serialize(int8_t t, const char* tag);
+  virtual void serialize(bool t, const char* tag);
+  virtual void serialize(int32_t t, const char* tag);
+  virtual void serialize(int64_t t, const char* tag);
+  virtual void serialize(float t, const char* tag);
+  virtual void serialize(double t, const char* tag);
+  virtual void serialize(const std::string& t, const char* tag);
+  virtual void serialize(const std::string& t, size_t len, const char* tag);
+  virtual void startRecord(const Record& s, const char* tag);
+  virtual void endRecord(const Record& s, const char* tag);
+  virtual void startVector(size_t len, const char* tag);
+  virtual void endVector(size_t len, const char* tag);
+  virtual void startMap(size_t len, const char* tag);
+  virtual void endMap(size_t len, const char* tag);
+  virtual ~OXmlArchive();
+};
+
+}
+#endif /*XMLARCHIVE_HH_*/

+ 115 - 0
src/java/org/apache/hadoop/record/BinaryInputArchive.java

@@ -0,0 +1,115 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.InputStream;
+
+
+/**
+ *
+ * @author milindb
+ */
+public class BinaryInputArchive implements InputArchive {
+    
+    private DataInput in;
+    
+    static BinaryInputArchive getArchive(InputStream strm) {
+        return new BinaryInputArchive(new DataInputStream(strm));
+    }
+    
+    static private class BinaryIndex implements Index {
+        private int nelems;
+        BinaryIndex(int nelems) {
+            this.nelems = nelems;
+        }
+        public boolean done() {
+            return (nelems <= 0);
+        }
+        public void incr() {
+            nelems--;
+        }
+    }
+    /** Creates a new instance of BinaryInputArchive */
+    public BinaryInputArchive(DataInput in) {
+        this.in = in;
+    }
+    
+    public byte readByte(String tag) throws IOException {
+        return in.readByte();
+    }
+    
+    public boolean readBool(String tag) throws IOException {
+        return in.readBoolean();
+    }
+    
+    public int readInt(String tag) throws IOException {
+        return Utils.readInt(in);
+    }
+    
+    public long readLong(String tag) throws IOException {
+        return Utils.readLong(in);
+    }
+    
+    public float readFloat(String tag) throws IOException {
+        return in.readFloat();
+    }
+    
+    public double readDouble(String tag) throws IOException {
+        return in.readDouble();
+    }
+    
+    public String readString(String tag) throws IOException {
+        int len = Utils.readInt(in);
+        byte[] chars = new byte[len];
+        in.readFully(chars);
+        return new String(chars, "UTF-8");
+    }
+    
+    public ByteArrayOutputStream readBuffer(String tag) throws IOException {
+        int len = Utils.readInt(in);
+        ByteArrayOutputStream buf = new ByteArrayOutputStream(len);
+        byte[] arr = new byte[len];
+        in.readFully(arr);
+        buf.write(arr, 0, len);
+        return buf;
+    }
+    
+    public void readRecord(Record r, String tag) throws IOException {
+        r.deserialize(this, tag);
+    }
+    
+    public void startRecord(String tag) throws IOException {}
+    
+    public void endRecord(String tag) throws IOException {}
+    
+    public Index startVector(String tag) throws IOException {
+        return new BinaryIndex(Utils.readInt(in));
+    }
+    
+    public void endVector(String tag) throws IOException {}
+    
+    public Index startMap(String tag) throws IOException {
+        return new BinaryIndex(Utils.readInt(in));
+    }
+    
+    public void endMap(String tag) throws IOException {}
+    
+}

+ 101 - 0
src/java/org/apache/hadoop/record/BinaryOutputArchive.java

@@ -0,0 +1,101 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record;
+
+import java.io.IOException;
+import java.io.ByteArrayOutputStream;
+import java.util.TreeMap;
+import java.util.ArrayList;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.OutputStream;
+
+/**
+ *
+ * @author milindb
+ */
+public class BinaryOutputArchive implements OutputArchive {
+    
+    private DataOutput out;
+    
+    static BinaryOutputArchive getArchive(OutputStream strm) {
+        return new BinaryOutputArchive(new DataOutputStream(strm));
+    }
+    
+    /** Creates a new instance of BinaryOutputArchive */
+    public BinaryOutputArchive(DataOutput out) {
+        this.out = out;
+    }
+    
+    public void writeByte(byte b, String tag) throws IOException {
+        out.writeByte(b);
+    }
+    
+    public void writeBool(boolean b, String tag) throws IOException {
+        out.writeBoolean(b);
+    }
+    
+    public void writeInt(int i, String tag) throws IOException {
+        Utils.writeInt(out, i);
+    }
+    
+    public void writeLong(long l, String tag) throws IOException {
+        Utils.writeLong(out, l);
+    }
+    
+    public void writeFloat(float f, String tag) throws IOException {
+        out.writeFloat(f);
+    }
+    
+    public void writeDouble(double d, String tag) throws IOException {
+        out.writeDouble(d);
+    }
+    
+    public void writeString(String s, String tag) throws IOException {
+        byte[] chars = s.getBytes("UTF-8");
+        Utils.writeInt(out, chars.length);
+        out.write(chars);
+    }
+    
+    public void writeBuffer(ByteArrayOutputStream buf, String tag)
+    throws IOException {
+        byte[] barr = buf.toByteArray();
+        Utils.writeInt(out, barr.length);
+        out.write(barr);
+    }
+    
+    public void writeRecord(Record r, String tag) throws IOException {
+        r.serialize(this, tag);
+    }
+    
+    public void startRecord(Record r, String tag) throws IOException {}
+    
+    public void endRecord(Record r, String tag) throws IOException {}
+    
+    public void startVector(ArrayList v, String tag) throws IOException {
+        Utils.writeInt(out, v.size());
+    }
+    
+    public void endVector(ArrayList v, String tag) throws IOException {}
+    
+    public void startMap(TreeMap v, String tag) throws IOException {
+        Utils.writeInt(out, v.size());
+    }
+    
+    public void endMap(TreeMap v, String tag) throws IOException {}
+    
+}

+ 209 - 0
src/java/org/apache/hadoop/record/CsvInputArchive.java

@@ -0,0 +1,209 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record;
+
+import java.io.InputStreamReader;
+import java.io.InputStream;
+import java.io.IOException;
+import java.io.ByteArrayOutputStream;
+import java.io.PushbackInputStream;
+import java.io.PushbackReader;
+import java.io.UnsupportedEncodingException;
+
+/**
+ *
+ * @author milindb
+ */
+class CsvInputArchive implements InputArchive {
+    
+    private PushbackReader stream;
+    
+    private class CsvIndex implements Index {
+        public boolean done() {
+            char c = '\0';
+            try {
+                c = (char) stream.read();
+                stream.unread(c);
+            } catch (IOException ex) {
+            }
+            return (c == '}') ? true : false;
+        }
+        public void incr() {}
+    }
+    
+    private void throwExceptionOnError(String tag) throws IOException {
+        throw new IOException("Error deserializing "+tag);
+    }
+    
+    private String readField(String tag) throws IOException {
+        try {
+            StringBuffer buf = new StringBuffer();
+            while (true) {
+                char c = (char) stream.read();
+                switch (c) {
+                    case ',':
+                        return buf.toString();
+                    case '}':
+                    case '\n':
+                    case '\r':
+                        stream.unread(c);
+                        return buf.toString();
+                    default:
+                        buf.append(c);
+                }
+            }
+        } catch (IOException ex) {
+            throw new IOException("Error reading "+tag);
+        }
+    }
+    
+    static CsvInputArchive getArchive(InputStream strm)
+    throws UnsupportedEncodingException {
+        return new CsvInputArchive(strm);
+    }
+    
+    /** Creates a new instance of CsvInputArchive */
+    public CsvInputArchive(InputStream in)
+    throws UnsupportedEncodingException {
+        stream = new PushbackReader(new InputStreamReader(in, "UTF-8"));
+    }
+    
+    public byte readByte(String tag) throws IOException {
+        return (byte) readLong(tag);
+    }
+    
+    public boolean readBool(String tag) throws IOException {
+        String sval = readField(tag);
+        return "T".equals(sval) ? true : false;
+    }
+    
+    public int readInt(String tag) throws IOException {
+        return (int) readLong(tag);
+    }
+    
+    public long readLong(String tag) throws IOException {
+        String sval = readField(tag);
+        try {
+            long lval = Long.parseLong(sval);
+            return lval;
+        } catch (NumberFormatException ex) {
+            throw new IOException("Error deserializing "+tag);
+        }
+    }
+    
+    public float readFloat(String tag) throws IOException {
+        return (float) readDouble(tag);
+    }
+    
+    public double readDouble(String tag) throws IOException {
+        String sval = readField(tag);
+        try {
+            double dval = Double.parseDouble(sval);
+            return dval;
+        } catch (NumberFormatException ex) {
+            throw new IOException("Error deserializing "+tag);
+        }
+    }
+    
+    public String readString(String tag) throws IOException {
+        String sval = readField(tag);
+        return Utils.fromCSVString(sval);
+        
+    }
+    
+    public ByteArrayOutputStream readBuffer(String tag) throws IOException {
+        String sval = readField(tag);
+        return Utils.fromCSVBuffer(sval);
+    }
+    
+    public void readRecord(Record r, String tag) throws IOException {
+        r.deserialize(this, tag);
+    }
+    
+    public void startRecord(String tag) throws IOException {
+        if (tag != null && !"".equals(tag)) {
+            char c1 = (char) stream.read();
+            char c2 = (char) stream.read();
+            if (c1 != 's' || c2 != '{') {
+                throw new IOException("Error deserializing "+tag);
+            }
+        }
+    }
+    
+    public void endRecord(String tag) throws IOException {
+        char c = (char) stream.read();
+        if (tag == null || "".equals(tag)) {
+            if (c != '\n' && c != '\r') {
+                throw new IOException("Error deserializing record.");
+            } else {
+                return;
+            }
+        }
+        
+        if (c != '}') {
+            throw new IOException("Error deserializing "+tag);
+        }
+        c = (char) stream.read();
+        if (c != ',') {
+            stream.unread(c);
+        }
+        
+        return;
+    }
+    
+    public Index startVector(String tag) throws IOException {
+        char c1 = (char) stream.read();
+        char c2 = (char) stream.read();
+        if (c1 != 'v' || c2 != '{') {
+            throw new IOException("Error deserializing "+tag);
+        }
+        return new CsvIndex();
+    }
+    
+    public void endVector(String tag) throws IOException {
+        char c = (char) stream.read();
+        if (c != '}') {
+            throw new IOException("Error deserializing "+tag);
+        }
+        c = (char) stream.read();
+        if (c != ',') {
+            stream.unread(c);
+        }
+        return;
+    }
+    
+    public Index startMap(String tag) throws IOException {
+        char c1 = (char) stream.read();
+        char c2 = (char) stream.read();
+        if (c1 != 'm' || c2 != '{') {
+            throw new IOException("Error deserializing "+tag);
+        }
+        return new CsvIndex();
+    }
+    
+    public void endMap(String tag) throws IOException {
+        char c = (char) stream.read();
+        if (c != '}') {
+            throw new IOException("Error deserializing "+tag);
+        }
+        c = (char) stream.read();
+        if (c != ',') {
+            stream.unread(c);
+        }
+        return;
+    }
+}

+ 147 - 0
src/java/org/apache/hadoop/record/CsvOutputArchive.java

@@ -0,0 +1,147 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record;
+
+import java.io.IOException;
+import java.io.ByteArrayOutputStream;
+import java.util.TreeMap;
+import java.util.ArrayList;
+import java.io.PrintStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+
+/**
+ *
+ * @author milindb
+ */
+public class CsvOutputArchive implements OutputArchive {
+
+    private PrintStream stream;
+    private boolean isFirst = true;
+    
+    static CsvOutputArchive getArchive(OutputStream strm)
+    throws UnsupportedEncodingException {
+        return new CsvOutputArchive(strm);
+    }
+    
+    private void throwExceptionOnError(String tag) throws IOException {
+        if (stream.checkError()) {
+            throw new IOException("Error serializing "+tag);
+        }
+    }
+ 
+    private void printCommaUnlessFirst() {
+        if (!isFirst) {
+            stream.print(",");
+        }
+        isFirst = false;
+    }
+    
+    /** Creates a new instance of CsvOutputArchive */
+    public CsvOutputArchive(OutputStream out)
+    throws UnsupportedEncodingException {
+        stream = new PrintStream(out, true, "UTF-8");
+    }
+    
+    public void writeByte(byte b, String tag) throws IOException {
+        writeLong((long)b, tag);
+    }
+    
+    public void writeBool(boolean b, String tag) throws IOException {
+        printCommaUnlessFirst();
+        String val = b ? "T" : "F";
+        stream.print(val);
+        throwExceptionOnError(tag);
+    }
+    
+    public void writeInt(int i, String tag) throws IOException {
+        writeLong((long)i, tag);
+    }
+    
+    public void writeLong(long l, String tag) throws IOException {
+        printCommaUnlessFirst();
+        stream.print(l);
+        throwExceptionOnError(tag);
+    }
+    
+    public void writeFloat(float f, String tag) throws IOException {
+        writeDouble((double)f, tag);
+    }
+    
+    public void writeDouble(double d, String tag) throws IOException {
+        printCommaUnlessFirst();
+        stream.print(d);
+        throwExceptionOnError(tag);
+    }
+    
+    public void writeString(String s, String tag) throws IOException {
+        printCommaUnlessFirst();
+        stream.print(Utils.toCSVString(s));
+        throwExceptionOnError(tag);
+    }
+    
+    public void writeBuffer(ByteArrayOutputStream buf, String tag)
+    throws IOException {
+        printCommaUnlessFirst();
+        stream.print(Utils.toCSVBuffer(buf));
+        throwExceptionOnError(tag);
+    }
+    
+    public void writeRecord(Record r, String tag) throws IOException {
+        r.serialize(this, tag);
+    }
+    
+    public void startRecord(Record r, String tag) throws IOException {
+        if (tag != null && !"".equals(tag)) {
+            printCommaUnlessFirst();
+            stream.print("s{");
+            isFirst = true;
+        }
+    }
+    
+    public void endRecord(Record r, String tag) throws IOException {
+        if (tag == null || "".equals(tag)) {
+            stream.print("\n");
+            isFirst = true;
+        } else {
+            stream.print("}");
+            isFirst = false;
+        }
+    }
+    
+    public void startVector(ArrayList v, String tag) throws IOException {
+        printCommaUnlessFirst();
+        stream.print("v{");
+        isFirst = true;
+    }
+    
+    public void endVector(ArrayList v, String tag) throws IOException {
+        stream.print("}");
+        isFirst = false;
+    }
+    
+    public void startMap(TreeMap v, String tag) throws IOException {
+        printCommaUnlessFirst();
+        stream.print("m{");
+        isFirst = true;
+    }
+    
+    public void endMap(TreeMap v, String tag) throws IOException {
+        stream.print("}");
+        isFirst = false;
+    }
+}

+ 37 - 0
src/java/org/apache/hadoop/record/Index.java

@@ -0,0 +1,37 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record;
+
+/**
+ * Interface that acts as an iterator for deserializing maps.
+ * The deserializer returns an instance that the record uses to
+ * read vectors and maps. An example of usage is as follows:
+ *
+ * <code>
+ * Index idx = startVector(...);
+ * while (!idx.done()) {
+ *   .... // read element of a vector
+ *   idx.incr();
+ * }
+ * </code>
+ *
+ * @author Milind Bhandarkar
+ */
+public interface Index {
+    public boolean done();
+    public void incr();
+}

+ 45 - 0
src/java/org/apache/hadoop/record/InputArchive.java

@@ -0,0 +1,45 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record;
+
+import java.io.IOException;
+import java.io.ByteArrayOutputStream;
+import java.util.TreeMap;
+import java.util.ArrayList;
+
+/**
+ * Interface that all the Deserializers have to implement.
+ *
+ * @author milindb
+ */
+public interface InputArchive {
+    public byte readByte(String tag) throws IOException;
+    public boolean readBool(String tag) throws IOException;
+    public int readInt(String tag) throws IOException;
+    public long readLong(String tag) throws IOException;
+    public float readFloat(String tag) throws IOException;
+    public double readDouble(String tag) throws IOException;
+    public String readString(String tag) throws IOException;
+    public ByteArrayOutputStream readBuffer(String tag) throws IOException;
+    public void readRecord(Record r, String tag) throws IOException;
+    public void startRecord(String tag) throws IOException;
+    public void endRecord(String tag) throws IOException;
+    public Index startVector(String tag) throws IOException;
+    public void endVector(String tag) throws IOException;
+    public Index startMap(String tag) throws IOException;
+    public void endMap(String tag) throws IOException;
+}

+ 47 - 0
src/java/org/apache/hadoop/record/OutputArchive.java

@@ -0,0 +1,47 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record;
+
+import java.io.IOException;
+import java.io.ByteArrayOutputStream;
+import java.util.TreeMap;
+import java.util.ArrayList;
+
+/**
+ * Interface that alll the serializers have to implement.
+ *
+ * @author milindb
+ */
+public interface OutputArchive {
+    public void writeByte(byte b, String tag) throws IOException;
+    public void writeBool(boolean b, String tag) throws IOException;
+    public void writeInt(int i, String tag) throws IOException;
+    public void writeLong(long l, String tag) throws IOException;
+    public void writeFloat(float f, String tag) throws IOException;
+    public void writeDouble(double d, String tag) throws IOException;
+    public void writeString(String s, String tag) throws IOException;
+    public void writeBuffer(ByteArrayOutputStream buf, String tag)
+        throws IOException;
+    public void writeRecord(Record r, String tag) throws IOException;
+    public void startRecord(Record r, String tag) throws IOException;
+    public void endRecord(Record r, String tag) throws IOException;
+    public void startVector(ArrayList v, String tag) throws IOException;
+    public void endVector(ArrayList v, String tag) throws IOException;
+    public void startMap(TreeMap v, String tag) throws IOException;
+    public void endMap(TreeMap v, String tag) throws IOException;
+
+}

+ 31 - 0
src/java/org/apache/hadoop/record/Record.java

@@ -0,0 +1,31 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record;
+
+import java.io.IOException;
+
+/**
+ * Interface that is implemented by generated classes.
+ * 
+ * @author Milind Bhandarkar
+ */
+public interface Record {
+    public void serialize(OutputArchive archive, String tag)
+        throws IOException;
+    public void deserialize(InputArchive archive, String tag)
+        throws IOException;
+}

+ 96 - 0
src/java/org/apache/hadoop/record/RecordReader.java

@@ -0,0 +1,96 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.DataInputStream;
+import java.io.UnsupportedEncodingException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.HashMap;
+import javax.xml.parsers.ParserConfigurationException;
+import org.xml.sax.SAXException;
+
+/**
+ * Front-end interface to deserializers. Also acts as a factory
+ * for deserializers.
+ *
+ * @author milindb
+ */
+public class RecordReader {
+    
+    private InputArchive archive;
+
+    static private HashMap archiveFactory;
+    
+    static {
+        archiveFactory = new HashMap();
+        Class[] params = { InputStream.class };
+        try {
+            archiveFactory.put("binary",
+                    BinaryInputArchive.class.getDeclaredMethod(
+                        "getArchive", params));
+            archiveFactory.put("csv",
+                    CsvInputArchive.class.getDeclaredMethod(
+                        "getArchive", params));
+            archiveFactory.put("xml",
+                    XmlInputArchive.class.getDeclaredMethod(
+                        "getArchive", params));
+        } catch (SecurityException ex) {
+            ex.printStackTrace();
+        } catch (NoSuchMethodException ex) {
+            ex.printStackTrace();
+        }
+    }
+    
+    static private InputArchive createArchive(InputStream in, String format)
+    throws IOException {
+        Method factory = (Method) archiveFactory.get(format);
+        if (factory != null) {
+            Object[] params = { in };
+            try {
+                return (InputArchive) factory.invoke(null, params);
+            } catch (IllegalArgumentException ex) {
+                ex.printStackTrace();
+            } catch (InvocationTargetException ex) {
+                ex.printStackTrace();
+            } catch (IllegalAccessException ex) {
+                ex.printStackTrace();
+            }
+        }
+        return null;
+    }
+    /**
+     * Creates a new instance of RecordReader.
+     * @param in Stream from which to deserialize a record
+     * @param format Deserialization format ("binary", "xml", or "csv")
+     */
+    public RecordReader(InputStream in, String format)
+    throws IOException {
+        archive = createArchive(in, format);
+    }
+    
+    /**
+     * Deserialize a record
+     * @param r Record to be deserialized
+     */
+    public void read(Record r) throws IOException {
+        r.deserialize(archive, "");
+    }
+    
+}

+ 115 - 0
src/java/org/apache/hadoop/record/RecordWriter.java

@@ -0,0 +1,115 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.DataOutputStream;
+import java.io.InputStream;
+import java.io.UnsupportedEncodingException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.HashMap;
+import javax.xml.parsers.ParserConfigurationException;
+import org.xml.sax.SAXException;
+
+/**
+ * Front-end for serializers. Also serves as a factory for serializers.
+ *
+ * @author milindb
+ */
+public class RecordWriter {
+    
+    private OutputArchive archive;
+    
+    static private OutputArchive getBinaryArchive(OutputStream out) {
+        return new BinaryOutputArchive(new DataOutputStream(out));
+    }
+    
+    static private OutputArchive getCsvArchive(OutputStream out)
+    throws IOException {
+        try {
+            return new CsvOutputArchive(out);
+        } catch (UnsupportedEncodingException ex) {
+            throw new IOException("Unsupported encoding UTF-8");
+        }
+    }
+    
+    static private OutputArchive getXmlArchive(OutputStream out)
+    throws IOException {
+        return new XmlOutputArchive(out);
+    }
+
+    static HashMap constructFactory() {
+        HashMap factory = new HashMap();
+        Class[] params = { OutputStream.class };
+        try {
+            factory.put("binary",
+                    BinaryOutputArchive.class.getDeclaredMethod(
+                        "getArchive", params));
+            factory.put("csv",
+                    CsvOutputArchive.class.getDeclaredMethod(
+                        "getArchive", params));
+            factory.put("xml",
+                    XmlOutputArchive.class.getDeclaredMethod(
+                        "getArchive", params));
+        } catch (SecurityException ex) {
+            ex.printStackTrace();
+        } catch (NoSuchMethodException ex) {
+            ex.printStackTrace();
+        }
+        return factory;
+    }
+    
+    static private HashMap archiveFactory = constructFactory();
+    
+    static private OutputArchive createArchive(OutputStream out,
+            String format)
+            throws IOException {
+        Method factory = (Method) archiveFactory.get(format);
+        if (factory != null) {
+            Object[] params = { out };
+            try {
+                return (OutputArchive) factory.invoke(null, params);
+            } catch (IllegalArgumentException ex) {
+                ex.printStackTrace();
+            } catch (InvocationTargetException ex) {
+                ex.printStackTrace();
+            } catch (IllegalAccessException ex) {
+                ex.printStackTrace();
+            }
+        }
+        return null;
+    }
+    /**
+     * Creates a new instance of RecordWriter
+     * @param out Output stream where the records will be serialized
+     * @param format Serialization format ("binary", "xml", or "csv")
+     */
+    public RecordWriter(OutputStream out, String format)
+    throws IOException {
+        archive = createArchive(out, format);
+    }
+    
+    /**
+     * Serialize a record
+     * @param r record to be serialized
+     */
+    public void write(Record r) throws IOException {
+        r.serialize(archive, "");
+    }
+}

+ 366 - 0
src/java/org/apache/hadoop/record/Utils.java

@@ -0,0 +1,366 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+
+/**
+ * Various utility functions for Hadooop record I/O runtime.
+ * @author milindb@yahoo-inc.com
+ */
+public class Utils {
+    
+    /** Cannot create a new instance of Utils */
+    private Utils() {
+    }
+   
+    /**
+     * Serializes an integer to a binary stream with zero-compressed encoding.
+     * For -120 <= i <= 127, only one byte is used with the actual value.
+     * For other values of i, the first byte value indicates whether the
+     * integer is positive or negative, and the number of bytes that follow.
+     * If the first byte value v is between -121 and -124, the following integer
+     * is positive, with number of bytes that follow are -(v+120).
+     * If the first byte value v is between -125 and -128, the following integer
+     * is negative, with number of bytes that follow are -(v+124). Bytes are
+     * stored in the high-non-zero-byte-first order.
+     *
+     * @param stream Binary output stream
+     * @param i Integer to be serialized
+     * @throws java.io.IOException 
+     */
+    static void writeInt(DataOutput stream, int i) throws IOException {
+        if (i >= -120 && i <= 127) {
+            stream.writeByte((byte)i);
+            return;
+        }
+        
+        int len = -120;
+        if (i < 0) {
+            i &= 0x7FFFFFFF; // reset the sign bit
+            len = -124;
+        }
+        
+        int tmp = i;
+        while (tmp != 0) {
+            tmp = tmp >> 8;
+            len--;
+        }
+        
+        stream.writeByte((byte)len);
+        
+        len = (len < -124) ? -(len + 124) : -(len+120);
+        
+        for (int idx = len; idx != 0; idx--) {
+            int shiftbits = (idx - 1) * 8;
+            int mask = 0xFF << shiftbits;
+            stream.writeByte((byte)((i & mask) >> shiftbits));
+        }
+    }
+    
+    /**
+     * Serializes a long to a binary stream with zero-compressed encoding.
+     * For -112 <= i <= 127, only one byte is used with the actual value.
+     * For other values of i, the first byte value indicates whether the
+     * long is positive or negative, and the number of bytes that follow.
+     * If the first byte value v is between -113 and -120, the following long
+     * is positive, with number of bytes that follow are -(v+112).
+     * If the first byte value v is between -121 and -128, the following long
+     * is negative, with number of bytes that follow are -(v+120). Bytes are
+     * stored in the high-non-zero-byte-first order.
+     * 
+     * @param stream Binary output stream
+     * @param i Long to be serialized
+     * @throws java.io.IOException 
+     */
+    static void writeLong(DataOutput stream, long i) throws IOException {
+        if (i >= -112 && i <= 127) {
+            stream.writeByte((byte)i);
+            return;
+        }
+        
+        int len = -112;
+        if (i < 0) {
+            i &= 0x7FFFFFFFFFFFFFFFL; // reset the sign bit
+            len = -120;
+        }
+        
+        long tmp = i;
+        while (tmp != 0) {
+            tmp = tmp >> 8;
+            len--;
+        }
+        
+        stream.writeByte((byte)len);
+        
+        len = (len < -120) ? -(len + 120) : -(len + 112);
+        
+        for (int idx = len; idx != 0; idx--) {
+            int shiftbits = (idx - 1) * 8;
+            long mask = 0xFFL << shiftbits;
+            stream.writeByte((byte)((i & mask) >> shiftbits));
+        }
+    }
+    
+    /**
+     * Reads a zero-compressed encoded integer from input stream and returns it.
+     * @param stream Binary input stream
+     * @throws java.io.IOException 
+     * @return deserialized integer from stream.
+     */
+    static int readInt(DataInput stream) throws IOException {
+        int len = stream.readByte();
+        if (len >= -120) {
+            return len;
+        }
+        len = (len < -124) ? -(len + 124) : -(len + 120);
+        byte[] barr = new byte[len];
+        stream.readFully(barr);
+        int i = 0;
+        for (int idx = 0; idx < len; idx++) {
+            i = i << 8;
+            i = i | (barr[idx] & 0xFF);
+        }
+        return i;
+    }
+    
+    /**
+     * Reads a zero-compressed encoded long from input stream and returns it.
+     * @param stream Binary input stream
+     * @throws java.io.IOException 
+     * @return deserialized long from stream.
+     */
+    static long readLong(DataInput stream) throws IOException {
+        int len = stream.readByte();
+        if (len >= -112) {
+            return len;
+        }
+        len = (len < -120) ? -(len + 120) : -(len + 112);
+        byte[] barr = new byte[len];
+        stream.readFully(barr);
+        long i = 0;
+        for (int idx = 0; idx < len; idx++) {
+            i = i << 8;
+            i = i | (barr[idx] & 0xFF);
+        }
+        return i;
+    }
+    
+    /**
+     * equals function that actually compares two buffers.
+     *
+     * @param one First buffer
+     * @param two Second buffer
+     * @return true if one and two contain exactly the same content, else false.
+     */
+    public static boolean bufEquals(ByteArrayOutputStream one,
+            ByteArrayOutputStream two) {
+        if (one == two) {
+            return true;
+        }
+        byte[] onearray = one.toByteArray();
+        byte[] twoarray = two.toByteArray();
+        boolean ret = (onearray.length == twoarray.length);
+        if (!ret) {
+            return ret;
+        }
+        for (int idx = 0; idx < onearray.length; idx++) {
+            if (onearray[idx] != twoarray[idx]) {
+                return false;
+            }
+        }
+        return true;
+    }
+    
+    /**
+     * 
+     * @param s 
+     * @return 
+     */
+    static String toXMLString(String s) {
+        String rets = "";
+        try {
+            rets = java.net.URLEncoder.encode(s, "UTF-8");
+        } catch (UnsupportedEncodingException ex) {
+            ex.printStackTrace();
+        }
+        return rets;
+    }
+    
+    /**
+     * 
+     * @param s 
+     * @return 
+     */
+    static String fromXMLString(String s) {
+        String rets = "";
+        try {
+            rets = java.net.URLDecoder.decode(s, "UTF-8");
+        } catch (UnsupportedEncodingException ex) {
+            ex.printStackTrace();
+        }
+        return rets;
+    }
+    
+    /**
+     * 
+     * @param s 
+     * @return 
+     */
+    static String toCSVString(String s) {
+        StringBuffer sb = new StringBuffer(s.length()+1);
+        sb.append('\'');
+        int len = s.length();
+        for (int i = 0; i < len; i++) {
+            char c = s.charAt(i);
+            switch(c) {
+                case '\0':
+                    sb.append("%00");
+                    break;
+                case '\n':
+                    sb.append("%0A");
+                    break;
+                case '\r':
+                    sb.append("%0D");
+                    break;
+                case ',':
+                    sb.append("%2C");
+                    break;
+                case '}':
+                    sb.append("%7D");
+                    break;
+                case '%':
+                    sb.append("%25");
+                    break;
+                default:
+                    sb.append(c);
+            }
+        }
+        return sb.toString();
+    }
+    
+    /**
+     * 
+     * @param s 
+     * @throws java.io.IOException 
+     * @return 
+     */
+    static String fromCSVString(String s) throws IOException {
+        if (s.charAt(0) != '\'') {
+            throw new IOException("Error deserializing string.");
+        }
+        int len = s.length();
+        StringBuffer sb = new StringBuffer(len-1);
+        for (int i = 1; i < len; i++) {
+            char c = s.charAt(i);
+            if (c == '%') {
+                char ch1 = s.charAt(i+1);
+                char ch2 = s.charAt(i+2);
+                i += 2;
+                if (ch1 == '0' && ch2 == '0') { sb.append('\0'); }
+                else if (ch1 == '0' && ch2 == 'A') { sb.append('\n'); }
+                else if (ch1 == '0' && ch2 == 'D') { sb.append('\r'); }
+                else if (ch1 == '2' && ch2 == 'C') { sb.append(','); }
+                else if (ch1 == '7' && ch2 == 'D') { sb.append('}'); }
+                else if (ch1 == '2' && ch2 == '5') { sb.append('%'); }
+                else {throw new IOException("Error deserializing string.");}
+            } else {
+                sb.append(c);
+            }
+        }
+        return sb.toString();
+    }
+    
+    /**
+     * 
+     * @param s 
+     * @return 
+     */
+    static String toXMLBuffer(ByteArrayOutputStream s) {
+        byte[] barr = s.toByteArray();
+        StringBuffer sb = new StringBuffer(2*barr.length);
+        for (int idx = 0; idx < barr.length; idx++) {
+            sb.append(Integer.toHexString((int)barr[idx]));
+        }
+        return sb.toString();
+    }
+    
+    /**
+     * 
+     * @param s 
+     * @throws java.io.IOException 
+     * @return 
+     */
+    static ByteArrayOutputStream fromXMLBuffer(String s)
+    throws IOException {
+        ByteArrayOutputStream stream =  new ByteArrayOutputStream();
+        if (s.length() == 0) { return stream; }
+        int blen = s.length()/2;
+        byte[] barr = new byte[blen];
+        for (int idx = 0; idx < blen; idx++) {
+            char c1 = s.charAt(2*idx);
+            char c2 = s.charAt(2*idx+1);
+            barr[idx] = Byte.parseByte(""+c1+c2, 16);
+        }
+        stream.write(barr);
+        return stream;
+    }
+    
+    /**
+     * 
+     * @param buf 
+     * @return 
+     */
+    static String toCSVBuffer(ByteArrayOutputStream buf) {
+        byte[] barr = buf.toByteArray();
+        StringBuffer sb = new StringBuffer(barr.length+1);
+        sb.append('#');
+        for(int idx = 0; idx < barr.length; idx++) {
+            sb.append(Integer.toHexString((int)barr[idx]));
+        }
+        return sb.toString();
+    }
+    
+    /**
+     * Converts a CSV-serialized representation of buffer to a new
+     * ByteArrayOutputStream.
+     * @param s CSV-serialized representation of buffer
+     * @throws java.io.IOException 
+     * @return Deserialized ByteArrayOutputStream
+     */
+    static ByteArrayOutputStream fromCSVBuffer(String s)
+    throws IOException {
+        if (s.charAt(0) != '#') {
+            throw new IOException("Error deserializing buffer.");
+        }
+        ByteArrayOutputStream stream =  new ByteArrayOutputStream();
+        if (s.length() == 1) { return stream; }
+        int blen = (s.length()-1)/2;
+        byte[] barr = new byte[blen];
+        for (int idx = 0; idx < blen; idx++) {
+            char c1 = s.charAt(2*idx+1);
+            char c2 = s.charAt(2*idx+2);
+            barr[idx] = Byte.parseByte(""+c1+c2, 16);
+        }
+        stream.write(barr);
+        return stream;
+    }
+}

+ 250 - 0
src/java/org/apache/hadoop/record/XmlInputArchive.java

@@ -0,0 +1,250 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record;
+
+import java.io.InputStream;
+import java.io.IOException;
+import java.io.ByteArrayOutputStream;
+import java.util.ArrayList;
+import java.util.Iterator;
+
+import org.xml.sax.*;
+import org.xml.sax.helpers.DefaultHandler;
+import javax.xml.parsers.SAXParserFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.parsers.SAXParser;
+/**
+ *
+ * @author milindb
+ */
+class XmlInputArchive implements InputArchive {
+    
+    static private class Value {
+        private String type;
+        private StringBuffer sb;
+        
+        public Value(String t) {
+            type = t;
+            sb = new StringBuffer();
+        }
+        public void addChars(char[] buf, int offset, int len) {
+            sb.append(buf, offset, len);
+        }
+        public String getValue() { return sb.toString(); }
+        public String getType() { return type; }
+    }
+    
+    private class XMLParser extends DefaultHandler {
+        private boolean charsValid = false;
+        
+        private ArrayList valList;
+        
+        private XMLParser(ArrayList vlist) {
+            valList = vlist;
+        }
+        
+        public void startDocument() throws SAXException {}
+        
+        public void endDocument() throws SAXException {}
+        
+        public void startElement(String ns,
+                String sname,
+                String qname,
+                Attributes attrs) throws SAXException {
+            charsValid = false;
+            if ("boolean".equals(qname) ||
+                    "i4".equals(qname) ||
+                    "int".equals(qname) ||
+                    "string".equals(qname) ||
+                    "double".equals(qname) ||
+                    "ex:i1".equals(qname) ||
+                    "ex:i8".equals(qname) ||
+                    "ex:float".equals(qname)) {
+                charsValid = true;
+                valList.add(new Value(qname));
+            } else if ("struct".equals(qname) ||
+                "array".equals(qname)) {
+                valList.add(new Value(qname));
+            }
+        }
+        
+        public void endElement(String ns,
+                String sname,
+                String qname) throws SAXException {
+            charsValid = false;
+            if ("struct".equals(qname) ||
+                    "array".equals(qname)) {
+                valList.add(new Value("/"+qname));
+            }
+        }
+        
+        public void characters(char buf[], int offset, int len)
+        throws SAXException {
+            if (charsValid) {
+                Value v = (Value) valList.get(valList.size()-1);
+                v.addChars(buf, offset,len);
+            }
+        }
+        
+    }
+    
+    private class XmlIndex implements Index {
+        public boolean done() {
+            Value v = (Value) valList.get(vIdx);
+            if ("/array".equals(v.getType())) {
+                valList.set(vIdx, null);
+                vIdx++;
+                return true;
+            } else {
+                return false;
+            }
+        }
+        public void incr() {}
+    }
+    
+    private ArrayList valList;
+    private int vLen;
+    private int vIdx;
+    
+    private Value next() throws IOException {
+        if (vIdx < vLen) {
+            Value v = (Value) valList.get(vIdx);
+            valList.set(vIdx, null);
+            vIdx++;
+            return v;
+        } else {
+            throw new IOException("Error in deserialization.");
+        }
+    }
+    
+    static XmlInputArchive getArchive(InputStream strm)
+    throws ParserConfigurationException, SAXException, IOException {
+        return new XmlInputArchive(strm);
+    }
+    
+    /** Creates a new instance of BinaryInputArchive */
+    public XmlInputArchive(InputStream in)
+    throws ParserConfigurationException, SAXException, IOException {
+        valList = new ArrayList();
+        DefaultHandler handler = new XMLParser(valList);
+        SAXParserFactory factory = SAXParserFactory.newInstance();
+        SAXParser parser = factory.newSAXParser();
+        parser.parse(in, handler);
+        vLen = valList.size();
+        vIdx = 0;
+    }
+    
+    public byte readByte(String tag) throws IOException {
+        Value v = next();
+        if (!"ex:i1".equals(v.getType())) {
+            throw new IOException("Error deserializing "+tag+".");
+        }
+        return Byte.parseByte(v.getValue());
+    }
+    
+    public boolean readBool(String tag) throws IOException {
+        Value v = next();
+        if (!"boolean".equals(v.getType())) {
+            throw new IOException("Error deserializing "+tag+".");
+        }
+        return "1".equals(v.getValue());
+    }
+    
+    public int readInt(String tag) throws IOException {
+        Value v = next();
+        if (!"i4".equals(v.getType()) &&
+                !"int".equals(v.getType())) {
+            throw new IOException("Error deserializing "+tag+".");
+        }
+        return Integer.parseInt(v.getValue());
+    }
+    
+    public long readLong(String tag) throws IOException {
+        Value v = next();
+        if (!"ex:i8".equals(v.getType())) {
+            throw new IOException("Error deserializing "+tag+".");
+        }
+        return Long.parseLong(v.getValue());
+    }
+    
+    public float readFloat(String tag) throws IOException {
+        Value v = next();
+        if (!"ex:float".equals(v.getType())) {
+            throw new IOException("Error deserializing "+tag+".");
+        }
+        return Float.parseFloat(v.getValue());
+    }
+    
+    public double readDouble(String tag) throws IOException {
+        Value v = next();
+        if (!"double".equals(v.getType())) {
+            throw new IOException("Error deserializing "+tag+".");
+        }
+        return Double.parseDouble(v.getValue());
+    }
+    
+    public String readString(String tag) throws IOException {
+        Value v = next();
+        if (!"string".equals(v.getType())) {
+            throw new IOException("Error deserializing "+tag+".");
+        }
+        return Utils.fromXMLString(v.getValue());
+    }
+    
+    public ByteArrayOutputStream readBuffer(String tag) throws IOException {
+        Value v = next();
+        if (!"string".equals(v.getType())) {
+            throw new IOException("Error deserializing "+tag+".");
+        }
+        return Utils.fromXMLBuffer(v.getValue());
+    }
+    
+    public void readRecord(Record r, String tag) throws IOException {
+        r.deserialize(this, tag);
+    }
+    
+    public void startRecord(String tag) throws IOException {
+        Value v = next();
+        if (!"struct".equals(v.getType())) {
+            throw new IOException("Error deserializing "+tag+".");
+        }
+    }
+    
+    public void endRecord(String tag) throws IOException {
+        Value v = next();
+        if (!"/struct".equals(v.getType())) {
+            throw new IOException("Error deserializing "+tag+".");
+        }
+    }
+    
+    public Index startVector(String tag) throws IOException {
+        Value v = next();
+        if (!"array".equals(v.getType())) {
+            throw new IOException("Error deserializing "+tag+".");
+        }
+        return new XmlIndex();
+    }
+    
+    public void endVector(String tag) throws IOException {}
+    
+    public Index startMap(String tag) throws IOException {
+        return startVector(tag);
+    }
+    
+    public void endMap(String tag) throws IOException { endVector(tag); }
+
+}

+ 251 - 0
src/java/org/apache/hadoop/record/XmlOutputArchive.java

@@ -0,0 +1,251 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record;
+
+import java.io.IOException;
+import java.io.ByteArrayOutputStream;
+import java.util.TreeMap;
+import java.util.ArrayList;
+import java.io.PrintStream;
+import java.io.OutputStream;
+import java.util.Stack;
+
+/**
+ *
+ * @author milindb
+ */
+class XmlOutputArchive implements OutputArchive {
+
+    private PrintStream stream;
+    
+    private int indent = 0;
+    
+    private Stack compoundStack;
+    
+    private void putIndent() {
+        StringBuffer sb = new StringBuffer("");
+        for (int idx = 0; idx < indent; idx++) {
+            sb.append("  ");
+        }
+        stream.print(sb.toString());
+    }
+    
+    private void addIndent() {
+        indent++;
+    }
+    
+    private void closeIndent() {
+        indent--;
+    }
+    
+    private void printBeginEnvelope(String tag) {
+        if (!compoundStack.empty()) {
+            String s = (String) compoundStack.peek();
+            if ("struct".equals(s)) {
+                putIndent();
+                stream.print("<member>\n");
+                addIndent();
+                putIndent();
+                stream.print("<name>"+tag+"</name>\n");
+                putIndent();
+                stream.print("<value>");
+            } else if ("vector".equals(s)) {
+                stream.print("<value>");
+            } else if ("map".equals(s)) {
+                stream.print("<value>");
+            }
+        } else {
+            stream.print("<value>");
+        }
+    }
+    
+    private void printEndEnvelope(String tag) {
+        if (!compoundStack.empty()) {
+            String s = (String) compoundStack.peek();
+            if ("struct".equals(s)) {
+                stream.print("</value>\n");
+                closeIndent();
+                putIndent();
+                stream.print("</member>\n");
+            } else if ("vector".equals(s)) {
+                stream.print("</value>\n");
+            } else if ("map".equals(s)) {
+                stream.print("</value>\n");
+            }
+        } else {
+            stream.print("</value>\n");
+        }
+    }
+    
+    private void insideVector(String tag) {
+        printBeginEnvelope(tag);
+        compoundStack.push("vector");
+    }
+    
+    private void outsideVector(String tag) throws IOException {
+        String s = (String) compoundStack.pop();
+        if (!"vector".equals(s)) {
+            throw new IOException("Error serializing vector.");
+        }
+        printEndEnvelope(tag);
+    }
+    
+    private void insideMap(String tag) {
+        printBeginEnvelope(tag);
+        compoundStack.push("map");
+    }
+    
+    private void outsideMap(String tag) throws IOException {
+        String s = (String) compoundStack.pop();
+        if (!"map".equals(s)) {
+            throw new IOException("Error serializing map.");
+        }
+        printEndEnvelope(tag);
+    }
+    
+    private void insideRecord(String tag) {
+        printBeginEnvelope(tag);
+        compoundStack.push("struct");
+    }
+    
+    private void outsideRecord(String tag) throws IOException {
+        String s = (String) compoundStack.pop();
+        if (!"struct".equals(s)) {
+            throw new IOException("Error serializing record.");
+        }
+        printEndEnvelope(tag);
+    }
+    
+    static XmlOutputArchive getArchive(OutputStream strm) {
+        return new XmlOutputArchive(strm);
+    }
+    
+    /** Creates a new instance of XmlOutputArchive */
+    public XmlOutputArchive(OutputStream out) {
+        stream = new PrintStream(out);
+        compoundStack = new Stack();
+    }
+    
+    public void writeByte(byte b, String tag) throws IOException {
+        printBeginEnvelope(tag);
+        stream.print("<ex:i1>");
+        stream.print(Byte.toString(b));
+        stream.print("</ex:i1>");
+        printEndEnvelope(tag);
+    }
+    
+    public void writeBool(boolean b, String tag) throws IOException {
+        printBeginEnvelope(tag);
+        stream.print("<boolean>");
+        stream.print(b ? "1" : "0");
+        stream.print("</boolean>");
+        printEndEnvelope(tag);
+    }
+    
+    public void writeInt(int i, String tag) throws IOException {
+        printBeginEnvelope(tag);
+        stream.print("<i4>");
+        stream.print(Integer.toString(i));
+        stream.print("</i4>");
+        printEndEnvelope(tag);
+    }
+    
+    public void writeLong(long l, String tag) throws IOException {
+        printBeginEnvelope(tag);
+        stream.print("<ex:i8>");
+        stream.print(Long.toString(l));
+        stream.print("</ex:i8>");
+        printEndEnvelope(tag);
+    }
+    
+    public void writeFloat(float f, String tag) throws IOException {
+        printBeginEnvelope(tag);
+        stream.print("<ex:float>");
+        stream.print(Float.toString(f));
+        stream.print("</ex:float>");
+        printEndEnvelope(tag);
+    }
+    
+    public void writeDouble(double d, String tag) throws IOException {
+        printBeginEnvelope(tag);
+        stream.print("<double>");
+        stream.print(Double.toString(d));
+        stream.print("</double>");
+        printEndEnvelope(tag);
+    }
+    
+    public void writeString(String s, String tag) throws IOException {
+        printBeginEnvelope(tag);
+        stream.print("<string>");
+        stream.print(Utils.toXMLString(s));
+        stream.print("</string>");
+        printEndEnvelope(tag);
+    }
+    
+    public void writeBuffer(ByteArrayOutputStream buf, String tag)
+    throws IOException {
+        printBeginEnvelope(tag);
+        stream.print("<string>");
+        stream.print(Utils.toXMLBuffer(buf));
+        stream.print("</string>");
+        printEndEnvelope(tag);
+    }
+    
+    public void writeRecord(Record r, String tag) throws IOException {
+        r.serialize(this, tag);
+    }
+    
+    public void startRecord(Record r, String tag) throws IOException {
+        insideRecord(tag);
+        stream.print("<struct>\n");
+        addIndent();
+    }
+    
+    public void endRecord(Record r, String tag) throws IOException {
+        closeIndent();
+        putIndent();
+        stream.print("</struct>");
+        outsideRecord(tag);
+    }
+    
+    public void startVector(ArrayList v, String tag) throws IOException {
+        insideVector(tag);
+        stream.print("<array>\n");
+        addIndent();
+    }
+    
+    public void endVector(ArrayList v, String tag) throws IOException {
+        closeIndent();
+        putIndent();
+        stream.print("</array>");
+        outsideVector(tag);
+    }
+    
+    public void startMap(TreeMap v, String tag) throws IOException {
+        insideMap(tag);
+        stream.print("<array>\n");
+        addIndent();
+    }
+    
+    public void endMap(TreeMap v, String tag) throws IOException {
+        closeIndent();
+        putIndent();
+        stream.print("</array>");
+        outsideMap(tag);
+    }
+
+}

+ 64 - 0
src/java/org/apache/hadoop/record/compiler/CppGenerator.java

@@ -0,0 +1,64 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler;
+
+import java.util.ArrayList;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Iterator;
+
+/**
+ *
+ * @author milindb
+ */
+class CppGenerator {
+    private String mName;
+    private ArrayList mInclFiles;
+    private ArrayList mRecList;
+    
+    /** Creates a new instance of CppGenerator */
+    public CppGenerator(String name, ArrayList ilist, ArrayList rlist) {
+        mName = name;
+        mInclFiles = ilist;
+        mRecList = rlist;
+    }
+    
+    public void genCode() throws IOException {
+        FileWriter cc = new FileWriter(mName+".cc");
+        FileWriter hh = new FileWriter(mName+".hh");
+        hh.write("#ifndef __"+mName.toUpperCase().replace('.','_')+"__\n");
+        hh.write("#define __"+mName.toUpperCase().replace('.','_')+"__\n");
+        
+        hh.write("#include \"recordio.hh\"\n");
+        for (Iterator i = mInclFiles.iterator(); i.hasNext();) {
+            JFile f = (JFile) i.next();
+            hh.write("#include \""+f.getName()+".hh\"\n");
+        }
+        cc.write("#include \""+mName+".hh\"\n");
+        
+        for (Iterator i = mRecList.iterator(); i.hasNext();) {
+            JRecord jr = (JRecord) i.next();
+            jr.genCppCode(hh, cc);
+        }
+        
+        hh.write("#endif //"+mName.toUpperCase().replace('.','_')+"__\n");
+        
+        hh.close();
+        cc.close();
+    }
+}

+ 41 - 0
src/java/org/apache/hadoop/record/compiler/JBoolean.java

@@ -0,0 +1,41 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler;
+
+/**
+ *
+ * @author milindb
+ */
+public class JBoolean extends JType {
+    
+    /** Creates a new instance of JBoolean */
+    public JBoolean() {
+        super("bool", "boolean", "Bool", "Boolean", "toBoolean");
+    }
+    
+    public String getSignature() {
+        return "z";
+    }
+    
+    public String genJavaCompareTo(String fname) {
+        return "    ret = ("+fname+" == peer."+fname+")? 0 : ("+fname+"?1:-1);\n";
+    }
+    
+    public String genJavaHashCode(String fname) {
+        return "     ret = ("+fname+")?0:1;\n";
+    }
+}

+ 67 - 0
src/java/org/apache/hadoop/record/compiler/JBuffer.java

@@ -0,0 +1,67 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler;
+
+/**
+ *
+ * @author milindb
+ */
+public class JBuffer extends JCompType {
+    
+    /** Creates a new instance of JBuffer */
+    public JBuffer() {
+        super(" ::std::string", "java.io.ByteArrayOutputStream", "Buffer", "java.io.ByteArrayOutputStream");
+    }
+    
+    public String genCppGetSet(String fname, int fIdx) {
+        String cgetFunc = "  virtual const "+getCppType()+"& get"+fname+"() const {\n";
+        cgetFunc += "    return m"+fname+";\n";
+        cgetFunc += "  }\n";
+        String getFunc = "  virtual "+getCppType()+"& get"+fname+"() {\n";
+        getFunc += "    bs_.set("+fIdx+");return m"+fname+";\n";
+        getFunc += "  }\n";
+        return cgetFunc + getFunc;
+    }
+    
+    public String getSignature() {
+        return "B";
+    }
+    
+    public String genJavaReadWrapper(String fname, String tag, boolean decl) {
+        String ret = "";
+        if (decl) {
+            ret = "    java.io.ByteArrayOutputStream "+fname+";\n";
+        }
+        return ret + "        java.io.ByteArrayOutputStream "+fname+"=a_.readBuffer(\""+tag+"\");\n";
+    }
+    
+    public String genJavaWriteWrapper(String fname, String tag) {
+        return "        a_.writeBuffer("+fname+"\""+tag+"\");\n";
+    }
+    
+    public String genJavaCompareTo(String fname) {
+        return "";
+    }
+    
+    public String genJavaEquals(String fname, String peer) {
+        return "    ret = org.apache.hadoop.record.Utils.bufEquals("+fname+","+peer+");\n";
+    }
+    
+    public String genJavaHashCode(String fname) {
+        return "    ret = "+fname+".toString().hashCode();\n";
+    }
+}

+ 33 - 0
src/java/org/apache/hadoop/record/compiler/JByte.java

@@ -0,0 +1,33 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler;
+
+/**
+ *
+ * @author milindb
+ */
+public class JByte extends JType {
+    
+    /** Creates a new instance of JByte */
+    public JByte() {
+        super("int8_t", "byte", "Byte", "Byte", "toByte");
+    }
+    
+    public String getSignature() {
+        return "b";
+    }
+}

+ 51 - 0
src/java/org/apache/hadoop/record/compiler/JCompType.java

@@ -0,0 +1,51 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler;
+
+/**
+ *
+ * @author milindb
+ */
+abstract class JCompType extends JType {
+    
+    /** Creates a new instance of JCompType */
+    public JCompType(String cppType, String javaType, String suffix, String wrapper) {
+        super(cppType, javaType, suffix, wrapper, null);
+    }
+    
+    public String genCppGetSet(String fname, int fIdx) {
+        String cgetFunc = "  virtual const "+getCppType()+"& get"+fname+"() const {\n";
+        cgetFunc += "    return m"+fname+";\n";
+        cgetFunc += "  }\n";
+        String getFunc = "  virtual "+getCppType()+"& get"+fname+"() {\n";
+        getFunc += "    bs_.set("+fIdx+");return m"+fname+";\n";
+        getFunc += "  }\n";
+        return cgetFunc + getFunc;
+    }
+    
+    public String genJavaCompareTo(String fname) {
+        return "    ret = "+fname+".compareTo(peer."+fname+");\n";
+    }
+    
+    public String genJavaEquals(String fname, String peer) {
+        return "    ret = "+fname+".equals("+peer+");\n";
+    }
+    
+    public String genJavaHashCode(String fname) {
+        return "    ret = "+fname+".hashCode();\n";
+    }
+}

+ 38 - 0
src/java/org/apache/hadoop/record/compiler/JDouble.java

@@ -0,0 +1,38 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler;
+
+/**
+ *
+ * @author milindb
+ */
+public class JDouble extends JType {
+    
+    /** Creates a new instance of JDouble */
+    public JDouble() {
+        super("double", "double", "Double", "Double", "toDouble");
+    }
+    
+    public String getSignature() {
+        return "d";
+    }
+    
+    public String genJavaHashCode(String fname) {
+        String tmp = "Double.doubleToLongBits("+fname+")";
+        return "    ret = (int)("+tmp+"^("+tmp+">>>32));\n";
+    }
+}

+ 93 - 0
src/java/org/apache/hadoop/record/compiler/JField.java

@@ -0,0 +1,93 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler;
+
+/**
+ *
+ * @author milindb
+ */
+public class JField {
+    private JType mType;
+    private String mName;
+    /**
+     * Creates a new instance of JField
+     */
+    public JField(JType type, String name) {
+        mType = type;
+        mName = name;
+    }
+    
+    public String getSignature() {
+        return mType.getSignature();
+    }
+    
+    public String genCppDecl() {
+        return mType.genCppDecl(mName);
+    }
+    
+    public String genJavaDecl() {
+        return mType.genJavaDecl(mName);
+    }
+    
+    public String genJavaConstructorParam(int fIdx) {
+        return mType.genJavaConstructorParam(fIdx);
+    }
+    
+    public String getName() {
+        return "m"+mName;
+    }
+    
+    public String getTag() {
+        return mName;
+    }
+    
+    public JType getType() {
+        return mType;
+    }
+    
+    public String genCppGetSet(int fIdx) {
+        return mType.genCppGetSet(mName, fIdx);
+    }
+    
+    public String genJavaGetSet(int fIdx) {
+        return mType.genJavaGetSet(mName, fIdx);
+    }
+    
+    public String genJavaWriteMethodName() {
+        return mType.genJavaWriteMethod(getName(), getTag());
+    }
+    
+    public String genJavaReadMethodName() {
+        return mType.genJavaReadMethod(getName(), getTag());
+    }
+    
+    public String genJavaCompareTo() {
+        return mType.genJavaCompareTo(getName());
+    }
+    
+    public String genJavaEquals() {
+        return mType.genJavaEquals(getName(), "peer."+getName());
+    }
+    
+    public String genJavaHashCode() {
+        return mType.genJavaHashCode(getName());
+    }
+
+    public String genJavaConstructorSet(int fIdx) {
+        return mType.genJavaConstructorSet(mName, fIdx);
+    }
+}

+ 56 - 0
src/java/org/apache/hadoop/record/compiler/JFile.java

@@ -0,0 +1,56 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+/**
+ *
+ * @author milindb@yahoo-inc.com
+ */
+public class JFile {
+    
+    private String mName;
+    private ArrayList mInclFiles;
+    private ArrayList mRecords;
+    
+    /** Creates a new instance of JFile */
+    public JFile(String name, ArrayList inclFiles, ArrayList recList) {
+        mName = name;
+        mInclFiles = inclFiles;
+        mRecords = recList;
+    }
+        
+    String getName() {
+        int idx = mName.lastIndexOf('/');
+        return (idx > 0) ? mName.substring(idx) : mName; 
+    }
+    
+    public void genCode(String language) throws IOException {
+        if ("c++".equals(language)) {
+            CppGenerator gen = new CppGenerator(mName, mInclFiles, mRecords);
+            gen.genCode();
+        } else if ("java".equals(language)) {
+            JavaGenerator gen = new JavaGenerator(mName, mInclFiles, mRecords);
+            gen.genCode();
+        } else {
+            System.out.println("Cannnot recognize language:"+language);
+            System.exit(1);
+        }
+    }
+}

+ 37 - 0
src/java/org/apache/hadoop/record/compiler/JFloat.java

@@ -0,0 +1,37 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler;
+
+/**
+ *
+ * @author milindb
+ */
+public class JFloat extends JType {
+    
+    /** Creates a new instance of JFloat */
+    public JFloat() {
+        super("float", "float", "Float", "Float", "toFloat");
+    }
+    
+    public String getSignature() {
+        return "f";
+    }
+    
+    public String genJavaHashCode(String fname) {
+        return "    ret = Float.floatToIntBits("+fname+");\n";
+    }
+}

+ 33 - 0
src/java/org/apache/hadoop/record/compiler/JInt.java

@@ -0,0 +1,33 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler;
+
+/**
+ *
+ * @author milindb
+ */
+public class JInt extends JType {
+    
+    /** Creates a new instance of JInt */
+    public JInt() {
+        super("int32_t", "int", "Int", "Integer", "toInt");
+    }
+    
+    public String getSignature() {
+        return "i";
+    }
+}

+ 37 - 0
src/java/org/apache/hadoop/record/compiler/JLong.java

@@ -0,0 +1,37 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler;
+
+/**
+ *
+ * @author milindb
+ */
+public class JLong extends JType {
+    
+    /** Creates a new instance of JLong */
+    public JLong() {
+        super("int64_t", "long", "Long", "Long", "toLong");
+    }
+    
+    public String getSignature() {
+        return "l";
+    }
+    
+    public String genJavaHashCode(String fname) {
+        return "    ret = (int) ("+fname+"^("+fname+">>>32));\n";
+    }
+}

+ 99 - 0
src/java/org/apache/hadoop/record/compiler/JMap.java

@@ -0,0 +1,99 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler;
+
+/**
+ *
+ * @author milindb
+ */
+public class JMap extends JCompType {
+   
+    static private int level = 0;
+    
+    static private String getLevel() { return Integer.toString(level); }
+    
+    static private void incrLevel() { level++; }
+    
+    static private void decrLevel() { level--; }
+    
+    static private String getId(String id) { return id+getLevel(); }
+    
+    private JType mKey;
+    private JType mValue;
+    
+    /** Creates a new instance of JMap */
+    public JMap(JType t1, JType t2) {
+        super(" ::std::map<"+t1.getCppType()+","+t2.getCppType()+">",
+                "java.util.TreeMap", "Map", "java.util.TreeMap");
+        mKey = t1;
+        mValue = t2;
+    }
+    
+    public String getSignature() {
+        return "{" + mKey.getSignature() + mValue.getSignature() +"}";
+    }
+    
+    public String genJavaCompareTo(String fname) {
+        return "";
+    }
+    
+    public String genJavaReadWrapper(String fname, String tag, boolean decl) {
+        StringBuffer ret = new StringBuffer("");
+        if (decl) {
+            ret.append("    java.util.TreeMap "+fname+";\n");
+        }
+        ret.append("    {\n");
+        incrLevel();
+        ret.append("      org.apache.hadoop.record.Index "+getId("midx")+" = a_.startMap(\""+tag+"\");\n");
+        ret.append("      "+fname+"=new java.util.TreeMap();\n");
+        ret.append("      for (; !"+getId("midx")+".done(); "+getId("midx")+".incr()) {\n");
+        ret.append(mKey.genJavaReadWrapper(getId("k"),getId("k"),true));
+        ret.append(mValue.genJavaReadWrapper(getId("v"),getId("v"),true));
+        ret.append("        "+fname+".put("+getId("k")+","+getId("v")+");\n");
+        ret.append("      }\n");
+        ret.append("    a_.endMap(\""+tag+"\");\n");
+        decrLevel();
+        ret.append("    }\n");
+        return ret.toString();
+    }
+    
+    public String genJavaReadMethod(String fname, String tag) {
+        return genJavaReadWrapper(fname, tag, false);
+    }
+    
+    public String genJavaWriteWrapper(String fname, String tag) {
+        StringBuffer ret = new StringBuffer("    {\n");
+        incrLevel();
+        ret.append("      a_.startMap("+fname+",\""+tag+"\");\n");
+        ret.append("      java.util.Set "+getId("es")+" = "+fname+".entrySet();\n");
+        ret.append("      for(java.util.Iterator "+getId("midx")+" = "+getId("es")+".iterator(); "+getId("midx")+".hasNext(); ) {\n");
+        ret.append("        java.util.Map.Entry "+getId("me")+" = (java.util.Map.Entry) "+getId("midx")+".next();\n");
+        ret.append("        "+mKey.getJavaWrapperType()+" "+getId("k")+" = ("+mKey.getJavaWrapperType()+") "+getId("me")+".getKey();\n");
+        ret.append("        "+mValue.getJavaWrapperType()+" "+getId("v")+" = ("+mValue.getJavaWrapperType()+") "+getId("me")+".getValue();\n");
+        ret.append(mKey.genJavaWriteWrapper(getId("k"),getId("k")));
+        ret.append(mValue.genJavaWriteWrapper(getId("v"),getId("v")));
+        ret.append("      }\n");
+        ret.append("      a_.endMap("+fname+",\""+tag+"\");\n");
+        ret.append("    }\n");
+        decrLevel();
+        return ret.toString();
+    }
+    
+    public String genJavaWriteMethod(String fname, String tag) {
+        return genJavaWriteWrapper(fname, tag);
+    }
+}

+ 384 - 0
src/java/org/apache/hadoop/record/compiler/JRecord.java

@@ -0,0 +1,384 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+
+/**
+ *
+ * @author milindb
+ */
+public class JRecord extends JCompType {
+
+    private String mFQName;
+    private String mName;
+    private String mModule;
+    private ArrayList mFields;
+    
+    /**
+     * Creates a new instance of JRecord
+     */
+    public JRecord(String name, ArrayList flist) {
+        super(name.replaceAll("\\.","::"), name, "Record", name);
+        mFQName = name;
+        int idx = name.lastIndexOf('.');
+        mName = name.substring(idx+1);
+        mModule = name.substring(0, idx);
+        mFields = flist;
+    }
+    
+    public String getName() {
+        return mName;
+    }
+    
+    public String getJavaFQName() {
+        return mFQName;
+    }
+    
+    public String getCppFQName() {
+        return mFQName.replaceAll("\\.", "::");
+    }
+    
+    public String getJavaPackage() {
+        return mModule;
+    }
+    
+    public String getCppNameSpace() {
+        return mModule.replaceAll("\\.", "::");
+    }
+    
+    public ArrayList getFields() {
+        return mFields;
+    }
+    
+    public String getSignature() {
+        StringBuffer sb = new StringBuffer();
+        sb.append("L").append(mName).append("(");
+        for (Iterator i = mFields.iterator(); i.hasNext();) {
+            String s = ((JField) i.next()).getSignature();
+            sb.append(s);
+        }
+        sb.append(")");
+        return sb.toString();
+    }
+    
+    public String genCppDecl(String fname) {
+        return "  "+mName+" "+fname+";\n";
+    }
+    
+    public String genJavaDecl (String fname) {
+        return "  private "+mName+" "+fname+";\n";
+    }
+    
+    public String genJavaReadWrapper(String fname, String tag, boolean decl) {
+        StringBuffer ret = new StringBuffer("");
+        if (decl) {
+            ret.append("    "+getJavaFQName()+" "+fname+";\n");
+        }
+        ret.append("    "+fname+"= new "+getJavaFQName()+"();\n");
+        ret.append("    a_.readRecord("+fname+",\""+tag+"\");\n");
+        return ret.toString();
+    }
+    
+    public String genJavaWriteWrapper(String fname, String tag) {
+        return "    a_.writeRecord("+fname+",\""+tag+"\");\n";
+    }
+    
+    public void genCppCode(FileWriter hh, FileWriter cc)
+        throws IOException {
+        String[] ns = getCppNameSpace().split("::");
+        for (int i = 0; i < ns.length; i++) {
+            hh.write("namespace "+ns[i]+" {\n");
+        }
+        
+        hh.write("class "+getName()+" : public ::hadoop::Record {\n");
+        hh.write("private:\n");
+        
+        for (Iterator i = mFields.iterator(); i.hasNext();) {
+            JField jf = (JField) i.next();
+            hh.write(jf.genCppDecl());
+        }
+        hh.write("  std::bitset<"+mFields.size()+"> bs_;\n");
+        hh.write("public:\n");
+        hh.write("  virtual void serialize(::hadoop::OArchive& a_, const char* tag);\n");
+        hh.write("  virtual void deserialize(::hadoop::IArchive& a_, const char* tag);\n");
+        hh.write("  virtual const ::std::string& type() const;\n");
+        hh.write("  virtual const ::std::string& signature() const;\n");
+        hh.write("  virtual bool validate() const;\n");
+        hh.write("  virtual bool operator<(const "+getName()+"& peer_);\n");
+        hh.write("  virtual bool operator==(const "+getName()+"& peer_);\n");
+        hh.write("  virtual ~"+getName()+"() {};\n");
+        int fIdx = 0;
+        for (Iterator i = mFields.iterator(); i.hasNext(); fIdx++) {
+            JField jf = (JField) i.next();
+            hh.write(jf.genCppGetSet(fIdx));
+        }
+        hh.write("}; // end record "+getName()+"\n");
+        for (int i=0; i<ns.length; i++) {
+            hh.write("} // end namespace "+ns[i]+"\n");
+        }
+        cc.write("void "+getCppFQName()+"::serialize(::hadoop::OArchive& a_, const char* tag) {\n");
+        cc.write("  if (!validate()) throw new ::hadoop::IOException(\"All fields not set.\");\n");
+        cc.write("  a_.startRecord(*this,tag);\n");
+        fIdx = 0;
+        for (Iterator i = mFields.iterator(); i.hasNext(); fIdx++) {
+            JField jf = (JField) i.next();
+            String name = jf.getName();
+            if (jf.getType() instanceof JBuffer) {
+                cc.write("  a_.serialize("+name+","+name+".length(),\""+jf.getTag()+"\");\n");
+            } else {
+                cc.write("  a_.serialize("+name+",\""+jf.getTag()+"\");\n");
+            }
+            cc.write("  bs_.reset("+fIdx+");\n");
+        }
+        cc.write("  a_.endRecord(*this,tag);\n");
+        cc.write("  return;\n");
+        cc.write("}\n");
+        
+        cc.write("void "+getCppFQName()+"::deserialize(::hadoop::IArchive& a_, const char* tag) {\n");
+        cc.write("  a_.startRecord(*this,tag);\n");
+        fIdx = 0;
+        for (Iterator i = mFields.iterator(); i.hasNext(); fIdx++) {
+            JField jf = (JField) i.next();
+            String name = jf.getName();
+            if (jf.getType() instanceof JBuffer) {
+                cc.write("  { size_t len=0; a_.deserialize("+name+",len,\""+jf.getTag()+"\");}\n");
+            } else {
+                cc.write("  a_.deserialize("+name+",\""+jf.getTag()+"\");\n");
+            }
+            cc.write("  bs_.set("+fIdx+");\n");
+        }
+        cc.write("  a_.endRecord(*this,tag);\n");
+        cc.write("  return;\n");
+        cc.write("}\n");
+        
+        cc.write("bool "+getCppFQName()+"::validate() const {\n");
+        cc.write("  if (bs_.size() != bs_.count()) return false;\n");
+        for (Iterator i = mFields.iterator(); i.hasNext(); fIdx++) {
+            JField jf = (JField) i.next();
+            JType type = jf.getType();
+            if (type instanceof JRecord) {
+                cc.write("  if (!"+jf.getName()+".validate()) return false;\n");
+            }
+        }
+        cc.write("  return true;\n");
+        cc.write("}\n");
+        
+        cc.write("bool "+getCppFQName()+"::operator< (const "+getCppFQName()+"& peer_) {\n");
+        cc.write("  return (1\n");
+        for (Iterator i = mFields.iterator(); i.hasNext();) {
+            JField jf = (JField) i.next();
+            String name = jf.getName();
+            cc.write("    && ("+name+" < peer_."+name+")\n");
+        }
+        cc.write("  );\n");
+        cc.write("}\n");
+        
+        cc.write("bool "+getCppFQName()+"::operator== (const "+getCppFQName()+"& peer_) {\n");
+        cc.write("  return (1\n");
+        for (Iterator i = mFields.iterator(); i.hasNext();) {
+            JField jf = (JField) i.next();
+            String name = jf.getName();
+            cc.write("    && ("+name+" == peer_."+name+")\n");
+        }
+        cc.write("  );\n");
+        cc.write("}\n");
+        
+        cc.write("const ::std::string&"+getCppFQName()+"::type() const {\n");
+        cc.write("  static const ::std::string type_(\""+mName+"\");\n");
+        cc.write("  return type_;\n");
+        cc.write("}\n");
+        
+        cc.write("const ::std::string&"+getCppFQName()+"::signature() const {\n");
+        cc.write("  static const ::std::string sig_(\""+getSignature()+"\");\n");
+        cc.write("  return sig_;\n");
+        cc.write("}\n");
+        
+    }
+    
+    public void genJavaCode() throws IOException {
+        String pkg = getJavaPackage();
+        String pkgpath = pkg.replaceAll("\\.", "/");
+        File pkgdir = new File(pkgpath);
+        if (!pkgdir.exists()) {
+            // create the pkg directory
+            boolean ret = pkgdir.mkdirs();
+            if (!ret) {
+                System.out.println("Cannnot create directory: "+pkgpath);
+                System.exit(1);
+            }
+        } else if (!pkgdir.isDirectory()) {
+            // not a directory
+            System.out.println(pkgpath+" is not a directory.");
+            System.exit(1);
+        }
+        File jfile = new File(pkgdir, getName()+".java");
+        FileWriter jj = new FileWriter(jfile);
+        jj.write("// File generated by hadoop record compiler. Do not edit.\n");
+        jj.write("package "+getJavaPackage()+";\n\n");
+        jj.write("public class "+getName()+" implements org.apache.hadoop.record.Record, org.apache.hadoop.io.WritableComparable {\n");
+        for (Iterator i = mFields.iterator(); i.hasNext();) {
+            JField jf = (JField) i.next();
+            jj.write(jf.genJavaDecl());
+        }
+        jj.write("  private java.util.BitSet bs_;\n");
+        jj.write("  public "+getName()+"() {\n");
+        jj.write("    bs_ = new java.util.BitSet("+(mFields.size()+1)+");\n");
+        jj.write("    bs_.set("+mFields.size()+");\n");
+        jj.write("  }\n");
+        
+        jj.write("  public "+getName()+"(\n");
+        int fIdx = 0;
+        int fLen = mFields.size();
+        for (Iterator i = mFields.iterator(); i.hasNext(); fIdx++) {
+            JField jf = (JField) i.next();
+            jj.write(jf.genJavaConstructorParam(fIdx));
+            jj.write((fLen-1 == fIdx)?"":",\n");
+        }
+        jj.write(") {\n");
+        jj.write("    bs_ = new java.util.BitSet("+(mFields.size()+1)+");\n");
+        jj.write("    bs_.set("+mFields.size()+");\n");
+        fIdx = 0;
+        for (Iterator i = mFields.iterator(); i.hasNext(); fIdx++) {
+            JField jf = (JField) i.next();
+            jj.write(jf.genJavaConstructorSet(fIdx));
+        }
+        jj.write("  }\n");
+        fIdx = 0;
+        for (Iterator i = mFields.iterator(); i.hasNext(); fIdx++) {
+            JField jf = (JField) i.next();
+            jj.write(jf.genJavaGetSet(fIdx));
+        }
+        jj.write("  public void serialize(org.apache.hadoop.record.OutputArchive a_, String tag) throws java.io.IOException {\n");
+        jj.write("    if (!validate()) throw new java.io.IOException(\"All fields not set:\");\n");
+        jj.write("    a_.startRecord(this,tag);\n");
+        fIdx = 0;
+        for (Iterator i = mFields.iterator(); i.hasNext(); fIdx++) {
+            JField jf = (JField) i.next();
+            jj.write(jf.genJavaWriteMethodName());
+            jj.write("    bs_.clear("+fIdx+");\n");
+        }
+        jj.write("    a_.endRecord(this,tag);\n");
+        jj.write("  }\n");
+        
+        jj.write("  public void deserialize(org.apache.hadoop.record.InputArchive a_, String tag) throws java.io.IOException {\n");
+        jj.write("    a_.startRecord(tag);\n");
+        fIdx = 0;
+        for (Iterator i = mFields.iterator(); i.hasNext(); fIdx++) {
+            JField jf = (JField) i.next();
+            jj.write(jf.genJavaReadMethodName());
+            jj.write("    bs_.set("+fIdx+");\n");
+        }
+        jj.write("    a_.endRecord(tag);\n");
+        jj.write("}\n");
+        
+        jj.write("  public String toString() {\n");
+        jj.write("    try {\n");
+        jj.write("      java.io.ByteArrayOutputStream s =\n");
+        jj.write("        new java.io.ByteArrayOutputStream();\n");
+        jj.write("      org.apache.hadoop.record.CsvOutputArchive a_ = \n");
+        jj.write("        new org.apache.hadoop.record.CsvOutputArchive(s);\n");
+        jj.write("      a_.startRecord(this,\"\");\n");
+        fIdx = 0;
+        for (Iterator i = mFields.iterator(); i.hasNext(); fIdx++) {
+            JField jf = (JField) i.next();
+            jj.write(jf.genJavaWriteMethodName());
+        }
+        jj.write("      a_.endRecord(this,\"\");\n");
+        jj.write("      return new String(s.toByteArray(), \"UTF-8\");\n");
+        jj.write("    } catch (Throwable ex) {\n");
+        jj.write("      ex.printStackTrace();\n");
+        jj.write("    }\n");
+        jj.write("    return \"ERROR\";\n");
+        jj.write("  }\n");
+        
+        jj.write("  public void write(java.io.DataOutput out) throws java.io.IOException {\n");
+        jj.write("    org.apache.hadoop.record.BinaryOutputArchive archive = new org.apache.hadoop.record.BinaryOutputArchive(out);\n");
+        jj.write("    serialize(archive, \"\");\n");
+        jj.write("  }\n");
+        
+        jj.write("  public void readFields(java.io.DataInput in) throws java.io.IOException {\n");
+        jj.write("    org.apache.hadoop.record.BinaryInputArchive archive = new org.apache.hadoop.record.BinaryInputArchive(in);\n");
+        jj.write("    deserialize(archive, \"\");\n");
+        jj.write("  }\n");
+        
+        jj.write("  private boolean validate() {\n");
+        jj.write("    if (bs_.cardinality() != bs_.length()) return false;\n");
+        for (Iterator i = mFields.iterator(); i.hasNext(); fIdx++) {
+            JField jf = (JField) i.next();
+            JType type = jf.getType();
+            if (type instanceof JRecord) {
+                jj.write("    if (!"+jf.getName()+".validate()) return false;\n");
+            }
+        }
+        jj.write("    return true;\n");
+        jj.write("}\n");
+        
+        jj.write("  public int compareTo (Object peer_) throws ClassCastException {\n");
+        jj.write("    if (!(peer_ instanceof "+getName()+")) {\n");
+        jj.write("      throw new ClassCastException(\"Comparing different types of records.\");\n");
+        jj.write("    }\n");
+        jj.write("    "+getName()+" peer = ("+getName()+") peer_;\n");
+        jj.write("    int ret = 0;\n");
+        for (Iterator i = mFields.iterator(); i.hasNext(); fIdx++) {
+            JField jf = (JField) i.next();
+            jj.write(jf.genJavaCompareTo());
+            jj.write("    if (ret != 0) return ret;\n");
+        }
+        jj.write("     return ret;\n");
+        jj.write("  }\n");
+        
+        jj.write("  public boolean equals(Object peer_) {\n");
+        jj.write("    if (!(peer_ instanceof "+getName()+")) {\n");
+        jj.write("      return false;\n");
+        jj.write("    }\n");
+        jj.write("    if (peer_ == this) {\n");
+        jj.write("      return true;\n");
+        jj.write("    }\n");
+        jj.write("    "+getName()+" peer = ("+getName()+") peer_;\n");
+        jj.write("    boolean ret = false;\n");
+        for (Iterator i = mFields.iterator(); i.hasNext(); fIdx++) {
+            JField jf = (JField) i.next();
+            jj.write(jf.genJavaEquals());
+            jj.write("    if (!ret) return ret;\n");
+        }
+        jj.write("     return ret;\n");
+        jj.write("  }\n");
+        
+        jj.write("  public int hashCode() {\n");
+        jj.write("    int result = 17;\n");
+        jj.write("    int ret;\n");
+        for (Iterator i = mFields.iterator(); i.hasNext(); fIdx++) {
+            JField jf = (JField) i.next();
+            jj.write(jf.genJavaHashCode());
+            jj.write("    result = 37*result + ret;\n");
+        }
+        jj.write("    return result;\n");
+        jj.write("  }\n");
+        jj.write("  public static String signature() {\n");
+        jj.write("    return \""+getSignature()+"\";\n");
+        jj.write("  }\n");
+        
+        jj.write("}\n");
+        
+        jj.close();
+    }
+}

+ 47 - 0
src/java/org/apache/hadoop/record/compiler/JString.java

@@ -0,0 +1,47 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler;
+
+/**
+ *
+ * @author milindb
+ */
+public class JString extends JCompType {
+    
+    /** Creates a new instance of JString */
+    public JString() {
+        super(" ::std::string", "String", "String", "String");
+    }
+    
+    public String getSignature() {
+        return "s";
+    }
+    
+    public String genJavaReadWrapper(String fname, String tag, boolean decl) {
+        String ret = "";
+        if (decl) {
+            ret = "    String "+fname+";\n";
+        }
+        return ret + "        "+fname+"=a_.readString(\""+tag+"\");\n";
+    }
+    
+    public String genJavaWriteWrapper(String fname, String tag) {
+        return "        a_.writeString("+fname+",\""+tag+"\");\n";
+    }
+    
+    
+}

+ 127 - 0
src/java/org/apache/hadoop/record/compiler/JType.java

@@ -0,0 +1,127 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler;
+
+/**
+ *
+ * @author milindb
+ */
+public abstract class JType {
+    
+    private String mCppName;
+    private String mJavaName;
+    private String mMethodSuffix;
+    private String mWrapper;
+    private String mUnwrapMethod;
+    
+    /**
+     * Creates a new instance of JType
+     */
+    public JType(String cppname, String javaname, String suffix, String wrapper, String unwrap) {
+        mCppName = cppname;
+        mJavaName = javaname;
+        mMethodSuffix = suffix;
+        mWrapper = wrapper;
+        mUnwrapMethod = unwrap;
+    }
+    
+    abstract public String getSignature();
+    
+    public String genCppDecl(String fname) {
+        return "  "+mCppName+" m"+fname+";\n"; 
+    }
+    
+    public String genJavaDecl (String fname) {
+        return "  private "+mJavaName+" m"+fname+";\n";
+    }
+    
+    public String genJavaConstructorParam (int fIdx) {
+        return "        "+mJavaName+" m"+fIdx;
+    }
+    
+    public String genCppGetSet(String fname, int fIdx) {
+        String getFunc = "  virtual "+mCppName+" get"+fname+"() const {\n";
+        getFunc += "    return m"+fname+";\n";
+        getFunc += "  }\n";
+        String setFunc = "  virtual void set"+fname+"("+mCppName+" m_) {\n";
+        setFunc += "    m"+fname+"=m_; bs_.set("+fIdx+");\n";
+        setFunc += "  }\n";
+        return getFunc+setFunc;
+    }
+    
+    public String genJavaGetSet(String fname, int fIdx) {
+        String getFunc = "  public "+mJavaName+" get"+fname+"() {\n";
+        getFunc += "    return m"+fname+";\n";
+        getFunc += "  }\n";
+        String setFunc = "  public void set"+fname+"("+mJavaName+" m_) {\n";
+        setFunc += "    m"+fname+"=m_; bs_.set("+fIdx+");\n";
+        setFunc += "  }\n";
+        return getFunc+setFunc;
+    }
+    
+    public String getCppType() {
+        return mCppName;
+    }
+    
+    public String getJavaType() {
+        return mJavaName;
+    }
+   
+    public String getJavaWrapperType() {
+        return mWrapper;
+    }
+    
+    public String getMethodSuffix() {
+        return mMethodSuffix;
+    }
+    
+    public String genJavaWriteMethod(String fname, String tag) {
+        return "    a_.write"+mMethodSuffix+"("+fname+",\""+tag+"\");\n";
+    }
+    
+    public String genJavaReadMethod(String fname, String tag) {
+        return "    "+fname+"=a_.read"+mMethodSuffix+"(\""+tag+"\");\n";
+    }
+    
+    public String genJavaReadWrapper(String fname, String tag, boolean decl) {
+        String ret = "";
+        if (decl) {
+            ret = "    "+mWrapper+" "+fname+";\n";
+        }
+        return ret + "    "+fname+"=new "+mWrapper+"(a_.read"+mMethodSuffix+"(\""+tag+"\"));\n";
+    }
+    
+    public String genJavaWriteWrapper(String fname, String tag) {
+        return "        a_.write"+mMethodSuffix+"("+fname+"."+mUnwrapMethod+"(),\""+tag+"\");\n";
+    }
+    
+    public String genJavaCompareTo(String fname) {
+        return "    ret = ("+fname+" == peer."+fname+")? 0 :(("+fname+"<peer."+fname+")?-1:1);\n";
+    }
+    
+    public String genJavaEquals(String fname, String peer) {
+        return "    ret = ("+fname+"=="+peer+");\n";
+    }
+    
+    public String genJavaHashCode(String fname) {
+        return "    ret = (int)"+fname+";\n";
+    }
+
+    String genJavaConstructorSet(String fname, int fIdx) {
+        return "    m"+fname+"=m"+fIdx+"; bs_.set("+fIdx+");\n";
+    }
+}

+ 92 - 0
src/java/org/apache/hadoop/record/compiler/JVector.java

@@ -0,0 +1,92 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler;
+
+/**
+ *
+ * @author milindb
+ */
+public class JVector extends JCompType {
+    
+    static private int level = 0;
+    
+    static private String getId(String id) { return id+getLevel(); }
+    
+    static private String getLevel() { return Integer.toString(level); }
+    
+    static private void incrLevel() { level++; }
+    
+    static private void decrLevel() { level--; }
+    
+    private JType mElement;
+    
+    /** Creates a new instance of JVector */
+    public JVector(JType t) {
+        super(" ::std::vector<"+t.getCppType()+">", "java.util.ArrayList", "Vector", "java.util.ArrayList");
+        mElement = t;
+    }
+    
+    public String getSignature() {
+        return "[" + mElement.getSignature() + "]";
+    }
+    
+    public String genJavaCompareTo(String fname) {
+        return "";
+    }
+    
+    public String genJavaReadWrapper(String fname, String tag, boolean decl) {
+        StringBuffer ret = new StringBuffer("");
+        if (decl) {
+            ret.append("      java.util.ArrayList "+fname+";\n");
+        }
+        ret.append("    {\n");
+        incrLevel();
+        ret.append("      org.apache.hadoop.record.Index "+getId("vidx")+" = a_.startVector(\""+tag+"\");\n");
+        ret.append("      "+fname+"=new java.util.ArrayList();\n");
+        ret.append("      for (; !"+getId("vidx")+".done(); "+getId("vidx")+".incr()) {\n");
+        ret.append(mElement.genJavaReadWrapper(getId("e"), getId("e"), true));
+        ret.append("        "+fname+".add("+getId("e")+");\n");
+        ret.append("      }\n");
+        ret.append("    a_.endVector(\""+tag+"\");\n");
+        decrLevel();
+        ret.append("    }\n");
+        return ret.toString();
+    }
+    
+    public String genJavaReadMethod(String fname, String tag) {
+        return genJavaReadWrapper(fname, tag, false);
+    }
+    
+    public String genJavaWriteWrapper(String fname, String tag) {
+        StringBuffer ret = new StringBuffer("    {\n");
+        incrLevel();
+        ret.append("      a_.startVector("+fname+",\""+tag+"\");\n");
+        ret.append("      int "+getId("len")+" = "+fname+".size();\n");
+        ret.append("      for(int "+getId("vidx")+" = 0; "+getId("vidx")+"<"+getId("len")+"; "+getId("vidx")+"++) {\n");
+        ret.append("        "+mElement.getJavaWrapperType()+" "+getId("e")+" = ("+mElement.getJavaWrapperType()+") "+fname+".get("+getId("vidx")+");\n");
+        ret.append(mElement.genJavaWriteWrapper(getId("e"), getId("e")));
+        ret.append("      }\n");
+        ret.append("      a_.endVector("+fname+",\""+tag+"\");\n");
+        ret.append("    }\n");
+        decrLevel();
+        return ret.toString();
+    }
+    
+    public String genJavaWriteMethod(String fname, String tag) {
+        return genJavaWriteWrapper(fname, tag);
+    }
+}

+ 47 - 0
src/java/org/apache/hadoop/record/compiler/JavaGenerator.java

@@ -0,0 +1,47 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler;
+
+import java.util.ArrayList;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Iterator;
+
+/**
+ *
+ * @author milindb
+ */
+class JavaGenerator {
+    private String mName;
+    private ArrayList mInclFiles;
+    private ArrayList mRecList;
+    
+    /** Creates a new instance of JavaGenerator */
+    JavaGenerator(String name, ArrayList incl, ArrayList records) {
+        mName = name;
+        mInclFiles = incl;
+        mRecList = records;
+    }
+    
+    public void genCode() throws IOException {
+        for (Iterator i = mRecList.iterator(); i.hasNext(); ) {
+            JRecord rec = (JRecord) i.next();
+            rec.genJavaCode();
+        }
+    }
+}

+ 208 - 0
src/java/org/apache/hadoop/record/compiler/generated/ParseException.java

@@ -0,0 +1,208 @@
+/* Generated By:JavaCC: Do not edit this line. ParseException.java Version 3.0 */
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler.generated;
+
+/**
+ * This exception is thrown when parse errors are encountered.
+ * You can explicitly create objects of this exception type by
+ * calling the method generateParseException in the generated
+ * parser.
+ *
+ * You can modify this class to customize your error reporting
+ * mechanisms so long as you retain the public fields.
+ */
+public class ParseException extends Exception {
+
+  /**
+   * This constructor is used by the method "generateParseException"
+   * in the generated parser.  Calling this constructor generates
+   * a new object of this type with the fields "currentToken",
+   * "expectedTokenSequences", and "tokenImage" set.  The boolean
+   * flag "specialConstructor" is also set to true to indicate that
+   * this constructor was used to create this object.
+   * This constructor calls its super class with the empty string
+   * to force the "toString" method of parent class "Throwable" to
+   * print the error message in the form:
+   *     ParseException: <result of getMessage>
+   */
+  public ParseException(Token currentTokenVal,
+                        int[][] expectedTokenSequencesVal,
+                        String[] tokenImageVal
+                       )
+  {
+    super("");
+    specialConstructor = true;
+    currentToken = currentTokenVal;
+    expectedTokenSequences = expectedTokenSequencesVal;
+    tokenImage = tokenImageVal;
+  }
+
+  /**
+   * The following constructors are for use by you for whatever
+   * purpose you can think of.  Constructing the exception in this
+   * manner makes the exception behave in the normal way - i.e., as
+   * documented in the class "Throwable".  The fields "errorToken",
+   * "expectedTokenSequences", and "tokenImage" do not contain
+   * relevant information.  The JavaCC generated code does not use
+   * these constructors.
+   */
+
+  public ParseException() {
+    super();
+    specialConstructor = false;
+  }
+
+  public ParseException(String message) {
+    super(message);
+    specialConstructor = false;
+  }
+
+  /**
+   * This variable determines which constructor was used to create
+   * this object and thereby affects the semantics of the
+   * "getMessage" method (see below).
+   */
+  protected boolean specialConstructor;
+
+  /**
+   * This is the last token that has been consumed successfully.  If
+   * this object has been created due to a parse error, the token
+   * followng this token will (therefore) be the first error token.
+   */
+  public Token currentToken;
+
+  /**
+   * Each entry in this array is an array of integers.  Each array
+   * of integers represents a sequence of tokens (by their ordinal
+   * values) that is expected at this point of the parse.
+   */
+  public int[][] expectedTokenSequences;
+
+  /**
+   * This is a reference to the "tokenImage" array of the generated
+   * parser within which the parse error occurred.  This array is
+   * defined in the generated ...Constants interface.
+   */
+  public String[] tokenImage;
+
+  /**
+   * This method has the standard behavior when this object has been
+   * created using the standard constructors.  Otherwise, it uses
+   * "currentToken" and "expectedTokenSequences" to generate a parse
+   * error message and returns it.  If this object has been created
+   * due to a parse error, and you do not catch it (it gets thrown
+   * from the parser), then this method is called during the printing
+   * of the final stack trace, and hence the correct error message
+   * gets displayed.
+   */
+  public String getMessage() {
+    if (!specialConstructor) {
+      return super.getMessage();
+    }
+    StringBuffer expected = new StringBuffer();
+    int maxSize = 0;
+    for (int i = 0; i < expectedTokenSequences.length; i++) {
+      if (maxSize < expectedTokenSequences[i].length) {
+        maxSize = expectedTokenSequences[i].length;
+      }
+      for (int j = 0; j < expectedTokenSequences[i].length; j++) {
+        expected.append(tokenImage[expectedTokenSequences[i][j]]).append(" ");
+      }
+      if (expectedTokenSequences[i][expectedTokenSequences[i].length - 1] != 0) {
+        expected.append("...");
+      }
+      expected.append(eol).append("    ");
+    }
+    String retval = "Encountered \"";
+    Token tok = currentToken.next;
+    for (int i = 0; i < maxSize; i++) {
+      if (i != 0) retval += " ";
+      if (tok.kind == 0) {
+        retval += tokenImage[0];
+        break;
+      }
+      retval += add_escapes(tok.image);
+      tok = tok.next; 
+    }
+    retval += "\" at line " + currentToken.next.beginLine + ", column " + currentToken.next.beginColumn;
+    retval += "." + eol;
+    if (expectedTokenSequences.length == 1) {
+      retval += "Was expecting:" + eol + "    ";
+    } else {
+      retval += "Was expecting one of:" + eol + "    ";
+    }
+    retval += expected.toString();
+    return retval;
+  }
+
+  /**
+   * The end of line string for this machine.
+   */
+  protected String eol = System.getProperty("line.separator", "\n");
+ 
+  /**
+   * Used to convert raw characters to their escaped version
+   * when these raw version cannot be used as part of an ASCII
+   * string literal.
+   */
+  protected String add_escapes(String str) {
+      StringBuffer retval = new StringBuffer();
+      char ch;
+      for (int i = 0; i < str.length(); i++) {
+        switch (str.charAt(i))
+        {
+           case 0 :
+              continue;
+           case '\b':
+              retval.append("\\b");
+              continue;
+           case '\t':
+              retval.append("\\t");
+              continue;
+           case '\n':
+              retval.append("\\n");
+              continue;
+           case '\f':
+              retval.append("\\f");
+              continue;
+           case '\r':
+              retval.append("\\r");
+              continue;
+           case '\"':
+              retval.append("\\\"");
+              continue;
+           case '\'':
+              retval.append("\\\'");
+              continue;
+           case '\\':
+              retval.append("\\\\");
+              continue;
+           default:
+              if ((ch = str.charAt(i)) < 0x20 || ch > 0x7e) {
+                 String s = "0000" + Integer.toString(ch, 16);
+                 retval.append("\\u" + s.substring(s.length() - 4, s.length()));
+              } else {
+                 retval.append(ch);
+              }
+              continue;
+        }
+      }
+      return retval.toString();
+   }
+
+}

+ 517 - 0
src/java/org/apache/hadoop/record/compiler/generated/Rcc.java

@@ -0,0 +1,517 @@
+/* Generated By:JavaCC: Do not edit this line. Rcc.java */
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler.generated;
+
+import org.apache.hadoop.record.compiler.*;
+import java.util.ArrayList;
+import java.util.Hashtable;
+import java.util.Iterator;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+public class Rcc implements RccConstants {
+    private static String language = "java";
+    private static ArrayList recFiles = new ArrayList();
+    private static JFile curFile;
+    private static Hashtable recTab;
+    private static String curDir = System.getProperty("user.dir");
+    private static String curFileName;
+    private static String curModuleName;
+
+    public static void main(String args[]) {
+        for (int i=0; i<args.length; i++) {
+            if ("-l".equalsIgnoreCase(args[i]) ||
+                "--language".equalsIgnoreCase(args[i])) {
+                language = args[i+1].toLowerCase();
+                i++;
+            } else {
+                recFiles.add(args[i]);
+            }
+        }
+        if (!"c++".equals(language) && !"java".equals(language)) {
+            System.out.println("Cannot recognize language:" + language);
+            System.exit(1);
+        }
+        if (recFiles.size() == 0) {
+            System.out.println("No record files specified. Exiting.");
+            System.exit(1);
+        }
+        for (int i=0; i<recFiles.size(); i++) {
+            curFileName = (String) recFiles.get(i);
+            File file = new File(curDir, curFileName);
+            try {
+                FileReader reader = new FileReader(file);
+                Rcc parser = new Rcc(reader);
+                try {
+                    recTab = new Hashtable();
+                    curFile = parser.Input();
+                    System.out.println((String) recFiles.get(i) +
+                        " Parsed Successfully");
+                } catch (ParseException e) {
+                    System.out.println(e.toString());
+                    System.exit(1);
+                }
+                try {
+                    reader.close();
+                } catch (IOException e) {
+                }
+            } catch (FileNotFoundException e) {
+                System.out.println("File " + (String) recFiles.get(i) +
+                    " Not found.");
+                System.exit(1);
+            }
+            try {
+                curFile.genCode(language);
+            } catch (IOException e) {
+                System.out.println(e.toString());
+                System.exit(1);
+            }
+        }
+    }
+
+  final public JFile Input() throws ParseException {
+    ArrayList ilist = new ArrayList();
+    ArrayList rlist = new ArrayList();
+    JFile i;
+    ArrayList l;
+    label_1:
+    while (true) {
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case INCLUDE_TKN:
+        i = Include();
+          ilist.add(i);
+        break;
+      case MODULE_TKN:
+        l = Module();
+          rlist.addAll(l);
+        break;
+      default:
+        jj_la1[0] = jj_gen;
+        jj_consume_token(-1);
+        throw new ParseException();
+      }
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case MODULE_TKN:
+      case INCLUDE_TKN:
+        ;
+        break;
+      default:
+        jj_la1[1] = jj_gen;
+        break label_1;
+      }
+    }
+    jj_consume_token(0);
+      {if (true) return new JFile(curFileName, ilist, rlist);}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public JFile Include() throws ParseException {
+    String fname;
+    Token t;
+    jj_consume_token(INCLUDE_TKN);
+    t = jj_consume_token(CSTRING_TKN);
+        JFile ret = null;
+        fname = t.image.replaceAll("^\"", "").replaceAll("\"$","");
+        File file = new File(curDir, fname);
+        String tmpDir = curDir;
+        String tmpFile = curFileName;
+        curDir = file.getParent();
+        curFileName = file.getName();
+        try {
+            FileReader reader = new FileReader(file);
+            Rcc parser = new Rcc(reader);
+            try {
+                ret = parser.Input();
+                System.out.println(fname + " Parsed Successfully");
+            } catch (ParseException e) {
+                System.out.println(e.toString());
+                System.exit(1);
+            }
+            try {
+                reader.close();
+            } catch (IOException e) {
+            }
+        } catch (FileNotFoundException e) {
+            System.out.println("File " + fname +
+                " Not found.");
+            System.exit(1);
+        }
+        curDir = tmpDir;
+        curFileName = tmpFile;
+        {if (true) return ret;}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public ArrayList Module() throws ParseException {
+    String mName;
+    ArrayList rlist;
+    jj_consume_token(MODULE_TKN);
+    mName = ModuleName();
+      curModuleName = mName;
+    jj_consume_token(LBRACE_TKN);
+    rlist = RecordList();
+    jj_consume_token(RBRACE_TKN);
+      {if (true) return rlist;}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public String ModuleName() throws ParseException {
+    String name = "";
+    Token t;
+    t = jj_consume_token(IDENT_TKN);
+      name += t.image;
+    label_2:
+    while (true) {
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case DOT_TKN:
+        ;
+        break;
+      default:
+        jj_la1[2] = jj_gen;
+        break label_2;
+      }
+      jj_consume_token(DOT_TKN);
+      t = jj_consume_token(IDENT_TKN);
+          name += "." + t.image;
+    }
+      {if (true) return name;}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public ArrayList RecordList() throws ParseException {
+    ArrayList rlist = new ArrayList();
+    JRecord r;
+    label_3:
+    while (true) {
+      r = Record();
+          rlist.add(r);
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case RECORD_TKN:
+        ;
+        break;
+      default:
+        jj_la1[3] = jj_gen;
+        break label_3;
+      }
+    }
+      {if (true) return rlist;}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public JRecord Record() throws ParseException {
+    String rname;
+    ArrayList flist = new ArrayList();
+    Token t;
+    JField f;
+    jj_consume_token(RECORD_TKN);
+    t = jj_consume_token(IDENT_TKN);
+      rname = t.image;
+    jj_consume_token(LBRACE_TKN);
+    label_4:
+    while (true) {
+      f = Field();
+          flist.add(f);
+      jj_consume_token(SEMICOLON_TKN);
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case BYTE_TKN:
+      case BOOLEAN_TKN:
+      case INT_TKN:
+      case LONG_TKN:
+      case FLOAT_TKN:
+      case DOUBLE_TKN:
+      case USTRING_TKN:
+      case BUFFER_TKN:
+      case VECTOR_TKN:
+      case MAP_TKN:
+      case IDENT_TKN:
+        ;
+        break;
+      default:
+        jj_la1[4] = jj_gen;
+        break label_4;
+      }
+    }
+    jj_consume_token(RBRACE_TKN);
+        String fqn = curModuleName + "." + rname;
+        JRecord r = new JRecord(fqn, flist);
+        recTab.put(fqn, r);
+        {if (true) return r;}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public JField Field() throws ParseException {
+    JType jt;
+    Token t;
+    jt = Type();
+    t = jj_consume_token(IDENT_TKN);
+      {if (true) return new JField(jt, t.image);}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public JType Type() throws ParseException {
+    JType jt;
+    Token t;
+    String rname;
+    switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+    case MAP_TKN:
+      jt = Map();
+      {if (true) return jt;}
+      break;
+    case VECTOR_TKN:
+      jt = Vector();
+      {if (true) return jt;}
+      break;
+    case BYTE_TKN:
+      jj_consume_token(BYTE_TKN);
+      {if (true) return new JByte();}
+      break;
+    case BOOLEAN_TKN:
+      jj_consume_token(BOOLEAN_TKN);
+      {if (true) return new JBoolean();}
+      break;
+    case INT_TKN:
+      jj_consume_token(INT_TKN);
+      {if (true) return new JInt();}
+      break;
+    case LONG_TKN:
+      jj_consume_token(LONG_TKN);
+      {if (true) return new JLong();}
+      break;
+    case FLOAT_TKN:
+      jj_consume_token(FLOAT_TKN);
+      {if (true) return new JFloat();}
+      break;
+    case DOUBLE_TKN:
+      jj_consume_token(DOUBLE_TKN);
+      {if (true) return new JDouble();}
+      break;
+    case USTRING_TKN:
+      jj_consume_token(USTRING_TKN);
+      {if (true) return new JString();}
+      break;
+    case BUFFER_TKN:
+      jj_consume_token(BUFFER_TKN);
+      {if (true) return new JBuffer();}
+      break;
+    case IDENT_TKN:
+      rname = ModuleName();
+        if (rname.indexOf('.', 0) < 0) {
+            rname = curModuleName + "." + rname;
+        }
+        JRecord r = (JRecord) recTab.get(rname);
+        if (r == null) {
+            System.out.println("Type " + rname + " not known. Exiting.");
+            System.exit(1);
+        }
+        {if (true) return r;}
+      break;
+    default:
+      jj_la1[5] = jj_gen;
+      jj_consume_token(-1);
+      throw new ParseException();
+    }
+    throw new Error("Missing return statement in function");
+  }
+
+  final public JMap Map() throws ParseException {
+    JType jt1;
+    JType jt2;
+    jj_consume_token(MAP_TKN);
+    jj_consume_token(LT_TKN);
+    jt1 = Type();
+    jj_consume_token(COMMA_TKN);
+    jt2 = Type();
+    jj_consume_token(GT_TKN);
+      {if (true) return new JMap(jt1, jt2);}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public JVector Vector() throws ParseException {
+    JType jt;
+    jj_consume_token(VECTOR_TKN);
+    jj_consume_token(LT_TKN);
+    jt = Type();
+    jj_consume_token(GT_TKN);
+      {if (true) return new JVector(jt);}
+    throw new Error("Missing return statement in function");
+  }
+
+  public RccTokenManager token_source;
+  SimpleCharStream jj_input_stream;
+  public Token token, jj_nt;
+  private int jj_ntk;
+  private int jj_gen;
+  final private int[] jj_la1 = new int[6];
+  static private int[] jj_la1_0;
+  static private int[] jj_la1_1;
+  static {
+      jj_la1_0();
+      jj_la1_1();
+   }
+   private static void jj_la1_0() {
+      jj_la1_0 = new int[] {0x2800,0x2800,0x40000000,0x1000,0xffc000,0xffc000,};
+   }
+   private static void jj_la1_1() {
+      jj_la1_1 = new int[] {0x0,0x0,0x0,0x0,0x1,0x1,};
+   }
+
+  public Rcc(java.io.InputStream stream) {
+     this(stream, null);
+  }
+  public Rcc(java.io.InputStream stream, String encoding) {
+    try { jj_input_stream = new SimpleCharStream(stream, encoding, 1, 1); } catch(java.io.UnsupportedEncodingException e) { throw new RuntimeException(e); }
+    token_source = new RccTokenManager(jj_input_stream);
+    token = new Token();
+    jj_ntk = -1;
+    jj_gen = 0;
+    for (int i = 0; i < 6; i++) jj_la1[i] = -1;
+  }
+
+  public void ReInit(java.io.InputStream stream) {
+     ReInit(stream, null);
+  }
+  public void ReInit(java.io.InputStream stream, String encoding) {
+    try { jj_input_stream.ReInit(stream, encoding, 1, 1); } catch(java.io.UnsupportedEncodingException e) { throw new RuntimeException(e); }
+    token_source.ReInit(jj_input_stream);
+    token = new Token();
+    jj_ntk = -1;
+    jj_gen = 0;
+    for (int i = 0; i < 6; i++) jj_la1[i] = -1;
+  }
+
+  public Rcc(java.io.Reader stream) {
+    jj_input_stream = new SimpleCharStream(stream, 1, 1);
+    token_source = new RccTokenManager(jj_input_stream);
+    token = new Token();
+    jj_ntk = -1;
+    jj_gen = 0;
+    for (int i = 0; i < 6; i++) jj_la1[i] = -1;
+  }
+
+  public void ReInit(java.io.Reader stream) {
+    jj_input_stream.ReInit(stream, 1, 1);
+    token_source.ReInit(jj_input_stream);
+    token = new Token();
+    jj_ntk = -1;
+    jj_gen = 0;
+    for (int i = 0; i < 6; i++) jj_la1[i] = -1;
+  }
+
+  public Rcc(RccTokenManager tm) {
+    token_source = tm;
+    token = new Token();
+    jj_ntk = -1;
+    jj_gen = 0;
+    for (int i = 0; i < 6; i++) jj_la1[i] = -1;
+  }
+
+  public void ReInit(RccTokenManager tm) {
+    token_source = tm;
+    token = new Token();
+    jj_ntk = -1;
+    jj_gen = 0;
+    for (int i = 0; i < 6; i++) jj_la1[i] = -1;
+  }
+
+  final private Token jj_consume_token(int kind) throws ParseException {
+    Token oldToken;
+    if ((oldToken = token).next != null) token = token.next;
+    else token = token.next = token_source.getNextToken();
+    jj_ntk = -1;
+    if (token.kind == kind) {
+      jj_gen++;
+      return token;
+    }
+    token = oldToken;
+    jj_kind = kind;
+    throw generateParseException();
+  }
+
+  final public Token getNextToken() {
+    if (token.next != null) token = token.next;
+    else token = token.next = token_source.getNextToken();
+    jj_ntk = -1;
+    jj_gen++;
+    return token;
+  }
+
+  final public Token getToken(int index) {
+    Token t = token;
+    for (int i = 0; i < index; i++) {
+      if (t.next != null) t = t.next;
+      else t = t.next = token_source.getNextToken();
+    }
+    return t;
+  }
+
+  final private int jj_ntk() {
+    if ((jj_nt=token.next) == null)
+      return (jj_ntk = (token.next=token_source.getNextToken()).kind);
+    else
+      return (jj_ntk = jj_nt.kind);
+  }
+
+  private java.util.Vector jj_expentries = new java.util.Vector();
+  private int[] jj_expentry;
+  private int jj_kind = -1;
+
+  public ParseException generateParseException() {
+    jj_expentries.removeAllElements();
+    boolean[] la1tokens = new boolean[33];
+    for (int i = 0; i < 33; i++) {
+      la1tokens[i] = false;
+    }
+    if (jj_kind >= 0) {
+      la1tokens[jj_kind] = true;
+      jj_kind = -1;
+    }
+    for (int i = 0; i < 6; i++) {
+      if (jj_la1[i] == jj_gen) {
+        for (int j = 0; j < 32; j++) {
+          if ((jj_la1_0[i] & (1<<j)) != 0) {
+            la1tokens[j] = true;
+          }
+          if ((jj_la1_1[i] & (1<<j)) != 0) {
+            la1tokens[32+j] = true;
+          }
+        }
+      }
+    }
+    for (int i = 0; i < 33; i++) {
+      if (la1tokens[i]) {
+        jj_expentry = new int[1];
+        jj_expentry[0] = i;
+        jj_expentries.addElement(jj_expentry);
+      }
+    }
+    int[][] exptokseq = new int[jj_expentries.size()][];
+    for (int i = 0; i < jj_expentries.size(); i++) {
+      exptokseq[i] = (int[])jj_expentries.elementAt(i);
+    }
+    return new ParseException(token, exptokseq, tokenImage);
+  }
+
+  final public void enable_tracing() {
+  }
+
+  final public void disable_tracing() {
+  }
+
+}

+ 86 - 0
src/java/org/apache/hadoop/record/compiler/generated/RccConstants.java

@@ -0,0 +1,86 @@
+/* Generated By:JavaCC: Do not edit this line. RccConstants.java */
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler.generated;
+
+public interface RccConstants {
+
+  int EOF = 0;
+  int MODULE_TKN = 11;
+  int RECORD_TKN = 12;
+  int INCLUDE_TKN = 13;
+  int BYTE_TKN = 14;
+  int BOOLEAN_TKN = 15;
+  int INT_TKN = 16;
+  int LONG_TKN = 17;
+  int FLOAT_TKN = 18;
+  int DOUBLE_TKN = 19;
+  int USTRING_TKN = 20;
+  int BUFFER_TKN = 21;
+  int VECTOR_TKN = 22;
+  int MAP_TKN = 23;
+  int LBRACE_TKN = 24;
+  int RBRACE_TKN = 25;
+  int LT_TKN = 26;
+  int GT_TKN = 27;
+  int SEMICOLON_TKN = 28;
+  int COMMA_TKN = 29;
+  int DOT_TKN = 30;
+  int CSTRING_TKN = 31;
+  int IDENT_TKN = 32;
+
+  int DEFAULT = 0;
+  int WithinOneLineComment = 1;
+  int WithinMultiLineComment = 2;
+
+  String[] tokenImage = {
+    "<EOF>",
+    "\" \"",
+    "\"\\t\"",
+    "\"\\n\"",
+    "\"\\r\"",
+    "\"//\"",
+    "<token of kind 6>",
+    "<token of kind 7>",
+    "\"/*\"",
+    "\"*/\"",
+    "<token of kind 10>",
+    "\"module\"",
+    "\"class\"",
+    "\"include\"",
+    "\"byte\"",
+    "\"boolean\"",
+    "\"int\"",
+    "\"long\"",
+    "\"float\"",
+    "\"double\"",
+    "\"ustring\"",
+    "\"buffer\"",
+    "\"vector\"",
+    "\"map\"",
+    "\"{\"",
+    "\"}\"",
+    "\"<\"",
+    "\">\"",
+    "\";\"",
+    "\",\"",
+    "\".\"",
+    "<CSTRING_TKN>",
+    "<IDENT_TKN>",
+  };
+
+}

+ 841 - 0
src/java/org/apache/hadoop/record/compiler/generated/RccTokenManager.java

@@ -0,0 +1,841 @@
+/* Generated By:JavaCC: Do not edit this line. RccTokenManager.java */
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler.generated;
+import org.apache.hadoop.record.compiler.*;
+import java.util.ArrayList;
+import java.util.Hashtable;
+import java.util.Iterator;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+public class RccTokenManager implements RccConstants
+{
+  public  java.io.PrintStream debugStream = System.out;
+  public  void setDebugStream(java.io.PrintStream ds) { debugStream = ds; }
+private final int jjMoveStringLiteralDfa0_1()
+{
+   return jjMoveNfa_1(1, 0);
+}
+private final void jjCheckNAdd(int state)
+{
+   if (jjrounds[state] != jjround)
+   {
+      jjstateSet[jjnewStateCnt++] = state;
+      jjrounds[state] = jjround;
+   }
+}
+private final void jjAddStates(int start, int end)
+{
+   do {
+      jjstateSet[jjnewStateCnt++] = jjnextStates[start];
+   } while (start++ != end);
+}
+private final void jjCheckNAddTwoStates(int state1, int state2)
+{
+   jjCheckNAdd(state1);
+   jjCheckNAdd(state2);
+}
+private final void jjCheckNAddStates(int start, int end)
+{
+   do {
+      jjCheckNAdd(jjnextStates[start]);
+   } while (start++ != end);
+}
+private final void jjCheckNAddStates(int start)
+{
+   jjCheckNAdd(jjnextStates[start]);
+   jjCheckNAdd(jjnextStates[start + 1]);
+}
+private final int jjMoveNfa_1(int startState, int curPos)
+{
+   int[] nextStates;
+   int startsAt = 0;
+   jjnewStateCnt = 5;
+   int i = 1;
+   jjstateSet[0] = startState;
+   int j, kind = 0x7fffffff;
+   for (;;)
+   {
+      if (++jjround == 0x7fffffff)
+         ReInitRounds();
+      if (curChar < 64)
+      {
+         long l = 1L << curChar;
+         MatchLoop: do
+         {
+            switch(jjstateSet[--i])
+            {
+               case 1:
+                  if ((0x2400L & l) != 0L)
+                  {
+                     if (kind > 6)
+                        kind = 6;
+                  }
+                  else if (curChar == 47)
+                     jjstateSet[jjnewStateCnt++] = 0;
+                  if (curChar == 13)
+                     jjstateSet[jjnewStateCnt++] = 3;
+                  break;
+               case 0:
+                  if (curChar == 47 && kind > 6)
+                     kind = 6;
+                  break;
+               case 2:
+                  if ((0x2400L & l) != 0L && kind > 6)
+                     kind = 6;
+                  break;
+               case 3:
+                  if (curChar == 10 && kind > 6)
+                     kind = 6;
+                  break;
+               case 4:
+                  if (curChar == 13)
+                     jjstateSet[jjnewStateCnt++] = 3;
+                  break;
+               default : break;
+            }
+         } while(i != startsAt);
+      }
+      else if (curChar < 128)
+      {
+         long l = 1L << (curChar & 077);
+         MatchLoop: do
+         {
+            switch(jjstateSet[--i])
+            {
+               default : break;
+            }
+         } while(i != startsAt);
+      }
+      else
+      {
+         int i2 = (curChar & 0xff) >> 6;
+         long l2 = 1L << (curChar & 077);
+         MatchLoop: do
+         {
+            switch(jjstateSet[--i])
+            {
+               default : break;
+            }
+         } while(i != startsAt);
+      }
+      if (kind != 0x7fffffff)
+      {
+         jjmatchedKind = kind;
+         jjmatchedPos = curPos;
+         kind = 0x7fffffff;
+      }
+      ++curPos;
+      if ((i = jjnewStateCnt) == (startsAt = 5 - (jjnewStateCnt = startsAt)))
+         return curPos;
+      try { curChar = input_stream.readChar(); }
+      catch(java.io.IOException e) { return curPos; }
+   }
+}
+private final int jjStopStringLiteralDfa_0(int pos, long active0)
+{
+   switch (pos)
+   {
+      case 0:
+         if ((active0 & 0xfff800L) != 0L)
+         {
+            jjmatchedKind = 32;
+            return 4;
+         }
+         return -1;
+      case 1:
+         if ((active0 & 0xfff800L) != 0L)
+         {
+            jjmatchedKind = 32;
+            jjmatchedPos = 1;
+            return 4;
+         }
+         return -1;
+      case 2:
+         if ((active0 & 0x7ef800L) != 0L)
+         {
+            jjmatchedKind = 32;
+            jjmatchedPos = 2;
+            return 4;
+         }
+         if ((active0 & 0x810000L) != 0L)
+            return 4;
+         return -1;
+      case 3:
+         if ((active0 & 0x24000L) != 0L)
+            return 4;
+         if ((active0 & 0x7cb800L) != 0L)
+         {
+            jjmatchedKind = 32;
+            jjmatchedPos = 3;
+            return 4;
+         }
+         return -1;
+      case 4:
+         if ((active0 & 0x41000L) != 0L)
+            return 4;
+         if ((active0 & 0x78a800L) != 0L)
+         {
+            jjmatchedKind = 32;
+            jjmatchedPos = 4;
+            return 4;
+         }
+         return -1;
+      case 5:
+         if ((active0 & 0x680800L) != 0L)
+            return 4;
+         if ((active0 & 0x10a000L) != 0L)
+         {
+            jjmatchedKind = 32;
+            jjmatchedPos = 5;
+            return 4;
+         }
+         return -1;
+      default :
+         return -1;
+   }
+}
+private final int jjStartNfa_0(int pos, long active0)
+{
+   return jjMoveNfa_0(jjStopStringLiteralDfa_0(pos, active0), pos + 1);
+}
+private final int jjStopAtPos(int pos, int kind)
+{
+   jjmatchedKind = kind;
+   jjmatchedPos = pos;
+   return pos + 1;
+}
+private final int jjStartNfaWithStates_0(int pos, int kind, int state)
+{
+   jjmatchedKind = kind;
+   jjmatchedPos = pos;
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) { return pos + 1; }
+   return jjMoveNfa_0(state, pos + 1);
+}
+private final int jjMoveStringLiteralDfa0_0()
+{
+   switch(curChar)
+   {
+      case 44:
+         return jjStopAtPos(0, 29);
+      case 46:
+         return jjStopAtPos(0, 30);
+      case 47:
+         return jjMoveStringLiteralDfa1_0(0x120L);
+      case 59:
+         return jjStopAtPos(0, 28);
+      case 60:
+         return jjStopAtPos(0, 26);
+      case 62:
+         return jjStopAtPos(0, 27);
+      case 98:
+         return jjMoveStringLiteralDfa1_0(0x20c000L);
+      case 99:
+         return jjMoveStringLiteralDfa1_0(0x1000L);
+      case 100:
+         return jjMoveStringLiteralDfa1_0(0x80000L);
+      case 102:
+         return jjMoveStringLiteralDfa1_0(0x40000L);
+      case 105:
+         return jjMoveStringLiteralDfa1_0(0x12000L);
+      case 108:
+         return jjMoveStringLiteralDfa1_0(0x20000L);
+      case 109:
+         return jjMoveStringLiteralDfa1_0(0x800800L);
+      case 117:
+         return jjMoveStringLiteralDfa1_0(0x100000L);
+      case 118:
+         return jjMoveStringLiteralDfa1_0(0x400000L);
+      case 123:
+         return jjStopAtPos(0, 24);
+      case 125:
+         return jjStopAtPos(0, 25);
+      default :
+         return jjMoveNfa_0(0, 0);
+   }
+}
+private final int jjMoveStringLiteralDfa1_0(long active0)
+{
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) {
+      jjStopStringLiteralDfa_0(0, active0);
+      return 1;
+   }
+   switch(curChar)
+   {
+      case 42:
+         if ((active0 & 0x100L) != 0L)
+            return jjStopAtPos(1, 8);
+         break;
+      case 47:
+         if ((active0 & 0x20L) != 0L)
+            return jjStopAtPos(1, 5);
+         break;
+      case 97:
+         return jjMoveStringLiteralDfa2_0(active0, 0x800000L);
+      case 101:
+         return jjMoveStringLiteralDfa2_0(active0, 0x400000L);
+      case 108:
+         return jjMoveStringLiteralDfa2_0(active0, 0x41000L);
+      case 110:
+         return jjMoveStringLiteralDfa2_0(active0, 0x12000L);
+      case 111:
+         return jjMoveStringLiteralDfa2_0(active0, 0xa8800L);
+      case 115:
+         return jjMoveStringLiteralDfa2_0(active0, 0x100000L);
+      case 117:
+         return jjMoveStringLiteralDfa2_0(active0, 0x200000L);
+      case 121:
+         return jjMoveStringLiteralDfa2_0(active0, 0x4000L);
+      default :
+         break;
+   }
+   return jjStartNfa_0(0, active0);
+}
+private final int jjMoveStringLiteralDfa2_0(long old0, long active0)
+{
+   if (((active0 &= old0)) == 0L)
+      return jjStartNfa_0(0, old0); 
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) {
+      jjStopStringLiteralDfa_0(1, active0);
+      return 2;
+   }
+   switch(curChar)
+   {
+      case 97:
+         return jjMoveStringLiteralDfa3_0(active0, 0x1000L);
+      case 99:
+         return jjMoveStringLiteralDfa3_0(active0, 0x402000L);
+      case 100:
+         return jjMoveStringLiteralDfa3_0(active0, 0x800L);
+      case 102:
+         return jjMoveStringLiteralDfa3_0(active0, 0x200000L);
+      case 110:
+         return jjMoveStringLiteralDfa3_0(active0, 0x20000L);
+      case 111:
+         return jjMoveStringLiteralDfa3_0(active0, 0x48000L);
+      case 112:
+         if ((active0 & 0x800000L) != 0L)
+            return jjStartNfaWithStates_0(2, 23, 4);
+         break;
+      case 116:
+         if ((active0 & 0x10000L) != 0L)
+            return jjStartNfaWithStates_0(2, 16, 4);
+         return jjMoveStringLiteralDfa3_0(active0, 0x104000L);
+      case 117:
+         return jjMoveStringLiteralDfa3_0(active0, 0x80000L);
+      default :
+         break;
+   }
+   return jjStartNfa_0(1, active0);
+}
+private final int jjMoveStringLiteralDfa3_0(long old0, long active0)
+{
+   if (((active0 &= old0)) == 0L)
+      return jjStartNfa_0(1, old0); 
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) {
+      jjStopStringLiteralDfa_0(2, active0);
+      return 3;
+   }
+   switch(curChar)
+   {
+      case 97:
+         return jjMoveStringLiteralDfa4_0(active0, 0x40000L);
+      case 98:
+         return jjMoveStringLiteralDfa4_0(active0, 0x80000L);
+      case 101:
+         if ((active0 & 0x4000L) != 0L)
+            return jjStartNfaWithStates_0(3, 14, 4);
+         break;
+      case 102:
+         return jjMoveStringLiteralDfa4_0(active0, 0x200000L);
+      case 103:
+         if ((active0 & 0x20000L) != 0L)
+            return jjStartNfaWithStates_0(3, 17, 4);
+         break;
+      case 108:
+         return jjMoveStringLiteralDfa4_0(active0, 0xa000L);
+      case 114:
+         return jjMoveStringLiteralDfa4_0(active0, 0x100000L);
+      case 115:
+         return jjMoveStringLiteralDfa4_0(active0, 0x1000L);
+      case 116:
+         return jjMoveStringLiteralDfa4_0(active0, 0x400000L);
+      case 117:
+         return jjMoveStringLiteralDfa4_0(active0, 0x800L);
+      default :
+         break;
+   }
+   return jjStartNfa_0(2, active0);
+}
+private final int jjMoveStringLiteralDfa4_0(long old0, long active0)
+{
+   if (((active0 &= old0)) == 0L)
+      return jjStartNfa_0(2, old0); 
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) {
+      jjStopStringLiteralDfa_0(3, active0);
+      return 4;
+   }
+   switch(curChar)
+   {
+      case 101:
+         return jjMoveStringLiteralDfa5_0(active0, 0x208000L);
+      case 105:
+         return jjMoveStringLiteralDfa5_0(active0, 0x100000L);
+      case 108:
+         return jjMoveStringLiteralDfa5_0(active0, 0x80800L);
+      case 111:
+         return jjMoveStringLiteralDfa5_0(active0, 0x400000L);
+      case 115:
+         if ((active0 & 0x1000L) != 0L)
+            return jjStartNfaWithStates_0(4, 12, 4);
+         break;
+      case 116:
+         if ((active0 & 0x40000L) != 0L)
+            return jjStartNfaWithStates_0(4, 18, 4);
+         break;
+      case 117:
+         return jjMoveStringLiteralDfa5_0(active0, 0x2000L);
+      default :
+         break;
+   }
+   return jjStartNfa_0(3, active0);
+}
+private final int jjMoveStringLiteralDfa5_0(long old0, long active0)
+{
+   if (((active0 &= old0)) == 0L)
+      return jjStartNfa_0(3, old0); 
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) {
+      jjStopStringLiteralDfa_0(4, active0);
+      return 5;
+   }
+   switch(curChar)
+   {
+      case 97:
+         return jjMoveStringLiteralDfa6_0(active0, 0x8000L);
+      case 100:
+         return jjMoveStringLiteralDfa6_0(active0, 0x2000L);
+      case 101:
+         if ((active0 & 0x800L) != 0L)
+            return jjStartNfaWithStates_0(5, 11, 4);
+         else if ((active0 & 0x80000L) != 0L)
+            return jjStartNfaWithStates_0(5, 19, 4);
+         break;
+      case 110:
+         return jjMoveStringLiteralDfa6_0(active0, 0x100000L);
+      case 114:
+         if ((active0 & 0x200000L) != 0L)
+            return jjStartNfaWithStates_0(5, 21, 4);
+         else if ((active0 & 0x400000L) != 0L)
+            return jjStartNfaWithStates_0(5, 22, 4);
+         break;
+      default :
+         break;
+   }
+   return jjStartNfa_0(4, active0);
+}
+private final int jjMoveStringLiteralDfa6_0(long old0, long active0)
+{
+   if (((active0 &= old0)) == 0L)
+      return jjStartNfa_0(4, old0); 
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) {
+      jjStopStringLiteralDfa_0(5, active0);
+      return 6;
+   }
+   switch(curChar)
+   {
+      case 101:
+         if ((active0 & 0x2000L) != 0L)
+            return jjStartNfaWithStates_0(6, 13, 4);
+         break;
+      case 103:
+         if ((active0 & 0x100000L) != 0L)
+            return jjStartNfaWithStates_0(6, 20, 4);
+         break;
+      case 110:
+         if ((active0 & 0x8000L) != 0L)
+            return jjStartNfaWithStates_0(6, 15, 4);
+         break;
+      default :
+         break;
+   }
+   return jjStartNfa_0(5, active0);
+}
+static final long[] jjbitVec0 = {
+   0x0L, 0x0L, 0xffffffffffffffffL, 0xffffffffffffffffL
+};
+private final int jjMoveNfa_0(int startState, int curPos)
+{
+   int[] nextStates;
+   int startsAt = 0;
+   jjnewStateCnt = 5;
+   int i = 1;
+   jjstateSet[0] = startState;
+   int j, kind = 0x7fffffff;
+   for (;;)
+   {
+      if (++jjround == 0x7fffffff)
+         ReInitRounds();
+      if (curChar < 64)
+      {
+         long l = 1L << curChar;
+         MatchLoop: do
+         {
+            switch(jjstateSet[--i])
+            {
+               case 0:
+                  if (curChar == 34)
+                     jjCheckNAdd(1);
+                  break;
+               case 1:
+                  if ((0xfffffffbffffffffL & l) != 0L)
+                     jjCheckNAddTwoStates(1, 2);
+                  break;
+               case 2:
+                  if (curChar == 34 && kind > 31)
+                     kind = 31;
+                  break;
+               case 4:
+                  if ((0x3ff000000000000L & l) == 0L)
+                     break;
+                  if (kind > 32)
+                     kind = 32;
+                  jjstateSet[jjnewStateCnt++] = 4;
+                  break;
+               default : break;
+            }
+         } while(i != startsAt);
+      }
+      else if (curChar < 128)
+      {
+         long l = 1L << (curChar & 077);
+         MatchLoop: do
+         {
+            switch(jjstateSet[--i])
+            {
+               case 0:
+                  if ((0x7fffffe07fffffeL & l) == 0L)
+                     break;
+                  if (kind > 32)
+                     kind = 32;
+                  jjCheckNAdd(4);
+                  break;
+               case 1:
+                  jjAddStates(0, 1);
+                  break;
+               case 4:
+                  if ((0x7fffffe87fffffeL & l) == 0L)
+                     break;
+                  if (kind > 32)
+                     kind = 32;
+                  jjCheckNAdd(4);
+                  break;
+               default : break;
+            }
+         } while(i != startsAt);
+      }
+      else
+      {
+         int i2 = (curChar & 0xff) >> 6;
+         long l2 = 1L << (curChar & 077);
+         MatchLoop: do
+         {
+            switch(jjstateSet[--i])
+            {
+               case 1:
+                  if ((jjbitVec0[i2] & l2) != 0L)
+                     jjAddStates(0, 1);
+                  break;
+               default : break;
+            }
+         } while(i != startsAt);
+      }
+      if (kind != 0x7fffffff)
+      {
+         jjmatchedKind = kind;
+         jjmatchedPos = curPos;
+         kind = 0x7fffffff;
+      }
+      ++curPos;
+      if ((i = jjnewStateCnt) == (startsAt = 5 - (jjnewStateCnt = startsAt)))
+         return curPos;
+      try { curChar = input_stream.readChar(); }
+      catch(java.io.IOException e) { return curPos; }
+   }
+}
+private final int jjMoveStringLiteralDfa0_2()
+{
+   switch(curChar)
+   {
+      case 42:
+         return jjMoveStringLiteralDfa1_2(0x200L);
+      default :
+         return 1;
+   }
+}
+private final int jjMoveStringLiteralDfa1_2(long active0)
+{
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) {
+      return 1;
+   }
+   switch(curChar)
+   {
+      case 47:
+         if ((active0 & 0x200L) != 0L)
+            return jjStopAtPos(1, 9);
+         break;
+      default :
+         return 2;
+   }
+   return 2;
+}
+static final int[] jjnextStates = {
+   1, 2, 
+};
+public static final String[] jjstrLiteralImages = {
+"", null, null, null, null, null, null, null, null, null, null, 
+"\155\157\144\165\154\145", "\143\154\141\163\163", "\151\156\143\154\165\144\145", "\142\171\164\145", 
+"\142\157\157\154\145\141\156", "\151\156\164", "\154\157\156\147", "\146\154\157\141\164", 
+"\144\157\165\142\154\145", "\165\163\164\162\151\156\147", "\142\165\146\146\145\162", 
+"\166\145\143\164\157\162", "\155\141\160", "\173", "\175", "\74", "\76", "\73", "\54", "\56", null, null, };
+public static final String[] lexStateNames = {
+   "DEFAULT", 
+   "WithinOneLineComment", 
+   "WithinMultiLineComment", 
+};
+public static final int[] jjnewLexState = {
+   -1, -1, -1, -1, -1, 1, 0, -1, 2, 0, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 
+   -1, -1, -1, -1, -1, -1, -1, -1, 
+};
+static final long[] jjtoToken = {
+   0x1fffff801L, 
+};
+static final long[] jjtoSkip = {
+   0x37eL, 
+};
+static final long[] jjtoSpecial = {
+   0x360L, 
+};
+static final long[] jjtoMore = {
+   0x480L, 
+};
+protected SimpleCharStream input_stream;
+private final int[] jjrounds = new int[5];
+private final int[] jjstateSet = new int[10];
+StringBuffer image;
+int jjimageLen;
+int lengthOfMatch;
+protected char curChar;
+public RccTokenManager(SimpleCharStream stream){
+   if (SimpleCharStream.staticFlag)
+      throw new Error("ERROR: Cannot use a static CharStream class with a non-static lexical analyzer.");
+   input_stream = stream;
+}
+public RccTokenManager(SimpleCharStream stream, int lexState){
+   this(stream);
+   SwitchTo(lexState);
+}
+public void ReInit(SimpleCharStream stream)
+{
+   jjmatchedPos = jjnewStateCnt = 0;
+   curLexState = defaultLexState;
+   input_stream = stream;
+   ReInitRounds();
+}
+private final void ReInitRounds()
+{
+   int i;
+   jjround = 0x80000001;
+   for (i = 5; i-- > 0;)
+      jjrounds[i] = 0x80000000;
+}
+public void ReInit(SimpleCharStream stream, int lexState)
+{
+   ReInit(stream);
+   SwitchTo(lexState);
+}
+public void SwitchTo(int lexState)
+{
+   if (lexState >= 3 || lexState < 0)
+      throw new TokenMgrError("Error: Ignoring invalid lexical state : " + lexState + ". State unchanged.", TokenMgrError.INVALID_LEXICAL_STATE);
+   else
+      curLexState = lexState;
+}
+
+protected Token jjFillToken()
+{
+   Token t = Token.newToken(jjmatchedKind);
+   t.kind = jjmatchedKind;
+   String im = jjstrLiteralImages[jjmatchedKind];
+   t.image = (im == null) ? input_stream.GetImage() : im;
+   t.beginLine = input_stream.getBeginLine();
+   t.beginColumn = input_stream.getBeginColumn();
+   t.endLine = input_stream.getEndLine();
+   t.endColumn = input_stream.getEndColumn();
+   return t;
+}
+
+int curLexState = 0;
+int defaultLexState = 0;
+int jjnewStateCnt;
+int jjround;
+int jjmatchedPos;
+int jjmatchedKind;
+
+public Token getNextToken() 
+{
+  int kind;
+  Token specialToken = null;
+  Token matchedToken;
+  int curPos = 0;
+
+  EOFLoop :
+  for (;;)
+  {   
+   try   
+   {     
+      curChar = input_stream.BeginToken();
+   }     
+   catch(java.io.IOException e)
+   {        
+      jjmatchedKind = 0;
+      matchedToken = jjFillToken();
+      matchedToken.specialToken = specialToken;
+      return matchedToken;
+   }
+   image = null;
+   jjimageLen = 0;
+
+   for (;;)
+   {
+     switch(curLexState)
+     {
+       case 0:
+         try { input_stream.backup(0);
+            while (curChar <= 32 && (0x100002600L & (1L << curChar)) != 0L)
+               curChar = input_stream.BeginToken();
+         }
+         catch (java.io.IOException e1) { continue EOFLoop; }
+         jjmatchedKind = 0x7fffffff;
+         jjmatchedPos = 0;
+         curPos = jjMoveStringLiteralDfa0_0();
+         break;
+       case 1:
+         jjmatchedKind = 0x7fffffff;
+         jjmatchedPos = 0;
+         curPos = jjMoveStringLiteralDfa0_1();
+         if (jjmatchedPos == 0 && jjmatchedKind > 7)
+         {
+            jjmatchedKind = 7;
+         }
+         break;
+       case 2:
+         jjmatchedKind = 0x7fffffff;
+         jjmatchedPos = 0;
+         curPos = jjMoveStringLiteralDfa0_2();
+         if (jjmatchedPos == 0 && jjmatchedKind > 10)
+         {
+            jjmatchedKind = 10;
+         }
+         break;
+     }
+     if (jjmatchedKind != 0x7fffffff)
+     {
+        if (jjmatchedPos + 1 < curPos)
+           input_stream.backup(curPos - jjmatchedPos - 1);
+        if ((jjtoToken[jjmatchedKind >> 6] & (1L << (jjmatchedKind & 077))) != 0L)
+        {
+           matchedToken = jjFillToken();
+           matchedToken.specialToken = specialToken;
+       if (jjnewLexState[jjmatchedKind] != -1)
+         curLexState = jjnewLexState[jjmatchedKind];
+           return matchedToken;
+        }
+        else if ((jjtoSkip[jjmatchedKind >> 6] & (1L << (jjmatchedKind & 077))) != 0L)
+        {
+           if ((jjtoSpecial[jjmatchedKind >> 6] & (1L << (jjmatchedKind & 077))) != 0L)
+           {
+              matchedToken = jjFillToken();
+              if (specialToken == null)
+                 specialToken = matchedToken;
+              else
+              {
+                 matchedToken.specialToken = specialToken;
+                 specialToken = (specialToken.next = matchedToken);
+              }
+              SkipLexicalActions(matchedToken);
+           }
+           else 
+              SkipLexicalActions(null);
+         if (jjnewLexState[jjmatchedKind] != -1)
+           curLexState = jjnewLexState[jjmatchedKind];
+           continue EOFLoop;
+        }
+        jjimageLen += jjmatchedPos + 1;
+      if (jjnewLexState[jjmatchedKind] != -1)
+        curLexState = jjnewLexState[jjmatchedKind];
+        curPos = 0;
+        jjmatchedKind = 0x7fffffff;
+        try {
+           curChar = input_stream.readChar();
+           continue;
+        }
+        catch (java.io.IOException e1) { }
+     }
+     int error_line = input_stream.getEndLine();
+     int error_column = input_stream.getEndColumn();
+     String error_after = null;
+     boolean EOFSeen = false;
+     try { input_stream.readChar(); input_stream.backup(1); }
+     catch (java.io.IOException e1) {
+        EOFSeen = true;
+        error_after = curPos <= 1 ? "" : input_stream.GetImage();
+        if (curChar == '\n' || curChar == '\r') {
+           error_line++;
+           error_column = 0;
+        }
+        else
+           error_column++;
+     }
+     if (!EOFSeen) {
+        input_stream.backup(1);
+        error_after = curPos <= 1 ? "" : input_stream.GetImage();
+     }
+     throw new TokenMgrError(EOFSeen, curLexState, error_line, error_column, error_after, curChar, TokenMgrError.LEXICAL_ERROR);
+   }
+  }
+}
+
+void SkipLexicalActions(Token matchedToken)
+{
+   switch(jjmatchedKind)
+   {
+      default :
+         break;
+   }
+}
+}

+ 455 - 0
src/java/org/apache/hadoop/record/compiler/generated/SimpleCharStream.java

@@ -0,0 +1,455 @@
+/* Generated By:JavaCC: Do not edit this line. SimpleCharStream.java Version 4.0 */
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler.generated;
+
+/**
+ * An implementation of interface CharStream, where the stream is assumed to
+ * contain only ASCII characters (without unicode processing).
+ */
+
+public class SimpleCharStream
+{
+  public static final boolean staticFlag = false;
+  int bufsize;
+  int available;
+  int tokenBegin;
+  public int bufpos = -1;
+  protected int bufline[];
+  protected int bufcolumn[];
+
+  protected int column = 0;
+  protected int line = 1;
+
+  protected boolean prevCharIsCR = false;
+  protected boolean prevCharIsLF = false;
+
+  protected java.io.Reader inputStream;
+
+  protected char[] buffer;
+  protected int maxNextCharInd = 0;
+  protected int inBuf = 0;
+  protected int tabSize = 8;
+
+  protected void setTabSize(int i) { tabSize = i; }
+  protected int getTabSize(int i) { return tabSize; }
+
+
+  protected void ExpandBuff(boolean wrapAround)
+  {
+     char[] newbuffer = new char[bufsize + 2048];
+     int newbufline[] = new int[bufsize + 2048];
+     int newbufcolumn[] = new int[bufsize + 2048];
+
+     try
+     {
+        if (wrapAround)
+        {
+           System.arraycopy(buffer, tokenBegin, newbuffer, 0, bufsize - tokenBegin);
+           System.arraycopy(buffer, 0, newbuffer,
+                                             bufsize - tokenBegin, bufpos);
+           buffer = newbuffer;
+
+           System.arraycopy(bufline, tokenBegin, newbufline, 0, bufsize - tokenBegin);
+           System.arraycopy(bufline, 0, newbufline, bufsize - tokenBegin, bufpos);
+           bufline = newbufline;
+
+           System.arraycopy(bufcolumn, tokenBegin, newbufcolumn, 0, bufsize - tokenBegin);
+           System.arraycopy(bufcolumn, 0, newbufcolumn, bufsize - tokenBegin, bufpos);
+           bufcolumn = newbufcolumn;
+
+           maxNextCharInd = (bufpos += (bufsize - tokenBegin));
+        }
+        else
+        {
+           System.arraycopy(buffer, tokenBegin, newbuffer, 0, bufsize - tokenBegin);
+           buffer = newbuffer;
+
+           System.arraycopy(bufline, tokenBegin, newbufline, 0, bufsize - tokenBegin);
+           bufline = newbufline;
+
+           System.arraycopy(bufcolumn, tokenBegin, newbufcolumn, 0, bufsize - tokenBegin);
+           bufcolumn = newbufcolumn;
+
+           maxNextCharInd = (bufpos -= tokenBegin);
+        }
+     }
+     catch (Throwable t)
+     {
+        throw new Error(t.getMessage());
+     }
+
+
+     bufsize += 2048;
+     available = bufsize;
+     tokenBegin = 0;
+  }
+
+  protected void FillBuff() throws java.io.IOException
+  {
+     if (maxNextCharInd == available)
+     {
+        if (available == bufsize)
+        {
+           if (tokenBegin > 2048)
+           {
+              bufpos = maxNextCharInd = 0;
+              available = tokenBegin;
+           }
+           else if (tokenBegin < 0)
+              bufpos = maxNextCharInd = 0;
+           else
+              ExpandBuff(false);
+        }
+        else if (available > tokenBegin)
+           available = bufsize;
+        else if ((tokenBegin - available) < 2048)
+           ExpandBuff(true);
+        else
+           available = tokenBegin;
+     }
+
+     int i;
+     try {
+        if ((i = inputStream.read(buffer, maxNextCharInd,
+                                    available - maxNextCharInd)) == -1)
+        {
+           inputStream.close();
+           throw new java.io.IOException();
+        }
+        else
+           maxNextCharInd += i;
+        return;
+     }
+     catch(java.io.IOException e) {
+        --bufpos;
+        backup(0);
+        if (tokenBegin == -1)
+           tokenBegin = bufpos;
+        throw e;
+     }
+  }
+
+  public char BeginToken() throws java.io.IOException
+  {
+     tokenBegin = -1;
+     char c = readChar();
+     tokenBegin = bufpos;
+
+     return c;
+  }
+
+  protected void UpdateLineColumn(char c)
+  {
+     column++;
+
+     if (prevCharIsLF)
+     {
+        prevCharIsLF = false;
+        line += (column = 1);
+     }
+     else if (prevCharIsCR)
+     {
+        prevCharIsCR = false;
+        if (c == '\n')
+        {
+           prevCharIsLF = true;
+        }
+        else
+           line += (column = 1);
+     }
+
+     switch (c)
+     {
+        case '\r' :
+           prevCharIsCR = true;
+           break;
+        case '\n' :
+           prevCharIsLF = true;
+           break;
+        case '\t' :
+           column--;
+           column += (tabSize - (column % tabSize));
+           break;
+        default :
+           break;
+     }
+
+     bufline[bufpos] = line;
+     bufcolumn[bufpos] = column;
+  }
+
+  public char readChar() throws java.io.IOException
+  {
+     if (inBuf > 0)
+     {
+        --inBuf;
+
+        if (++bufpos == bufsize)
+           bufpos = 0;
+
+        return buffer[bufpos];
+     }
+
+     if (++bufpos >= maxNextCharInd)
+        FillBuff();
+
+     char c = buffer[bufpos];
+
+     UpdateLineColumn(c);
+     return (c);
+  }
+
+  /**
+   * @deprecated 
+   * @see #getEndColumn
+   */
+
+  public int getColumn() {
+     return bufcolumn[bufpos];
+  }
+
+  /**
+   * @deprecated 
+   * @see #getEndLine
+   */
+
+  public int getLine() {
+     return bufline[bufpos];
+  }
+
+  public int getEndColumn() {
+     return bufcolumn[bufpos];
+  }
+
+  public int getEndLine() {
+     return bufline[bufpos];
+  }
+
+  public int getBeginColumn() {
+     return bufcolumn[tokenBegin];
+  }
+
+  public int getBeginLine() {
+     return bufline[tokenBegin];
+  }
+
+  public void backup(int amount) {
+
+    inBuf += amount;
+    if ((bufpos -= amount) < 0)
+       bufpos += bufsize;
+  }
+
+  public SimpleCharStream(java.io.Reader dstream, int startline,
+  int startcolumn, int buffersize)
+  {
+    inputStream = dstream;
+    line = startline;
+    column = startcolumn - 1;
+
+    available = bufsize = buffersize;
+    buffer = new char[buffersize];
+    bufline = new int[buffersize];
+    bufcolumn = new int[buffersize];
+  }
+
+  public SimpleCharStream(java.io.Reader dstream, int startline,
+                          int startcolumn)
+  {
+     this(dstream, startline, startcolumn, 4096);
+  }
+
+  public SimpleCharStream(java.io.Reader dstream)
+  {
+     this(dstream, 1, 1, 4096);
+  }
+  public void ReInit(java.io.Reader dstream, int startline,
+  int startcolumn, int buffersize)
+  {
+    inputStream = dstream;
+    line = startline;
+    column = startcolumn - 1;
+
+    if (buffer == null || buffersize != buffer.length)
+    {
+      available = bufsize = buffersize;
+      buffer = new char[buffersize];
+      bufline = new int[buffersize];
+      bufcolumn = new int[buffersize];
+    }
+    prevCharIsLF = prevCharIsCR = false;
+    tokenBegin = inBuf = maxNextCharInd = 0;
+    bufpos = -1;
+  }
+
+  public void ReInit(java.io.Reader dstream, int startline,
+                     int startcolumn)
+  {
+     ReInit(dstream, startline, startcolumn, 4096);
+  }
+
+  public void ReInit(java.io.Reader dstream)
+  {
+     ReInit(dstream, 1, 1, 4096);
+  }
+  public SimpleCharStream(java.io.InputStream dstream, String encoding, int startline,
+  int startcolumn, int buffersize) throws java.io.UnsupportedEncodingException
+  {
+     this(encoding == null ? new java.io.InputStreamReader(dstream) : new java.io.InputStreamReader(dstream, encoding), startline, startcolumn, buffersize);
+  }
+
+  public SimpleCharStream(java.io.InputStream dstream, int startline,
+  int startcolumn, int buffersize)
+  {
+     this(new java.io.InputStreamReader(dstream), startline, startcolumn, buffersize);
+  }
+
+  public SimpleCharStream(java.io.InputStream dstream, String encoding, int startline,
+                          int startcolumn) throws java.io.UnsupportedEncodingException
+  {
+     this(dstream, encoding, startline, startcolumn, 4096);
+  }
+
+  public SimpleCharStream(java.io.InputStream dstream, int startline,
+                          int startcolumn)
+  {
+     this(dstream, startline, startcolumn, 4096);
+  }
+
+  public SimpleCharStream(java.io.InputStream dstream, String encoding) throws java.io.UnsupportedEncodingException
+  {
+     this(dstream, encoding, 1, 1, 4096);
+  }
+
+  public SimpleCharStream(java.io.InputStream dstream)
+  {
+     this(dstream, 1, 1, 4096);
+  }
+
+  public void ReInit(java.io.InputStream dstream, String encoding, int startline,
+                          int startcolumn, int buffersize) throws java.io.UnsupportedEncodingException
+  {
+     ReInit(encoding == null ? new java.io.InputStreamReader(dstream) : new java.io.InputStreamReader(dstream, encoding), startline, startcolumn, buffersize);
+  }
+
+  public void ReInit(java.io.InputStream dstream, int startline,
+                          int startcolumn, int buffersize)
+  {
+     ReInit(new java.io.InputStreamReader(dstream), startline, startcolumn, buffersize);
+  }
+
+  public void ReInit(java.io.InputStream dstream, String encoding) throws java.io.UnsupportedEncodingException
+  {
+     ReInit(dstream, encoding, 1, 1, 4096);
+  }
+
+  public void ReInit(java.io.InputStream dstream)
+  {
+     ReInit(dstream, 1, 1, 4096);
+  }
+  public void ReInit(java.io.InputStream dstream, String encoding, int startline,
+                     int startcolumn) throws java.io.UnsupportedEncodingException
+  {
+     ReInit(dstream, encoding, startline, startcolumn, 4096);
+  }
+  public void ReInit(java.io.InputStream dstream, int startline,
+                     int startcolumn)
+  {
+     ReInit(dstream, startline, startcolumn, 4096);
+  }
+  public String GetImage()
+  {
+     if (bufpos >= tokenBegin)
+        return new String(buffer, tokenBegin, bufpos - tokenBegin + 1);
+     else
+        return new String(buffer, tokenBegin, bufsize - tokenBegin) +
+                              new String(buffer, 0, bufpos + 1);
+  }
+
+  public char[] GetSuffix(int len)
+  {
+     char[] ret = new char[len];
+
+     if ((bufpos + 1) >= len)
+        System.arraycopy(buffer, bufpos - len + 1, ret, 0, len);
+     else
+     {
+        System.arraycopy(buffer, bufsize - (len - bufpos - 1), ret, 0,
+                                                          len - bufpos - 1);
+        System.arraycopy(buffer, 0, ret, len - bufpos - 1, bufpos + 1);
+     }
+
+     return ret;
+  }
+
+  public void Done()
+  {
+     buffer = null;
+     bufline = null;
+     bufcolumn = null;
+  }
+
+  /**
+   * Method to adjust line and column numbers for the start of a token.
+   */
+  public void adjustBeginLineColumn(int newLine, int newCol)
+  {
+     int start = tokenBegin;
+     int len;
+
+     if (bufpos >= tokenBegin)
+     {
+        len = bufpos - tokenBegin + inBuf + 1;
+     }
+     else
+     {
+        len = bufsize - tokenBegin + bufpos + 1 + inBuf;
+     }
+
+     int i = 0, j = 0, k = 0;
+     int nextColDiff = 0, columnDiff = 0;
+
+     while (i < len &&
+            bufline[j = start % bufsize] == bufline[k = ++start % bufsize])
+     {
+        bufline[j] = newLine;
+        nextColDiff = columnDiff + bufcolumn[k] - bufcolumn[j];
+        bufcolumn[j] = newCol + columnDiff;
+        columnDiff = nextColDiff;
+        i++;
+     } 
+
+     if (i < len)
+     {
+        bufline[j] = newLine++;
+        bufcolumn[j] = newCol + columnDiff;
+
+        while (i++ < len)
+        {
+           if (bufline[j = start % bufsize] != bufline[++start % bufsize])
+              bufline[j] = newLine++;
+           else
+              bufline[j] = newLine;
+        }
+     }
+
+     line = bufline[j];
+     column = bufcolumn[j];
+  }
+
+}

+ 97 - 0
src/java/org/apache/hadoop/record/compiler/generated/Token.java

@@ -0,0 +1,97 @@
+/* Generated By:JavaCC: Do not edit this line. Token.java Version 3.0 */
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler.generated;
+
+/**
+ * Describes the input token stream.
+ */
+
+public class Token {
+
+  /**
+   * An integer that describes the kind of this token.  This numbering
+   * system is determined by JavaCCParser, and a table of these numbers is
+   * stored in the file ...Constants.java.
+   */
+  public int kind;
+
+  /**
+   * beginLine and beginColumn describe the position of the first character
+   * of this token; endLine and endColumn describe the position of the
+   * last character of this token.
+   */
+  public int beginLine, beginColumn, endLine, endColumn;
+
+  /**
+   * The string image of the token.
+   */
+  public String image;
+
+  /**
+   * A reference to the next regular (non-special) token from the input
+   * stream.  If this is the last token from the input stream, or if the
+   * token manager has not read tokens beyond this one, this field is
+   * set to null.  This is true only if this token is also a regular
+   * token.  Otherwise, see below for a description of the contents of
+   * this field.
+   */
+  public Token next;
+
+  /**
+   * This field is used to access special tokens that occur prior to this
+   * token, but after the immediately preceding regular (non-special) token.
+   * If there are no such special tokens, this field is set to null.
+   * When there are more than one such special token, this field refers
+   * to the last of these special tokens, which in turn refers to the next
+   * previous special token through its specialToken field, and so on
+   * until the first special token (whose specialToken field is null).
+   * The next fields of special tokens refer to other special tokens that
+   * immediately follow it (without an intervening regular token).  If there
+   * is no such token, this field is null.
+   */
+  public Token specialToken;
+
+  /**
+   * Returns the image.
+   */
+  public String toString()
+  {
+     return image;
+  }
+
+  /**
+   * Returns a new Token object, by default. However, if you want, you
+   * can create and return subclass objects based on the value of ofKind.
+   * Simply add the cases to the switch for all those special cases.
+   * For example, if you have a subclass of Token called IDToken that
+   * you want to create if ofKind is ID, simlpy add something like :
+   *
+   *    case MyParserConstants.ID : return new IDToken();
+   *
+   * to the following switch statement. Then you can cast matchedToken
+   * variable to the appropriate type and use it in your lexical actions.
+   */
+  public static final Token newToken(int ofKind)
+  {
+     switch(ofKind)
+     {
+       default : return new Token();
+     }
+  }
+
+}

+ 149 - 0
src/java/org/apache/hadoop/record/compiler/generated/TokenMgrError.java

@@ -0,0 +1,149 @@
+/* Generated By:JavaCC: Do not edit this line. TokenMgrError.java Version 3.0 */
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler.generated;
+
+public class TokenMgrError extends Error
+{
+   /*
+    * Ordinals for various reasons why an Error of this type can be thrown.
+    */
+
+   /**
+    * Lexical error occured.
+    */
+   static final int LEXICAL_ERROR = 0;
+
+   /**
+    * An attempt wass made to create a second instance of a static token manager.
+    */
+   static final int STATIC_LEXER_ERROR = 1;
+
+   /**
+    * Tried to change to an invalid lexical state.
+    */
+   static final int INVALID_LEXICAL_STATE = 2;
+
+   /**
+    * Detected (and bailed out of) an infinite loop in the token manager.
+    */
+   static final int LOOP_DETECTED = 3;
+
+   /**
+    * Indicates the reason why the exception is thrown. It will have
+    * one of the above 4 values.
+    */
+   int errorCode;
+
+   /**
+    * Replaces unprintable characters by their espaced (or unicode escaped)
+    * equivalents in the given string
+    */
+   protected static final String addEscapes(String str) {
+      StringBuffer retval = new StringBuffer();
+      char ch;
+      for (int i = 0; i < str.length(); i++) {
+        switch (str.charAt(i))
+        {
+           case 0 :
+              continue;
+           case '\b':
+              retval.append("\\b");
+              continue;
+           case '\t':
+              retval.append("\\t");
+              continue;
+           case '\n':
+              retval.append("\\n");
+              continue;
+           case '\f':
+              retval.append("\\f");
+              continue;
+           case '\r':
+              retval.append("\\r");
+              continue;
+           case '\"':
+              retval.append("\\\"");
+              continue;
+           case '\'':
+              retval.append("\\\'");
+              continue;
+           case '\\':
+              retval.append("\\\\");
+              continue;
+           default:
+              if ((ch = str.charAt(i)) < 0x20 || ch > 0x7e) {
+                 String s = "0000" + Integer.toString(ch, 16);
+                 retval.append("\\u" + s.substring(s.length() - 4, s.length()));
+              } else {
+                 retval.append(ch);
+              }
+              continue;
+        }
+      }
+      return retval.toString();
+   }
+
+   /**
+    * Returns a detailed message for the Error when it is thrown by the
+    * token manager to indicate a lexical error.
+    * Parameters : 
+    *    EOFSeen     : indicates if EOF caused the lexicl error
+    *    curLexState : lexical state in which this error occured
+    *    errorLine   : line number when the error occured
+    *    errorColumn : column number when the error occured
+    *    errorAfter  : prefix that was seen before this error occured
+    *    curchar     : the offending character
+    * Note: You can customize the lexical error message by modifying this method.
+    */
+   protected static String LexicalError(boolean EOFSeen, int lexState, int errorLine, int errorColumn, String errorAfter, char curChar) {
+      return("Lexical error at line " +
+           errorLine + ", column " +
+           errorColumn + ".  Encountered: " +
+           (EOFSeen ? "<EOF> " : ("\"" + addEscapes(String.valueOf(curChar)) + "\"") + " (" + (int)curChar + "), ") +
+           "after : \"" + addEscapes(errorAfter) + "\"");
+   }
+
+   /**
+    * You can also modify the body of this method to customize your error messages.
+    * For example, cases like LOOP_DETECTED and INVALID_LEXICAL_STATE are not
+    * of end-users concern, so you can return something like : 
+    *
+    *     "Internal Error : Please file a bug report .... "
+    *
+    * from this method for such cases in the release version of your parser.
+    */
+   public String getMessage() {
+      return super.getMessage();
+   }
+
+   /*
+    * Constructors of various flavors follow.
+    */
+
+   public TokenMgrError() {
+   }
+
+   public TokenMgrError(String message, int reason) {
+      super(message);
+      errorCode = reason;
+   }
+
+   public TokenMgrError(boolean EOFSeen, int lexState, int errorLine, int errorColumn, String errorAfter, char curChar, int reason) {
+      this(LexicalError(EOFSeen, lexState, errorLine, errorColumn, errorAfter, curChar), reason);
+   }
+}

+ 366 - 0
src/java/org/apache/hadoop/record/compiler/generated/rcc.jj

@@ -0,0 +1,366 @@
+options {
+STATIC=false;
+}
+
+PARSER_BEGIN(Rcc)
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.compiler.generated;
+
+import org.apache.hadoop.record.compiler.*;
+import java.util.ArrayList;
+import java.util.Hashtable;
+import java.util.Iterator;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+public class Rcc {
+    private static String language = "java";
+    private static ArrayList recFiles = new ArrayList();
+    private static JFile curFile;
+    private static Hashtable recTab;
+    private static String curDir = System.getProperty("user.dir");
+    private static String curFileName;
+    private static String curModuleName;
+
+    public static void main(String args[]) {
+        for (int i=0; i<args.length; i++) {
+            if ("-l".equalsIgnoreCase(args[i]) ||
+                "--language".equalsIgnoreCase(args[i])) {
+                language = args[i+1].toLowerCase();
+                i++;
+            } else {
+                recFiles.add(args[i]);
+            }
+        }
+        if (!"c++".equals(language) && !"java".equals(language)) {
+            System.out.println("Cannot recognize language:" + language);
+            System.exit(1);
+        }
+        if (recFiles.size() == 0) {
+            System.out.println("No record files specified. Exiting.");
+            System.exit(1);
+        }
+        for (int i=0; i<recFiles.size(); i++) {
+            curFileName = (String) recFiles.get(i);
+            File file = new File(curDir, curFileName);
+            try {
+                FileReader reader = new FileReader(file);
+                Rcc parser = new Rcc(reader);
+                try {
+                    recTab = new Hashtable();
+                    curFile = parser.Input();
+                    System.out.println((String) recFiles.get(i) +
+                        " Parsed Successfully");
+                } catch (ParseException e) {
+                    System.out.println(e.toString());
+                    System.exit(1);
+                }
+                try {
+                    reader.close();
+                } catch (IOException e) {
+                }
+            } catch (FileNotFoundException e) {
+                System.out.println("File " + (String) recFiles.get(i) +
+                    " Not found.");
+                System.exit(1);
+            }
+            try {
+                curFile.genCode(language);
+            } catch (IOException e) {
+                System.out.println(e.toString());
+                System.exit(1);
+            }
+        }
+    }
+}
+
+PARSER_END(Rcc)
+
+SKIP :
+{
+  " "
+| "\t"
+| "\n"
+| "\r"
+}
+
+SPECIAL_TOKEN :
+{
+  "//" : WithinOneLineComment
+}
+
+<WithinOneLineComment> SPECIAL_TOKEN :
+{
+  <("//" | "\n" | "\r" | "\r\n" )> : DEFAULT
+}
+
+<WithinOneLineComment> MORE :
+{
+  <~[]>
+}
+
+SPECIAL_TOKEN :
+{
+  "/*" : WithinMultiLineComment
+}
+
+<WithinMultiLineComment> SPECIAL_TOKEN :
+{
+  "*/" : DEFAULT
+}
+
+<WithinMultiLineComment> MORE :
+{
+  <~[]>
+}
+
+TOKEN :
+{
+    <MODULE_TKN: "module">
+|   <RECORD_TKN: "class">
+|   <INCLUDE_TKN: "include">
+|   <BYTE_TKN: "byte">
+|   <BOOLEAN_TKN: "boolean">
+|   <INT_TKN: "int">
+|   <LONG_TKN: "long">
+|   <FLOAT_TKN: "float">
+|   <DOUBLE_TKN: "double">
+|   <USTRING_TKN: "ustring">
+|   <BUFFER_TKN: "buffer">
+|   <VECTOR_TKN: "vector">
+|   <MAP_TKN: "map">
+|   <LBRACE_TKN: "{">
+|   <RBRACE_TKN: "}">
+|   <LT_TKN: "<">
+|   <GT_TKN: ">">
+|   <SEMICOLON_TKN: ";">
+|   <COMMA_TKN: ",">
+|   <DOT_TKN: ".">
+|   <CSTRING_TKN: "\"" ( ~["\""] )+ "\"">
+|   <IDENT_TKN: ["A"-"Z","a"-"z"] (["a"-"z","A"-"Z","0"-"9","_"])*>
+}
+
+JFile Input() :
+{
+    ArrayList ilist = new ArrayList();
+    ArrayList rlist = new ArrayList();
+    JFile i;
+    ArrayList l;
+}
+{
+    (
+        i = Include()
+        { ilist.add(i); }
+    |   l = Module()
+        { rlist.addAll(l); }
+    )+
+    <EOF>
+    { return new JFile(curFileName, ilist, rlist); }
+}
+
+JFile Include() :
+{
+    String fname;
+    Token t;
+}
+{
+    <INCLUDE_TKN>
+    t = <CSTRING_TKN>
+    {
+        JFile ret = null;
+        fname = t.image.replaceAll("^\"", "").replaceAll("\"$","");
+        File file = new File(curDir, fname);
+        String tmpDir = curDir;
+        String tmpFile = curFileName;
+        curDir = file.getParent();
+        curFileName = file.getName();
+        try {
+            FileReader reader = new FileReader(file);
+            Rcc parser = new Rcc(reader);
+            try {
+                ret = parser.Input();
+                System.out.println(fname + " Parsed Successfully");
+            } catch (ParseException e) {
+                System.out.println(e.toString());
+                System.exit(1);
+            }
+            try {
+                reader.close();
+            } catch (IOException e) {
+            }
+        } catch (FileNotFoundException e) {
+            System.out.println("File " + fname +
+                " Not found.");
+            System.exit(1);
+        }
+        curDir = tmpDir;
+        curFileName = tmpFile;
+        return ret;
+    }
+}
+
+ArrayList Module() :
+{
+    String mName;
+    ArrayList rlist;
+}
+{
+    <MODULE_TKN>
+    mName = ModuleName()
+    { curModuleName = mName; }
+    <LBRACE_TKN>
+    rlist = RecordList()
+    <RBRACE_TKN>
+    { return rlist; }
+}
+
+String ModuleName() :
+{
+    String name = "";
+    Token t;
+}
+{
+    t = <IDENT_TKN>
+    { name += t.image; }
+    (
+        <DOT_TKN>
+        t = <IDENT_TKN>
+        { name += "." + t.image; }
+    )*
+    { return name; }
+}
+
+ArrayList RecordList() :
+{
+    ArrayList rlist = new ArrayList();
+    JRecord r;
+}
+{
+    (
+        r = Record()
+        { rlist.add(r); }
+    )+
+    { return rlist; }
+}
+
+JRecord Record() :
+{
+    String rname;
+    ArrayList flist = new ArrayList();
+    Token t;
+    JField f;
+}
+{
+    <RECORD_TKN>
+    t = <IDENT_TKN>
+    { rname = t.image; }
+    <LBRACE_TKN>
+    (
+        f = Field()
+        { flist.add(f); }
+        <SEMICOLON_TKN>
+    )+
+    <RBRACE_TKN>
+    {
+        String fqn = curModuleName + "." + rname;
+        JRecord r = new JRecord(fqn, flist);
+        recTab.put(fqn, r);
+        return r;
+    }
+}
+
+JField Field() :
+{
+    JType jt;
+    Token t;
+}
+{
+    jt = Type()
+    t = <IDENT_TKN>
+    { return new JField(jt, t.image); }
+}
+
+JType Type() :
+{
+    JType jt;
+    Token t;
+    String rname;
+}
+{
+    jt = Map()
+    { return jt; }
+|   jt = Vector()
+    { return jt; }
+|   <BYTE_TKN>
+    { return new JByte(); }
+|   <BOOLEAN_TKN>
+    { return new JBoolean(); }
+|   <INT_TKN>
+    { return new JInt(); }
+|   <LONG_TKN>
+    { return new JLong(); }
+|   <FLOAT_TKN>
+    { return new JFloat(); }
+|   <DOUBLE_TKN>
+    { return new JDouble(); }
+|   <USTRING_TKN>
+    { return new JString(); }
+|   <BUFFER_TKN>
+    { return new JBuffer(); }
+|   rname = ModuleName()
+    {
+        if (rname.indexOf('.', 0) < 0) {
+            rname = curModuleName + "." + rname;
+        }
+        JRecord r = (JRecord) recTab.get(rname);
+        if (r == null) {
+            System.out.println("Type " + rname + " not known. Exiting.");
+            System.exit(1);
+        }
+        return r;
+    }
+}
+
+JMap Map() :
+{
+    JType jt1;
+    JType jt2;
+}
+{
+    <MAP_TKN>
+    <LT_TKN>
+    jt1 = Type()
+    <COMMA_TKN>
+    jt2 = Type()
+    <GT_TKN>
+    { return new JMap(jt1, jt2); }
+}
+
+JVector Vector() :
+{
+    JType jt;
+}
+{
+    <VECTOR_TKN>
+    <LT_TKN>
+    jt = Type()
+    <GT_TKN>
+    { return new JVector(jt); }
+}

+ 785 - 0
src/java/org/apache/hadoop/record/package.html

@@ -0,0 +1,785 @@
+<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN">
+
+<html>
+  <head>
+    <title>Hadoop Record I/O</title>
+  </head>
+  <body>
+  Hadoop record I/O contains classes and a record description language
+  translator for simplifying serialization and deserialization of records in a
+  language-neutral manner.
+  
+  <h2>Introduction</h2>
+  
+  Software systems of any significant complexity require mechanisms for data 
+interchange with the outside world. These interchanges typically involve the
+marshaling and unmarshaling of logical units of data to and from data streams
+(files, network connections, memory buffers etc.). Applications usually have
+some code for serializing and deserializing the data types that they manipulate
+embedded in them. The work of serialization has several features that make
+automatic code generation for it worthwhile. Given a particular output encoding
+(binary, XML, etc.), serialization of primitive types and simple compositions
+of primitives (structs, vectors etc.) is a very mechanical task. Manually
+written serialization code can be susceptible to bugs especially when records
+have a large number of fields or a record definition changes between software
+versions. Lastly, it can be very useful for applications written in different
+programming languages to be able to share and interchange data. This can be 
+made a lot easier by describing the data records manipulated by these
+applications in a language agnostic manner and using the descriptions to derive
+implementations of serialization in multiple target languages. 
+
+This document describes Hadoop Record I/O, a mechanism that is aimed 
+at
+<ul> 
+<li> enabling the specification of simple serializable data types (records) 
+<li> enabling the generation of code in multiple target languages for
+marshaling and unmarshaling such types
+<li> providing target language specific support that will enable application 
+programmers to incorporate generated code into their applications
+</ul>
+
+The goals of Hadoop Record I/O are similar to those of mechanisms such as XDR,
+ASN.1, PADS and ICE. While these systems all include a DDL that enables
+the specification of most record types, they differ widely in what else they
+focus on. The focus in Hadoop Record I/O is on data marshaling and
+multi-lingual support.  We take a translator-based approach to serialization.
+Hadoop users have to describe their data in a simple data description
+language. The Hadoop DDL translator rcc generates code that users
+can invoke in order to read/write their data from/to simple stream 
+abstractions. Next we list explicitly some of the goals and non-goals of
+Hadoop Record I/O.
+
+
+<h3>Goals</h3>
+
+<ul>
+<li> Support for commonly used primitive types. Hadoop should include as
+primitives commonly used builtin types from programming languages we intend to
+support.
+
+<li> Support for common data compositions (including recursive compositions).
+Hadoop should support widely used composite types such as structs and
+vectors.
+
+<li> Code generation in multiple target languages. Hadoop should be capable of
+generating serialization code in multiple target languages and should be
+easily extensible to new target languages. The initial target languages are
+C++ and Java.
+
+<li> Support for generated target languages. Hadooop should include support
+in the form of headers, libraries, packages for supported target languages 
+that enable easy inclusion and use of generated code in applications.
+
+<li> Support for multiple output encodings. Candidates include
+packed binary, comma-separated text, XML etc.
+
+<li> Support for specifying record types in a backwards/forwards compatible
+manner. This will probably be in the form of support for optional fields in
+records. This version of the document does not include a description of the
+planned mechanism, we intend to include it in the next iteration.
+
+</ul>
+
+<h3>Non-Goals</h3>
+
+<ul>
+  <li> Serializing existing arbitrary C++ classes.
+  <li> Serializing complex data structures such as trees, linked lists etc.
+  <li> Built-in indexing schemes, compression, or check-sums.
+  <li> Dynamic construction of objects from an XML schema.
+</ul>
+
+The remainder of this document describes the features of Hadoop record I/O
+in more detail. Section 2 describes the data types supported by the system.
+Section 3 lays out the DDL syntax with some examples of simple records. 
+Section 4 describes the process of code generation with rcc. Section 5
+describes target language mappings and support for Hadoop types. We include a
+fairly complete description of C++ mappings with intent to include Java and
+others in upcoming iterations of this document. The last section talks about
+supported output encodings.
+
+
+<h2>Data Types and Streams</h2>
+
+This section describes the primitive and composite types supported by Hadoop.
+We aim to support a set of types that can be used to simply and efficiently
+express a wide range of record types in different programming languages.
+
+<h3>Primitive Types</h3>
+
+For the most part, the primitive types of Hadoop map directly to primitive
+types in high level programming languages. Special cases are the
+ustring (a Unicode string) and buffer types, which we believe
+find wide use and which are usually implemented in library code and not
+available as language built-ins. Hadoop also supplies these via library code
+when a target language built-in is not present and there is no widely
+adopted "standard" implementation. The complete list of primitive types is:
+
+<ul>
+  <li> byte: An 8-bit unsigned integer.
+  <li> boolean: A boolean value.
+  <li> int: A 32-bit signed integer.
+  <li> long: A 64-bit signed integer.
+  <li> float: A single precision floating point number as described by
+    IEEE-754.
+  <li> double: A double precision floating point number as described by
+    IEEE-754.
+  <li> ustring: A string consisting of Unicode characters.
+  <li> buffer: An arbitrary sequence of bytes. 
+</ul>
+
+
+<h3>Composite Types</h3>
+Hadoop supports a small set of composite types that enable the description
+of simple aggregate types and containers. A composite type is serialized
+by sequentially serializing it constituent elements. The supported
+composite types are:
+
+<ul>
+
+  <li> record: An aggregate type like a C-struct. This is a list of
+typed fields that are together considered a single unit of data. A record
+is serialized by sequentially serializing its constituent fields. In addition
+to serialization a record has comparison operations (equality and less-than)
+implemented for it, these are defined as memberwise comparisons.
+
+  <li>vector: A sequence of entries of the same data type, primitive
+or composite.
+
+  <li> map: An associative container mapping instances of a key type to
+instances of a value type. The key and value types may themselves be primitive
+or composite types. 
+
+</ul>
+
+<h3>Streams</h3>
+
+Hadoop generates code for serializing and deserializing record types to
+abstract streams. For each target language Hadoop defines very simple input
+and output stream interfaces. Application writers can usually develop
+concrete implementations of these by putting a one method wrapper around
+an existing stream implementation.
+
+
+<h2>DDL Syntax and Examples</h2>
+
+We now describe the syntax of the Hadoop data description language. This is
+followed by a few examples of DDL usage.
+ 
+<h3>Hadoop DDL Syntax</h3>
+
+<pre><code>
+recfile = *include module *record
+include = "include" path
+path = (relative-path / absolute-path)
+module = "module" module-name
+module-name = name *("." name)
+record := "class" name "{" 1*(field) "}"
+field := type name ";"
+name :=  ALPHA (ALPHA / DIGIT / "_" )*
+type := (ptype / ctype)
+ptype := ("byte" / "boolean" / "int" |
+          "long" / "float" / "double"
+          "ustring" / "buffer")
+ctype := (("vector" "<" type ">") /
+          ("map" "<" type "," type ">" ) ) / name)
+</code></pre>
+
+A DDL file describes one or more record types. It begins with zero or
+more include declarations, a single mandatory module declaration
+followed by zero or more class declarations. The semantics of each of
+these declarations are described below:
+
+<ul>
+
+<li>include: An include declaration specifies a DDL file to be
+referenced when generating code for types in the current DDL file. Record types
+in the current compilation unit may refer to types in all included files.
+File inclusion is recursive. An include does not trigger code
+generation for the referenced file.
+
+<li> module: Every Hadoop DDL file must have a single module
+declaration that follows the list of includes and precedes all record
+declarations. A module declaration identifies a scope within which
+the names of all types in the current file are visible. Module names are
+mapped to C++ namespaces, Java packages etc. in generated code.
+
+<li> class: Records types are specified through class
+declarations. A class declaration is like a Java class declaration.
+It specifies a named record type and a list of fields that constitute records
+of the type. Usage is illustrated in the following examples.
+
+</ul>
+
+<h3>Examples</h3>
+
+<ul>
+<li>A simple DDL file links.jr with just one record declaration. 
+<pre><code>
+module links {
+    class Link {
+        ustring URL;
+        boolean isRelative;
+        ustring anchorText;
+    };
+}
+</code></pre>
+
+<li> A DDL file outlinks.jr which includes another
+<pre><code>
+include "links.jr"
+
+module outlinks {
+    class OutLinks {
+        ustring baseURL;
+        vector<links.Link> outLinks;
+    };
+}
+</code></pre>
+</ul>
+
+<h2>Code Generation</h2>
+
+The Hadoop translator is written in Java. Invocation is done by executing a 
+wrapper shell script named named rcc. It takes a list of
+record description files as a mandatory argument and an
+optional language argument (the default is Java) --language or
+-l. Thus a typical invocation would look like:
+<pre><code>
+$ rcc -l C++ <filename> ...
+</code></pre>
+
+
+<h2>Target Language Mappings and Support</h2>
+
+For all target languages, the unit of code generation is a record type. 
+For each record type, Hadoop generates code for serialization and
+deserialization, record comparison and access to record members.
+
+<h3>C++</h3>
+
+Support for including Hadoop generated C++ code in applications comes in the
+form of a header file recordio.hh which needs to be included in source
+that uses Hadoop types and a library librecordio.a which applications need
+to be linked with. The header declares the Hadoop C++ namespace which defines
+appropriate types for the various primitives, the basic interfaces for
+records and streams and enumerates the supported serialization encodings.
+Declarations of these interfaces and a description of their semantics follow:
+
+<pre><code>
+namespace hadoop {
+
+  enum RecFormat { kBinary, kXML, kCSV };
+
+  class InStream {
+  public:
+    virtual ssize_t read(void *buf, size_t n) = 0;
+  };
+
+  class OutStream {
+  public:
+    virtual ssize_t write(const void *buf, size_t n) = 0;
+  };
+
+  class IOError : public runtime_error {
+  public:
+    explicit IOError(const std::string& msg);
+  };
+
+  class IArchive;
+  class OArchive;
+
+  class RecordReader {
+  public:
+    RecordReader(InStream& in, RecFormat fmt);
+    virtual ~RecordReader(void);
+
+    virtual void read(Record& rec);
+  };
+
+  class RecordWriter {
+  public:
+    RecordWriter(OutStream& out, RecFormat fmt);
+    virtual ~RecordWriter(void);
+
+    virtual void write(Record& rec);
+  };
+
+
+  class Record {
+  public:
+    virtual std::string type(void) const = 0;
+    virtual std::string signature(void) const = 0;
+  protected:
+    virtual bool validate(void) const = 0;
+
+    virtual void
+    serialize(OArchive& oa, const std::string& tag) const = 0;
+
+    virtual void
+    deserialize(IArchive& ia, const std::string& tag) = 0;
+  };
+}
+</code></pre>
+
+<ul>
+
+<li> RecFormat: An enumeration of the serialization encodings supported
+by this implementation of Hadoop.
+
+<li> InStream: A simple abstraction for an input stream. This has a 
+single public read method that reads n bytes from the stream into
+the buffer buf. Has the same semantics as a blocking read system
+call. Returns the number of bytes read or -1 if an error occurs.
+
+<li> OutStream: A simple abstraction for an output stream. This has a 
+single write method that writes n bytes to the stream from the
+buffer buf. Has the same semantics as a blocking write system
+call. Returns the number of bytes written or -1 if an error occurs.
+
+<li> RecordReader: A RecordReader reads records one at a time from
+an underlying stream in a specified record format. The reader is instantiated
+with a stream and a serialization format. It has a read method that
+takes an instance of a record and deserializes the record from the stream.
+
+<li> RecordWriter: A RecordWriter writes records one at a
+time to an underlying stream in a specified record format. The writer is
+instantiated with a stream and a serialization format. It has a
+write method that takes an instance of a record and serializes the
+record to the stream.
+
+<li> Record: The base class for all generated record types. This has two
+public methods type and signature that return the typename and the
+type signature of the record.
+
+</ul>
+
+Two files are generated for each record file (note: not for each record). If a
+record file is named "name.jr", the generated files are 
+"name.jr.cc" and "name.jr.hh" containing serialization 
+implementations and record type declarations respectively.
+
+For each record in the DDL file, the generated header file will contain a
+class definition corresponding to the record type, method definitions for the
+generated type will be present in the '.cc' file.  The generated class will
+inherit from the abstract class hadoop::Record. The DDL files
+module declaration determines the namespace the record belongs to.
+Each '.' delimited token in the module declaration results in the
+creation of a namespace. For instance, the declaration module docs.links
+results in the creation of a docs namespace and a nested 
+docs::links namespace. In the preceding examples, the Link class
+is placed in the links namespace. The header file corresponding to
+the links.jr file will contain:
+
+<pre><code>
+namespace links {
+  class Link : public hadoop::Record {
+    // ....
+  };
+};
+</code></pre>
+
+Each field within the record will cause the generation of a private member
+declaration of the appropriate type in the class declaration, and one or more
+acccessor methods. The generated class will implement the serialize and
+deserialize methods defined in hadoop::Record+. It will also 
+implement the inspection methods type and signature from
+hadoop::Record. A default constructor and virtual destructor will also
+be generated. Serialization code will read/write records into streams that
+implement the hadoop::InStream and the hadoop::OutStream interfaces.
+
+For each member of a record an accessor method is generated that returns 
+either the member or a reference to the member. For members that are returned 
+by value, a setter method is also generated. This is true for primitive 
+data members of the types byte, int, long, boolean, float and 
+double. For example, for a int field called MyField the folowing
+code is generated.
+
+<pre><code>
+...
+private:
+  int32_t mMyField;
+  ...
+public:
+  int32_t getMyField(void) const {
+    return mMyField;
+  };
+
+  void setMyField(int32_t m) {
+    mMyField = m;
+  };
+  ...
+</code></pre>
+
+For a ustring or buffer or composite field. The generated code
+only contains accessors that return a reference to the field. A const
+and a non-const accessor are generated. For example:
+
+<pre><code>
+...
+private:
+  std::string mMyBuf;
+  ...
+public:
+
+  std::string& getMyBuf() {
+    return mMyBuf;
+  };
+
+  const std::string& getMyBuf() const {
+    return mMyBuf;
+  };
+  ...
+</code></pre>
+
+<h4>Examples</h4>
+
+Suppose the inclrec.jr file contains:
+<pre><code>
+module inclrec {
+    class RI {
+        int      I32;
+        double   D;
+        ustring  S;
+    };
+}
+</code></pre>
+
+and the testrec.jr file contains:
+
+<pre><code>
+include "inclrec.jr"
+module testrec {
+    class R {
+        vector<float> VF;
+        RI            Rec;
+        buffer        Buf;
+    };
+}
+</code></pre>
+
+Then the invocation of rcc such as:
+<pre><code>
+$ rcc -l c++ inclrec.jr testrec.jr
+</code></pre>
+will result in generation of four files:
+inclrec.jr.{cc,hh} and testrec.jr.{cc,hh}.
+
+The inclrec.jr.hh will contain:
+
+<pre><code>
+#ifndef _INCLREC_JR_HH_
+#define _INCLREC_JR_HH_
+
+#include "recordio.hh"
+
+namespace inclrec {
+  
+  class RI : public hadoop::Record {
+
+  private:
+
+    int32_t      mI32;
+    double       mD;
+    std::string  mS;
+
+  public:
+
+    RI(void);
+    virtual ~RI(void);
+
+    virtual bool operator==(const RI& peer) const;
+    virtual bool operator<(const RI& peer) const;
+
+    virtual int32_t getI32(void) const { return mI32; }
+    virtual void setI32(int32_t v) { mI32 = v; }
+
+    virtual double getD(void) const { return mD; }
+    virtual void setD(double v) { mD = v; }
+
+    virtual std::string& getS(void) const { return mS; }
+    virtual const std::string& getS(void) const { return mS; }
+
+    virtual std::string type(void) const;
+    virtual std::string signature(void) const;
+
+  protected:
+
+    virtual void serialize(hadoop::OArchive& a) const;
+    virtual void deserialize(hadoop::IArchive& a);
+
+    virtual bool validate(void);
+  };
+} // end namespace inclrec
+
+#endif /* _INCLREC_JR_HH_ */
+
+</code></pre>
+
+The testrec.jr.hh file will contain:
+
+
+<pre><code>
+
+#ifndef _TESTREC_JR_HH_
+#define _TESTREC_JR_HH_
+
+#include "inclrec.jr.hh"
+
+namespace testrec {
+  class R : public hadoop::Record {
+
+  private:
+
+    std::vector<float> mVF;
+    inclrec::RI        mRec;
+    std::string        mBuf;
+
+  public:
+
+    R(void);
+    virtual ~R(void);
+
+    virtual bool operator==(const R& peer) const;
+    virtual bool operator<(const R& peer) const;
+
+    virtual std::vector<float>& getVF(void) const;
+    virtual const std::vector<float>& getVF(void) const;
+
+    virtual std::string& getBuf(void) const ;
+    virtual const std::string& getBuf(void) const;
+
+    virtual inclrec::RI& getRec(void) const;
+    virtual const inclrec::RI& getRec(void) const;
+    
+    virtual bool serialize(hadoop::OutArchive& a) const;
+    virtual bool deserialize(hadoop::InArchive& a);
+    
+    virtual std::string type(void) const;
+    virtual std::string signature(void) const;
+  };
+}; // end namespace testrec
+#endif /* _TESTREC_JR_HH_ */
+
+</code></pre>
+
+<h3>Java</h3>
+
+Code generation for Java is similar to that for C++. A Java class is generated
+for each record type with private members corresponding to the fields. Getters
+and setters for fields are also generated. Some differences arise in the
+way comparison is expressed and in the mapping of modules to packages and
+classes to files. For equality testing, an equals method is generated
+for each record type. As per Java requirements a hashCode method is also
+generated. For comparison a compareTo method is generated for each
+record type. This has the semantics as defined by the Java Comparable
+interface, that is, the method returns a negative integer, zero, or a positive
+integer as the invoked object is less than, equal to, or greater than the
+comparison parameter.
+
+A .java file is generated per record type as opposed to per DDL
+file as in C++. The module declaration translates to a Java
+package declaration. The module name maps to an identical Java package
+name. In addition to this mapping, the DDL compiler creates the appropriate
+directory hierarchy for the package and places the generated .java
+files in the correct directories.
+
+<h2>Mapping Summary</h2>
+
+<pre><code>
+DDL Type        C++ Type            Java Type 
+
+boolean         bool                boolean
+byte            int8_t              byte
+int             int32_t             int
+long            int64_t             long
+float           float               float
+double          double              double
+ustring         std::string         java.lang.String
+buffer          std::string         java.io.ByteArrayOutputStream
+class type      class type          class type
+vector<type>    std::vector<type>   java.util.ArrayList
+map<type,type>  std::map<type,type> java.util.TreeMap
+</code></pre>
+
+<h2>Data encodings</h2>
+
+This section describes the format of the data encodings supported by Hadoop.
+Currently, three data encodings are supported, namely binary, CSV and XML.
+
+<h3>Binary Serialization Format</h3>
+
+The binary data encoding format is fairly dense. Serialization of composite
+types is simply defined as a concatenation of serializations of the constituent
+elements (lengths are included in vectors and maps).
+
+Composite types are serialized as follows:
+<ul>
+<li> class: Sequence of serialized members.
+<li> vector: The number of elements serialized as an int. Followed by a
+sequence of serialized elements.
+<li> map: The number of key value pairs serialized as an int. Followed
+by a sequence of serialized (key,value) pairs.
+</ul>
+
+Serialization of primitives is more interesting, with a zero compression
+optimization for integral types and normalization to UTF-8 for strings. 
+Primitive types are serialized as follows:
+
+<ul>
+<li> byte: Represented by 1 byte, as is.
+<li> boolean: Represented by 1-byte (0 or 1)
+<li> int/long: Integers and longs are serialized zero compressed.
+Represented as 1-byte if -120 <= value < 128. Otherwise, serialized as a
+sequence of 2-5 bytes for ints, 2-9 bytes for longs. The first byte represents
+the number of trailing bytes, N, as the negative number (-120-N). For example,
+the number 1024 (0x400) is represented by the byte sequence 'x86 x04 x00'.
+This doesn't help much for 4-byte integers but does a reasonably good job with
+longs without bit twiddling.
+<li> float/double: Serialized in IEEE 754 single and double precision
+format in network byte order. This is the format used by Java.
+<li> ustring: Serialized as 4-byte zero compressed length followed by
+data encoded as UTF-8. Strings are normalized to UTF-8 regardless of native
+language representation.
+<li> buffer: Serialized as a 4-byte zero compressed length followed by the
+raw bytes in the buffer.
+</ul>
+
+
+<h3>CSV Serialization Format</h3>
+
+The CSV serialization format has a lot more structure than the "standard"
+Excel CSV format, but we believe the additional structure is useful because
+
+<ul>
+<li> it makes parsing a lot easier without detracting too much from legibility
+<li> the delimiters around composites make it obvious when one is reading a
+sequence of Hadoop records
+</ul>
+
+Serialization formats for the various types are detailed in the grammar that
+follows. The notable feature of the formats is the use of delimiters for 
+indicating the certain field types.
+
+<ul>
+<li> A string field begins with a single quote (').
+<li> A buffer field begins with a sharp (#).
+<li> A class, vector or map begins with 's{', 'v{' or 'm{' respectively and
+ends with '}'.
+</ul>
+
+The CSV format can be described by the following grammar:
+
+<pre><code>
+record = primitive / struct / vector / map
+primitive = boolean / int / long / float / double / ustring / buffer
+
+boolean = "T" / "F"
+int = ["-"] 1*DIGIT
+long = ";" ["-"] 1*DIGIT
+float = ["-"] 1*DIGIT "." 1*DIGIT ["E" / "e" ["-"] 1*DIGIT]
+double = ";" ["-"] 1*DIGIT "." 1*DIGIT ["E" / "e" ["-"] 1*DIGIT]
+
+ustring = "'" *(UTF8 char except NULL, LF, % and , / "%00" / "%0a" / "%25" / "%2c" )
+
+buffer = "#" *(BYTE except NULL, LF, % and , / "%00" / "%0a" / "%25" / "%2c" )
+
+struct = "s{" record *("," record) "}"
+vector = "v{" [record *("," record)] "}"
+map = "m{" [*(record "," record)] "}"
+</code></pre>
+
+<h3>XML Serialization Format</h3>
+
+The XML serialization format is the same used by Apache XML-RPC
+(http://ws.apache.org/xmlrpc/types.html). This is an extension of the original
+XML-RPC format and adds some additional data types. All record I/O types are
+not directly expressible in this format, and access to a DDL is required in
+order to convert these to valid types. All types primitive or composite are
+represented by &lt;value&gt; elements. The particular XML-RPC type is
+indicated by a nested element in the &lt;value&gt; element. The encoding for
+records is always UTF-8. Primitive types are serialized as follows:
+
+<ul>
+<li> byte: XML tag &lt;ex:i1&gt;. Values: 1-byte unsigned 
+integers represented in US-ASCII
+<li> boolean: XML tag &lt;boolean&gt;. Values: "0" or "1"
+<li> int: XML tags &lt;i4&gt; or &lt;int&gt;. Values: 4-byte
+signed integers represented in US-ASCII.
+<li> long: XML tag &lt;ex:i8&gt;. Values: 8-byte signed integers
+represented in US-ASCII.
+<li> float: XML tag &lt;ex:float&gt;. Values: Single precision
+floating point numbers represented in US-ASCII.
+<li> double: XML tag &lt;double&gt;. Values: Double precision
+floating point numbers represented in US-ASCII.
+<li> ustring: XML tag &lt;;string&gt;. Values: String values
+represented as UTF-8. XML does not permit all Unicode characters in literal
+data. In particular, NULLs and control chars are not allowed. Additionally,
+XML processors are required to replace carriage returns with line feeds and to
+replace CRLF sequences with line feeds. Programming languages that we work
+with do not impose these restrictions on string types. To work around these
+restrictions, disallowed characters and CRs are percent escaped in strings.
+The '%' character is also percent escaped.
+<li> buffer: XML tag &lt;string&&gt;. Values: Arbitrary binary
+data. Represented as hexBinary, each byte is replaced by its 2-byte
+hexadecimal representation.
+</ul>
+
+Composite types are serialized as follows:
+
+<ul>
+<li> class: XML tag &lt;struct&gt;. A struct is a sequence of
+&lt;member&gt; elements. Each &lt;member&gt; element has a &lt;name&gt;
+element and a &lt;value&gt; element. The &lt;name&gt; is a string that must
+match /[a-zA-Z][a-zA-Z0-9_]*/. The value of the member is represented
+by a &lt;value&gt; element.
+
+<li> vector: XML tag &lt;array&lt;. An &lt;array&gt; contains a
+single &lt;data&gt; element. The &lt;data&gt; element is a sequence of
+&lt;value&gt; elements each of which represents an element of the vector.
+
+<li> map: XML tag &lt;array&gt;. Same as vector.
+
+</ul>
+
+For example:
+
+<pre><code>
+class {
+  int           MY_INT;            // value 5
+  vector<float> MY_VEC;            // values 0.1, -0.89, 2.45e4
+  buffer        MY_BUF;            // value '\00\n\tabc%'
+}
+</code></pre>
+
+is serialized as
+
+<pre><code class="XML">
+&lt;value&gt;
+  &lt;struct&gt;
+    &lt;member&gt;
+      &lt;name&gt;MY_INT&lt;/name&gt;
+      &lt;value&gt;&lt;i4&gt;5&lt;/i4&gt;&lt;/value&gt;
+    &lt;/member&gt;
+    &lt;member&gt;
+      &lt;name&gt;MY_VEC&lt;/name&gt;
+      &lt;value&gt;
+        &lt;array&gt;
+          &lt;data&gt;
+            &lt;value&gt;&lt;ex:float&gt;0.1&lt;/ex:float&gt;&lt;/value&gt;
+            &lt;value&gt;&lt;ex:float&gt;-0.89&lt;/ex:float&gt;&lt;/value&gt;
+            &lt;value&gt;&lt;ex:float&gt;2.45e4&lt;/ex:float&gt;&lt;/value&gt;
+          &lt;/data&gt;
+        &lt;/array&gt;
+      &lt;/value&gt;
+    &lt;/member&gt;
+    &lt;member&gt;
+      &lt;name&gt;MY_BUF&lt;/name&gt;
+      &lt;value&gt;&lt;string&gt;%00\n\tabc%25&lt;/string&gt;&lt;/value&gt;
+    &lt;/member&gt;
+  &lt;/struct&gt;
+&lt;/value&gt; 
+</code></pre>
+
+  </body>
+</html>

+ 6 - 0
src/test/ddl/buffer.jr

@@ -0,0 +1,6 @@
+module org.apache.hadoop.record.test {
+    class RecBuffer {
+        buffer Data;
+    }
+}
+

+ 6 - 0
src/test/ddl/int.jr

@@ -0,0 +1,6 @@
+module org.apache.hadoop.record.test {
+    class RecInt {
+        int Data;
+    }
+}
+

+ 18 - 0
src/test/ddl/links.jr

@@ -0,0 +1,18 @@
+include "location.jr"
+
+module org.apache.hadoop.record.Links {
+    class Link {
+        ustring URL;
+        vector<org.apache.hadoop.record.Location.LinkLocation> Locations;
+        boolean IsRelative;
+        double Weight;
+    }
+
+    class HTTP_Transaction {
+        ustring URL;
+        ustring Request;
+        map<ustring,ustring> ResponseHeader;
+        buffer Source;
+    }
+} // end module
+

+ 8 - 0
src/test/ddl/location.jr

@@ -0,0 +1,8 @@
+module org.apache.hadoop.record.Location {
+    class LinkLocation {
+        int RowNum;
+        int ColNum;
+        ustring AnchorText;
+    }
+}
+

+ 6 - 0
src/test/ddl/string.jr

@@ -0,0 +1,6 @@
+module org.apache.hadoop.record.test {
+    class RecString {
+        ustring Data;
+    }
+}
+

+ 19 - 0
src/test/ddl/test.jr

@@ -0,0 +1,19 @@
+module org.apache.hadoop.record.test {
+    class RecRecord0 {
+        ustring     StringVal;
+    }
+
+    class RecRecord1 {
+        boolean         BoolVal;
+        byte            ByteVal;
+        int             IntVal;
+        long            LongVal;
+        float           FloatVal;
+        double          DoubleVal;
+        ustring         StringVal;
+        buffer          BufferVal;
+        vector<ustring> VectorVal;
+        map<ustring,ustring>   MapVal;
+    }
+}
+

+ 124 - 0
src/test/org/apache/hadoop/record/test/FromCpp.java

@@ -0,0 +1,124 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.test;
+
+import org.apache.hadoop.record.RecordReader;
+import org.apache.hadoop.record.RecordWriter;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.TreeMap;
+import junit.framework.*;
+
+/**
+ *
+ * @author milindb
+ */
+public class FromCpp extends TestCase {
+    
+    public FromCpp(String testName) {
+        super(testName);
+    }
+
+    protected void setUp() throws Exception {
+    }
+
+    protected void tearDown() throws Exception {
+    }
+    
+    public void testBinary() {
+        File tmpfile;
+        try {
+            tmpfile = new File("/temp/hadooptmp.dat");
+            RecRecord1 r1 = new RecRecord1();
+            r1.setBoolVal(true);
+            r1.setByteVal((byte)0x66);
+            r1.setFloatVal(3.145F);
+            r1.setDoubleVal(1.5234);
+            r1.setIntVal(4567);
+            r1.setLongVal(0x5a5a5a5a5a5aL);
+            r1.setStringVal("random text");
+            r1.setBufferVal(new ByteArrayOutputStream(20));
+            r1.setVectorVal(new ArrayList());
+            r1.setMapVal(new TreeMap());
+            FileInputStream istream = new FileInputStream(tmpfile);
+            RecordReader in = new RecordReader(istream, "binary");
+            RecRecord1 r2 = new RecRecord1();
+            in.read(r2);
+            istream.close();
+            assertTrue(r1.equals(r2));
+        } catch (IOException ex) {
+            ex.printStackTrace();
+        } 
+    }
+    
+    public void testCsv() {
+        File tmpfile;
+        try {
+            tmpfile = new File("/temp/hadooptmp.txt");
+            RecRecord1 r1 = new RecRecord1();
+            r1.setBoolVal(true);
+            r1.setByteVal((byte)0x66);
+            r1.setFloatVal(3.145F);
+            r1.setDoubleVal(1.5234);
+            r1.setIntVal(4567);
+            r1.setLongVal(0x5a5a5a5a5a5aL);
+            r1.setStringVal("random text");
+            r1.setBufferVal(new ByteArrayOutputStream(20));
+            r1.setVectorVal(new ArrayList());
+            r1.setMapVal(new TreeMap());
+            FileInputStream istream = new FileInputStream(tmpfile);
+            RecordReader in = new RecordReader(istream, "csv");
+            RecRecord1 r2 = new RecRecord1();
+            in.read(r2);
+            istream.close();
+            assertTrue(r1.equals(r2));
+        } catch (IOException ex) {
+            ex.printStackTrace();
+        } 
+    }
+
+    public void testXml() {
+        File tmpfile;
+        try {
+            tmpfile = new File("/temp/hadooptmp.xml");
+            RecRecord1 r1 = new RecRecord1();
+            r1.setBoolVal(true);
+            r1.setByteVal((byte)0x66);
+            r1.setFloatVal(3.145F);
+            r1.setDoubleVal(1.5234);
+            r1.setIntVal(4567);
+            r1.setLongVal(0x5a5a5a5a5a5aL);
+            r1.setStringVal("random text");
+            r1.setBufferVal(new ByteArrayOutputStream(20));
+            r1.setVectorVal(new ArrayList());
+            r1.setMapVal(new TreeMap());
+            FileInputStream istream = new FileInputStream(tmpfile);
+            RecordReader in = new RecordReader(istream, "xml");
+            RecRecord1 r2 = new RecRecord1();
+            in.read(r2);
+            istream.close();
+            assertTrue(r1.equals(r2));
+        } catch (IOException ex) {
+            ex.printStackTrace();
+        } 
+    }
+
+}

+ 95 - 0
src/test/org/apache/hadoop/record/test/RecBuffer.java

@@ -0,0 +1,95 @@
+// File generated by hadoop record compiler. Do not edit.
+package org.apache.hadoop.record.test;
+
+public class RecBuffer implements org.apache.hadoop.record.Record, org.apache.hadoop.io.WritableComparable {
+  private java.io.ByteArrayOutputStream mData;
+  private java.util.BitSet bs_;
+  public RecBuffer() {
+    bs_ = new java.util.BitSet(2);
+    bs_.set(1);
+  }
+  public RecBuffer(
+        java.io.ByteArrayOutputStream m0) {
+    bs_ = new java.util.BitSet(2);
+    bs_.set(1);
+    mData=m0; bs_.set(0);
+  }
+  public java.io.ByteArrayOutputStream getData() {
+    return mData;
+  }
+  public void setData(java.io.ByteArrayOutputStream m_) {
+    mData=m_; bs_.set(0);
+  }
+  public void serialize(org.apache.hadoop.record.OutputArchive a_, String tag) throws java.io.IOException {
+    if (!validate()) throw new java.io.IOException("All fields not set:");
+    a_.startRecord(this,tag);
+    a_.writeBuffer(mData,"Data");
+    bs_.clear(0);
+    a_.endRecord(this,tag);
+  }
+  public void deserialize(org.apache.hadoop.record.InputArchive a_, String tag) throws java.io.IOException {
+    a_.startRecord(tag);
+    mData=a_.readBuffer("Data");
+    bs_.set(0);
+    a_.endRecord(tag);
+}
+  public String toString() {
+    try {
+      java.io.ByteArrayOutputStream s =
+        new java.io.ByteArrayOutputStream();
+      org.apache.hadoop.record.CsvOutputArchive a_ = 
+        new org.apache.hadoop.record.CsvOutputArchive(s);
+      a_.startRecord(this,"");
+    a_.writeBuffer(mData,"Data");
+      a_.endRecord(this,"");
+      return new String(s.toByteArray(), "UTF-8");
+    } catch (Throwable ex) {
+      ex.printStackTrace();
+    }
+    return "ERROR";
+  }
+  public void write(java.io.DataOutput out) throws java.io.IOException {
+    org.apache.hadoop.record.BinaryOutputArchive archive = new org.apache.hadoop.record.BinaryOutputArchive(out);
+    serialize(archive, "");
+  }
+  public void readFields(java.io.DataInput in) throws java.io.IOException {
+    org.apache.hadoop.record.BinaryInputArchive archive = new org.apache.hadoop.record.BinaryInputArchive(in);
+    deserialize(archive, "");
+  }
+  private boolean validate() {
+    if (bs_.cardinality() != bs_.length()) return false;
+    return true;
+}
+  public int compareTo (Object peer_) throws ClassCastException {
+    if (!(peer_ instanceof RecBuffer)) {
+      throw new ClassCastException("Comparing different types of records.");
+    }
+    RecBuffer peer = (RecBuffer) peer_;
+    int ret = 0;
+    if (ret != 0) return ret;
+     return ret;
+  }
+  public boolean equals(Object peer_) {
+    if (!(peer_ instanceof RecBuffer)) {
+      return false;
+    }
+    if (peer_ == this) {
+      return true;
+    }
+    RecBuffer peer = (RecBuffer) peer_;
+    boolean ret = false;
+    ret = org.apache.hadoop.record.Utils.bufEquals(mData,peer.mData);
+    if (!ret) return ret;
+     return ret;
+  }
+  public int hashCode() {
+    int result = 17;
+    int ret;
+    ret = mData.toString().hashCode();
+    result = 37*result + ret;
+    return result;
+  }
+  public static String signature() {
+    return "LRecBuffer(B)";
+  }
+}

+ 96 - 0
src/test/org/apache/hadoop/record/test/RecInt.java

@@ -0,0 +1,96 @@
+// File generated by hadoop record compiler. Do not edit.
+package org.apache.hadoop.record.test;
+
+public class RecInt implements org.apache.hadoop.record.Record, org.apache.hadoop.io.WritableComparable {
+  private int mData;
+  private java.util.BitSet bs_;
+  public RecInt() {
+    bs_ = new java.util.BitSet(2);
+    bs_.set(1);
+  }
+  public RecInt(
+        int m0) {
+    bs_ = new java.util.BitSet(2);
+    bs_.set(1);
+    mData=m0; bs_.set(0);
+  }
+  public int getData() {
+    return mData;
+  }
+  public void setData(int m_) {
+    mData=m_; bs_.set(0);
+  }
+  public void serialize(org.apache.hadoop.record.OutputArchive a_, String tag) throws java.io.IOException {
+    if (!validate()) throw new java.io.IOException("All fields not set:");
+    a_.startRecord(this,tag);
+    a_.writeInt(mData,"Data");
+    bs_.clear(0);
+    a_.endRecord(this,tag);
+  }
+  public void deserialize(org.apache.hadoop.record.InputArchive a_, String tag) throws java.io.IOException {
+    a_.startRecord(tag);
+    mData=a_.readInt("Data");
+    bs_.set(0);
+    a_.endRecord(tag);
+}
+  public String toString() {
+    try {
+      java.io.ByteArrayOutputStream s =
+        new java.io.ByteArrayOutputStream();
+      org.apache.hadoop.record.CsvOutputArchive a_ = 
+        new org.apache.hadoop.record.CsvOutputArchive(s);
+      a_.startRecord(this,"");
+    a_.writeInt(mData,"Data");
+      a_.endRecord(this,"");
+      return new String(s.toByteArray(), "UTF-8");
+    } catch (Throwable ex) {
+      ex.printStackTrace();
+    }
+    return "ERROR";
+  }
+  public void write(java.io.DataOutput out) throws java.io.IOException {
+    org.apache.hadoop.record.BinaryOutputArchive archive = new org.apache.hadoop.record.BinaryOutputArchive(out);
+    serialize(archive, "");
+  }
+  public void readFields(java.io.DataInput in) throws java.io.IOException {
+    org.apache.hadoop.record.BinaryInputArchive archive = new org.apache.hadoop.record.BinaryInputArchive(in);
+    deserialize(archive, "");
+  }
+  private boolean validate() {
+    if (bs_.cardinality() != bs_.length()) return false;
+    return true;
+}
+  public int compareTo (Object peer_) throws ClassCastException {
+    if (!(peer_ instanceof RecInt)) {
+      throw new ClassCastException("Comparing different types of records.");
+    }
+    RecInt peer = (RecInt) peer_;
+    int ret = 0;
+    ret = (mData == peer.mData)? 0 :((mData<peer.mData)?-1:1);
+    if (ret != 0) return ret;
+     return ret;
+  }
+  public boolean equals(Object peer_) {
+    if (!(peer_ instanceof RecInt)) {
+      return false;
+    }
+    if (peer_ == this) {
+      return true;
+    }
+    RecInt peer = (RecInt) peer_;
+    boolean ret = false;
+    ret = (mData==peer.mData);
+    if (!ret) return ret;
+     return ret;
+  }
+  public int hashCode() {
+    int result = 17;
+    int ret;
+    ret = (int)mData;
+    result = 37*result + ret;
+    return result;
+  }
+  public static String signature() {
+    return "LRecInt(i)";
+  }
+}

+ 96 - 0
src/test/org/apache/hadoop/record/test/RecRecord0.java

@@ -0,0 +1,96 @@
+// File generated by hadoop record compiler. Do not edit.
+package org.apache.hadoop.record.test;
+
+public class RecRecord0 implements org.apache.hadoop.record.Record, org.apache.hadoop.io.WritableComparable {
+  private String mStringVal;
+  private java.util.BitSet bs_;
+  public RecRecord0() {
+    bs_ = new java.util.BitSet(2);
+    bs_.set(1);
+  }
+  public RecRecord0(
+        String m0) {
+    bs_ = new java.util.BitSet(2);
+    bs_.set(1);
+    mStringVal=m0; bs_.set(0);
+  }
+  public String getStringVal() {
+    return mStringVal;
+  }
+  public void setStringVal(String m_) {
+    mStringVal=m_; bs_.set(0);
+  }
+  public void serialize(org.apache.hadoop.record.OutputArchive a_, String tag) throws java.io.IOException {
+    if (!validate()) throw new java.io.IOException("All fields not set:");
+    a_.startRecord(this,tag);
+    a_.writeString(mStringVal,"StringVal");
+    bs_.clear(0);
+    a_.endRecord(this,tag);
+  }
+  public void deserialize(org.apache.hadoop.record.InputArchive a_, String tag) throws java.io.IOException {
+    a_.startRecord(tag);
+    mStringVal=a_.readString("StringVal");
+    bs_.set(0);
+    a_.endRecord(tag);
+}
+  public String toString() {
+    try {
+      java.io.ByteArrayOutputStream s =
+        new java.io.ByteArrayOutputStream();
+      org.apache.hadoop.record.CsvOutputArchive a_ = 
+        new org.apache.hadoop.record.CsvOutputArchive(s);
+      a_.startRecord(this,"");
+    a_.writeString(mStringVal,"StringVal");
+      a_.endRecord(this,"");
+      return new String(s.toByteArray(), "UTF-8");
+    } catch (Throwable ex) {
+      ex.printStackTrace();
+    }
+    return "ERROR";
+  }
+  public void write(java.io.DataOutput out) throws java.io.IOException {
+    org.apache.hadoop.record.BinaryOutputArchive archive = new org.apache.hadoop.record.BinaryOutputArchive(out);
+    serialize(archive, "");
+  }
+  public void readFields(java.io.DataInput in) throws java.io.IOException {
+    org.apache.hadoop.record.BinaryInputArchive archive = new org.apache.hadoop.record.BinaryInputArchive(in);
+    deserialize(archive, "");
+  }
+  private boolean validate() {
+    if (bs_.cardinality() != bs_.length()) return false;
+    return true;
+}
+  public int compareTo (Object peer_) throws ClassCastException {
+    if (!(peer_ instanceof RecRecord0)) {
+      throw new ClassCastException("Comparing different types of records.");
+    }
+    RecRecord0 peer = (RecRecord0) peer_;
+    int ret = 0;
+    ret = mStringVal.compareTo(peer.mStringVal);
+    if (ret != 0) return ret;
+     return ret;
+  }
+  public boolean equals(Object peer_) {
+    if (!(peer_ instanceof RecRecord0)) {
+      return false;
+    }
+    if (peer_ == this) {
+      return true;
+    }
+    RecRecord0 peer = (RecRecord0) peer_;
+    boolean ret = false;
+    ret = mStringVal.equals(peer.mStringVal);
+    if (!ret) return ret;
+     return ret;
+  }
+  public int hashCode() {
+    int result = 17;
+    int ret;
+    ret = mStringVal.hashCode();
+    result = 37*result + ret;
+    return result;
+  }
+  public static String signature() {
+    return "LRecRecord0(s)";
+  }
+}

+ 331 - 0
src/test/org/apache/hadoop/record/test/RecRecord1.java

@@ -0,0 +1,331 @@
+// File generated by hadoop record compiler. Do not edit.
+package org.apache.hadoop.record.test;
+
+public class RecRecord1 implements org.apache.hadoop.record.Record, org.apache.hadoop.io.WritableComparable {
+  private boolean mBoolVal;
+  private byte mByteVal;
+  private int mIntVal;
+  private long mLongVal;
+  private float mFloatVal;
+  private double mDoubleVal;
+  private String mStringVal;
+  private java.io.ByteArrayOutputStream mBufferVal;
+  private java.util.ArrayList mVectorVal;
+  private java.util.TreeMap mMapVal;
+  private java.util.BitSet bs_;
+  public RecRecord1() {
+    bs_ = new java.util.BitSet(11);
+    bs_.set(10);
+  }
+  public RecRecord1(
+        boolean m0,
+        byte m1,
+        int m2,
+        long m3,
+        float m4,
+        double m5,
+        String m6,
+        java.io.ByteArrayOutputStream m7,
+        java.util.ArrayList m8,
+        java.util.TreeMap m9) {
+    bs_ = new java.util.BitSet(11);
+    bs_.set(10);
+    mBoolVal=m0; bs_.set(0);
+    mByteVal=m1; bs_.set(1);
+    mIntVal=m2; bs_.set(2);
+    mLongVal=m3; bs_.set(3);
+    mFloatVal=m4; bs_.set(4);
+    mDoubleVal=m5; bs_.set(5);
+    mStringVal=m6; bs_.set(6);
+    mBufferVal=m7; bs_.set(7);
+    mVectorVal=m8; bs_.set(8);
+    mMapVal=m9; bs_.set(9);
+  }
+  public boolean getBoolVal() {
+    return mBoolVal;
+  }
+  public void setBoolVal(boolean m_) {
+    mBoolVal=m_; bs_.set(0);
+  }
+  public byte getByteVal() {
+    return mByteVal;
+  }
+  public void setByteVal(byte m_) {
+    mByteVal=m_; bs_.set(1);
+  }
+  public int getIntVal() {
+    return mIntVal;
+  }
+  public void setIntVal(int m_) {
+    mIntVal=m_; bs_.set(2);
+  }
+  public long getLongVal() {
+    return mLongVal;
+  }
+  public void setLongVal(long m_) {
+    mLongVal=m_; bs_.set(3);
+  }
+  public float getFloatVal() {
+    return mFloatVal;
+  }
+  public void setFloatVal(float m_) {
+    mFloatVal=m_; bs_.set(4);
+  }
+  public double getDoubleVal() {
+    return mDoubleVal;
+  }
+  public void setDoubleVal(double m_) {
+    mDoubleVal=m_; bs_.set(5);
+  }
+  public String getStringVal() {
+    return mStringVal;
+  }
+  public void setStringVal(String m_) {
+    mStringVal=m_; bs_.set(6);
+  }
+  public java.io.ByteArrayOutputStream getBufferVal() {
+    return mBufferVal;
+  }
+  public void setBufferVal(java.io.ByteArrayOutputStream m_) {
+    mBufferVal=m_; bs_.set(7);
+  }
+  public java.util.ArrayList getVectorVal() {
+    return mVectorVal;
+  }
+  public void setVectorVal(java.util.ArrayList m_) {
+    mVectorVal=m_; bs_.set(8);
+  }
+  public java.util.TreeMap getMapVal() {
+    return mMapVal;
+  }
+  public void setMapVal(java.util.TreeMap m_) {
+    mMapVal=m_; bs_.set(9);
+  }
+  public void serialize(org.apache.hadoop.record.OutputArchive a_, String tag) throws java.io.IOException {
+    if (!validate()) throw new java.io.IOException("All fields not set:");
+    a_.startRecord(this,tag);
+    a_.writeBool(mBoolVal,"BoolVal");
+    bs_.clear(0);
+    a_.writeByte(mByteVal,"ByteVal");
+    bs_.clear(1);
+    a_.writeInt(mIntVal,"IntVal");
+    bs_.clear(2);
+    a_.writeLong(mLongVal,"LongVal");
+    bs_.clear(3);
+    a_.writeFloat(mFloatVal,"FloatVal");
+    bs_.clear(4);
+    a_.writeDouble(mDoubleVal,"DoubleVal");
+    bs_.clear(5);
+    a_.writeString(mStringVal,"StringVal");
+    bs_.clear(6);
+    a_.writeBuffer(mBufferVal,"BufferVal");
+    bs_.clear(7);
+    {
+      a_.startVector(mVectorVal,"VectorVal");
+      int len1 = mVectorVal.size();
+      for(int vidx1 = 0; vidx1<len1; vidx1++) {
+        String e1 = (String) mVectorVal.get(vidx1);
+        a_.writeString(e1,"e1");
+      }
+      a_.endVector(mVectorVal,"VectorVal");
+    }
+    bs_.clear(8);
+    {
+      a_.startMap(mMapVal,"MapVal");
+      java.util.Set es1 = mMapVal.entrySet();
+      for(java.util.Iterator midx1 = es1.iterator(); midx1.hasNext(); ) {
+        java.util.Map.Entry me1 = (java.util.Map.Entry) midx1.next();
+        String k1 = (String) me1.getKey();
+        String v1 = (String) me1.getValue();
+        a_.writeString(k1,"k1");
+        a_.writeString(v1,"v1");
+      }
+      a_.endMap(mMapVal,"MapVal");
+    }
+    bs_.clear(9);
+    a_.endRecord(this,tag);
+  }
+  public void deserialize(org.apache.hadoop.record.InputArchive a_, String tag) throws java.io.IOException {
+    a_.startRecord(tag);
+    mBoolVal=a_.readBool("BoolVal");
+    bs_.set(0);
+    mByteVal=a_.readByte("ByteVal");
+    bs_.set(1);
+    mIntVal=a_.readInt("IntVal");
+    bs_.set(2);
+    mLongVal=a_.readLong("LongVal");
+    bs_.set(3);
+    mFloatVal=a_.readFloat("FloatVal");
+    bs_.set(4);
+    mDoubleVal=a_.readDouble("DoubleVal");
+    bs_.set(5);
+    mStringVal=a_.readString("StringVal");
+    bs_.set(6);
+    mBufferVal=a_.readBuffer("BufferVal");
+    bs_.set(7);
+    {
+      org.apache.hadoop.record.Index vidx1 = a_.startVector("VectorVal");
+      mVectorVal=new java.util.ArrayList();
+      for (; !vidx1.done(); vidx1.incr()) {
+    String e1;
+        e1=a_.readString("e1");
+        mVectorVal.add(e1);
+      }
+    a_.endVector("VectorVal");
+    }
+    bs_.set(8);
+    {
+      org.apache.hadoop.record.Index midx1 = a_.startMap("MapVal");
+      mMapVal=new java.util.TreeMap();
+      for (; !midx1.done(); midx1.incr()) {
+    String k1;
+        k1=a_.readString("k1");
+    String v1;
+        v1=a_.readString("v1");
+        mMapVal.put(k1,v1);
+      }
+    a_.endMap("MapVal");
+    }
+    bs_.set(9);
+    a_.endRecord(tag);
+}
+  public String toString() {
+    try {
+      java.io.ByteArrayOutputStream s =
+        new java.io.ByteArrayOutputStream();
+      org.apache.hadoop.record.CsvOutputArchive a_ = 
+        new org.apache.hadoop.record.CsvOutputArchive(s);
+      a_.startRecord(this,"");
+    a_.writeBool(mBoolVal,"BoolVal");
+    a_.writeByte(mByteVal,"ByteVal");
+    a_.writeInt(mIntVal,"IntVal");
+    a_.writeLong(mLongVal,"LongVal");
+    a_.writeFloat(mFloatVal,"FloatVal");
+    a_.writeDouble(mDoubleVal,"DoubleVal");
+    a_.writeString(mStringVal,"StringVal");
+    a_.writeBuffer(mBufferVal,"BufferVal");
+    {
+      a_.startVector(mVectorVal,"VectorVal");
+      int len1 = mVectorVal.size();
+      for(int vidx1 = 0; vidx1<len1; vidx1++) {
+        String e1 = (String) mVectorVal.get(vidx1);
+        a_.writeString(e1,"e1");
+      }
+      a_.endVector(mVectorVal,"VectorVal");
+    }
+    {
+      a_.startMap(mMapVal,"MapVal");
+      java.util.Set es1 = mMapVal.entrySet();
+      for(java.util.Iterator midx1 = es1.iterator(); midx1.hasNext(); ) {
+        java.util.Map.Entry me1 = (java.util.Map.Entry) midx1.next();
+        String k1 = (String) me1.getKey();
+        String v1 = (String) me1.getValue();
+        a_.writeString(k1,"k1");
+        a_.writeString(v1,"v1");
+      }
+      a_.endMap(mMapVal,"MapVal");
+    }
+      a_.endRecord(this,"");
+      return new String(s.toByteArray(), "UTF-8");
+    } catch (Throwable ex) {
+      ex.printStackTrace();
+    }
+    return "ERROR";
+  }
+  public void write(java.io.DataOutput out) throws java.io.IOException {
+    org.apache.hadoop.record.BinaryOutputArchive archive = new org.apache.hadoop.record.BinaryOutputArchive(out);
+    serialize(archive, "");
+  }
+  public void readFields(java.io.DataInput in) throws java.io.IOException {
+    org.apache.hadoop.record.BinaryInputArchive archive = new org.apache.hadoop.record.BinaryInputArchive(in);
+    deserialize(archive, "");
+  }
+  private boolean validate() {
+    if (bs_.cardinality() != bs_.length()) return false;
+    return true;
+}
+  public int compareTo (Object peer_) throws ClassCastException {
+    if (!(peer_ instanceof RecRecord1)) {
+      throw new ClassCastException("Comparing different types of records.");
+    }
+    RecRecord1 peer = (RecRecord1) peer_;
+    int ret = 0;
+    ret = (mBoolVal == peer.mBoolVal)? 0 : (mBoolVal?1:-1);
+    if (ret != 0) return ret;
+    ret = (mByteVal == peer.mByteVal)? 0 :((mByteVal<peer.mByteVal)?-1:1);
+    if (ret != 0) return ret;
+    ret = (mIntVal == peer.mIntVal)? 0 :((mIntVal<peer.mIntVal)?-1:1);
+    if (ret != 0) return ret;
+    ret = (mLongVal == peer.mLongVal)? 0 :((mLongVal<peer.mLongVal)?-1:1);
+    if (ret != 0) return ret;
+    ret = (mFloatVal == peer.mFloatVal)? 0 :((mFloatVal<peer.mFloatVal)?-1:1);
+    if (ret != 0) return ret;
+    ret = (mDoubleVal == peer.mDoubleVal)? 0 :((mDoubleVal<peer.mDoubleVal)?-1:1);
+    if (ret != 0) return ret;
+    ret = mStringVal.compareTo(peer.mStringVal);
+    if (ret != 0) return ret;
+    if (ret != 0) return ret;
+    if (ret != 0) return ret;
+    if (ret != 0) return ret;
+     return ret;
+  }
+  public boolean equals(Object peer_) {
+    if (!(peer_ instanceof RecRecord1)) {
+      return false;
+    }
+    if (peer_ == this) {
+      return true;
+    }
+    RecRecord1 peer = (RecRecord1) peer_;
+    boolean ret = false;
+    ret = (mBoolVal==peer.mBoolVal);
+    if (!ret) return ret;
+    ret = (mByteVal==peer.mByteVal);
+    if (!ret) return ret;
+    ret = (mIntVal==peer.mIntVal);
+    if (!ret) return ret;
+    ret = (mLongVal==peer.mLongVal);
+    if (!ret) return ret;
+    ret = (mFloatVal==peer.mFloatVal);
+    if (!ret) return ret;
+    ret = (mDoubleVal==peer.mDoubleVal);
+    if (!ret) return ret;
+    ret = mStringVal.equals(peer.mStringVal);
+    if (!ret) return ret;
+    ret = org.apache.hadoop.record.Utils.bufEquals(mBufferVal,peer.mBufferVal);
+    if (!ret) return ret;
+    ret = mVectorVal.equals(peer.mVectorVal);
+    if (!ret) return ret;
+    ret = mMapVal.equals(peer.mMapVal);
+    if (!ret) return ret;
+     return ret;
+  }
+  public int hashCode() {
+    int result = 17;
+    int ret;
+     ret = (mBoolVal)?0:1;
+    result = 37*result + ret;
+    ret = (int)mByteVal;
+    result = 37*result + ret;
+    ret = (int)mIntVal;
+    result = 37*result + ret;
+    ret = (int) (mLongVal^(mLongVal>>>32));
+    result = 37*result + ret;
+    ret = Float.floatToIntBits(mFloatVal);
+    result = 37*result + ret;
+    ret = (int)(Double.doubleToLongBits(mDoubleVal)^(Double.doubleToLongBits(mDoubleVal)>>>32));
+    result = 37*result + ret;
+    ret = mStringVal.hashCode();
+    result = 37*result + ret;
+    ret = mBufferVal.toString().hashCode();
+    result = 37*result + ret;
+    ret = mVectorVal.hashCode();
+    result = 37*result + ret;
+    ret = mMapVal.hashCode();
+    result = 37*result + ret;
+    return result;
+  }
+  public static String signature() {
+    return "LRecRecord1(zbilfdsB[s]{ss})";
+  }
+}

+ 96 - 0
src/test/org/apache/hadoop/record/test/RecString.java

@@ -0,0 +1,96 @@
+// File generated by hadoop record compiler. Do not edit.
+package org.apache.hadoop.record.test;
+
+public class RecString implements org.apache.hadoop.record.Record, org.apache.hadoop.io.WritableComparable {
+  private String mData;
+  private java.util.BitSet bs_;
+  public RecString() {
+    bs_ = new java.util.BitSet(2);
+    bs_.set(1);
+  }
+  public RecString(
+        String m0) {
+    bs_ = new java.util.BitSet(2);
+    bs_.set(1);
+    mData=m0; bs_.set(0);
+  }
+  public String getData() {
+    return mData;
+  }
+  public void setData(String m_) {
+    mData=m_; bs_.set(0);
+  }
+  public void serialize(org.apache.hadoop.record.OutputArchive a_, String tag) throws java.io.IOException {
+    if (!validate()) throw new java.io.IOException("All fields not set:");
+    a_.startRecord(this,tag);
+    a_.writeString(mData,"Data");
+    bs_.clear(0);
+    a_.endRecord(this,tag);
+  }
+  public void deserialize(org.apache.hadoop.record.InputArchive a_, String tag) throws java.io.IOException {
+    a_.startRecord(tag);
+    mData=a_.readString("Data");
+    bs_.set(0);
+    a_.endRecord(tag);
+}
+  public String toString() {
+    try {
+      java.io.ByteArrayOutputStream s =
+        new java.io.ByteArrayOutputStream();
+      org.apache.hadoop.record.CsvOutputArchive a_ = 
+        new org.apache.hadoop.record.CsvOutputArchive(s);
+      a_.startRecord(this,"");
+    a_.writeString(mData,"Data");
+      a_.endRecord(this,"");
+      return new String(s.toByteArray(), "UTF-8");
+    } catch (Throwable ex) {
+      ex.printStackTrace();
+    }
+    return "ERROR";
+  }
+  public void write(java.io.DataOutput out) throws java.io.IOException {
+    org.apache.hadoop.record.BinaryOutputArchive archive = new org.apache.hadoop.record.BinaryOutputArchive(out);
+    serialize(archive, "");
+  }
+  public void readFields(java.io.DataInput in) throws java.io.IOException {
+    org.apache.hadoop.record.BinaryInputArchive archive = new org.apache.hadoop.record.BinaryInputArchive(in);
+    deserialize(archive, "");
+  }
+  private boolean validate() {
+    if (bs_.cardinality() != bs_.length()) return false;
+    return true;
+}
+  public int compareTo (Object peer_) throws ClassCastException {
+    if (!(peer_ instanceof RecString)) {
+      throw new ClassCastException("Comparing different types of records.");
+    }
+    RecString peer = (RecString) peer_;
+    int ret = 0;
+    ret = mData.compareTo(peer.mData);
+    if (ret != 0) return ret;
+     return ret;
+  }
+  public boolean equals(Object peer_) {
+    if (!(peer_ instanceof RecString)) {
+      return false;
+    }
+    if (peer_ == this) {
+      return true;
+    }
+    RecString peer = (RecString) peer_;
+    boolean ret = false;
+    ret = mData.equals(peer.mData);
+    if (!ret) return ret;
+     return ret;
+  }
+  public int hashCode() {
+    int result = 17;
+    int ret;
+    ret = mData.hashCode();
+    result = 37*result + ret;
+    return result;
+  }
+  public static String signature() {
+    return "LRecString(s)";
+  }
+}

+ 452 - 0
src/test/org/apache/hadoop/record/test/TestMapRed.java

@@ -0,0 +1,452 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.test;
+
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.*;
+import junit.framework.TestCase;
+import java.io.*;
+import java.util.*;
+
+
+/**********************************************************
+ * MapredLoadTest generates a bunch of work that exercises
+ * a Hadoop Map-Reduce system (and DFS, too).  It goes through
+ * the following steps:
+ *
+ * 1) Take inputs 'range' and 'counts'.
+ * 2) Generate 'counts' random integers between 0 and range-1.
+ * 3) Create a file that lists each integer between 0 and range-1,
+ *    and lists the number of times that integer was generated.
+ * 4) Emit a (very large) file that contains all the integers
+ *    in the order generated.
+ * 5) After the file has been generated, read it back and count
+ *    how many times each int was generated.
+ * 6) Compare this big count-map against the original one.  If
+ *    they match, then SUCCESS!  Otherwise, FAILURE!
+ *
+ * OK, that's how we can think about it.  What are the map-reduce
+ * steps that get the job done?
+ *
+ * 1) In a non-mapred thread, take the inputs 'range' and 'counts'.
+ * 2) In a non-mapread thread, generate the answer-key and write to disk.
+ * 3) In a mapred job, divide the answer key into K jobs.
+ * 4) A mapred 'generator' task consists of K map jobs.  Each reads
+ *    an individual "sub-key", and generates integers according to
+ *    to it (though with a random ordering).
+ * 5) The generator's reduce task agglomerates all of those files
+ *    into a single one.
+ * 6) A mapred 'reader' task consists of M map jobs.  The output
+ *    file is cut into M pieces. Each of the M jobs counts the 
+ *    individual ints in its chunk and creates a map of all seen ints.
+ * 7) A mapred job integrates all the count files into a single one.
+ *
+ **********************************************************/
+public class TestMapRed extends TestCase {
+    /**
+     * Modified to make it a junit test.
+     * The RandomGen Job does the actual work of creating
+     * a huge file of assorted numbers.  It receives instructions
+     * as to how many times each number should be counted.  Then
+     * it emits those numbers in a crazy order.
+     *
+     * The map() function takes a key/val pair that describes
+     * a value-to-be-emitted (the key) and how many times it 
+     * should be emitted (the value), aka "numtimes".  map() then
+     * emits a series of intermediate key/val pairs.  It emits
+     * 'numtimes' of these.  The key is a random number and the
+     * value is the 'value-to-be-emitted'.
+     *
+     * The system collates and merges these pairs according to
+     * the random number.  reduce() function takes in a key/value
+     * pair that consists of a crazy random number and a series
+     * of values that should be emitted.  The random number key
+     * is now dropped, and reduce() emits a pair for every intermediate value.
+     * The emitted key is an intermediate value.  The emitted value
+     * is just a blank string.  Thus, we've created a huge file
+     * of numbers in random order, but where each number appears
+     * as many times as we were instructed.
+     */
+    static public class RandomGenMapper implements Mapper {
+        Random r = new Random();
+        public void configure(JobConf job) {
+        }
+
+        public void map(WritableComparable key, Writable val, OutputCollector out, Reporter reporter) throws IOException {
+            int randomVal = ((RecInt) key).getData();
+            int randomCount = ((RecInt) val).getData();
+
+            for (int i = 0; i < randomCount; i++) {
+                out.collect(new RecInt(Math.abs(r.nextInt())),
+                        new RecString(Integer.toString(randomVal)));
+            }
+        }
+        public void close() {
+        }
+    }
+    /**
+     */
+    static public class RandomGenReducer implements Reducer {
+        public void configure(JobConf job) {
+        }
+
+        public void reduce(WritableComparable key,
+                Iterator it,
+                OutputCollector out,
+                Reporter reporter)
+                throws IOException {
+            int keyint = ((RecInt) key).getData();
+            while (it.hasNext()) {
+                String val = ((RecString) it.next()).getData();
+                out.collect(new RecInt(Integer.parseInt(val)),
+                        new RecString(""));
+            }
+        }
+        public void close() {
+        }
+    }
+
+    /**
+     * The RandomCheck Job does a lot of our work.  It takes
+     * in a num/string keyspace, and transforms it into a
+     * key/count(int) keyspace.
+     *
+     * The map() function just emits a num/1 pair for every
+     * num/string input pair.
+     *
+     * The reduce() function sums up all the 1s that were
+     * emitted for a single key.  It then emits the key/total
+     * pair.
+     *
+     * This is used to regenerate the random number "answer key".
+     * Each key here is a random number, and the count is the
+     * number of times the number was emitted.
+     */
+    static public class RandomCheckMapper implements Mapper {
+        public void configure(JobConf job) {
+        }
+
+        public void map(WritableComparable key, Writable val, OutputCollector out, Reporter reporter) throws IOException {
+            int pos = ((RecInt) key).getData();
+            String str = ((RecString) val).getData();
+            out.collect(new RecInt(pos), new RecString("1"));
+        }
+        public void close() {
+        }
+    }
+    /**
+     */
+    static public class RandomCheckReducer implements Reducer {
+        public void configure(JobConf job) {
+        }
+        
+        public void reduce(WritableComparable key, Iterator it, OutputCollector out, Reporter reporter) throws IOException {
+            int keyint = ((RecInt) key).getData();
+            int count = 0;
+            while (it.hasNext()) {
+                it.next();
+                count++;
+            }
+            out.collect(new RecInt(keyint), new RecString(Integer.toString(count)));
+        }
+        public void close() {
+        }
+    }
+
+    /**
+     * The Merge Job is a really simple one.  It takes in
+     * an int/int key-value set, and emits the same set.
+     * But it merges identical keys by adding their values.
+     *
+     * Thus, the map() function is just the identity function
+     * and reduce() just sums.  Nothing to see here!
+     */
+    static public class MergeMapper implements Mapper {
+        public void configure(JobConf job) {
+        }
+
+        public void map(WritableComparable key, Writable val, OutputCollector out, Reporter reporter) throws IOException {
+            int keyint = ((RecInt) key).getData();
+            String valstr = ((RecString) val).getData();
+            out.collect(new RecInt(keyint), new RecInt(Integer.parseInt(valstr)));
+        }
+        public void close() {
+        }
+    }
+    static public class MergeReducer implements Reducer {
+        public void configure(JobConf job) {
+        }
+        
+        public void reduce(WritableComparable key, Iterator it, OutputCollector out, Reporter reporter) throws IOException {
+            int keyint = ((RecInt) key).getData();
+            int total = 0;
+            while (it.hasNext()) {
+                total += ((RecInt) it.next()).getData();
+            }
+            out.collect(new RecInt(keyint), new RecInt(total));
+        }
+        public void close() {
+        }
+    }
+
+    private static int range = 10;
+    private static int counts = 100;
+    private static Random r = new Random();
+    private static Configuration conf = new Configuration();
+
+    /**
+       public TestMapRed(int range, int counts, Configuration conf) throws IOException {
+       this.range = range;
+       this.counts = counts;
+       this.conf = conf;
+       }
+    **/
+
+    public void testMapred() throws Exception {
+	launch();
+    }
+
+    /**
+     * 
+     */
+    public static void launch() throws Exception {
+        //
+        // Generate distribution of ints.  This is the answer key.
+        //
+        int countsToGo = counts;
+        int dist[] = new int[range];
+        for (int i = 0; i < range; i++) {
+            double avgInts = (1.0 * countsToGo) / (range - i);
+            dist[i] = (int) Math.max(0, Math.round(avgInts + (Math.sqrt(avgInts) * r.nextGaussian())));
+            countsToGo -= dist[i];
+        }
+        if (countsToGo > 0) {
+            dist[dist.length-1] += countsToGo;
+        }
+
+        //
+        // Write the answer key to a file.  
+        //
+        FileSystem fs = FileSystem.get(conf);
+        File testdir = new File("mapred.loadtest");
+        fs.mkdirs(testdir);
+
+        File randomIns = new File(testdir, "genins");
+        fs.mkdirs(randomIns);
+
+        File answerkey = new File(randomIns, "answer.key");
+        SequenceFile.Writer out = new SequenceFile.Writer(fs, answerkey.getPath(), RecInt.class, RecInt.class);
+        try {
+            for (int i = 0; i < range; i++) {
+                RecInt k = new RecInt();
+                RecInt v = new RecInt();
+                k.setData(i);
+                v.setData(dist[i]);
+                out.append(k, v);
+            }
+        } finally {
+            out.close();
+        }
+
+        //
+        // Now we need to generate the random numbers according to
+        // the above distribution.
+        //
+        // We create a lot of map tasks, each of which takes at least
+        // one "line" of the distribution.  (That is, a certain number
+        // X is to be generated Y number of times.)
+        //
+        // A map task emits Y key/val pairs.  The val is X.  The key
+        // is a randomly-generated number.
+        //
+        // The reduce task gets its input sorted by key.  That is, sorted
+        // in random order.  It then emits a single line of text that
+        // for the given values.  It does not emit the key.
+        //
+        // Because there's just one reduce task, we emit a single big
+        // file of random numbers.
+        //
+        File randomOuts = new File(testdir, "genouts");
+        //fs.mkdirs(randomOuts);
+
+
+        JobConf genJob = new JobConf(conf);
+        genJob.setInputDir(randomIns);
+        genJob.setInputKeyClass(RecInt.class);
+        genJob.setInputValueClass(RecInt.class);
+        genJob.setInputFormat(SequenceFileInputFormat.class);
+        genJob.setMapperClass(RandomGenMapper.class);
+
+        genJob.setOutputDir(randomOuts);
+        genJob.setOutputKeyClass(RecInt.class);
+        genJob.setOutputValueClass(RecString.class);
+        genJob.setOutputFormat(SequenceFileOutputFormat.class);
+        genJob.setReducerClass(RandomGenReducer.class);
+        genJob.setNumReduceTasks(1);
+
+        JobClient.runJob(genJob);
+
+        //
+        // Next, we read the big file in and regenerate the 
+        // original map.  It's split into a number of parts.
+        // (That number is 'intermediateReduces'.)
+        //
+        // We have many map tasks, each of which read at least one
+        // of the output numbers.  For each number read in, the
+        // map task emits a key/value pair where the key is the
+        // number and the value is "1".
+        //
+        // We have a single reduce task, which receives its input
+        // sorted by the key emitted above.  For each key, there will
+        // be a certain number of "1" values.  The reduce task sums
+        // these values to compute how many times the given key was
+        // emitted.
+        //
+        // The reduce task then emits a key/val pair where the key
+        // is the number in question, and the value is the number of
+        // times the key was emitted.  This is the same format as the
+        // original answer key (except that numbers emitted zero times
+        // will not appear in the regenerated key.)  The answer set
+        // is split into a number of pieces.  A final MapReduce job
+        // will merge them.
+        //
+        // There's not really a need to go to 10 reduces here 
+        // instead of 1.  But we want to test what happens when
+        // you have multiple reduces at once.
+        //
+        int intermediateReduces = 10;
+        File intermediateOuts = new File(testdir, "intermediateouts");
+        //fs.mkdirs(intermediateOuts);
+        JobConf checkJob = new JobConf(conf);
+        checkJob.setInputDir(randomOuts);
+        checkJob.setInputKeyClass(RecInt.class);
+        checkJob.setInputValueClass(RecString.class);
+        checkJob.setInputFormat(SequenceFileInputFormat.class);
+        checkJob.setMapperClass(RandomCheckMapper.class);
+
+        checkJob.setOutputDir(intermediateOuts);
+        checkJob.setOutputKeyClass(RecInt.class);
+        checkJob.setOutputValueClass(RecString.class);
+        checkJob.setOutputFormat(SequenceFileOutputFormat.class);
+        checkJob.setReducerClass(RandomCheckReducer.class);
+        checkJob.setNumReduceTasks(intermediateReduces);
+
+        JobClient.runJob(checkJob);
+
+        //
+        // OK, now we take the output from the last job and
+        // merge it down to a single file.  The map() and reduce()
+        // functions don't really do anything except reemit tuples.
+        // But by having a single reduce task here, we end up merging
+        // all the files.
+        //
+        File finalOuts = new File(testdir, "finalouts");        
+        //fs.mkdirs(finalOuts);
+        JobConf mergeJob = new JobConf(conf);
+        mergeJob.setInputDir(intermediateOuts);
+        mergeJob.setInputKeyClass(RecInt.class);
+        mergeJob.setInputValueClass(RecString.class);
+        mergeJob.setInputFormat(SequenceFileInputFormat.class);
+        mergeJob.setMapperClass(MergeMapper.class);
+        
+        mergeJob.setOutputDir(finalOuts);
+        mergeJob.setOutputKeyClass(RecInt.class);
+        mergeJob.setOutputValueClass(RecInt.class);
+        mergeJob.setOutputFormat(SequenceFileOutputFormat.class);
+        mergeJob.setReducerClass(MergeReducer.class);
+        mergeJob.setNumReduceTasks(1);
+        
+        JobClient.runJob(mergeJob);
+        
+ 
+        //
+        // Finally, we compare the reconstructed answer key with the
+        // original one.  Remember, we need to ignore zero-count items
+        // in the original key.
+        //
+        boolean success = true;
+        File recomputedkey = new File(finalOuts, "part-00000");
+        SequenceFile.Reader in = new SequenceFile.Reader(fs, recomputedkey.getPath(), conf);
+        int totalseen = 0;
+        try {
+            RecInt key = new RecInt();
+            RecInt val = new RecInt();            
+            for (int i = 0; i < range; i++) {
+                if (dist[i] == 0) {
+                    continue;
+                }
+                if (! in.next(key, val)) {
+                    System.err.println("Cannot read entry " + i);
+                    success = false;
+                    break;
+                } else {
+                    if ( !((key.getData() == i ) && (val.getData() == dist[i]))) {
+                        System.err.println("Mismatch!  Pos=" + key.getData() + ", i=" + i + ", val=" + val.getData() + ", dist[i]=" + dist[i]);
+                        success = false;
+                    }
+                    totalseen += val.getData();
+                }
+            }
+            if (success) {
+                if (in.next(key, val)) {
+                    System.err.println("Unnecessary lines in recomputed key!");
+                    success = false;
+                }
+            }
+        } finally {
+            in.close();
+        }
+        int originalTotal = 0;
+        for (int i = 0; i < dist.length; i++) {
+            originalTotal += dist[i];
+        }
+        System.out.println("Original sum: " + originalTotal);
+        System.out.println("Recomputed sum: " + totalseen);
+
+        //
+        // Write to "results" whether the test succeeded or not.
+        //
+        File resultFile = new File(testdir, "results");
+        BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(fs.create(resultFile)));
+        try {
+            bw.write("Success=" + success + "\n");
+            System.out.println("Success=" + success);            
+        } finally {
+            bw.close();
+        }
+	fs.delete(testdir);
+    }
+
+    /**
+     * Launches all the tasks in order.
+     */
+    public static void main(String[] argv) throws Exception {
+        if (argv.length < 2) {
+            System.err.println("Usage: TestMapRed <range> <counts>");
+            System.err.println();
+            System.err.println("Note: a good test will have a <counts> value that is substantially larger than the <range>");
+            return;
+        }
+
+        int i = 0;
+        int range = Integer.parseInt(argv[i++]);
+        int counts = Integer.parseInt(argv[i++]);
+	launch();
+    }
+}

+ 138 - 0
src/test/org/apache/hadoop/record/test/TestRecordIO.java

@@ -0,0 +1,138 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.test;
+
+import java.io.IOException;
+import junit.framework.*;
+import org.apache.hadoop.record.RecordWriter;
+import org.apache.hadoop.record.RecordReader;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.util.ArrayList;
+import java.util.TreeMap;
+
+/**
+ *
+ * @author milindb
+ */
+public class TestRecordIO extends TestCase {
+    
+    public TestRecordIO(String testName) {
+        super(testName);
+    }
+
+    protected void setUp() throws Exception {
+    }
+
+    protected void tearDown() throws Exception {
+    }
+    
+    public void testBinary() {
+        File tmpfile;
+        try {
+            tmpfile = File.createTempFile("hadooprec", ".dat");
+            FileOutputStream ostream = new FileOutputStream(tmpfile);
+            RecordWriter out = new RecordWriter(ostream, "binary");
+            RecRecord1 r1 = new RecRecord1();
+            r1.setBoolVal(true);
+            r1.setByteVal((byte)0x66);
+            r1.setFloatVal(3.145F);
+            r1.setDoubleVal(1.5234);
+            r1.setIntVal(4567);
+            r1.setLongVal(0x5a5a5a5a5a5aL);
+            r1.setStringVal("random text");
+            r1.setBufferVal(new ByteArrayOutputStream(20));
+            r1.setVectorVal(new ArrayList());
+            r1.setMapVal(new TreeMap());
+            out.write(r1);
+            ostream.close();
+            FileInputStream istream = new FileInputStream(tmpfile);
+            RecordReader in = new RecordReader(istream, "binary");
+            RecRecord1 r2 = new RecRecord1();
+            in.read(r2);
+            istream.close();
+            tmpfile.delete();
+            assertTrue("Serialized and deserialized records do not match.", r1.equals(r2));
+        } catch (IOException ex) {
+            ex.printStackTrace();
+        } 
+    }
+    
+    public void testCsv() {
+        File tmpfile;
+        try {
+            tmpfile = File.createTempFile("hadooprec", ".txt");
+            FileOutputStream ostream = new FileOutputStream(tmpfile);
+            RecordWriter out = new RecordWriter(ostream, "csv");
+            RecRecord1 r1 = new RecRecord1();
+            r1.setBoolVal(true);
+            r1.setByteVal((byte)0x66);
+            r1.setFloatVal(3.145F);
+            r1.setDoubleVal(1.5234);
+            r1.setIntVal(4567);
+            r1.setLongVal(0x5a5a5a5a5a5aL);
+            r1.setStringVal("random text");
+            r1.setBufferVal(new ByteArrayOutputStream(20));
+            r1.setVectorVal(new ArrayList());
+            r1.setMapVal(new TreeMap());
+            out.write(r1);
+            ostream.close();
+            FileInputStream istream = new FileInputStream(tmpfile);
+            RecordReader in = new RecordReader(istream, "csv");
+            RecRecord1 r2 = new RecRecord1();
+            in.read(r2);
+            istream.close();
+            tmpfile.delete();
+            assertTrue("Serialized and deserialized records do not match.", r1.equals(r2));
+        } catch (IOException ex) {
+            ex.printStackTrace();
+        } 
+    }
+
+    public void testXml() {
+        File tmpfile;
+        try {
+            tmpfile = File.createTempFile("hadooprec", ".xml");
+            FileOutputStream ostream = new FileOutputStream(tmpfile);
+            RecordWriter out = new RecordWriter(ostream, "xml");
+            RecRecord1 r1 = new RecRecord1();
+            r1.setBoolVal(true);
+            r1.setByteVal((byte)0x66);
+            r1.setFloatVal(3.145F);
+            r1.setDoubleVal(1.5234);
+            r1.setIntVal(4567);
+            r1.setLongVal(0x5a5a5a5a5a5aL);
+            r1.setStringVal("random text");
+            r1.setBufferVal(new ByteArrayOutputStream(20));
+            r1.setVectorVal(new ArrayList());
+            r1.setMapVal(new TreeMap());
+            out.write(r1);
+            ostream.close();
+            FileInputStream istream = new FileInputStream(tmpfile);
+            RecordReader in = new RecordReader(istream, "xml");
+            RecRecord1 r2 = new RecRecord1();
+            in.read(r2);
+            istream.close();
+            tmpfile.delete();
+            assertTrue("Serialized and deserialized records do not match.", r1.equals(r2));
+        } catch (IOException ex) {
+            ex.printStackTrace();
+        } 
+    }
+}

+ 126 - 0
src/test/org/apache/hadoop/record/test/TestWritable.java

@@ -0,0 +1,126 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.test;
+
+import java.io.*;
+import java.util.*;
+import junit.framework.TestCase;
+import java.util.logging.*;
+
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputFormatBase;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.SequenceFileInputFormat;
+
+public class TestWritable extends TestCase {
+  private static final Logger LOG = InputFormatBase.LOG;
+
+  private static int MAX_LENGTH = 10000;
+  private static Configuration conf = new Configuration();
+
+  public void testFormat() throws Exception {
+    JobConf job = new JobConf(conf);
+    FileSystem fs = FileSystem.getNamed("local", conf);
+    Path dir = new Path(System.getProperty("test.build.data",".") + "/mapred");
+    Path file = new Path(dir, "test.seq");
+    
+    Reporter reporter = new Reporter() {
+        public void setStatus(String status) throws IOException {}
+      };
+    
+    int seed = new Random().nextInt();
+    //LOG.info("seed = "+seed);
+    Random random = new Random(seed);
+
+    fs.delete(dir);
+
+    job.setInputPath(dir);
+
+    // for a variety of lengths
+    for (int length = 0; length < MAX_LENGTH;
+         length+= random.nextInt(MAX_LENGTH/10)+1) {
+
+      //LOG.info("creating; entries = " + length);
+
+      // create a file with length entries
+      SequenceFile.Writer writer =
+        new SequenceFile.Writer(fs, file,
+                                RecInt.class, RecBuffer.class);
+      try {
+        for (int i = 0; i < length; i++) {
+          RecInt key = new RecInt();
+          key.setData(i);
+          byte[] data = new byte[random.nextInt(10)];
+          random.nextBytes(data);
+          RecBuffer value = new RecBuffer();
+          ByteArrayOutputStream strm = new ByteArrayOutputStream(data.length);
+          strm.write(data);
+          value.setData(strm);
+          writer.append(key, value);
+        }
+      } finally {
+        writer.close();
+      }
+
+      // try splitting the file in a variety of sizes
+      InputFormat format = new SequenceFileInputFormat();
+      RecInt key = new RecInt();
+      RecBuffer value = new RecBuffer();
+      for (int i = 0; i < 3; i++) {
+        int numSplits =
+          random.nextInt(MAX_LENGTH/(SequenceFile.SYNC_INTERVAL/20))+1;
+        //LOG.info("splitting: requesting = " + numSplits);
+        FileSplit[] splits = format.getSplits(fs, job, numSplits);
+        //LOG.info("splitting: got =        " + splits.length);
+
+        // check each split
+        BitSet bits = new BitSet(length);
+        for (int j = 0; j < splits.length; j++) {
+          RecordReader reader =
+            format.getRecordReader(fs, splits[j], job, reporter);
+          try {
+            int count = 0;
+            while (reader.next(key, value)) {
+              // if (bits.get(key.get())) {
+              // LOG.info("splits["+j+"]="+splits[j]+" : " + key.get());
+              // LOG.info("@"+reader.getPos());
+              // }
+              assertFalse("Key in multiple partitions.", bits.get(key.getData()));
+              bits.set(key.getData());
+              count++;
+            }
+            //LOG.info("splits["+j+"]="+splits[j]+" count=" + count);
+          } finally {
+            reader.close();
+          }
+        }
+        assertEquals("Some keys in no partition.", length, bits.cardinality());
+      }
+
+    }
+  }
+
+  public static void main(String[] args) throws Exception {
+    new TestWritable().testFormat();
+  }
+}

+ 117 - 0
src/test/org/apache/hadoop/record/test/ToCpp.java

@@ -0,0 +1,117 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.record.test;
+
+import java.io.IOException;
+import junit.framework.*;
+import org.apache.hadoop.record.RecordWriter;
+import org.apache.hadoop.record.RecordReader;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.util.ArrayList;
+import java.util.TreeMap;
+
+/**
+ *
+ * @author milindb
+ */
+public class ToCpp extends TestCase {
+    
+    public ToCpp(String testName) {
+        super(testName);
+    }
+
+    protected void setUp() throws Exception {
+    }
+
+    protected void tearDown() throws Exception {
+    }
+    
+    public void testBinary() {
+        File tmpfile;
+        try {
+            tmpfile = new File("/temp/hadooptemp.dat");
+            FileOutputStream ostream = new FileOutputStream(tmpfile);
+            RecordWriter out = new RecordWriter(ostream, "binary");
+            RecRecord1 r1 = new RecRecord1();
+            r1.setBoolVal(true);
+            r1.setByteVal((byte)0x66);
+            r1.setFloatVal(3.145F);
+            r1.setDoubleVal(1.5234);
+            r1.setIntVal(4567);
+            r1.setLongVal(0x5a5a5a5a5a5aL);
+            r1.setStringVal("random text");
+            r1.setBufferVal(new ByteArrayOutputStream(20));
+            r1.setVectorVal(new ArrayList());
+            r1.setMapVal(new TreeMap());
+            out.write(r1);
+            ostream.close();
+        } catch (IOException ex) {
+            ex.printStackTrace();
+        } 
+    }
+    
+    public void testCsv() {
+        File tmpfile;
+        try {
+            tmpfile = new File("/temp/hadooptemp.txt");
+            FileOutputStream ostream = new FileOutputStream(tmpfile);
+            RecordWriter out = new RecordWriter(ostream, "csv");
+            RecRecord1 r1 = new RecRecord1();
+            r1.setBoolVal(true);
+            r1.setByteVal((byte)0x66);
+            r1.setFloatVal(3.145F);
+            r1.setDoubleVal(1.5234);
+            r1.setIntVal(4567);
+            r1.setLongVal(0x5a5a5a5a5a5aL);
+            r1.setStringVal("random text");
+            r1.setBufferVal(new ByteArrayOutputStream(20));
+            r1.setVectorVal(new ArrayList());
+            r1.setMapVal(new TreeMap());
+            out.write(r1);
+            ostream.close();
+        } catch (IOException ex) {
+            ex.printStackTrace();
+        } 
+    }
+
+    public void testXml() {
+        File tmpfile;
+        try {
+            tmpfile = new File("/temp/hadooptemp.xml");
+            FileOutputStream ostream = new FileOutputStream(tmpfile);
+            RecordWriter out = new RecordWriter(ostream, "xml");
+            RecRecord1 r1 = new RecRecord1();
+            r1.setBoolVal(true);
+            r1.setByteVal((byte)0x66);
+            r1.setFloatVal(3.145F);
+            r1.setDoubleVal(1.5234);
+            r1.setIntVal(4567);
+            r1.setLongVal(0x5a5a5a5a5a5aL);
+            r1.setStringVal("random text");
+            r1.setBufferVal(new ByteArrayOutputStream(20));
+            r1.setVectorVal(new ArrayList());
+            r1.setMapVal(new TreeMap());
+            out.write(r1);
+            ostream.close();
+        } catch (IOException ex) {
+            ex.printStackTrace();
+        } 
+    }
+}