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 [5/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...

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

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

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

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

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

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

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

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

Added: lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/TestWritable.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/TestWritable.java?rev=399509&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/TestWritable.java (added)
+++ lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/TestWritable.java 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.
+ */
+
+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();
+  }
+}

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