avro-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From cutt...@apache.org
Subject svn commit: r787828 - in /hadoop/avro/trunk: ./ src/java/org/apache/avro/ src/java/org/apache/avro/file/ src/java/org/apache/avro/generic/ src/java/org/apache/avro/io/ src/java/org/apache/avro/ipc/ src/java/org/apache/avro/reflect/ src/test/java/org/ap...
Date Tue, 23 Jun 2009 20:38:47 GMT
Author: cutting
Date: Tue Jun 23 20:38:46 2009
New Revision: 787828

URL: http://svn.apache.org/viewvc?rev=787828&view=rev
Log:
AVRO-57.  Make ValueReader/Writer abstract, named Decoder/Encoder.  Add concrete implementations named BinaryDecoder/Encoder.

Added:
    hadoop/avro/trunk/src/java/org/apache/avro/io/BinaryDecoder.java   (contents, props changed)
      - copied, changed from r786682, hadoop/avro/trunk/src/java/org/apache/avro/io/ValueReader.java
    hadoop/avro/trunk/src/java/org/apache/avro/io/BinaryEncoder.java   (contents, props changed)
      - copied, changed from r786682, hadoop/avro/trunk/src/java/org/apache/avro/io/ValueWriter.java
    hadoop/avro/trunk/src/java/org/apache/avro/io/BlockingBinaryEncoder.java   (contents, props changed)
      - copied, changed from r786682, hadoop/avro/trunk/src/java/org/apache/avro/io/BlockingValueWriter.java
    hadoop/avro/trunk/src/java/org/apache/avro/io/Decoder.java
    hadoop/avro/trunk/src/java/org/apache/avro/io/Encoder.java
    hadoop/avro/trunk/src/test/java/org/apache/avro/io/TestBinaryDecoder.java   (contents, props changed)
      - copied, changed from r786682, hadoop/avro/trunk/src/test/java/org/apache/avro/io/TestValueReader.java
Removed:
    hadoop/avro/trunk/src/java/org/apache/avro/io/BlockingValueWriter.java
    hadoop/avro/trunk/src/java/org/apache/avro/io/ValueReader.java
    hadoop/avro/trunk/src/java/org/apache/avro/io/ValueWriter.java
    hadoop/avro/trunk/src/test/java/org/apache/avro/io/TestValueReader.java
Modified:
    hadoop/avro/trunk/CHANGES.txt
    hadoop/avro/trunk/src/java/org/apache/avro/file/DataFileReader.java
    hadoop/avro/trunk/src/java/org/apache/avro/file/DataFileWriter.java
    hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumReader.java
    hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumWriter.java
    hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericRequestor.java
    hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericResponder.java
    hadoop/avro/trunk/src/java/org/apache/avro/io/DatumReader.java
    hadoop/avro/trunk/src/java/org/apache/avro/io/DatumWriter.java
    hadoop/avro/trunk/src/java/org/apache/avro/ipc/Requestor.java
    hadoop/avro/trunk/src/java/org/apache/avro/ipc/Responder.java
    hadoop/avro/trunk/src/java/org/apache/avro/package.html
    hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectDatumWriter.java
    hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectRequestor.java
    hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectResponder.java
    hadoop/avro/trunk/src/test/java/org/apache/avro/TestSchema.java
    hadoop/avro/trunk/src/test/java/org/apache/avro/io/TestBlockingIO.java

Modified: hadoop/avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/CHANGES.txt?rev=787828&r1=787827&r2=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/CHANGES.txt (original)
+++ hadoop/avro/trunk/CHANGES.txt Tue Jun 23 20:38:46 2009
@@ -11,6 +11,10 @@
 
     AVRO-2. Optimized RPC handshake protocol for Java.  (cutting)
 
+    AVRO-57. Make ValueWriter an abstract class named Encoder and make
+    ValueReader an abstract class named Decoder, and add concrete
+    implementations named BinaryEncoder and BinaryDecoder. (cutting)
+
   NEW FEATURES
 
     AVRO-6. Permit easier implementation of alternate generic data

Modified: hadoop/avro/trunk/src/java/org/apache/avro/file/DataFileReader.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/file/DataFileReader.java?rev=787828&r1=787827&r2=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/file/DataFileReader.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/file/DataFileReader.java Tue Jun 23 20:38:46 2009
@@ -28,7 +28,8 @@
 
 import org.apache.avro.Schema;
 import org.apache.avro.io.DatumReader;
-import org.apache.avro.io.ValueReader;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.BinaryDecoder;
 
 /** Read files written by {@link DataFileWriter}.
  * @see DataFileWriter
@@ -38,7 +39,7 @@
   private Schema schema;
   private DatumReader<D> reader;
   private SeekableBufferedInput in;
-  private ValueReader vin;
+  private Decoder vin;
 
   private Map<String,byte[]> meta = new HashMap<String,byte[]>();
 
@@ -61,7 +62,7 @@
     in.seek(length-4);
     int footerSize=(in.read()<<24)+(in.read()<<16)+(in.read()<<8)+in.read();
     in.seek(length-footerSize);
-    this.vin = new ValueReader(in);
+    this.vin = new BinaryDecoder(in);
     long l = vin.readMapStart();
     if (l > 0) {
       do {

Modified: hadoop/avro/trunk/src/java/org/apache/avro/file/DataFileWriter.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/file/DataFileWriter.java?rev=787828&r1=787827&r2=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/file/DataFileWriter.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/file/DataFileWriter.java Tue Jun 23 20:38:46 2009
@@ -30,7 +30,8 @@
 
 import org.apache.avro.Schema;
 import org.apache.avro.io.DatumWriter;
-import org.apache.avro.io.ValueWriter;
+import org.apache.avro.io.Encoder;
+import org.apache.avro.io.BinaryEncoder;
 
 /** Stores in a file a sequence of data conforming to a schema.  The schema is
  * stored in the file with the data.  Each datum in a file is of the same
@@ -53,7 +54,7 @@
   private DatumWriter<D> dout;
 
   private BufferedFileOutputStream out;
-  private ValueWriter vout;
+  private Encoder vout;
 
   private Map<String,byte[]> meta = new HashMap<String,byte[]>();
 
@@ -62,7 +63,7 @@
 
   private ByteArrayOutputStream buffer =
     new ByteArrayOutputStream(SYNC_INTERVAL*2);
-  private ValueWriter bufOut = new ValueWriter(buffer);
+  private Encoder bufOut = new BinaryEncoder(buffer);
 
   private byte[] sync;                          // 16 random bytes
   {
@@ -81,7 +82,7 @@
                         DatumWriter<D> dout) throws IOException {
     this.schema = schema;
     this.out = new BufferedFileOutputStream(outs);
-    this.vout = new ValueWriter(out);
+    this.vout = new BinaryEncoder(out);
     this.dout = dout;
     
     dout.setSchema(schema);

Modified: hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumReader.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumReader.java?rev=787828&r1=787827&r2=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumReader.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumReader.java Tue Jun 23 20:38:46 2009
@@ -32,7 +32,7 @@
 import org.apache.avro.Schema.Field;
 import org.apache.avro.Schema.Type;
 import org.apache.avro.io.DatumReader;
-import org.apache.avro.io.ValueReader;
+import org.apache.avro.io.Decoder;
 import org.apache.avro.util.Utf8;
 
 /** {@link DatumReader} for generic Java objects. */
@@ -54,13 +54,13 @@
   public void setSchema(Schema actual) { this.actual = actual; }
 
   @SuppressWarnings("unchecked")
