You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cu...@apache.org on 2006/05/04 04:04:07 UTC

svn commit: r399509 [1/5] - in /lucene/hadoop/trunk: ./ bin/ src/c++/ src/c++/librecordio/ src/c++/librecordio/test/ src/java/org/apache/hadoop/record/ src/java/org/apache/hadoop/record/compiler/ src/java/org/apache/hadoop/record/compiler/generated/ sr...

Author: cutting
Date: Wed May  3 19:04:01 2006
New Revision: 399509

URL: http://svn.apache.org/viewcvs?rev=399509&view=rev
Log:
HADOOP-65.  Initial version of multi-language record system.  Contributed by Milind Bhandarkar.

Added:
    lucene/hadoop/trunk/bin/rcc   (with props)
    lucene/hadoop/trunk/src/c++/
    lucene/hadoop/trunk/src/c++/librecordio/
    lucene/hadoop/trunk/src/c++/librecordio/Makefile
    lucene/hadoop/trunk/src/c++/librecordio/archive.hh
    lucene/hadoop/trunk/src/c++/librecordio/binarchive.cc
    lucene/hadoop/trunk/src/c++/librecordio/binarchive.hh
    lucene/hadoop/trunk/src/c++/librecordio/csvarchive.cc
    lucene/hadoop/trunk/src/c++/librecordio/csvarchive.hh
    lucene/hadoop/trunk/src/c++/librecordio/exception.cc
    lucene/hadoop/trunk/src/c++/librecordio/exception.hh
    lucene/hadoop/trunk/src/c++/librecordio/filestream.cc
    lucene/hadoop/trunk/src/c++/librecordio/filestream.hh
    lucene/hadoop/trunk/src/c++/librecordio/recordio.cc
    lucene/hadoop/trunk/src/c++/librecordio/recordio.hh
    lucene/hadoop/trunk/src/c++/librecordio/test/
    lucene/hadoop/trunk/src/c++/librecordio/test/Makefile
    lucene/hadoop/trunk/src/c++/librecordio/test/test.cc
    lucene/hadoop/trunk/src/c++/librecordio/test/test.hh
    lucene/hadoop/trunk/src/c++/librecordio/test/test.jr
    lucene/hadoop/trunk/src/c++/librecordio/test/testFromJava.cc
    lucene/hadoop/trunk/src/c++/librecordio/test/testFromJava.hh
    lucene/hadoop/trunk/src/c++/librecordio/xmlarchive.cc
    lucene/hadoop/trunk/src/c++/librecordio/xmlarchive.hh
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/BinaryInputArchive.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/BinaryOutputArchive.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/CsvInputArchive.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/CsvOutputArchive.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/Index.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/InputArchive.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/OutputArchive.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/Record.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/RecordReader.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/RecordWriter.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/Utils.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/XmlInputArchive.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/XmlOutputArchive.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/CppGenerator.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/JBoolean.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/JBuffer.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/JByte.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/JCompType.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/JDouble.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/JField.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/JFile.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/JFloat.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/JInt.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/JLong.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/JMap.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/JRecord.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/JString.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/JType.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/JVector.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/JavaGenerator.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/ParseException.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/Rcc.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/RccConstants.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/RccTokenManager.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/SimpleCharStream.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/Token.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/TokenMgrError.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/rcc.jj
    lucene/hadoop/trunk/src/java/org/apache/hadoop/record/package.html
    lucene/hadoop/trunk/src/test/ddl/
    lucene/hadoop/trunk/src/test/ddl/buffer.jr
    lucene/hadoop/trunk/src/test/ddl/int.jr
    lucene/hadoop/trunk/src/test/ddl/links.jr
    lucene/hadoop/trunk/src/test/ddl/location.jr
    lucene/hadoop/trunk/src/test/ddl/string.jr
    lucene/hadoop/trunk/src/test/ddl/test.jr
    lucene/hadoop/trunk/src/test/org/apache/hadoop/record/
    lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/
    lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/FromCpp.java
    lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/RecBuffer.java
    lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/RecInt.java
    lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/RecRecord0.java
    lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/RecRecord1.java
    lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/RecString.java
    lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/TestMapRed.java
    lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/TestRecordIO.java
    lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/TestWritable.java
    lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/ToCpp.java
Modified:
    lucene/hadoop/trunk/CHANGES.txt
    lucene/hadoop/trunk/build.xml

Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/CHANGES.txt?rev=399509&r1=399508&r2=399509&view=diff
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Wed May  3 19:04:01 2006
@@ -171,6 +171,12 @@
     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
 

Added: lucene/hadoop/trunk/bin/rcc
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/bin/rcc?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/bin/rcc (added)
+++ lucene/hadoop/trunk/bin/rcc Wed May  3 19:04:01 2006
@@ -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 "$@"