-  public D read(D reuse, ValueReader in) throws IOException {
+  public D read(D reuse, Decoder in) throws IOException {
     return (D) read(reuse, actual, expected != null ? expected : actual, in);
   }
   
   /** Called to read data.*/
   protected Object read(Object old, Schema actual,
-                        Schema expected, ValueReader in) throws IOException {
+                        Schema expected, Decoder in) throws IOException {
     if (actual.getType() == Type.UNION)           // resolve unions
       actual = actual.getTypes().get((int)in.readIndex());
     if (expected.getType() == Type.UNION)
@@ -125,7 +125,7 @@
   /** Called to read a record instance. May be overridden for alternate record
    * representations.*/
   protected Object readRecord(Object old, Schema actual, Schema expected,
-                              ValueReader in) throws IOException {
+                              Decoder in) throws IOException {
     /* TODO: We may want to compute the expected and actual mapping and cache
      * the mapping (keyed by <actual, expected>). */
     String recordName = expected.getName();
@@ -243,7 +243,7 @@
 
   /** Called to read an enum value. May be overridden for alternate enum
    * representations.  By default, returns the symbol as a String. */
-  protected Object readEnum(Schema actual, Schema expected, ValueReader in)
+  protected Object readEnum(Schema actual, Schema expected, Decoder in)
     throws IOException {
     String name = expected.getName();
     if (name != null && !name.equals(actual.getName()))
@@ -258,7 +258,7 @@
   /** Called to read an array instance.  May be overridden for alternate array
    * representations.*/
   protected Object readArray(Object old, Schema actual, Schema expected,
-                             ValueReader in) throws IOException {
+                             Decoder in) throws IOException {
     Schema actualType = actual.getElementType();
     Schema expectedType = expected.getElementType();
     long l = in.readArrayStart();
@@ -294,7 +294,7 @@
   /** Called to read a map instance.  May be overridden for alternate map
    * representations.*/
   protected Object readMap(Object old, Schema actual, Schema expected,
-                           ValueReader in) throws IOException {
+                           Decoder in) throws IOException {
     Schema aValue = actual.getValueType();
     Schema eValue = expected.getValueType();
     long l = in.readMapStart();
@@ -321,7 +321,7 @@
   /** Called to read a fixed value. May be overridden for alternate fixed
    * representations.  By default, returns {@link GenericFixed}. */
   protected Object readFixed(Object old, Schema actual, Schema expected,
-                             ValueReader in)
+                             Decoder in)
     throws IOException {
     if (!actual.equals(expected))
       throw new AvroTypeException("Expected "+expected+", found "+actual);
@@ -387,8 +387,8 @@
 
   /** Called to read strings.  Subclasses may override to use a different
    * string representation.  By default, this calls {@link
-   * ValueReader#readString(Utf8)}.*/
-  protected Object readString(Object old, ValueReader in) throws IOException {
+   * Decoder#readString(Utf8)}.*/
+  protected Object readString(Object old, Decoder in) throws IOException {
     return in.readString((Utf8)old);
   }
 
@@ -399,8 +399,8 @@
 
   /** Called to read byte arrays.  Subclasses may override to use a different
    * byte array representation.  By default, this calls {@link
-   * ValueReader#readBytes(ByteBuffer)}.*/
-  protected Object readBytes(Object old, ValueReader in) throws IOException {
+   * Decoder#readBytes(ByteBuffer)}.*/
+  protected Object readBytes(Object old, Decoder in) throws IOException {
     return in.readBytes((ByteBuffer)old);
   }
 
@@ -412,7 +412,7 @@
   private static final Schema STRING_SCHEMA = Schema.create(Type.STRING);
 
   /** Skip an instance of a schema. */
-  public static void skip(Schema schema, ValueReader in) throws IOException {
+  public static void skip(Schema schema, Decoder in) throws IOException {
     switch (schema.getType()) {
     case RECORD:
       for (Map.Entry<String, Schema> entry : schema.getFieldSchemas())

Modified: hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumWriter.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumWriter.java?rev=787828&r1=787827&r2=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumWriter.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumWriter.java Tue Jun 23 20:38:46 2009
@@ -28,7 +28,7 @@
 import org.apache.avro.Schema;
 import org.apache.avro.Schema.Field;
 import org.apache.avro.io.DatumWriter;
-import org.apache.avro.io.ValueWriter;
+import org.apache.avro.io.Encoder;
 import org.apache.avro.util.Utf8;
 
 /** {@link DatumWriter} for generic Java objects. */
@@ -43,12 +43,12 @@
 
   public void setSchema(Schema root) { this.root = root; }
 
-  public void write(D datum, ValueWriter out) throws IOException {
+  public void write(D datum, Encoder out) throws IOException {
     write(root, datum, out);
   }
   
   /** Called to write data.*/
-  protected void write(Schema schema, Object datum, ValueWriter out)
+  protected void write(Schema schema, Object datum, Encoder out)
     throws IOException {
     switch (schema.getType()) {
     case RECORD: writeRecord(schema, datum, out); break;
@@ -75,7 +75,7 @@
 
   /** Called to write a record.  May be overridden for alternate record
    * representations.*/
-  protected void writeRecord(Schema schema, Object datum, ValueWriter out)
+  protected void writeRecord(Schema schema, Object datum, Encoder out)
     throws IOException {
     for (Entry<String, Field> entry : schema.getFields().entrySet()) {
       Field field = entry.getValue();
@@ -92,14 +92,14 @@
   
   /** Called to write an enum value.  May be overridden for alternate enum
    * representations.*/
-  protected void writeEnum(Schema schema, Object datum, ValueWriter out)
+  protected void writeEnum(Schema schema, Object datum, Encoder out)
     throws IOException {
     out.writeEnum(schema.getEnumOrdinal((String)datum));
   }
   
   /** Called to write a array.  May be overridden for alternate array
    * representations.*/
-  protected void writeArray(Schema schema, Object datum, ValueWriter out)
+  protected void writeArray(Schema schema, Object datum, Encoder out)
     throws IOException {
     Schema element = schema.getElementType();
     long size = getArraySize(datum);
@@ -129,7 +129,7 @@
   
   /** Called to write a map.  May be overridden for alternate map
    * representations.*/
-  protected void writeMap(Schema schema, Object datum, ValueWriter out)
+  protected void writeMap(Schema schema, Object datum, Encoder out)
     throws IOException {
     Schema value = schema.getValueType();
     int size = getMapSize(datum);
@@ -159,13 +159,13 @@
   
   /** Called to write a string.  May be overridden for alternate string
    * representations.*/
-  protected void writeString(Object datum, ValueWriter out) throws IOException {
+  protected void writeString(Object datum, Encoder out) throws IOException {
     out.writeString((Utf8)datum);
   }
 
   /** Called to write a bytes.  May be overridden for alternate bytes
    * representations.*/
-  protected void writeBytes(Object datum, ValueWriter out) throws IOException {
+  protected void writeBytes(Object datum, Encoder out) throws IOException {
     out.writeBytes((ByteBuffer)datum);
   }
 
@@ -205,7 +205,7 @@
 
   /** Called to write a fixed value.  May be overridden for alternate fixed
    * representations.*/
-  protected void writeFixed(Schema schema, Object datum, ValueWriter out)
+  protected void writeFixed(Schema schema, Object datum, Encoder out)
     throws IOException {
     out.writeFixed(((GenericFixed)datum).bytes(), 0, schema.getFixedSize());
   }

Modified: hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericRequestor.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericRequestor.java?rev=787828&r1=787827&r2=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericRequestor.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericRequestor.java Tue Jun 23 20:38:46 2009
@@ -22,8 +22,8 @@
 
 import org.apache.avro.Protocol;
 import org.apache.avro.Schema;
-import org.apache.avro.io.ValueReader;
-import org.apache.avro.io.ValueWriter;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.Encoder;
 import org.apache.avro.ipc.AvroRemoteException;
 import org.apache.avro.ipc.Requestor;
 import org.apache.avro.ipc.Transceiver;
@@ -40,16 +40,16 @@
     super(protocol, transceiver);
   }
 
-  public void writeRequest(Schema schema, Object request, ValueWriter out)
+  public void writeRequest(Schema schema, Object request, Encoder out)
     throws IOException {
     new GenericDatumWriter<Object>(schema).write(request, out);
   }
 
-  public Object readResponse(Schema schema, ValueReader in) throws IOException {
+  public Object readResponse(Schema schema, Decoder in) throws IOException {
     return new GenericDatumReader<Object>(schema).read(null, in);
   }
 
-  public AvroRemoteException readError(Schema schema, ValueReader in)
+  public AvroRemoteException readError(Schema schema, Decoder in)
     throws IOException {
     return new AvroRemoteException(new GenericDatumReader<Object>(schema).read(null,in));
   }

Modified: hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericResponder.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericResponder.java?rev=787828&r1=787827&r2=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericResponder.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericResponder.java Tue Jun 23 20:38:46 2009
@@ -22,8 +22,8 @@
 
 import org.apache.avro.Protocol;
 import org.apache.avro.Schema;
-import org.apache.avro.io.ValueReader;
-import org.apache.avro.io.ValueWriter;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.Encoder;
 import org.apache.avro.ipc.AvroRemoteException;
 import org.apache.avro.ipc.Responder;
 
@@ -35,19 +35,19 @@
   }
 
   /** Reads a request message. */
-  public Object readRequest(Schema schema, ValueReader in) throws IOException {
+  public Object readRequest(Schema schema, Decoder in) throws IOException {
     return new GenericDatumReader<Object>(schema).read(null, in);
   }
 
   /** Writes a response message. */
-  public void writeResponse(Schema schema, Object response, ValueWriter out)
+  public void writeResponse(Schema schema, Object response, Encoder out)
     throws IOException {
     new GenericDatumWriter<Object>(schema).write(response, out);
   }
 
   /** Writes an error message. */
   public void writeError(Schema schema, AvroRemoteException error,
-                         ValueWriter out) throws IOException {
+                         Encoder out) throws IOException {
     new GenericDatumWriter<Object>(schema).write(error.getValue(), out);
   }
 

Copied: hadoop/avro/trunk/src/java/org/apache/avro/io/BinaryDecoder.java (from r786682, hadoop/avro/trunk/src/java/org/apache/avro/io/ValueReader.java)
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/io/BinaryDecoder.java?p2=hadoop/avro/trunk/src/java/org/apache/avro/io/BinaryDecoder.java&p1=hadoop/avro/trunk/src/java/org/apache/avro/io/ValueReader.java&r1=786682&r2=787828&rev=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/io/ValueReader.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/io/BinaryDecoder.java Tue Jun 23 20:38:46 2009
@@ -38,10 +38,10 @@
  *  and similar methods for maps).  See {@link #readArrayStart} for
  *  details on these methods.)
  *
- *  @see ValueWriter
+ *  @see Encoder
  */
 
-public class ValueReader {
+public class BinaryDecoder extends Decoder {
   private InputStream in;
   
   private class ByteReader {
@@ -79,33 +79,22 @@
   
   private final ByteReader byteReader;
 
-  public ValueReader(InputStream in) {
+  public BinaryDecoder(InputStream in) {
     this.in = in;
     byteReader = (in instanceof ByteBufferInputStream) ?
         new ReuseByteReader((ByteBufferInputStream) in) : new ByteReader();
   }
   
-  /** Start reading against a different input stream.  Stateful
-    * subclasses will reset their states to their initial state. */
+  @Override
   public void init(InputStream in) {
     this.in = in;
   }
 
-  /**
-   * "Reads" a null value.  (Doesn't actually read anything, but
-   * advances the state of the parser if the implementation is
-   * stateful.)
-   *  @throws AvroTypeException If this is a stateful reader and
-   *          null is not the type of the next value to be read
-   */
+  @Override
   public void readNull() throws IOException { }
 
-  /**
-   * Reads a boolean value written by {@link ValueWriter#writeBoolean}.
-   * @throws AvroTypeException If this is a stateful reader and
-   * boolean is not the type of the next value to be read
-   */
 
+  @Override
   public boolean readBoolean() throws IOException {
     int n = in.read();
     if (n < 0) {
@@ -114,13 +103,7 @@
     return n == 1;
   }
 
-  /**
-   * Reads an integer written by {@link ValueWriter#writeInt}.
-   * @throws AvroTypeException If encoded value is larger than
-   *          32-bits
-   * @throws AvroTypeException If this is a stateful reader and
-   *          int is not the type of the next value to be read
-   */
+  @Override
   public int readInt() throws IOException {
     long result = readLong();
     if (result < Integer.MIN_VALUE || Integer.MAX_VALUE < result) {
@@ -129,11 +112,7 @@
     return (int)result;
   }
 
-  /**
-   * Reads a long written by {@link ValueWriter#writeLong}.
-   * @throws AvroTypeException If this is a stateful reader and
-   *          long is not the type of the next value to be read
-   */
+  @Override
   public long readLong() throws IOException {
     long n = 0;
     for (int shift = 0; ; shift += 7) {
@@ -150,11 +129,7 @@
     return (n >>> 1) ^ -(n & 1); // back to two's-complement
   }
 
-  /**
-   * Reads a float written by {@link ValueWriter#writeFloat}.
-   * @throws AvroTypeException If this is a stateful reader and
-   * is not the type of the next value to be read
-   */
+  @Override
   public float readFloat() throws IOException {
     int n = 0;
     for (int i = 0, shift = 0; i < 4; i++, shift += 8) {
@@ -168,11 +143,7 @@
     return Float.intBitsToFloat(n);
   }
 
-  /**
-   * Reads a double written by {@link ValueWriter#writeDouble}.
-   * @throws AvroTypeException If this is a stateful reader and
-   *           is not the type of the next value to be read
-   */
+  @Override
   public double readDouble() throws IOException {
     long n = 0;
     for (int i = 0, shift = 0; i < 8; i++, shift += 8) {
@@ -186,11 +157,7 @@
     return Double.longBitsToDouble(n);
   }
     
-  /**
-   * Reads a char-string written by {@link ValueWriter#writeString}.
-   * @throws AvroTypeException If this is a stateful reader and
-   * char-string is not the type of the next value to be read
-   */
+  @Override
   public Utf8 readString(Utf8 old) throws IOException {
     int length = readInt();
     Utf8 result = (old != null ? old : new Utf8());
@@ -199,81 +166,39 @@
     return result;
   }
     
-  /**
-   * Discards a char-string written by {@link ValueWriter#writeString}.
-   *  @throws AvroTypeException If this is a stateful reader and
-   *          char-string is not the type of the next value to be read
-   */
+  @Override
   public void skipString() throws IOException {
     doSkipBytes(readInt());
   }
 
-  /**
-   * Reads a byte-string written by {@link ValueWriter#writeBytes}.
-   * if <tt>old</tt> is not null and has sufficient capacity to take in
-   * the bytes being read, the bytes are returned in <tt>old</tt>.
-   * @throws AvroTypeException If this is a stateful reader and
-   *          byte-string is not the type of the next value to be read
-   */
+  @Override
   public ByteBuffer readBytes(ByteBuffer old) throws IOException {
     int length = readInt();
     return byteReader.read(old, length);
   }
 
-  /**
-   * Discards a byte-string written by {@link ValueWriter#writeBytes}.
-   *  @throws AvroTypeException If this is a stateful reader and
-   *          byte-string is not the type of the next value to be read
-   */
+  @Override
   public void skipBytes() throws IOException {
     doSkipBytes(readInt());
   }
   
-  /**
-   * Reads fixed sized binary object.
-   * @param bytes The buffer to store the contents being read.
-   * @param start The position where the data needs to be written.
-   * @param length  The size of the binary object.
-   * @throws AvroTypeException If this is a stateful reader and
-   *          fixed sized binary object is not the type of the next
-   *          value to be read or the length is incorrect.
-   * @throws IOException
-   */
+  @Override
   public void readFixed(byte[] bytes, int start, int length)
     throws IOException {
     doReadBytes(bytes, start, length);
   }
 
-  /**
-   * A shorthand for <tt>readFixed(bytes, 0, bytes.length)</tt>.
-   * @throws AvroTypeException If this is a stateful reader and
-   *          fixed sized binary object is not the type of the next
-   *          value to be read or the length is incorrect.
-   * @throws IOException
-   */
+  @Override
   public void readFixed(byte[] bytes) throws IOException {
     readFixed(bytes, 0, bytes.length);
   }
   
-  /**
-   * Discards fixed sized binary object.
-   * @param length  The size of the binary object to be skipped.
-   * @throws AvroTypeException If this is a stateful reader and
-   *          fixed sized binary object is not the type of the next
-   *          value to be read or the length is incorrect.
-   * @throws IOException
-   */
+  @Override
   public void skipFixed(int length) throws IOException {
     doSkipBytes(length);
   }
 
-  /**
-   * Reads an enumeration.
-   * @return The enumeration's value.
-   * @throws AvroTypeException If this is a stateful reader and
-   *          enumeration is not the type of the next value to be read.
-   * @throws IOException
-   */
+  @Override
   public int readEnum() throws IOException {
     return readInt();
   }
@@ -305,7 +230,7 @@
     }
   }
 
-/**
+  /**
    * Returns the number of items to follow in the current array or map.
    * Returns 0 if there are no more items in the current array and the array/map
    * has ended.
@@ -343,128 +268,37 @@
     return result;
   }
 
-  /**
-   * Reads and returns the size of the first block of an array.  If
-   * this method returns non-zero, then the caller should read the
-   * indicated number of items, and then call {@link
-   * #arrayNext} to find out the number of items in the next
-   * block.  The typical pattern for consuming an array looks like:
-   * <pre>
-   *   for(long i = in.readArrayStart(); i != 0; i = in.arrayNext()) {
-   *     for (long j = 0; j < i; j++) {
-   *       read next element of the array;
-   *     }
-   *   }
-   * </pre>
-   *  @throws AvroTypeException If this is a stateful reader and
-   *          array is not the type of the next value to be read */
+  @Override
   public long readArrayStart() throws IOException {
     return doReadItemCount();
   }
 
-  /**
-   * Processes the next block of an array andreturns the number of items in
-   * the block and let's the caller
-   * read those items.
-   * @throws AvroTypeException When called outside of an
-   *         array context
-   */
+  @Override
   public long arrayNext() throws IOException {
     return doReadItemCount();
   }
 
-  /**
-   * Used for quickly skipping through an array.  Note you can
-   * either skip the entire array, or read the entire array (with
-   * {@link #readArrayStart}), but you can't mix the two on the
-   * same array.
-   *
-   * This method will skip through as many items as it can, all of
-   * them if possible.  It will return zero if there are no more
-   * items to skip through, or an item count if it needs the client's
-   * help in skipping.  The typical usage pattern is:
-   * <pre>
-   *   for(long i = in.skipArray(); i != 0; i = i.skipArray()) {
-   *     for (long j = 0; j < i; j++) {
-   *       read and discard the next element of the array;
-   *     }
-   *   }
-   * </pre>
-   * Note that this method can automatically skip through items if a
-   * byte-count is found in the underlying data, or if a schema has
-   * been provided to the implementation, but
-   * otherwise the client will have to skip through items itself.
-   *
-   *  @throws AvroTypeException If this is a stateful reader and
-   *          array is not the type of the next value to be read
-   */
+  @Override
   public long skipArray() throws IOException {
     return doSkipItems();
   }
 
-  /**
-   * Reads and returns the size of the next block of map-entries.
-   * Similar to {@link #readArrayStart}.
-   *
-   *  As an example, let's say you want to read a map of records,
-   *  the record consisting of an Long field and a Boolean field.
-   *  Your code would look something like this:
-   * <pre>
-   *   Map<String,Record> m = new HashMap<String,Record>();
-   *   Record reuse = new Record();
-   *   for(long i = in.readMapStart(); i != 0; i = in.readMapNext()) {
-   *     for (long j = 0; j < i; j++) {
-   *       String key = in.readString();
-   *       reuse.intField = in.readInt();
-   *       reuse.boolField = in.readBoolean();
-   *       m.put(key, reuse);
-   *     }
-   *   }
-   * </pre>
-   * @throws AvroTypeException If this is a stateful reader and
-   *         map is not the type of the next value to be read
-   */
+  @Override
   public long readMapStart() throws IOException {
     return doReadItemCount();
   }
 
-  /**
-   * Processes the next block of map entries and returns the count of them.
-   * Similar to {@link #arrayNext}.  See {@link #readMapStart} for details.
-   * @throws AvroTypeException When called outside of a
-   *         map context
-   */
+  @Override
   public long mapNext() throws IOException {
     return doReadItemCount();
   }
 
-  /**
-   * Support for quickly skipping through a map similar to {@link #skipArray}.
-   *
-   * As an example, let's say you want to skip a map of records,
-   * the record consisting of an Long field and a Boolean field.
-   * Your code would look something like this:
-   * <pre>
-   *   for(long i = in.skipMap(); i != 0; i = in.skipMap()) {
-   *     for (long j = 0; j < i; j++) {
-   *       in.skipString();  // Discard key
-   *       in.readInt(); // Discard int-field of value
-   *       in.readBoolean(); // Discard boolean-field of value
-   *     }
-   *   }
-   * </pre>
-   *  @throws AvroTypeException If this is a stateful reader and
-   *          array is not the type of the next value to be read */
-
+  @Override
   public long skipMap() throws IOException {
     return doSkipItems();
   }
 
-  /**
-   * Reads the tag of a union written by {@link ValueWriter#writeIndex}.
-   * @throws AvroTypeException If this is a stateful reader and
-   *         union is not the type of the next value to be read
-   */
+  @Override
   public int readIndex() throws IOException {
     return readInt();
   }

Propchange: hadoop/avro/trunk/src/java/org/apache/avro/io/BinaryDecoder.java
------------------------------------------------------------------------------
    svn:mergeinfo = 

Copied: hadoop/avro/trunk/src/java/org/apache/avro/io/BinaryEncoder.java (from r786682, hadoop/avro/trunk/src/java/org/apache/avro/io/ValueWriter.java)
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/io/BinaryEncoder.java?p2=hadoop/avro/trunk/src/java/org/apache/avro/io/BinaryEncoder.java&p1=hadoop/avro/trunk/src/java/org/apache/avro/io/ValueWriter.java&r1=786682&r2=787828&rev=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/io/ValueWriter.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/io/BinaryEncoder.java Tue Jun 23 20:38:46 2009
@@ -31,19 +31,19 @@
  * This class has two types of methods.  One type of methods support
  * the writing of leaf values (for example, {@link #writeLong} and
  * {@link #writeString}).  These methods have analogs in {@link
- * ValueReader}.
+ * Decoder}.
  *
  * The other type of methods support the writing of maps and arrays.
  * These methods are {@link #writeArrayStart}, {@link
  * #startItem}, and {@link #writeArrayEnd} (and similar methods for
- * maps).  Some implementations of {@link ValueWriter} handle the
+ * maps).  Some implementations of {@link Encoder} handle the
  * buffering required to break large maps and arrays into blocks,
  * which is necessary for applications that want to do streaming.
  * (See {@link #writeArrayStart} for details on these methods.)
  *
- *  @see ValueReader
+ *  @see Decoder
  */
-public class ValueWriter {
+public class BinaryEncoder extends Encoder {
   protected OutputStream out;
   
   private interface ByteWriter {
@@ -81,278 +81,124 @@
 
   /** Create a writer that sends its output to the underlying stream
    *  <code>out</code>. */
-  public ValueWriter(OutputStream out) {
+  public BinaryEncoder(OutputStream out) {
     this.out = out;
     this.byteWriter = (out instanceof ByteBufferOutputStream) ?
         new ReuseByteWriter((ByteBufferOutputStream) out) :
           new SimpleByteWriter(out);
   }
 
-  /** Redirect output (and reset the parser state if we're checking). */
+  @Override
   public void init(OutputStream out) throws IOException {
     flush();
     this.out = out;
   }
 
-  /**
-   * Writes any buffered output to the underlying stream.
-   */
+  @Override
   public void flush() throws IOException {
     out.flush();
   }
 
-  /**
-   * "Writes" a null value.  (Doesn't actually write anything, but
-   * advances the state of the parser if this class is stateful.)
-   * @throws AvroTypeException If this is a stateful writer and a
-   *         null is not expected
-   */
+  @Override
   public void writeNull() throws IOException { }
   
-  /**
-   * Write a boolean value.
-   * @throws AvroTypeException If this is a stateful writer and a
-   * boolean is not expected
-   */
+  @Override
   public void writeBoolean(boolean b) throws IOException {
     out.write(b ? 1 : 0);
   }
 
-  /**
-   * Writes a 32-bit integer.
-   * @throws AvroTypeException If this is a stateful writer and an
-   * integer is not expected
-   */
+  @Override
   public void writeInt(int n) throws IOException {
     encodeLong(n, out);
   }
 
-  /**
-   * Write a 64-bit integer.
-   * @throws AvroTypeException If this is a stateful writer and a
-   * long is not expected
-   */
+  @Override
   public void writeLong(long n) throws IOException {
     encodeLong(n, out);
   }
   
-  /** Write a float.
-   * @throws IOException 
-   * @throws AvroTypeException If this is a stateful writer and a
-   * float is not expected
-   */
+  @Override
   public void writeFloat(float f) throws IOException {
     encodeFloat(f, out);
   }
 
-  /**
-   * Write a double.
-   * @throws AvroTypeException If this is a stateful writer and a
-   * double is not expected
-   */
+  @Override
   public void writeDouble(double d) throws IOException {
     encodeDouble(d, out);
   }
 
-  /**
-   * Write a Unicode character string.
-   * @throws AvroTypeException If this is a stateful writer and a
-   * char-string is not expected
-   */
+  @Override
   public void writeString(Utf8 utf8) throws IOException {
     encodeLong(utf8.getLength(), out);
     out.write(utf8.getBytes(), 0, utf8.getLength());
   }
 
-  /**
-   * Write a Unicode character string.
-   * @throws AvroTypeException If this is a stateful writer and a
-   * char-string is not expected
-   */
+  @Override
   public void writeString(String str) throws IOException {
     writeString(new Utf8(str));
   }
 
-  /**
-   * Write a byte string.
-   * @throws AvroTypeException If this is a stateful writer and a
-   *         byte-string is not expected
-   */
+  @Override
   public void writeBytes(ByteBuffer bytes) throws IOException {
     byteWriter.write(bytes);
   }
   
-  /**
-   * Write a byte string.
-   * @throws AvroTypeException If this is a stateful writer and a
-   * byte-string is not expected
-   */
+  @Override
   public void writeBytes(byte[] bytes, int start, int len) throws IOException {
     encodeLong(len, out);
     out.write(bytes, start, len);
   }
   
-  /**
-   * Writes a byte string.
-   * Equivalent to <tt>writeBytes(bytes, 0, bytes.length)</tt>
-   * @throws IOException 
-   * @throws AvroTypeException If this is a stateful writer and a
-   * byte-string is not expected
-   */
+  @Override
   public void writeBytes(byte[] bytes) throws IOException {
     writeBytes(bytes, 0, bytes.length);
   }
 
-  /**
-   * Writes a fixed size binary object.
-   * @param bytes The contents to write
-   * @param start The position within <tt>bytes</tt> where the contents
-   * start.
-   * @param len The number of bytes to write.
-   * @throws AvroTypeException If this is a stateful writer and a
-   * byte-string is not expected
-   * @throws IOException
-   */
+  @Override
   public void writeFixed(byte[] bytes, int start, int len) throws IOException {
 	  out.write(bytes, start, len);
   }
 
-  /**
-   * A shorthand for <tt>writeFixed(bytes, 0, bytes.length)</tt>
-   * @param bytes
-   */
+  @Override
   public void writeFixed(byte[] bytes) throws IOException {
     writeFixed(bytes, 0, bytes.length);
   }
   
-  /**
-   * Writes an enumeration.
-   * @param e
-   * @throws AvroTypeException If this is a stateful writer and an enumeration
-   * is not expected or the <tt>e</tt> is out of range.
-   * @throws IOException
-   */
+  @Override
   public void writeEnum(int e) throws IOException {
     encodeLong(e, out);
   }
 
-  /** Call this method to start writing an array.
-   *
-   *  When starting to serialize an array, call {@link
-   *  #writeArrayStart}. Then, before writing any data for any item
-   *  call {@link #setItemCount} followed by a sequence of
-   *  {@link #startItem()} and the item itself. The number of
-   *  {@link #startItem()} should match the number specified in
-   *  {@link #setItemCount}.
-   *  When actually writing the data of the item, you can call any {@link
-   *  ValueWriter} method (e.g., {@link #writeLong}).  When all items
-   *  of the array have been written, call {@link #writeArrayEnd}.
-   *
-   *  As an example, let's say you want to write an array of records,
-   *  the record consisting of an Long field and a Boolean field.
-   *  Your code would look something like this:
-   *  <pre>
-   *  out.writeArrayStart();
-   *  out.setItemCount(list.size());
-   *  for (Record r : list) {
-   *    out.startItem();
-   *    out.writeLong(r.longField);
-   *    out.writeBoolean(r.boolField);
-   *  }
-   *  out.writeArrayEnd();
-   *  </pre>
-   *  @throws AvroTypeException If this is a stateful writer and an
-   *          array is not expected
-   */
+  @Override
   public void writeArrayStart() throws IOException {
   }
 
-  /**
-   * Call this method before writing a batch of items in an array or a map.
-   * Then for each item, call {@link #startItem()} followed by any of the
-   * other write methods of {@link ValueWriter}. The number of calls
-   * to {@link #startItem()} must be equal to the count specified
-   * in {@link #setItemCount}. Once a batch is completed you
-   * can start another batch with {@link #setItemCount}.
-   * 
-   * @param itemCount The number of {@link #startItem()} calls to follow.
-   * @throws IOException
-   */
+  @Override
   public void setItemCount(long itemCount) throws IOException {
     if (itemCount > 0) {
       writeLong(itemCount);
     }
   }
   
-  /**
-   * Start a new item of an array or map.
-   * See {@link #writeArrayStart} for usage information.
-   * @throws AvroTypeException If called outside of an array or map context
-   */
+  @Override
   public void startItem() throws IOException {
   }
 
-  /**
-   * Call this method to finish writing an array.
-   * See {@link #writeArrayStart} for usage information.
-   *
-   * @throws AvroTypeException If items written does not match count
-   *          provided to {@link #writeArrayStart}
-   * @throws AvroTypeException If not currently inside an array
-   */
+  @Override
   public void writeArrayEnd() throws IOException {
     encodeLong(0, out);
   }
 
-  /**
-   * Call this to start a new map.  See
-   * {@link #writeArrayStart} for details on usage.
-   *
-   * As an example of usage, let's say you want to write a map of
-   * records, the record consisting of an Long field and a Boolean
-   * field.  Your code would look something like this:
-   * <pre>
-   * out.writeMapStart();
-   * out.setItemCount(list.size());
-   * for (Map.Entry<String,Record> entry : map.entrySet()) {
-   *   out.startItem();
-   *   out.writeString(entry.getKey());
-   *   out.writeLong(entry.getValue().longField);
-   *   out.writeBoolean(entry.getValue().boolField);
-   * }
-   * out.writeMapEnd();
-   * </pre>
-   * @throws AvroTypeException If this is a stateful writer and a
-   * map is not expected
-   */
+  @Override
   public void writeMapStart() throws IOException {
   }
 
-  /**
-   * Call this method to terminate the inner-most, currently-opened
-   * map.  See {@link #writeArrayStart} for more details.
-   *
-   * @throws AvroTypeException If items written does not match count
-   *          provided to {@link #writeMapStart}
-   * @throws AvroTypeException If not currently inside a map
-   */
+  @Override
   public void writeMapEnd() throws IOException {
     encodeLong(0, out);
   }
 
-  /** Call this method to write the tag of a union.
-   *
-   * As an example of usage, let's say you want to write a union,
-   * whose second branch is a record consisting of an Long field and
-   * a Boolean field.  Your code would look something like this:
-   * <pre>
-   * out.writeIndex(1);
-   * out.writeLong(record.longField);
-   * out.writeBoolean(record.boolField);
-   * </pre>
-   * @throws AvroTypeException If this is a stateful writer and a
-   * map is not expected
-   */
+  @Override
   public void writeIndex(int unionIndex) throws IOException {
     encodeLong(unionIndex, out);
   }

Propchange: hadoop/avro/trunk/src/java/org/apache/avro/io/BinaryEncoder.java
------------------------------------------------------------------------------
    svn:mergeinfo = 

Copied: hadoop/avro/trunk/src/java/org/apache/avro/io/BlockingBinaryEncoder.java (from r786682, hadoop/avro/trunk/src/java/org/apache/avro/io/BlockingValueWriter.java)
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/io/BlockingBinaryEncoder.java?p2=hadoop/avro/trunk/src/java/org/apache/avro/io/BlockingBinaryEncoder.java&p1=hadoop/avro/trunk/src/java/org/apache/avro/io/BlockingValueWriter.java&r1=786682&r2=787828&rev=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/io/BlockingValueWriter.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/io/BlockingBinaryEncoder.java Tue Jun 23 20:38:46 2009
@@ -27,14 +27,14 @@
 import org.apache.avro.Schema;
 import org.apache.avro.util.Utf8;
 
-/** A {@link ValueWriter} that writes large arrays and maps as a sequence of
+/** A {@link Encoder} that writes large arrays and maps as a sequence of
  * blocks.  So long as individual primitive values fit in memory, arbitrarily
  * long arrays and maps may be written and subsequently read without exhausting
  * memory.  Values are buffered until the specified block size would be
  * exceeded, minimizing block overhead.
- * @see ValueWriter
+ * @see Encoder
  */
-public class BlockingValueWriter extends ValueWriter {
+public class BlockingBinaryEncoder extends BinaryEncoder {
 
  /* Implementation note:
   *
@@ -213,11 +213,11 @@
   private static final int DEFAULT_BUFFER_SIZE = 64 * 1024;
   private static final int MIN_BUFFER_SIZE = 64;
 
-  public BlockingValueWriter(OutputStream out) {
+  public BlockingBinaryEncoder(OutputStream out) {
     this(out, DEFAULT_BUFFER_SIZE);
   }
 
-  public BlockingValueWriter(OutputStream out, int bufferSize) {
+  public BlockingBinaryEncoder(OutputStream out, int bufferSize) {
     super(out);
     if (bufferSize < MIN_BUFFER_SIZE) {
       throw new IllegalArgumentException("Buffer size too smll.");

Propchange: hadoop/avro/trunk/src/java/org/apache/avro/io/BlockingBinaryEncoder.java
------------------------------------------------------------------------------
    svn:mergeinfo = 

Modified: hadoop/avro/trunk/src/java/org/apache/avro/io/DatumReader.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/io/DatumReader.java?rev=787828&r1=787827&r2=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/io/DatumReader.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/io/DatumReader.java Tue Jun 23 20:38:46 2009
@@ -32,7 +32,7 @@
   /** Read a datum.  Traverse the schema, depth-first, reading all leaf values
    * in the schema into a datum that is returned.  If the provided datum is
    * non-null it may be reused and returned. */
-  D read(D reuse, ValueReader in) throws IOException;
+  D read(D reuse, Decoder in) throws IOException;
 
 }
 

Modified: hadoop/avro/trunk/src/java/org/apache/avro/io/DatumWriter.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/io/DatumWriter.java?rev=787828&r1=787827&r2=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/io/DatumWriter.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/io/DatumWriter.java Tue Jun 23 20:38:46 2009
@@ -31,5 +31,5 @@
 
   /** Write a datum.  Traverse the schema, depth first, writing each leaf value
    * in the schema from the datum to the output. */
-  void write(D datum, ValueWriter out) throws IOException;
+  void write(D datum, Encoder out) throws IOException;
 }

Added: hadoop/avro/trunk/src/java/org/apache/avro/io/Decoder.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/io/Decoder.java?rev=787828&view=auto
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/io/Decoder.java (added)
+++ hadoop/avro/trunk/src/java/org/apache/avro/io/Decoder.java Tue Jun 23 20:38:46 2009
@@ -0,0 +1,279 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro.io;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.avro.AvroTypeException;
+import org.apache.avro.util.Utf8;
+
+/**
+ * Low-level support for de-serializing Avro values.
+ *
+ *  This class has two types of methods.  One type of methods support
+ *  the reading of leaf values (for example, {@link #readLong} and
+ *  {@link #readString}).
+ *
+ *  The other type of methods support the writing of maps and arrays.
+ *  These methods are {@link #readArrayStart}, {@link #arrayNext},
+ *  and similar methods for maps).  See {@link #readArrayStart} for
+ *  details on these methods.)
+ *
+ *  @see Encoder
+ */
+
+public abstract class Decoder {
+  
+  /** Start reading against a different input stream.  Stateful
+    * subclasses will reset their states to their initial state. */
+  public abstract void init(InputStream in);
+
+  /**
+   * "Reads" a null value.  (Doesn't actually read anything, but
+   * advances the state of the parser if the implementation is
+   * stateful.)
+   *  @throws AvroTypeException If this is a stateful reader and
+   *          null is not the type of the next value to be read
+   */
+  public abstract void readNull() throws IOException;
+
+  /**
+   * Reads a boolean value written by {@link Encoder#writeBoolean}.
+   * @throws AvroTypeException If this is a stateful reader and
+   * boolean is not the type of the next value to be read
+   */
+
+  public abstract boolean readBoolean() throws IOException;
+
+  /**
+   * Reads an integer written by {@link Encoder#writeInt}.
+   * @throws AvroTypeException If encoded value is larger than
+   *          32-bits
+   * @throws AvroTypeException If this is a stateful reader and
+   *          int is not the type of the next value to be read
+   */
+  public abstract int readInt() throws IOException;
+
+  /**
+   * Reads a long written by {@link Encoder#writeLong}.
+   * @throws AvroTypeException If this is a stateful reader and
+   *          long is not the type of the next value to be read
+   */
+  public abstract long readLong() throws IOException;
+
+  /**
+   * Reads a float written by {@link Encoder#writeFloat}.
+   * @throws AvroTypeException If this is a stateful reader and
+   * is not the type of the next value to be read
+   */
+  public abstract float readFloat() throws IOException;
+
+  /**
+   * Reads a double written by {@link Encoder#writeDouble}.
+   * @throws AvroTypeException If this is a stateful reader and
+   *           is not the type of the next value to be read
+   */
+  public abstract double readDouble() throws IOException;
+    
+  /**
+   * Reads a char-string written by {@link Encoder#writeString}.
+   * @throws AvroTypeException If this is a stateful reader and
+   * char-string is not the type of the next value to be read
+   */
+  public abstract Utf8 readString(Utf8 old) throws IOException;
+    
+  /**
+   * Discards a char-string written by {@link Encoder#writeString}.
+   *  @throws AvroTypeException If this is a stateful reader and
+   *          char-string is not the type of the next value to be read
+   */
+  public abstract void skipString() throws IOException;
+
+  /**
+   * Reads a byte-string written by {@link Encoder#writeBytes}.
+   * if <tt>old</tt> is not null and has sufficient capacity to take in
+   * the bytes being read, the bytes are returned in <tt>old</tt>.
+   * @throws AvroTypeException If this is a stateful reader and
+   *          byte-string is not the type of the next value to be read
+   */
+  public abstract ByteBuffer readBytes(ByteBuffer old) throws IOException;
+
+  /**
+   * Discards a byte-string written by {@link Encoder#writeBytes}.
+   *  @throws AvroTypeException If this is a stateful reader and
+   *          byte-string is not the type of the next value to be read
+   */
+  public abstract void skipBytes() throws IOException;
+  
+  /**
+   * Reads fixed sized binary object.
+   * @param bytes The buffer to store the contents being read.
+   * @param start The position where the data needs to be written.
+   * @param length  The size of the binary object.
+   * @throws AvroTypeException If this is a stateful reader and
+   *          fixed sized binary object is not the type of the next
+   *          value to be read or the length is incorrect.
+   * @throws IOException
+   */
+  public abstract void readFixed(byte[] bytes, int start, int length)
+    throws IOException;
+
+  /**
+   * A shorthand for <tt>readFixed(bytes, 0, bytes.length)</tt>.
+   * @throws AvroTypeException If this is a stateful reader and
+   *          fixed sized binary object is not the type of the next
+   *          value to be read or the length is incorrect.
+   * @throws IOException
+   */
+  public abstract void readFixed(byte[] bytes) throws IOException;
+  
+  /**
+   * Discards fixed sized binary object.
+   * @param length  The size of the binary object to be skipped.
+   * @throws AvroTypeException If this is a stateful reader and
+   *          fixed sized binary object is not the type of the next
+   *          value to be read or the length is incorrect.
+   * @throws IOException
+   */
+  public abstract void skipFixed(int length) throws IOException;
+
+  /**
+   * Reads an enumeration.
+   * @return The enumeration's value.
+   * @throws AvroTypeException If this is a stateful reader and
+   *          enumeration is not the type of the next value to be read.
+   * @throws IOException
+   */
+  public abstract int readEnum() throws IOException;
+  
+  /**
+   * Reads and returns the size of the first block of an array.  If
+   * this method returns non-zero, then the caller should read the
+   * indicated number of items, and then call {@link
+   * #arrayNext} to find out the number of items in the next
+   * block.  The typical pattern for consuming an array looks like:
+   * <pre>
+   *   for(long i = in.readArrayStart(); i != 0; i = in.arrayNext()) {
+   *     for (long j = 0; j < i; j++) {
+   *       read next element of the array;
+   *     }
+   *   }
+   * </pre>
+   *  @throws AvroTypeException If this is a stateful reader and
+   *          array is not the type of the next value to be read */
+  public abstract long readArrayStart() throws IOException;
+
+  /**
+   * Processes the next block of an array andreturns the number of items in
+   * the block and let's the caller
+   * read those items.
+   * @throws AvroTypeException When called outside of an
+   *         array context
+   */
+  public abstract long arrayNext() throws IOException;
+
+  /**
+   * Used for quickly skipping through an array.  Note you can
+   * either skip the entire array, or read the entire array (with
+   * {@link #readArrayStart}), but you can't mix the two on the
+   * same array.
+   *
+   * This method will skip through as many items as it can, all of
+   * them if possible.  It will return zero if there are no more
+   * items to skip through, or an item count if it needs the client's
+   * help in skipping.  The typical usage pattern is:
+   * <pre>
+   *   for(long i = in.skipArray(); i != 0; i = i.skipArray()) {
+   *     for (long j = 0; j < i; j++) {
+   *       read and discard the next element of the array;
+   *     }
+   *   }
+   * </pre>
+   * Note that this method can automatically skip through items if a
+   * byte-count is found in the underlying data, or if a schema has
+   * been provided to the implementation, but
+   * otherwise the client will have to skip through items itself.
+   *
+   *  @throws AvroTypeException If this is a stateful reader and
+   *          array is not the type of the next value to be read
+   */
+  public abstract long skipArray() throws IOException;
+
+  /**
+   * Reads and returns the size of the next block of map-entries.
+   * Similar to {@link #readArrayStart}.
+   *
+   *  As an example, let's say you want to read a map of records,
+   *  the record consisting of an Long field and a Boolean field.
+   *  Your code would look something like this:
+   * <pre>
+   *   Map<String,Record> m = new HashMap<String,Record>();
+   *   Record reuse = new Record();
+   *   for(long i = in.readMapStart(); i != 0; i = in.readMapNext()) {
+   *     for (long j = 0; j < i; j++) {
+   *       String key = in.readString();
+   *       reuse.intField = in.readInt();
+   *       reuse.boolField = in.readBoolean();
+   *       m.put(key, reuse);
+   *     }
+   *   }
+   * </pre>
+   * @throws AvroTypeException If this is a stateful reader and
+   *         map is not the type of the next value to be read
+   */
+  public abstract long readMapStart() throws IOException;
+
+  /**
+   * Processes the next block of map entries and returns the count of them.
+   * Similar to {@link #arrayNext}.  See {@link #readMapStart} for details.
+   * @throws AvroTypeException When called outside of a
+   *         map context
+   */
+  public abstract long mapNext() throws IOException;
+
+  /**
+   * Support for quickly skipping through a map similar to {@link #skipArray}.
+   *
+   * As an example, let's say you want to skip a map of records,
+   * the record consisting of an Long field and a Boolean field.
+   * Your code would look something like this:
+   * <pre>
+   *   for(long i = in.skipMap(); i != 0; i = in.skipMap()) {
+   *     for (long j = 0; j < i; j++) {
+   *       in.skipString();  // Discard key
+   *       in.readInt(); // Discard int-field of value
+   *       in.readBoolean(); // Discard boolean-field of value
+   *     }
+   *   }
+   * </pre>
+   *  @throws AvroTypeException If this is a stateful reader and
+   *          array is not the type of the next value to be read */
+
+  public abstract long skipMap() throws IOException;
+
+  /**
+   * Reads the tag of a union written by {@link Encoder#writeIndex}.
+   * @throws AvroTypeException If this is a stateful reader and
+   *         union is not the type of the next value to be read
+   */
+  public abstract int readIndex() throws IOException;
+
+}

Added: hadoop/avro/trunk/src/java/org/apache/avro/io/Encoder.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/io/Encoder.java?rev=787828&view=auto
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/io/Encoder.java (added)
+++ hadoop/avro/trunk/src/java/org/apache/avro/io/Encoder.java Tue Jun 23 20:38:46 2009
@@ -0,0 +1,269 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro.io;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.avro.AvroTypeException;
+import org.apache.avro.util.Utf8;
+
+/**
+ * Low-level support for serializing Avro values.
+ *
+ * This class has two types of methods.  One type of methods support
+ * the writing of leaf values (for example, {@link #writeLong} and
+ * {@link #writeString}).  These methods have analogs in {@link
+ * Decoder}.
+ *
+ * The other type of methods support the writing of maps and arrays.
+ * These methods are {@link #writeArrayStart}, {@link
+ * #startItem}, and {@link #writeArrayEnd} (and similar methods for
+ * maps).  Some implementations of {@link Encoder} handle the
+ * buffering required to break large maps and arrays into blocks,
+ * which is necessary for applications that want to do streaming.
+ * (See {@link #writeArrayStart} for details on these methods.)
+ *
+ *  @see Decoder
+ */
+public abstract class Encoder {
+
+  /** Redirect output (and reset the parser state if we're checking). */
+  public abstract void init(OutputStream out) throws IOException;
+
+  /**
+   * Writes any buffered output to the underlying stream.
+   */
+  public abstract void flush() throws IOException;
+
+  /**
+   * "Writes" a null value.  (Doesn't actually write anything, but
+   * advances the state of the parser if this class is stateful.)
+   * @throws AvroTypeException If this is a stateful writer and a
+   *         null is not expected
+   */
+  public abstract void writeNull() throws IOException;
+  
+  /**
+   * Write a boolean value.
+   * @throws AvroTypeException If this is a stateful writer and a
+   * boolean is not expected
+   */
+  public abstract void writeBoolean(boolean b) throws IOException;
+
+  /**
+   * Writes a 32-bit integer.
+   * @throws AvroTypeException If this is a stateful writer and an
+   * integer is not expected
+   */
+  public abstract void writeInt(int n) throws IOException;
+
+  /**
+   * Write a 64-bit integer.
+   * @throws AvroTypeException If this is a stateful writer and a
+   * long is not expected
+   */
+  public abstract void writeLong(long n) throws IOException;
+  
+  /** Write a float.
+   * @throws IOException 
+   * @throws AvroTypeException If this is a stateful writer and a
+   * float is not expected
+   */
+  public abstract void writeFloat(float f) throws IOException;
+
+  /**
+   * Write a double.
+   * @throws AvroTypeException If this is a stateful writer and a
+   * double is not expected
+   */
+  public abstract void writeDouble(double d) throws IOException;
+
+  /**
+   * Write a Unicode character string.
+   * @throws AvroTypeException If this is a stateful writer and a
+   * char-string is not expected
+   */
+  public abstract void writeString(Utf8 utf8) throws IOException;
+
+  /**
+   * Write a Unicode character string.
+   * @throws AvroTypeException If this is a stateful writer and a
+   * char-string is not expected
+   */
+  public abstract void writeString(String str) throws IOException;
+
+  /**
+   * Write a byte string.
+   * @throws AvroTypeException If this is a stateful writer and a
+   *         byte-string is not expected
+   */
+  public abstract void writeBytes(ByteBuffer bytes) throws IOException;
+  
+  /**
+   * Write a byte string.
+   * @throws AvroTypeException If this is a stateful writer and a
+   * byte-string is not expected
+   */
+  public abstract void writeBytes(byte[] bytes, int start, int len) throws IOException;
+  
+  /**
+   * Writes a byte string.
+   * Equivalent to <tt>writeBytes(bytes, 0, bytes.length)</tt>
+   * @throws IOException 
+   * @throws AvroTypeException If this is a stateful writer and a
+   * byte-string is not expected
+   */
+  public abstract void writeBytes(byte[] bytes) throws IOException;
+
+  /**
+   * Writes a fixed size binary object.
+   * @param bytes The contents to write
+   * @param start The position within <tt>bytes</tt> where the contents
+   * start.
+   * @param len The number of bytes to write.
+   * @throws AvroTypeException If this is a stateful writer and a
+   * byte-string is not expected
+   * @throws IOException
+   */
+  public abstract void writeFixed(byte[] bytes, int start, int len) throws IOException;
+
+  /**
+   * A shorthand for <tt>writeFixed(bytes, 0, bytes.length)</tt>
+   * @param bytes
+   */
+  public abstract void writeFixed(byte[] bytes) throws IOException;
+  
+  /**
+   * Writes an enumeration.
+   * @param e
+   * @throws AvroTypeException If this is a stateful writer and an enumeration
+   * is not expected or the <tt>e</tt> is out of range.
+   * @throws IOException
+   */
+  public abstract void writeEnum(int e) throws IOException;
+
+  /** Call this method to start writing an array.
+   *
+   *  When starting to serialize an array, call {@link
+   *  #writeArrayStart}. Then, before writing any data for any item
+   *  call {@link #setItemCount} followed by a sequence of
+   *  {@link #startItem()} and the item itself. The number of
+   *  {@link #startItem()} should match the number specified in
+   *  {@link #setItemCount}.
+   *  When actually writing the data of the item, you can call any {@link
+   *  Encoder} method (e.g., {@link #writeLong}).  When all items
+   *  of the array have been written, call {@link #writeArrayEnd}.
+   *
+   *  As an example, let's say you want to write an array of records,
+   *  the record consisting of an Long field and a Boolean field.
+   *  Your code would look something like this:
+   *  <pre>
+   *  out.writeArrayStart();
+   *  out.setItemCount(list.size());
+   *  for (Record r : list) {
+   *    out.startItem();
+   *    out.writeLong(r.longField);
+   *    out.writeBoolean(r.boolField);
+   *  }
+   *  out.writeArrayEnd();
+   *  </pre>
+   *  @throws AvroTypeException If this is a stateful writer and an
+   *          array is not expected
+   */
+  public abstract void writeArrayStart() throws IOException;
+
+  /**
+   * Call this method before writing a batch of items in an array or a map.
+   * Then for each item, call {@link #startItem()} followed by any of the
+   * other write methods of {@link Encoder}. The number of calls
+   * to {@link #startItem()} must be equal to the count specified
+   * in {@link #setItemCount}. Once a batch is completed you
+   * can start another batch with {@link #setItemCount}.
+   * 
+   * @param itemCount The number of {@link #startItem()} calls to follow.
+   * @throws IOException
+   */
+  public abstract void setItemCount(long itemCount) throws IOException;
+  
+  /**
+   * Start a new item of an array or map.
+   * See {@link #writeArrayStart} for usage information.
+   * @throws AvroTypeException If called outside of an array or map context
+   */
+  public abstract void startItem() throws IOException;
+
+  /**
+   * Call this method to finish writing an array.
+   * See {@link #writeArrayStart} for usage information.
+   *
+   * @throws AvroTypeException If items written does not match count
+   *          provided to {@link #writeArrayStart}
+   * @throws AvroTypeException If not currently inside an array
+   */
+  public abstract void writeArrayEnd() throws IOException;
+
+  /**
+   * Call this to start a new map.  See
+   * {@link #writeArrayStart} for details on usage.
+   *
+   * As an example of usage, let's say you want to write a map of
+   * records, the record consisting of an Long field and a Boolean
+   * field.  Your code would look something like this:
+   * <pre>
+   * out.writeMapStart();
+   * out.setItemCount(list.size());
+   * for (Map.Entry<String,Record> entry : map.entrySet()) {
+   *   out.startItem();
+   *   out.writeString(entry.getKey());
+   *   out.writeLong(entry.getValue().longField);
+   *   out.writeBoolean(entry.getValue().boolField);
+   * }
+   * out.writeMapEnd();
+   * </pre>
+   * @throws AvroTypeException If this is a stateful writer and a
+   * map is not expected
+   */
+  public abstract void writeMapStart() throws IOException;
+
+  /**
+   * Call this method to terminate the inner-most, currently-opened
+   * map.  See {@link #writeArrayStart} for more details.
+   *
+   * @throws AvroTypeException If items written does not match count
+   *          provided to {@link #writeMapStart}
+   * @throws AvroTypeException If not currently inside a map
+   */
+  public abstract void writeMapEnd() throws IOException;
+
+  /** Call this method to write the tag of a union.
+   *
+   * As an example of usage, let's say you want to write a union,
+   * whose second branch is a record consisting of an Long field and
+   * a Boolean field.  Your code would look something like this:
+   * <pre>
+   * out.writeIndex(1);
+   * out.writeLong(record.longField);
+   * out.writeBoolean(record.boolField);
+   * </pre>
+   * @throws AvroTypeException If this is a stateful writer and a
+   * map is not expected
+   */
+  public abstract void writeIndex(int unionIndex) throws IOException;
+}

Modified: hadoop/avro/trunk/src/java/org/apache/avro/ipc/Requestor.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/ipc/Requestor.java?rev=787828&r1=787827&r2=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/ipc/Requestor.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/ipc/Requestor.java Tue Jun 23 20:38:46 2009
@@ -29,8 +29,10 @@
 import org.apache.avro.Protocol;
 import org.apache.avro.Schema;
 import org.apache.avro.Protocol.Message;
-import org.apache.avro.io.ValueReader;
-import org.apache.avro.io.ValueWriter;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.Encoder;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.avro.io.BinaryDecoder;
 import org.apache.avro.specific.SpecificDatumReader;
 import org.apache.avro.specific.SpecificDatumWriter;
 import org.apache.avro.util.Utf8;
@@ -59,11 +61,11 @@
   /** Writes a request message and reads a response or error message. */
   public Object request(String messageName, Object request)
     throws IOException {
-    ValueReader in;
+    Decoder in;
     Message m;
     do {
       ByteBufferOutputStream bbo = new ByteBufferOutputStream();
-      ValueWriter out = new ValueWriter(bbo);
+      Encoder out = new BinaryEncoder(bbo);
 
       if (!established)                           // if not established
         writeHandshake(out);                      // prepend handshake
@@ -80,7 +82,7 @@
         getTransceiver().transceive(bbo.getBufferList());
       
       ByteBufferInputStream bbi = new ByteBufferInputStream(response);
-      in = new ValueReader(bbi);
+      in = new BinaryDecoder(bbi);
       if (!established)                           // if not established
         readHandshake(in);                        // process handshake
     } while (!established);
@@ -107,7 +109,7 @@
   private static final SpecificDatumReader HANDSHAKE_READER =
     new SpecificDatumReader(HandshakeResponse._SCHEMA);
 
-  private void writeHandshake(ValueWriter out) throws IOException {
+  private void writeHandshake(Encoder out) throws IOException {
     MD5 localHash = new MD5();
     localHash.bytes(local.getMD5());
     String remoteName = transceiver.getRemoteName();
@@ -125,7 +127,7 @@
     HANDSHAKE_WRITER.write(handshake, out);
   }
 
-  private void readHandshake(ValueReader in) throws IOException {
+  private void readHandshake(Decoder in) throws IOException {
     HandshakeResponse handshake =
       (HandshakeResponse)HANDSHAKE_READER.read(null, in);
     switch (handshake.match) {
@@ -157,13 +159,13 @@
 
   /** Writes a request message. */
   public abstract void writeRequest(Schema schema, Object request,
-                                    ValueWriter out) throws IOException;
+                                    Encoder out) throws IOException;
 
   /** Reads a response message. */
-  public abstract Object readResponse(Schema schema, ValueReader in)
+  public abstract Object readResponse(Schema schema, Decoder in)
     throws IOException;
 
   /** Reads an error message. */
-  public abstract AvroRemoteException readError(Schema schema, ValueReader in)
+  public abstract AvroRemoteException readError(Schema schema, Decoder in)
     throws IOException;
 }

Modified: hadoop/avro/trunk/src/java/org/apache/avro/ipc/Responder.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/ipc/Responder.java?rev=787828&r1=787827&r2=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/ipc/Responder.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/ipc/Responder.java Tue Jun 23 20:38:46 2009
@@ -58,10 +58,10 @@
     ByteBufferInputStream bbi =
       new ByteBufferInputStream(transceiver.readBuffers());
     
-    ValueReader in = new ValueReader(bbi);
+    Decoder in = new BinaryDecoder(bbi);
     ByteBufferOutputStream bbo =
       new ByteBufferOutputStream();
-    ValueWriter out = new ValueWriter(bbo);
+    Encoder out = new BinaryEncoder(bbo);
     AvroRemoteException error = null;
     try {
       Protocol remote = handshake(transceiver, in, out);
@@ -100,7 +100,7 @@
       LOG.warn("system error", e);
       error = new AvroRemoteException(e);
       bbo = new ByteBufferOutputStream();
-      out = new ValueWriter(bbo);
+      out = new BinaryEncoder(bbo);
       out.writeBoolean(true);
       writeError(Protocol.SYSTEM_ERRORS, error, out);
     }
@@ -114,7 +114,7 @@
     new SpecificDatumReader(HandshakeRequest._SCHEMA);
 
   private Protocol handshake(Transceiver transceiver,
-                             ValueReader in, ValueWriter out)
+                             Decoder in, Encoder out)
     throws IOException {
     Protocol remote = remotes.get(transceiver);
     if (remote != null) return remote;            // already established
@@ -148,15 +148,15 @@
     throws AvroRemoteException;
 
   /** Reads a request message. */
-  public abstract Object readRequest(Schema schema, ValueReader in)
+  public abstract Object readRequest(Schema schema, Decoder in)
     throws IOException;
 
   /** Writes a response message. */
   public abstract void writeResponse(Schema schema, Object response,
-                                     ValueWriter out) throws IOException;
+                                     Encoder out) throws IOException;
 
   /** Writes an error message. */
   public abstract void writeError(Schema schema, AvroRemoteException error,
-                                  ValueWriter out) throws IOException;
+                                  Encoder out) throws IOException;
 
 }

Modified: hadoop/avro/trunk/src/java/org/apache/avro/package.html
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/package.html?rev=787828&r1=787827&r2=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/package.html (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/package.html Tue Jun 23 20:38:46 2009
@@ -35,9 +35,9 @@
 <p>Data of a given schema is always serialized identically, regardless
   of its in-memory representation, by traversing the schema and
   writing leaf values from the data structure with a {@link
-  org.apache.avro.io.ValueWriter}.  Deserializing similarly proceeds by
+  org.apache.avro.io.Encoder}.  Deserializing similarly proceeds by
   traversing the schema, reading leaf values with a {@link
-  org.apache.avro.io.ValueReader} and storing them in an in-memory data
+  org.apache.avro.io.Decoder} and storing them in an in-memory data
   structure.
 
 </body>

Modified: hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectDatumWriter.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectDatumWriter.java?rev=787828&r1=787827&r2=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectDatumWriter.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectDatumWriter.java Tue Jun 23 20:38:46 2009
@@ -25,7 +25,7 @@
 import org.apache.avro.Schema.Type;
 import org.apache.avro.generic.GenericDatumWriter;
 import org.apache.avro.io.DatumWriter;
-import org.apache.avro.io.ValueWriter;
+import org.apache.avro.io.Encoder;
 
 /** {@link DatumWriter} for existing classes via Java reflection. */
 public class ReflectDatumWriter extends GenericDatumWriter<Object> {
@@ -44,7 +44,7 @@
     }
   }
   
-  protected void writeEnum(Schema schema, Object datum, ValueWriter out)
+  protected void writeEnum(Schema schema, Object datum, Encoder out)
     throws IOException {
     out.writeInt(((Enum)datum).ordinal());
   }

Modified: hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectRequestor.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectRequestor.java?rev=787828&r1=787827&r2=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectRequestor.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectRequestor.java Tue Jun 23 20:38:46 2009
@@ -28,8 +28,8 @@
 import org.apache.avro.Schema;
 import org.apache.avro.io.DatumReader;
 import org.apache.avro.io.DatumWriter;
-import org.apache.avro.io.ValueReader;
-import org.apache.avro.io.ValueWriter;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.Encoder;
 import org.apache.avro.ipc.AvroRemoteException;
 import org.apache.avro.ipc.Requestor;
 import org.apache.avro.ipc.Transceiver;
@@ -57,7 +57,7 @@
     return new ReflectDatumReader(schema, packageName);
   }
 
-  public void writeRequest(Schema schema, Object request, ValueWriter out)
+  public void writeRequest(Schema schema, Object request, Encoder out)
     throws IOException {
     Object[] args = (Object[])request;
     int i = 0;
@@ -65,11 +65,11 @@
       getDatumWriter(param.getValue()).write(args[i++], out);
   }
     
-  public Object readResponse(Schema schema, ValueReader in) throws IOException {
+  public Object readResponse(Schema schema, Decoder in) throws IOException {
     return getDatumReader(schema).read(null, in);
   }
 
-  public AvroRemoteException readError(Schema schema, ValueReader in)
+  public AvroRemoteException readError(Schema schema, Decoder in)
     throws IOException {
     return (AvroRemoteException)getDatumReader(schema).read(null, in);
   }

Modified: hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectResponder.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectResponder.java?rev=787828&r1=787827&r2=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectResponder.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectResponder.java Tue Jun 23 20:38:46 2009
@@ -30,8 +30,8 @@
 import org.apache.avro.generic.GenericArray;
 import org.apache.avro.io.DatumReader;
 import org.apache.avro.io.DatumWriter;
-import org.apache.avro.io.ValueReader;
-import org.apache.avro.io.ValueWriter;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.Encoder;
 import org.apache.avro.ipc.AvroRemoteException;
 import org.apache.avro.ipc.Responder;
 import org.apache.avro.util.Utf8;
@@ -56,7 +56,7 @@
   }
 
   /** Reads a request message. */
-  public Object readRequest(Schema schema, ValueReader in) throws IOException {
+  public Object readRequest(Schema schema, Decoder in) throws IOException {
     Object[] args = new Object[schema.getFields().size()];
     int i = 0;
     for (Map.Entry<String, Schema> param : schema.getFieldSchemas())
@@ -65,14 +65,14 @@
   }
 
   /** Writes a response message. */
-  public void writeResponse(Schema schema, Object response, ValueWriter out)
+  public void writeResponse(Schema schema, Object response, Encoder out)
     throws IOException {
     getDatumWriter(schema).write(response, out);
   }
 
   /** Writes an error message. */
   public void writeError(Schema schema, AvroRemoteException error,
-                         ValueWriter out) throws IOException {
+                         Encoder out) throws IOException {
     getDatumWriter(schema).write(error, out);
   }
 

Modified: hadoop/avro/trunk/src/test/java/org/apache/avro/TestSchema.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/test/java/org/apache/avro/TestSchema.java?rev=787828&r1=787827&r2=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/test/java/org/apache/avro/TestSchema.java (original)
+++ hadoop/avro/trunk/src/test/java/org/apache/avro/TestSchema.java Tue Jun 23 20:38:46 2009
@@ -32,8 +32,10 @@
 import org.apache.avro.generic.GenericDatumWriter;
 import org.apache.avro.io.DatumReader;
 import org.apache.avro.io.DatumWriter;
-import org.apache.avro.io.ValueReader;
-import org.apache.avro.io.ValueWriter;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.Encoder;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.BinaryEncoder;
 import org.apache.avro.util.Utf8;
 import org.testng.annotations.Test;
 
@@ -195,7 +197,7 @@
     throws IOException {
     ByteArrayOutputStream out = new ByteArrayOutputStream();
     writer.setSchema(schema);
-    writer.write(datum, new ValueWriter(out));
+    writer.write(datum, new BinaryEncoder(out));
       
     byte[] data = out.toByteArray();
     // System.out.println("length = "+data.length);
@@ -203,7 +205,7 @@
     reader.setSchema(schema);
         
     Object decoded =
-      reader.read(null, new ValueReader(new ByteArrayInputStream(data)));
+      reader.read(null, new BinaryDecoder(new ByteArrayInputStream(data)));
       
     // System.out.println(GenericData.toString(datum));
     // System.out.println(GenericData.toString(decoded));
@@ -223,7 +225,7 @@
     Schema expected = Schema.parse(recordJson);
     DatumReader in = new GenericDatumReader(ACTUAL, expected);
     GenericData.Record record = (GenericData.Record)
-      in.read(null, new ValueReader(new ByteArrayInputStream(new byte[0])));
+      in.read(null, new BinaryDecoder(new ByteArrayInputStream(new byte[0])));
     assertEquals("Wrong default.", defaultValue, record.get("f"));
     assertEquals("Wrong toString", expected, Schema.parse(expected.toString()));
   }

Copied: hadoop/avro/trunk/src/test/java/org/apache/avro/io/TestBinaryDecoder.java (from r786682, hadoop/avro/trunk/src/test/java/org/apache/avro/io/TestValueReader.java)
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/test/java/org/apache/avro/io/TestBinaryDecoder.java?p2=hadoop/avro/trunk/src/test/java/org/apache/avro/io/TestBinaryDecoder.java&p1=hadoop/avro/trunk/src/test/java/org/apache/avro/io/TestValueReader.java&r1=786682&r2=787828&rev=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/test/java/org/apache/avro/io/TestValueReader.java (original)
+++ hadoop/avro/trunk/src/test/java/org/apache/avro/io/TestBinaryDecoder.java Tue Jun 23 20:38:46 2009
@@ -25,54 +25,54 @@
 import org.testng.annotations.Test;
 
 
-public class TestValueReader {
+public class TestBinaryDecoder {
   /** Verify EOFException throw at EOF */
 
   @Test(expectedExceptions=EOFException.class)
   public void testEOF_boolean() throws IOException {
-    new ValueReader(new ByteArrayInputStream(new byte[0])).readBoolean();
+    new BinaryDecoder(new ByteArrayInputStream(new byte[0])).readBoolean();
   }
   
   @Test(expectedExceptions=EOFException.class)
   public void testEOF_int() throws IOException {
-    new ValueReader(new ByteArrayInputStream(new byte[0])).readInt();
+    new BinaryDecoder(new ByteArrayInputStream(new byte[0])).readInt();
   }
   
   @Test(expectedExceptions=EOFException.class)
   public void testEOF_long() throws IOException {
-    new ValueReader(new ByteArrayInputStream(new byte[0])).readLong();
+    new BinaryDecoder(new ByteArrayInputStream(new byte[0])).readLong();
   }
   
   @Test(expectedExceptions=EOFException.class)
   public void testEOF_float() throws IOException {
-    new ValueReader(new ByteArrayInputStream(new byte[0])).readFloat();
+    new BinaryDecoder(new ByteArrayInputStream(new byte[0])).readFloat();
   }
   
   @Test(expectedExceptions=EOFException.class)
   public void testEOF_double() throws IOException {
-    new ValueReader(new ByteArrayInputStream(new byte[0])).readDouble();
+    new BinaryDecoder(new ByteArrayInputStream(new byte[0])).readDouble();
   }
   
   @Test(expectedExceptions=EOFException.class)
   public void testEOF_bytes() throws IOException {
-    new ValueReader(new ByteArrayInputStream(new byte[0])).readBytes(null);
+    new BinaryDecoder(new ByteArrayInputStream(new byte[0])).readBytes(null);
   }
   
   @Test(expectedExceptions=EOFException.class)
   public void testEOF_string() throws IOException {
-    new ValueReader(new ByteArrayInputStream(new byte[0])).
+    new BinaryDecoder(new ByteArrayInputStream(new byte[0])).
       readString(new Utf8("a"));
   }
   
   @Test(expectedExceptions=EOFException.class)
   public void testEOF_fixed() throws IOException {
-    new ValueReader(new ByteArrayInputStream(new byte[0])).
+    new BinaryDecoder(new ByteArrayInputStream(new byte[0])).
       readFixed(new byte[1]);
   }
 
   @Test(expectedExceptions=EOFException.class)
   public void testEOF_enum() throws IOException {
-    new ValueReader(new ByteArrayInputStream(new byte[0])).readEnum();
+    new BinaryDecoder(new ByteArrayInputStream(new byte[0])).readEnum();
   }
   
 }

Propchange: hadoop/avro/trunk/src/test/java/org/apache/avro/io/TestBinaryDecoder.java
------------------------------------------------------------------------------
    svn:mergeinfo = 

Modified: hadoop/avro/trunk/src/test/java/org/apache/avro/io/TestBlockingIO.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/test/java/org/apache/avro/io/TestBlockingIO.java?rev=787828&r1=787827&r2=787828&view=diff
==============================================================================
--- hadoop/avro/trunk/src/test/java/org/apache/avro/io/TestBlockingIO.java (original)
+++ hadoop/avro/trunk/src/test/java/org/apache/avro/io/TestBlockingIO.java Tue Jun 23 20:38:46 2009
@@ -19,7 +19,7 @@
   
   private static class Tests {
     private final JsonParser parser;
-    private final ValueReader input;
+    private final Decoder input;
     private final int depth;
     public Tests(int bufferSize, int depth, String input)
       throws JsonParseException, IOException {
@@ -31,13 +31,13 @@
           new ByteArrayInputStream(input.getBytes("UTF-8")));
       
       ByteArrayOutputStream os = new ByteArrayOutputStream();
-      ValueWriter cos = new BlockingValueWriter(os, bufferSize);
+      Encoder cos = new BlockingBinaryEncoder(os, bufferSize);
       serialize(cos, p, os);
       cos.flush();
       
       byte[] bb = os.toByteArray();
       // dump(bb);
-      this.input = new ValueReader(new ByteArrayInputStream(bb));
+      this.input = new BinaryDecoder(new ByteArrayInputStream(bb));
       this.parser =  f.createJsonParser(new ByteArrayInputStream(in));
     }
     
@@ -216,7 +216,7 @@
     t.skip(skipLevel);
   }
 
-  private static void skipMap(JsonParser parser, ValueReader input, int depth)
+  private static void skipMap(JsonParser parser, Decoder input, int depth)
     throws IOException, JsonParseException {
     for (long l = input.skipMap(); l != 0; l = input.skipMap()) {
       for (long i = 0; i < l; i++) {
@@ -230,7 +230,7 @@
     parser.skipChildren();
   }
 
-  private static void skipArray(JsonParser parser, ValueReader input, int depth)
+  private static void skipArray(JsonParser parser, Decoder input, int depth)
     throws IOException, JsonParseException {
     for (long l = input.skipArray(); l != 0; l = input.skipArray()) {
       for (long i = 0; i < l; i++) {
@@ -244,7 +244,7 @@
     parser.skipChildren();
   }
  
-  private static void checkString(String s, ValueReader input, int n)
+  private static void checkString(String s, Decoder input, int n)
     throws IOException, UnsupportedEncodingException {
     ByteBuffer buf = input.readBytes(null);
     Assert.assertEquals(n, buf.remaining());
@@ -253,7 +253,7 @@
     Assert.assertEquals(s, s2);
   }
   
-  private static void serialize(ValueWriter cos, JsonParser p,
+  private static void serialize(Encoder cos, JsonParser p,
       ByteArrayOutputStream os)
     throws JsonParseException, IOException {
     boolean[] isArray = new boolean[100];



Mime
View raw message