Propchange: lucene/hadoop/trunk/bin/rcc
------------------------------------------------------------------------------
    svn:executable = *

Modified: lucene/hadoop/trunk/build.xml
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/build.xml?rev=399509&r1=399508&r2=399509&view=diff
==============================================================================
--- lucene/hadoop/trunk/build.xml (original)
+++ lucene/hadoop/trunk/build.xml Wed May  3 19:04:01 2006
@@ -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"

Added: lucene/hadoop/trunk/src/c++/librecordio/Makefile
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/Makefile?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/Makefile (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/Makefile Wed May  3 19:04:01 2006
@@ -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

Added: lucene/hadoop/trunk/src/c++/librecordio/archive.hh
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/archive.hh?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/archive.hh (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/archive.hh Wed May  3 19:04:01 2006
@@ -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_*/

Added: lucene/hadoop/trunk/src/c++/librecordio/binarchive.cc
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/binarchive.cc?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/binarchive.cc (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/binarchive.cc Wed May  3 19:04:01 2006
@@ -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()
+{
+}

Added: lucene/hadoop/trunk/src/c++/librecordio/binarchive.hh
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/binarchive.hh?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/binarchive.hh (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/binarchive.hh Wed May  3 19:04:01 2006
@@ -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_*/

Added: lucene/hadoop/trunk/src/c++/librecordio/csvarchive.cc
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/csvarchive.cc?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/csvarchive.cc (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/csvarchive.cc Wed May  3 19:04:01 2006
@@ -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()
+{
+}

Added: lucene/hadoop/trunk/src/c++/librecordio/csvarchive.hh
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/csvarchive.hh?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/csvarchive.hh (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/csvarchive.hh Wed May  3 19:04:01 2006
@@ -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_*/

Added: lucene/hadoop/trunk/src/c++/librecordio/exception.cc
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/exception.cc?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/exception.cc (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/exception.cc Wed May  3 19:04:01 2006
@@ -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);
+  }
+
+}

Added: lucene/hadoop/trunk/src/c++/librecordio/exception.hh
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/exception.hh?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/exception.hh (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/exception.hh Wed May  3 19:04:01 2006
@@ -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

Added: lucene/hadoop/trunk/src/c++/librecordio/filestream.cc
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/filestream.cc?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/filestream.cc (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/filestream.cc Wed May  3 19:04:01 2006
@@ -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();
+  }
+}

Added: lucene/hadoop/trunk/src/c++/librecordio/filestream.hh
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/filestream.hh?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/filestream.hh (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/filestream.hh Wed May  3 19:04:01 2006
@@ -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_*/

Added: lucene/hadoop/trunk/src/c++/librecordio/recordio.cc
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/recordio.cc?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/recordio.cc (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/recordio.cc Wed May  3 19:04:01 2006
@@ -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);
+}
+

Added: lucene/hadoop/trunk/src/c++/librecordio/recordio.hh
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/recordio.hh?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/recordio.hh (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/recordio.hh Wed May  3 19:04:01 2006
@@ -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_*/
+

Added: lucene/hadoop/trunk/src/c++/librecordio/test/Makefile
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/test/Makefile?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/test/Makefile (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/test/Makefile Wed May  3 19:04:01 2006
@@ -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.*
+

Added: lucene/hadoop/trunk/src/c++/librecordio/test/test.cc
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/test/test.cc?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/test/test.cc (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/test/test.cc Wed May  3 19:04:01 2006
@@ -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;
+}
+

Added: lucene/hadoop/trunk/src/c++/librecordio/test/test.hh
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/test/test.hh?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/test/test.hh (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/test/test.hh Wed May  3 19:04:01 2006
@@ -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_*/

Added: lucene/hadoop/trunk/src/c++/librecordio/test/test.jr
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/test/test.jr?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/test/test.jr (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/test/test.jr Wed May  3 19:04:01 2006
@@ -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;
+    }
+}
+

Added: lucene/hadoop/trunk/src/c++/librecordio/test/testFromJava.cc
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/test/testFromJava.cc?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/test/testFromJava.cc (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/test/testFromJava.cc Wed May  3 19:04:01 2006
@@ -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;
+}
+

Added: lucene/hadoop/trunk/src/c++/librecordio/test/testFromJava.hh
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/test/testFromJava.hh?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/test/testFromJava.hh (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/test/testFromJava.hh Wed May  3 19:04:01 2006
@@ -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_*/

Added: lucene/hadoop/trunk/src/c++/librecordio/xmlarchive.cc
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/c%2B%2B/librecordio/xmlarchive.cc?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/c++/librecordio/xmlarchive.cc (added)
+++ lucene/hadoop/trunk/src/c++/librecordio/xmlarchive.cc Wed May  3 19:04:01 2006
@@ -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()
+{
+}