avro-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From cutt...@apache.org
Subject svn commit: r1074751 - in /avro/trunk: ./ lang/java/avro/src/main/java/org/apache/avro/ lang/java/avro/src/main/java/org/apache/avro/file/ lang/java/avro/src/main/java/org/apache/avro/generic/ lang/java/avro/src/main/java/org/apache/avro/io/ lang/java/...
Date Fri, 25 Feb 2011 23:45:31 GMT
Author: cutting
Date: Fri Feb 25 23:45:30 2011
New Revision: 1074751

URL: http://svn.apache.org/viewvc?rev=1074751&view=rev
Log:
AVRO-769. Java: Align Decoder/Encoder APIs.  Contributed by scottcarey.

Modified:
    avro/trunk/CHANGES.txt
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/file/DataFileReader.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/file/DataFileReader12.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/file/DataFileStream.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BinaryData.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BinaryDecoder.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BlockingBinaryEncoder.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BufferedBinaryEncoder.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/Decoder.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/DecoderFactory.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/DirectBinaryDecoder.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/EncoderFactory.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonDecoder.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonEncoder.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ParsingDecoder.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ResolvingDecoder.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ValidatingDecoder.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ValidatingEncoder.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/package.html
    avro/trunk/lang/java/avro/src/test/java/org/apache/avro/TestReflect.java
    avro/trunk/lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericDatumWriter.java
    avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/Perf.java
    avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestBinaryDecoder.java
    avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestBlockingIO.java
    avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestBlockingIO2.java
    avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestResolvingIO.java
    avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestValidatingIO.java
    avro/trunk/lang/java/ipc/src/main/java/org/apache/avro/ipc/Requestor.java
    avro/trunk/lang/java/ipc/src/main/java/org/apache/avro/ipc/Responder.java
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestSchema.java
    avro/trunk/lang/java/mapred/src/main/java/org/apache/avro/mapred/AvroSerialization.java
    avro/trunk/lang/java/mapred/src/main/java/org/apache/avro/mapred/tether/TetherKeySerialization.java
    avro/trunk/lang/java/mapred/src/test/java/org/apache/avro/mapred/tether/TetherTask.java
    avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/BinaryFragmentToJsonTool.java
    avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/DataFileWriteTool.java
    avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/JsonToBinaryFragmentTool.java
    avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/Util.java

Modified: avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/avro/trunk/CHANGES.txt?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/CHANGES.txt (original)
+++ avro/trunk/CHANGES.txt Fri Feb 25 23:45:30 2011
@@ -68,6 +68,14 @@ Avro 1.5.0 (unreleased)
       does not buffer output. 
     (scottcarey)
 
+    AVRO-769. Java: Align Decoder/Encoder APIs for consistency and long term
+    stability.  Avro's Decoder and Encoder APIs are aligned and now consist of 
+    only read and write operations.  EncoderFactory and DecoderFactory handle
+    all construction and common configuration.  Some specialized implementations
+    have separate configuration APIs. 
+     
+    (scottcarey)
+
     AVRO-670. Allow DataFileWriteTool to accept schema files as input with new
     --schema-file and --schema command-line flags. (Ron Bodkin via philz)
 

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/file/DataFileReader.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/file/DataFileReader.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/file/DataFileReader.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/file/DataFileReader.java Fri Feb 25 23:45:30 2011
@@ -75,7 +75,7 @@ public class DataFileReader<D>
                                                  Header header,
                                                  boolean sync)
       throws IOException {
-    DataFileReader<D> dreader = new DataFileReader(in, reader, header);
+    DataFileReader<D> dreader = new DataFileReader<D>(in, reader, header);
     // seek/sync to an (assumed) valid position
     if (sync)
       dreader.sync(in.tell());
@@ -112,7 +112,7 @@ public class DataFileReader<D>
    * saved while writing a file, use {@link #sync(long)} instead. */
   public void seek(long position) throws IOException {
     sin.seek(position);
-    vin = DecoderFactory.defaultFactory().createBinaryDecoder(this.sin, vin);
+    vin = DecoderFactory.get().binaryDecoder(this.sin, vin);
     datumIn = null;
     blockRemaining = 0;
     blockStart = position;
@@ -122,6 +122,7 @@ public class DataFileReader<D>
    * range of file entires, call this with the starting position, then check
    * {@link #pastSync(long)} with the end point before each call to {@link
    * #next()}. */
+  @Override
   public void sync(long position) throws IOException {
     seek(position);
     try {
@@ -161,6 +162,7 @@ public class DataFileReader<D>
   }
 
   /** Return true if past the next synchronization point after a position. */ 
+  @Override
   public boolean pastSync(long position) throws IOException {
     return ((blockStart >= position+SYNC_SIZE)||(blockStart >= sin.length()));
   }
@@ -211,7 +213,7 @@ public class DataFileReader<D>
       } else {
         return n;
       }
-    };
+    }
 
     @Override
     public long skip(long skip) throws IOException {

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/file/DataFileReader12.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/file/DataFileReader12.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/file/DataFileReader12.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/file/DataFileReader12.java Fri Feb 25 23:45:30 2011
@@ -123,6 +123,7 @@ public class DataFileReader12<D> impleme
   }
 
   /** Return the schema used in this file. */
+  @Override
   public Schema getSchema() { return schema; }
 
   // Iterator and Iterable implementation
@@ -149,6 +150,7 @@ public class DataFileReader12<D> impleme
   @Override public void remove() { throw new UnsupportedOperationException(); }
 
   /** Return the next datum in the file. */
+  @Override
   public synchronized D next(D reuse) throws IOException {
     while (blockCount == 0) {                     // at start of block
 
@@ -179,10 +181,11 @@ public class DataFileReader12<D> impleme
     in.seek(position);
     blockCount = 0;
     blockStart = position;
-    vin = DecoderFactory.defaultFactory().createBinaryDecoder(in, vin);
+    vin = DecoderFactory.get().binaryDecoder(in, vin);
   }
 
   /** Move to the next synchronization point after a position. */
+  @Override
   public synchronized void sync(long position) throws IOException {
     if (in.tell()+SYNC_SIZE >= in.length()) {
       seek(in.length());
@@ -206,14 +209,17 @@ public class DataFileReader12<D> impleme
   }
 
   /** Return true if past the next synchronization point after a position. */ 
+  @Override
   public boolean pastSync(long position) throws IOException {
     return ((blockStart >= position+SYNC_SIZE)||(blockStart >= in.length()));
   }
 
   /** Return the current position in the input. */
+  @Override
   public long tell() throws IOException { return in.tell(); }
 
   /** Close this reader. */
+  @Override
   public synchronized void close() throws IOException {
     in.close();
   }

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/file/DataFileStream.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/file/DataFileStream.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/file/DataFileStream.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/file/DataFileStream.java Fri Feb 25 23:45:30 2011
@@ -94,7 +94,7 @@ public class DataFileStream<D> implement
   /** Initialize the stream by reading from its head. */
   void initialize(InputStream in) throws IOException {
     this.header = new Header();
-    this.vin = DecoderFactory.defaultFactory().createBinaryDecoder(in, vin);
+    this.vin = DecoderFactory.get().binaryDecoder(in, vin);
     byte[] magic = new byte[DataFileConstants.MAGIC.length];
     try {
       vin.readFixed(magic);                         // read magic
@@ -198,7 +198,7 @@ public class DataFileStream<D> implement
           block = nextRawBlock(block);
           block.decompressUsing(codec);
           blockBuffer = block.getAsByteBuffer();
-          datumIn = DecoderFactory.defaultFactory().createBinaryDecoder(
+          datumIn = DecoderFactory.get().binaryDecoder(
               blockBuffer.array(), blockBuffer.arrayOffset() +
               blockBuffer.position(), blockBuffer.remaining(), datumIn);
         }

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java Fri Feb 25 23:45:30 2011
@@ -28,6 +28,7 @@ import org.apache.avro.Schema;
 import org.apache.avro.Schema.Field;
 import org.apache.avro.io.DatumReader;
 import org.apache.avro.io.Decoder;
+import org.apache.avro.io.DecoderFactory;
 import org.apache.avro.io.ResolvingDecoder;
 import org.apache.avro.util.Utf8;
 import org.apache.avro.util.WeakIdentityHashMap;
@@ -100,8 +101,8 @@ public class GenericDatumReader<D> imple
     }
     resolver = cache.get(expected);
     if (resolver == null) {
-      resolver = new ResolvingDecoder(Schema.applyAliases(actual, expected),
-                                      expected, null);
+      resolver = DecoderFactory.get().resolvingDecoder(
+          Schema.applyAliases(actual, expected), expected, null);
       cache.put(expected, resolver);
     }
     
@@ -112,10 +113,11 @@ public class GenericDatumReader<D> imple
     return resolver;
   }
 
+  @Override
   @SuppressWarnings("unchecked")
   public D read(D reuse, Decoder in) throws IOException {
     ResolvingDecoder resolver = getResolver(actual, expected);
-    resolver.init(in);
+    resolver.configure(in);
     D result = (D) read(reuse, expected, resolver);
     resolver.drain();
     return result;

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BinaryData.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BinaryData.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BinaryData.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BinaryData.java Fri Feb 25 23:45:30 2011
@@ -41,8 +41,8 @@ public class BinaryData {
     }
      public void set(byte[] data1, int off1, int len1, 
          byte[] data2, int off2, int len2) {
-       this.d1.init(data1, off1, len1);
-       this.d2.init(data2, off2, len2);
+       this.d1.configure(data1, off1, len1);
+       this.d2.configure(data2, off2, len2);
   }
   }                     // no public ctor
 
@@ -201,7 +201,7 @@ public class BinaryData {
       this.bytes = decoder.getBufferAccessor();
     }
     public void set(byte[] bytes, int start, int len) {
-      this.decoder.init(bytes, start, len);
+      this.decoder.configure(bytes, start, len);
     }
   }
 

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BinaryDecoder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BinaryDecoder.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BinaryDecoder.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BinaryDecoder.java Fri Feb 25 23:45:30 2011
@@ -29,8 +29,7 @@ import org.apache.avro.util.Utf8;
  * Instances are created using {@link DecoderFactory}.
  * <p/>
  * This class may read-ahead and buffer bytes from the source beyond what is
- * required to serve its read methods.  See {@link #inputStream} and 
- * {@link DecoderFactory#configureDirectDecoder(boolean)}.
+ * required to serve its read methods.
  * The number of unused bytes in the buffer can be accessed by
  * inputStream().remaining(), if the BinaryDecoder is not 'direct'.
  * 
@@ -59,43 +58,25 @@ public class BinaryDecoder extends Decod
   protected BinaryDecoder() {
   }
 
-  /**
-   * @deprecated Use {@link DecoderFactory} to create BinaryDecoder instances and
-   *             reinitialize them
-   */
-  @Deprecated
-  public BinaryDecoder(InputStream in) {
-    super();
-    init(in);
-  }
-
-  BinaryDecoder(int bufferSize, InputStream in) {
+  BinaryDecoder(InputStream in, int bufferSize) {
     super();
-    init(bufferSize, in);
+    configure(in, bufferSize);
   }
 
   BinaryDecoder(byte[] data, int offset, int length) {
     super();
-    init(data, offset, length);
-  }
-
-  /**
-   * @deprecated Use {@link DecoderFactory} to create BinaryDecoder instances and
-   *             reinitialize them
-   */
-  @Override
-  @Deprecated
-  public void init(InputStream in) {
-    init(DecoderFactory.DEFAULT_BUFFER_SIZE, in);
+    configure(data, offset, length);
   }
 
-  void init(int bufferSize, InputStream in) {
+  BinaryDecoder configure(InputStream in, int bufferSize) {
     configureSource(bufferSize, new InputStreamByteSource(in));
+    return this;
   }
 
-  void init(byte[] data, int offset, int length) {
+  BinaryDecoder configure(byte[] data, int offset, int length) {
     configureSource(DecoderFactory.DEFAULT_BUFFER_SIZE, new ByteArrayByteSource(
         data, offset, length));
+    return this;
   }
 
   /**
@@ -479,8 +460,6 @@ public class BinaryDecoder extends Decod
    * Avro data with other reads must access this InputStream to do so unless
    * the implementation is 'direct' and does not read beyond the minimum bytes
    * necessary from the source.  
-   * <p>
-   * See {@link DecoderFactory#configureDirectDecoder(boolean)}
    */
   public InputStream inputStream() {
     return source;

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BlockingBinaryEncoder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BlockingBinaryEncoder.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BlockingBinaryEncoder.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BlockingBinaryEncoder.java Fri Feb 25 23:45:30 2011
@@ -30,7 +30,7 @@ import org.apache.avro.Schema;
  * exhausting memory. Values are buffered until the specified block size would
  * be exceeded, minimizing block overhead.
  * <p/>
- * Use {@link EncoderFactory#blockingBinaryEncoder(OutputStream, BinaryEncoder)
+ * Use {@link EncoderFactory#blockingBinaryEncoder(OutputStream, BinaryEncoder)}
  * to construct and configure.
  * <p/>
  * BlockingBinaryEncoder buffers writes, data may not appear on the output until

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BufferedBinaryEncoder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BufferedBinaryEncoder.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BufferedBinaryEncoder.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/BufferedBinaryEncoder.java Fri Feb 25 23:45:30 2011
@@ -33,7 +33,7 @@ import org.apache.avro.AvroRuntimeExcept
  * is acceptable.
  * <p/>
  * To construct or reconfigure, use
- * {@link EncoderFactory#bufferedBinaryEncoder(OutputStream, BinaryEncoder)}.
+ * {@link EncoderFactory#binaryEncoder(OutputStream, BinaryEncoder)}.
  * <p/>
  * To change the buffer size, configure the factory instance used to 
  * create instances with {@link EncoderFactory#configureBufferSize(int)}

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/Decoder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/Decoder.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/Decoder.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/Decoder.java Fri Feb 25 23:45:30 2011
@@ -18,7 +18,6 @@
 package org.apache.avro.io;
 
 import java.io.IOException;
-import java.io.InputStream;
 import java.nio.ByteBuffer;
 
 import org.apache.avro.util.Utf8;
@@ -29,28 +28,18 @@ import org.apache.avro.util.Utf8;
  *  This class has two types of methods.  One type of methods support
  *  the reading of leaf values (for example, {@link #readLong} and
  *  {@link #readString}).
- *
+ *  <p/>
  *  The other type of methods support the reading of maps and arrays.
  *  These methods are {@link #readArrayStart}, {@link #arrayNext},
  *  and similar methods for maps).  See {@link #readArrayStart} for
  *  details on these methods.)
- *
+ *  <p/>
+ *  {@link DecoderFactory} is used to create and configure Decoder instances.
+ *  @see DecoderFactory
  *  @see Encoder
  */
 
 public abstract class Decoder {
-  
-  /** Start reading against a different input stream.  Stateful
-    * subclasses will reset their states to their initial state. 
-    * <p/>
-    * A Decoder may read ahead beyond the minimum bytes necessary,
-    * similar to a BufferedInputStream.
-    * In order to interleave reads on this stream's data between
-    * Decoder and another reader, the other reader must use the
-    * buffer-aware input stream returned by the Decoder implementation,
-    * if applicable.
-   * @throws IOException */
-  public abstract void init(InputStream in) throws IOException;
 
   /**
    * "Reads" a null value.  (Doesn't actually read anything, but

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/DecoderFactory.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/DecoderFactory.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/DecoderFactory.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/DecoderFactory.java Fri Feb 25 23:45:30 2011
@@ -17,8 +17,11 @@
  */
 package org.apache.avro.io;
 
+import java.io.IOException;
 import java.io.InputStream;
 
+import org.apache.avro.Schema;
+
 /**
  * A factory for creating and configuring {@link Decoder}s.
  * <p/>
@@ -31,10 +34,9 @@ import java.io.InputStream;
 
 public class DecoderFactory {
   private static final DecoderFactory DEFAULT_FACTORY = new DefaultDecoderFactory();
-  static final int DEFAULT_BUFFER_SIZE = 32 * 1000;
+  static final int DEFAULT_BUFFER_SIZE = 8192;
 
   int binaryDecoderBufferSize = DEFAULT_BUFFER_SIZE;
-  boolean preferDirect = false;
 
   /** Constructor for factory instances */
   public DecoderFactory() {
@@ -42,18 +44,26 @@ public class DecoderFactory {
   }
 
   /**
+   * @deprecated use the equivalent {@link #get()} instead
+   */
+  @Deprecated
+  public static DecoderFactory defaultFactory() {
+    return get();
+  }
+  
+  /**
    * Returns an immutable static DecoderFactory configured with default settings
    * All mutating methods throw IllegalArgumentExceptions. All creator methods
    * create objects with default settings.
    */
-  public static DecoderFactory defaultFactory() {
+  public static DecoderFactory get() {
     return DEFAULT_FACTORY;
   }
 
   /**
    * Configures this factory to use the specified buffer size when creating
    * Decoder instances that buffer their input. The default buffer size is
-   * 32000 bytes.
+   * 8192 bytes.
    * 
    * @param size The preferred buffer size. Valid values are in the range [32,
    *          16*1024*1024]. Values outside this range are rounded to the nearest
@@ -82,11 +92,57 @@ public class DecoderFactory {
     return this.binaryDecoderBufferSize;
   }
   
+  /** @deprecated use the equivalent
+   *  {@link #binaryDecoder(InputStream, BinaryDecoder)} instead */
+  @Deprecated
+  public BinaryDecoder createBinaryDecoder(InputStream in, BinaryDecoder reuse) {
+    return binaryDecoder(in, reuse);
+  }
+  
+  /**
+   * Creates or reinitializes a {@link BinaryDecoder} with the input stream
+   * provided as the source of data. If <i>reuse</i> is provided, it will be
+   * reinitialized to the given input stream.
+   * <p/>
+   * {@link BinaryDecoder} instances returned by this method buffer their input,
+   * reading up to {@link #getConfiguredBufferSize()} bytes past the minimum
+   * required to satisfy read requests in order to achieve better performance.
+   * If the buffering is not desired, use
+   * {@link #directBinaryDecoder(InputStream, BinaryDecoder)}.
+   * <p/>
+   * {@link BinaryDecoder#inputStream()} provides a view on the data that is
+   * buffer-aware, for users that need to interleave access to data
+   * with the Decoder API.
+   * 
+   * @param in
+   *          The InputStream to initialize to
+   * @param reuse
+   *          The BinaryDecoder to <i>attempt</i> to reuse given the factory
+   *          configuration. A BinaryDecoder implementation may not be
+   *          compatible with reuse, causing a new instance to be returned. If
+   *          null, a new instance is returned.
+   * @return A BinaryDecoder that uses <i>in</i> as its source of data. If
+   *         <i>reuse</i> is null, this will be a new instance. If <i>reuse</i>
+   *         is not null, then it may be reinitialized if compatible, otherwise
+   *         a new instance will be returned.
+   * @see BinaryDecoder
+   * @see Decoder
+   */
+  public BinaryDecoder binaryDecoder(InputStream in, BinaryDecoder reuse) {
+    if (null == reuse || !reuse.getClass().equals(BinaryDecoder.class)) {
+      return new BinaryDecoder(in, binaryDecoderBufferSize);
+    } else {
+      return ((BinaryDecoder)reuse).configure(in, binaryDecoderBufferSize);
+    }
+  }
+  
   /**
-   * Configures this factory to create "direct" BinaryDecoder instances when applicable.
+   * Creates or reinitializes a {@link BinaryDecoder} with the input stream
+   * provided as the source of data. If <i>reuse</i> is provided, it will be
+   * reinitialized to the given input stream.
    * <p/>
-   * The default is false, since buffering or 'read-ahead' decoders can be 
-   * twice as fast.  In most cases a normal BinaryDecoder is sufficient in combination with
+   * {@link BinaryDecoder} instances returned by this method do not buffer their input.
+   * In most cases a buffering BinaryDecoder is sufficient in combination with
    * {@link BinaryDecoder#inputStream()} which provides a buffer-aware view on
    * the data.
    * <p/>
@@ -95,95 +151,49 @@ public class DecoderFactory {
    * must never read beyond the minimum necessary bytes to service a {@link BinaryDecoder}
    * API read request.  
    * <p/>
-   * In the case that the performance of a normal BinaryDecoder does not outweigh the
+   * In the case that the improved performance of a buffering implementation does not outweigh the
    * inconvenience of its buffering semantics, a "direct" decoder can be
    * used.
-   * <p/>
-   * Generally, this distinction only applies to BinaryDecoder that read from an InputStream.
-   * @param useDirect If true, this factory will generate "direct" BinaryDecoder
-   * implementations when applicable. If false (the default) the faster buffering
-   * implementations will be generated.
-   * @return This factory, to enable method chaining:
-   * <pre>
-   * DecoderFactory myFactory = new DecoderFactory().configureDirectDecoder(true);
-   * </pre>
-   */
-  public DecoderFactory configureDirectDecoder(boolean useDirect) {
-    this.preferDirect = useDirect;
-    return this;
-  }
-
-  /**
-   * Creates or reinitializes a {@link BinaryDecoder} with the input stream
-   * provided as the source of data. If <i>reuse</i> is provided, it will be
-   * reinitialized to the given input stream.
-   * <p/>
-   * If this factory is configured to create "direct" BinaryDecoder instances,
-   * this will return a non-buffering variant. Otherwise, this instance will
-   * buffer the number of bytes configured by this factory, reading up to that
-   * many bytes from the InputStream ahead of the minimum required for Decoder
-   * API requests. {@link BinaryDecoder#inputStream()} provides a view on the data
-   * that is buffer-aware, for users that need to access possibly buffered data
-   * outside of the Decoder API.
-   * 
-   * @param in The InputStream to initialize to
-   * @param reuse The BinaryDecoder to <i>attempt<i/> to reuse given the factory
-   *          configuration. A specific BinaryDecoder implementation may not be
-   *          compatible with reuse. For example, a BinaryDecoder created as
-   *          'direct' can not be reinitialized to function in a non-'direct'
-   *          mode. If <i>reuse<i/> is null a new instance is always created.
+   * @param in
+   *          The InputStream to initialize to
+   * @param reuse
+   *          The BinaryDecoder to <i>attempt</i> to reuse given the factory
+   *          configuration. A BinaryDecoder implementation may not be
+   *          compatible with reuse, causing a new instance to be returned. If
+   *          null, a new instance is returned.
    * @return A BinaryDecoder that uses <i>in</i> as its source of data. If
    *         <i>reuse</i> is null, this will be a new instance. If <i>reuse</i>
    *         is not null, then it may be reinitialized if compatible, otherwise
    *         a new instance will be returned.
-   *         <p/>
-   *         example:
-   * 
-   *         <pre>
-   * DecoderFactory factory = new DecoderFactory();
-   * Decoder d = factory.createBinaryDecoder(input, null); // a new BinaryDecoder
-   * d = createBinaryDecoder(input2, d); // reinitializes d to read from input2
-   * factory.configureDirectDecoder(true);
-   * Decoder d2 = factory.createBinaryDecoder(input3, d); // a new BinaryDecoder
-   * </pre>
-   * 
-   *         <i>d2</i> above is not a reused instance of <i>d</d> because the
-   *         latter is not 'direct' and can't be reused to create a 'direct'
-   *         instance. Users must be careful to use the BinaryDecoder returned
-   *         from the factory and not assume that the factory passed in the
-   *         <i>reuse</i> argument
+   * @see DirectBinaryDecoder
+   * @see Decoder
    */
-  public BinaryDecoder createBinaryDecoder(InputStream in, BinaryDecoder reuse) {
-    if (null == reuse) {
-      if (preferDirect) {
-        return new DirectBinaryDecoder(in);
-      } else {
-        return new BinaryDecoder(binaryDecoderBufferSize, in);
-      }
+  public BinaryDecoder directBinaryDecoder(InputStream in, BinaryDecoder reuse) {
+    if (null == reuse || !reuse.getClass().equals(DirectBinaryDecoder.class)) {
+      return new DirectBinaryDecoder(in);
     } else {
-      if (!preferDirect) {
-        if(reuse.getClass() == BinaryDecoder.class) {
-          reuse.init(binaryDecoderBufferSize, in);
-          return reuse;
-        } else {
-          return new BinaryDecoder(binaryDecoderBufferSize, in);
-        }
-      } else {
-        if (reuse.getClass() == DirectBinaryDecoder.class) {
-          ((DirectBinaryDecoder)reuse).init(in);
-          return reuse;
-        } else {
-          return new DirectBinaryDecoder(in);
-        }
-      }
+      return ((DirectBinaryDecoder)reuse).configure(in);
     }
   }
 
+  /** @deprecated use {@link #binaryDecoder(byte[], int, int, BinaryDecoder)}
+   * instead */
+  @Deprecated
+  public BinaryDecoder createBinaryDecoder(byte[] bytes, int offset,
+      int length, BinaryDecoder reuse) {
+    if (null == reuse || !reuse.getClass().equals(BinaryDecoder.class)) {
+      return new BinaryDecoder(bytes, offset, length);
+    } else {
+      return reuse.configure(bytes, offset, length);
+    }
+  }
+  
   /**
    * Creates or reinitializes a {@link BinaryDecoder} with the byte array
    * provided as the source of data. If <i>reuse</i> is provided, it will
-   * attempt to reinitiailize <i>reuse</i> to the new byte array. This instance
+   * attempt to reinitialize <i>reuse</i> to the new byte array. This instance
    * will use the provided byte array as its buffer.
+   * <p/>
    * {@link BinaryDecoder#inputStream()} provides a view on the data that is
    * buffer-aware and can provide a view of the data not yet read by Decoder API
    * methods.
@@ -199,34 +209,108 @@ public class DecoderFactory {
    *         returned. Clients must not assume that <i>reuse</i> is
    *         reinitialized and returned.
    */
-  public BinaryDecoder createBinaryDecoder(byte[] bytes, int offset,
+  public BinaryDecoder binaryDecoder(byte[] bytes, int offset,
       int length, BinaryDecoder reuse) {
-    if (null != reuse && reuse.getClass() == BinaryDecoder.class) {
-      reuse.init(bytes, offset, length);
-      return reuse;
-    } else {
+    if (null == reuse || !reuse.getClass().equals(BinaryDecoder.class)) {
       return new BinaryDecoder(bytes, offset, length);
+    } else {
+      return reuse.configure(bytes, offset, length);
     }
   }
 
+  /** @deprecated use {@link #binaryDecoder(byte[], BinaryDecoder)} instead */
+//  @Deprecated
+//  public BinaryDecoder createBinaryDecoder(byte[] bytes, BinaryDecoder reuse) {
+//    return binaryDecoder(bytes, 0, bytes.length, reuse);
+//  }
+  
   /**
    * This method is shorthand for
    * <pre>
    * createBinaryDecoder(bytes, 0, bytes.length, reuse);
-   * </pre> {@link #createBinaryDecoder(byte[], int, int, BinaryDecoder)}
+   * </pre> {@link #binaryDecoder(byte[], int, int, BinaryDecoder)}
+   */
+  public BinaryDecoder binaryDecoder(byte[] bytes, BinaryDecoder reuse) {
+    return binaryDecoder(bytes, 0, bytes.length, reuse);
+  }
+
+  /**
+   * Creates a {@link JsonDecoder} using the InputStrim provided for reading
+   * data that conforms to the Schema provided.
+   * <p/>
+   * 
+   * @param schema
+   *          The Schema for data read from this JsonEncoder. Cannot be null.
+   * @param input
+   *          The InputStream to read from. Cannot be null.
+   * @return A JsonEncoder configured with <i>input</i> and <i>schema</i>
+   * @throws IOException
    */
-  public BinaryDecoder createBinaryDecoder(byte[] bytes, BinaryDecoder reuse) {
-    return createBinaryDecoder(bytes, 0, bytes.length, reuse);
+  public JsonDecoder jsonDecoder(Schema schema, InputStream input)
+      throws IOException {
+    return new JsonDecoder(schema, input);
+  }
+  
+  /**
+   * Creates a {@link JsonDecoder} using the String provided for reading data
+   * that conforms to the Schema provided.
+   * <p/>
+   * 
+   * @param schema
+   *          The Schema for data read from this JsonEncoder. Cannot be null.
+   * @param input
+   *          The String to read from. Cannot be null.
+   * @return A JsonEncoder configured with <i>input</i> and <i>schema</i>
+   * @throws IOException
+   */
+  public JsonDecoder jsonDecoder(Schema schema, String input)
+      throws IOException {
+    return new JsonDecoder(schema, input);
   }
 
+  /**
+   * Creates a {@link ValidatingDecoder} wrapping the Decoder provided. This
+   * ValidatingDecoder will ensure that operations against it conform to the
+   * schema provided.
+   * 
+   * @param schema
+   *          The Schema to validate against. Cannot be null.
+   * @param wrapped
+   *          The Decoder to wrap.
+   * @return A ValidatingDecoder configured with <i>wrapped</i> and
+   *         <i>schema</i>
+   * @throws IOException
+   */
+  public ValidatingDecoder validatingDecoder(Schema schema, Decoder wrapped)
+      throws IOException {
+    return new ValidatingDecoder(schema, wrapped);
+  }
+
+  /**
+   * Creates a {@link ResolvingDecoder} wrapping the Decoder provided. This
+   * ResolvingDecoder will resolve input conforming to the <i>writer</i> schema
+   * from the wrapped Decoder, and present it as the <i>reader</i> schema.
+   * 
+   * @param writer
+   *          The Schema that the source data is in. Cannot be null.
+   * @param reader
+   *          The Schema that the reader wishes to read the data as. Cannot be
+   *          null.
+   * @param wrapped
+   *          The Decoder to wrap.
+   * @return A ResolvingDecoder configured to resolve <i>writer</i> to
+   *         <i>reader</i> from <i>in</i>
+   * @throws IOException
+   */
+  public ResolvingDecoder resolvingDecoder(Schema writer, Schema reader,
+      Decoder wrapped) throws IOException {
+    return new ResolvingDecoder(writer, reader, wrapped);
+  }
+  
   private static class DefaultDecoderFactory extends DecoderFactory {
     @Override
     public DecoderFactory configureDecoderBufferSize(int bufferSize) {
       throw new IllegalArgumentException("This Factory instance is Immutable");
     }
-    @Override
-    public DecoderFactory configureDirectDecoder(boolean arg0) {
-      throw new IllegalArgumentException("This Factory instance is Immutable");
-    }
   }
 }

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/DirectBinaryDecoder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/DirectBinaryDecoder.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/DirectBinaryDecoder.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/DirectBinaryDecoder.java Fri Feb 25 23:45:30 2011
@@ -74,14 +74,14 @@ class DirectBinaryDecoder extends Binary
 
   DirectBinaryDecoder(InputStream in) {
     super();
-    init(in);
+    configure(in);
   }
 
-  @Override
-  public void init(InputStream in) {
+  DirectBinaryDecoder configure(InputStream in) {
     this.in = in;
     byteReader = (in instanceof ByteBufferInputStream) ?
             new ReuseByteReader((ByteBufferInputStream) in) : new ByteReader();
+    return this;
   }
 
   @Override

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/EncoderFactory.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/EncoderFactory.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/EncoderFactory.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/EncoderFactory.java Fri Feb 25 23:45:30 2011
@@ -25,7 +25,7 @@ import org.apache.avro.Schema;
 import org.codehaus.jackson.JsonGenerator;
 
 /**
- * A factory for creating and configuring {@link Encoders}s.
+ * A factory for creating and configuring {@link Encoder} instances.
  * <p/>
  * Factory methods that create Encoder instances are thread-safe.
  * Multiple instances with different configurations can be cached
@@ -91,6 +91,7 @@ public class EncoderFactory {
    * Returns this factory's configured default buffer size.  Used when creating
    * Encoder instances that buffer writes.
    * @see #configureBufferSize(int)
+   * @see #binaryEncoder(OutputStream, BinaryEncoder)
    * @return The preferred buffer size, in bytes.
    */
   public int getBufferSize() {
@@ -124,12 +125,13 @@ public class EncoderFactory {
 
   /**
    * Returns this factory's configured default block buffer size.  
-   * {@link BlockingBinaryEncoder} instances created by this factory will
-   * have block buffers of this size.
+   * {@link BinaryEncoder} instances created with
+   * #blockingBinaryEncoder(OutputStream, BinaryEncoder)
+   * will have block buffers of this size.
    * <p/>
-   * @see #configureBlockBufferSize
-   * @see BlockingBinaryEncoder
-   * @return The preferred buffer size, in bytes.
+   * @see #configureBlockSize(int)
+   * @see #blockingBinaryEncoder(OutputStream, BinaryEncoder)
+   * @return The preferred block size, in bytes.
    */
   public int getBlockSize() {
     return this.binaryBlockSize;
@@ -143,8 +145,8 @@ public class EncoderFactory {
    * <p/>
    * The {@link BinaryEncoder} implementation returned may buffer its output.
    * Data may not appear on the underlying OutputStream until
-   * {@link Encoder.flush()} is called.  The buffer size is configured with
-   * {@link #configureBufferSize(int).
+   * {@link Encoder#flush()} is called.  The buffer size is configured with
+   * {@link #configureBufferSize(int)}.
    * </p>  If buffering is not desired, and lower performance is acceptable, use 
    * {@link #directBinaryEncoder(OutputStream, BinaryEncoder)}
    * <p/>
@@ -180,7 +182,7 @@ public class EncoderFactory {
    * new instance, but this is not guaranteed, a new instance may be returned.
    * <p/>
    * The {@link BinaryEncoder} implementation returned does not buffer its
-   * output, calling {@link Encoder.flush()} will simply cause the wrapped
+   * output, calling {@link Encoder#flush()} will simply cause the wrapped
    * OutputStream to be flushed.
    * <p/>
    * Performance of unbuffered writes can be significantly slower than buffered
@@ -221,7 +223,7 @@ public class EncoderFactory {
    * new instance, but this is not guaranteed, a new instance may be returned.
    * <p/>
    * The {@link BinaryEncoder} implementation returned buffers its output,
-   * calling {@link Encoder.flush()} is required for output to appear on the underlying
+   * calling {@link Encoder#flush()} is required for output to appear on the underlying
    * OutputStream.
    * <p/>
    * The returned BinaryEncoder implements the Avro binary encoding using blocks
@@ -260,7 +262,7 @@ public class EncoderFactory {
    * data conforming to the Schema provided.
    * <p/>
    * {@link JsonEncoder} buffers its output. Data may not appear on the
-   * underlying OutputStream until {@link Encoder.flush()} is called.
+   * underlying OutputStream until {@link Encoder#flush()} is called.
    * <p/>
    * {@link JsonEncoder} is not thread-safe.
    * 
@@ -281,7 +283,7 @@ public class EncoderFactory {
    * output of data conforming to the Schema provided.
    * <p/>
    * {@link JsonEncoder} buffers its output. Data may not appear on the
-   * underlying output until {@link Encoder.flush()} is called.
+   * underlying output until {@link Encoder#flush()} is called.
    * <p/>
    * {@link JsonEncoder} is not thread-safe.
    * 
@@ -303,7 +305,7 @@ public class EncoderFactory {
    * to the schema provided.
    * <p/>
    * Many {@link Encoder}s buffer their output. Data may not appear on the
-   * underlying output until {@link Encoder.flush()} is called.
+   * underlying output until {@link Encoder#flush()} is called.
    * <p/>
    * {@link ValidatingEncoder} is not thread-safe.
    * 

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonDecoder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonDecoder.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonDecoder.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonDecoder.java Fri Feb 25 23:45:30 2011
@@ -40,45 +40,80 @@ public class JsonDecoder extends Parsing
   
   static final String CHARSET = "ISO-8859-1";
 
-  JsonDecoder(Symbol root, InputStream in) throws IOException {
+  private JsonDecoder(Symbol root, InputStream in) throws IOException {
     super(root);
-    init(in);
+    configure(in);
   }
   
-  JsonDecoder(Symbol root, String in) throws IOException {
+  private JsonDecoder(Symbol root, String in) throws IOException {
     super(root);
-    init(in);
+    configure(in);
   }
 
-  /** Creates a new JsonDecoder based on an InputStream. */
-  public JsonDecoder(Schema schema, InputStream in) throws IOException {
-    this(new JsonGrammarGenerator().generate(schema), in);
+  JsonDecoder(Schema schema, InputStream in) throws IOException {
+    this(getSymbol(schema), in);
   }
   
-  /** Creates a new JsonDecoder based on a String input. */
-  public JsonDecoder(Schema schema, String in) throws IOException {
-    this(new JsonGrammarGenerator().generate(schema), in);
+  JsonDecoder(Schema schema, String in) throws IOException {
+    this(getSymbol(schema), in);
   }
-
-  private void advance(Symbol symbol) throws IOException {
-    this.parser.processTrailingImplicitActions();
-    if (in.getCurrentToken() == null && this.parser.depth() == 1)
-      throw new EOFException();
-    parser.advance(symbol);
-  }
-
-  @Override
-  public void init(InputStream in) throws IOException {
+  
+  private static Symbol getSymbol(Schema schema) {
+    if (null == schema) {
+      throw new NullPointerException("Schema cannot be null!");
+    }
+    return new JsonGrammarGenerator().generate(schema);
+  }
+
+  /**
+   * Reconfigures this JsonDecoder to use the InputStream provided.
+   * <p/>
+   * If the InputStream provided is null, a NullPointerException is thrown.
+   * <p/>
+   * Otherwise, this JsonDecoder will reset its state and then
+   * reconfigure its input.
+   * @param in
+   *   The IntputStream to read from. Cannot be null.
+   * @throws IOException
+   * @return this JsonDecoder
+   */
+  public JsonDecoder configure(InputStream in) throws IOException {
+    if (null == in) {
+      throw new NullPointerException("InputStream to read from cannot be null!");
+    }
     parser.reset();
     this.in = jsonFactory.createJsonParser(in);
     this.in.nextToken();
+    return this;
   }
   
-  /** Re-initializes to start reading from a new String input. */
-  public void init(String in) throws IOException {
+  /**
+   * Reconfigures this JsonDecoder to use the String provided for input.
+   * <p/>
+   * If the String provided is null, a NullPointerException is thrown.
+   * <p/>
+   * Otherwise, this JsonDecoder will reset its state and then
+   * reconfigure its input.
+   * @param in
+   *   The String to read from. Cannot be null.
+   * @throws IOException
+   * @return this JsonDecoder
+   */
+  public JsonDecoder configure(String in) throws IOException {
+    if (null == in) {
+      throw new NullPointerException("String to read from cannot be null!");
+    }
     parser.reset();
     this.in = new JsonFactory().createJsonParser(in);
     this.in.nextToken();
+    return this;
+  }
+
+  private void advance(Symbol symbol) throws IOException {
+    this.parser.processTrailingImplicitActions();
+    if (in.getCurrentToken() == null && this.parser.depth() == 1)
+      throw new EOFException();
+    parser.advance(symbol);
   }
 
   @Override
@@ -384,6 +419,7 @@ public class JsonDecoder extends Parsing
     return n;
   }
 
+  @Override
   public Symbol doAction(Symbol input, Symbol top) throws IOException {
     if (top instanceof Symbol.FieldAdjustAction) {
       Symbol.FieldAdjustAction fa = (Symbol.FieldAdjustAction) top;
@@ -420,7 +456,5 @@ public class JsonDecoder extends Parsing
         ". Got " + in.getCurrentToken());
   }
 
-  public void setItemCount(long itemCount) throws IOException {
-  }
 }
 

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonEncoder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonEncoder.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonEncoder.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/JsonEncoder.java Fri Feb 25 23:45:30 2011
@@ -86,7 +86,7 @@ public class JsonEncoder extends Parsing
    * @param out
    *          The OutputStream to direct output to. Cannot be null.
    * @throws IOException
-   * @return
+   * @return this JsonEncoder
    */
   public JsonEncoder configure(OutputStream out) throws IOException {
     this.configure(getJsonGenerator(out));
@@ -104,14 +104,16 @@ public class JsonEncoder extends Parsing
    * @param generator
    *          The JsonGenerator to direct output to. Cannot be null.
    * @throws IOException
+   * @return this JsonEncoder
    */
-  public void configure(JsonGenerator generator) throws IOException {
+  public JsonEncoder configure(JsonGenerator generator) throws IOException {
     if (null == generator)
       throw new NullPointerException("JsonGenerator cannot be null");
     if (null != parser) {
       flush();
     }
     this.out = generator;
+    return this;
   }
 
   @Override

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ParsingDecoder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ParsingDecoder.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ParsingDecoder.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ParsingDecoder.java Fri Feb 25 23:45:30 2011
@@ -30,16 +30,18 @@ public abstract class ParsingDecoder ext
   implements ActionHandler, SkipHandler {
   protected final SkipParser parser;
 
-  public ParsingDecoder(Symbol root) throws IOException {
+  protected ParsingDecoder(Symbol root) throws IOException {
     this.parser = new SkipParser(root, this, this);
   }
 
   protected abstract void skipFixed() throws IOException;
   
+  @Override
   public void skipAction() throws IOException {
     parser.popSymbol();
   }
 
+  @Override
   public void skipTopSymbol() throws IOException {
     Symbol top = parser.topSymbol();
     if (top == Symbol.NULL) {

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ResolvingDecoder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ResolvingDecoder.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ResolvingDecoder.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ResolvingDecoder.java Fri Feb 25 23:45:30 2011
@@ -41,7 +41,7 @@ public class ResolvingDecoder extends Va
 
   private Decoder backup;
   
-  public ResolvingDecoder(Schema writer, Schema reader, Decoder in)
+  ResolvingDecoder(Schema writer, Schema reader, Decoder in)
     throws IOException {
     this(resolve(writer, reader), in);
   }
@@ -54,7 +54,7 @@ public class ResolvingDecoder extends Va
    * @param in  The underlying decoder.
    * @throws IOException
    */
-  public ResolvingDecoder(Object resolver, Decoder in)
+  private ResolvingDecoder(Object resolver, Decoder in)
     throws IOException {
     super((Symbol) resolver, in);
   }
@@ -66,13 +66,19 @@ public class ResolvingDecoder extends Va
    * in many ResolvingDecoders. This method is reasonably expensive, the
    * users are encouraged to cache the result.
    * 
-   * @param writer  The writer's schema.
-   * @param reader  The reader's schema.
+   * @param writer  The writer's schema. Cannot be null.
+   * @param reader  The reader's schema. Cannot be null.
    * @return  The opaque reolver.
    * @throws IOException
    */
   public static Object resolve(Schema writer, Schema reader)
     throws IOException {
+    if (null == writer) {
+      throw new NullPointerException("writer cannot be null!");
+    }
+    if (null == reader) {
+      throw new NullPointerException("reader cannot be null!");
+    }
     return new ResolvingGrammarGenerator().generate(writer, reader);
   }
 
@@ -226,8 +232,8 @@ public class ResolvingDecoder extends Va
     } else if (top instanceof Symbol.DefaultStartAction) {
       Symbol.DefaultStartAction dsa = (Symbol.DefaultStartAction) top;
       backup = in;
-      in = DecoderFactory.defaultFactory()
-        .createBinaryDecoder(dsa.contents, null);
+      in = DecoderFactory.get()
+        .binaryDecoder(dsa.contents, null);
     } else if (top == Symbol.DEFAULT_END_ACTION) {
       in = backup;
     } else {
@@ -251,8 +257,8 @@ public class ResolvingDecoder extends Va
     } else if (top instanceof Symbol.DefaultStartAction) {
       Symbol.DefaultStartAction dsa = (Symbol.DefaultStartAction) top;
       backup = in;
-      in = DecoderFactory.defaultFactory()
-        .createBinaryDecoder(dsa.contents, null);
+      in = DecoderFactory.get()
+        .binaryDecoder(dsa.contents, null);
     } else if (top == Symbol.DEFAULT_END_ACTION) {
       in = backup;
     }

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ValidatingDecoder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ValidatingDecoder.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ValidatingDecoder.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ValidatingDecoder.java Fri Feb 25 23:45:30 2011
@@ -18,7 +18,6 @@
 package org.apache.avro.io;
 
 import java.io.IOException;
-import java.io.InputStream;
 import java.nio.ByteBuffer;
 
 import org.apache.avro.AvroTypeException;
@@ -38,23 +37,25 @@ public class ValidatingDecoder extends P
 
   ValidatingDecoder(Symbol root, Decoder in) throws IOException {
     super(root);
-    this.in = in;
+    this.configure(in);
   }
 
-  public ValidatingDecoder(Schema schema, Decoder in) throws IOException {
-    this(new ValidatingGrammarGenerator().generate(schema), in);
+  ValidatingDecoder(Schema schema, Decoder in) throws IOException {
+    this(getSymbol(schema), in);
+  }
+  
+  private static Symbol getSymbol(Schema schema) {
+    if (null == schema) {
+      throw new NullPointerException("Schema cannot be null");
+    }
+    return new ValidatingGrammarGenerator().generate(schema);
   }
 
   /** Re-initialize, reading from a new underlying Decoder. */
-  public void init(Decoder in) throws IOException {
-    parser.reset();
+  public ValidatingDecoder configure(Decoder in) throws IOException {
+    this.parser.reset();
     this.in = in;
-  }
-
-  @Override
-  public void init(InputStream in) throws IOException {
-    parser.reset();
-    this.in.init(in);
+    return this;
   }
 
   @Override
@@ -139,6 +140,7 @@ public class ValidatingDecoder extends P
     in.skipFixed(length);
   }
 
+  @Override
   protected void skipFixed() throws IOException {
     parser.advance(Symbol.FIXED);
     Symbol.IntCheckAction top = (Symbol.IntCheckAction) parser.popSymbol();
@@ -231,6 +233,7 @@ public class ValidatingDecoder extends P
     return result;
   }
   
+  @Override
   public Symbol doAction(Symbol input, Symbol top) throws IOException {
     return null;
   }

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ValidatingEncoder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ValidatingEncoder.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ValidatingEncoder.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/ValidatingEncoder.java Fri Feb 25 23:45:30 2011
@@ -60,7 +60,7 @@ public class ValidatingEncoder extends P
 
   /**
    * Reconfigures this ValidatingEncoder to wrap the encoder provided.
-   * @param in
+   * @param encoder
    *   The Encoder to wrap for validation.
    */
   public void configure(Encoder encoder) {

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/package.html
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/package.html?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/package.html (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/package.html Fri Feb 25 23:45:30 2011
@@ -26,7 +26,7 @@
   org.apache.avro.io.DatumReader} and {@link
   org.apache.avro.io.DatumWriter} implementations.  Generic
   implementations are provided in the {@link org.apache.avro.generic}
-  package.  A {@link org.apache.avro.specific.SpecificCompiler
+  package.  A {@link org.apache.avro.compiler.specific.SpecificCompiler
   compiler} can generate specific java classes and interfaces for
   schemas and protocols.  Schemas may be automatically generated for
   existing Java classes by reflection using the {@link

Modified: avro/trunk/lang/java/avro/src/test/java/org/apache/avro/TestReflect.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/test/java/org/apache/avro/TestReflect.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/test/java/org/apache/avro/TestReflect.java (original)
+++ avro/trunk/lang/java/avro/src/test/java/org/apache/avro/TestReflect.java Fri Feb 25 23:45:30 2011
@@ -361,7 +361,7 @@ public class TestReflect {
     writer.write(object, factory.directBinaryEncoder(out, null));
     ReflectDatumReader<Object> reader = new ReflectDatumReader<Object>(s);
     Object after =
-      reader.read(null, DecoderFactory.defaultFactory().createBinaryDecoder(
+      reader.read(null, DecoderFactory.get().binaryDecoder(
           out.toByteArray(), null));
     assertEquals(object, after);
   }
@@ -401,7 +401,7 @@ public class TestReflect {
     ReflectDatumReader<SampleRecord> reader = 
       new ReflectDatumReader<SampleRecord>(schm);
     SampleRecord decoded =
-      reader.read(null, DecoderFactory.defaultFactory().createBinaryDecoder(
+      reader.read(null, DecoderFactory.get().binaryDecoder(
           out.toByteArray(), null));
     assertEquals(record, decoded);
   }
@@ -423,7 +423,7 @@ public class TestReflect {
     ReflectDatumReader<AnotherSampleRecord> reader = 
       new ReflectDatumReader<AnotherSampleRecord>(schm);
     ByteArrayInputStream in = new ByteArrayInputStream(out.toByteArray());
-    Decoder d = DecoderFactory.defaultFactory().createBinaryDecoder(in, null);
+    Decoder d = DecoderFactory.get().binaryDecoder(in, null);
     AnotherSampleRecord decoded = reader.read(null, d);
     assertEquals(a, decoded);
     decoded = reader.read(null, d);
@@ -559,7 +559,7 @@ public class TestReflect {
 
     ReflectDatumReader<Object> reader = new ReflectDatumReader<Object>(schema);
     Object decoded =
-      reader.read(null, DecoderFactory.defaultFactory().createBinaryDecoder(
+      reader.read(null, DecoderFactory.get().binaryDecoder(
           data, null));
       
     assertEquals(0, ReflectData.get().compare(datum, decoded, schema));

Modified: avro/trunk/lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericDatumWriter.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericDatumWriter.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericDatumWriter.java (original)
+++ avro/trunk/lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericDatumWriter.java Fri Feb 25 23:45:30 2011
@@ -24,6 +24,7 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 
 import org.apache.avro.Schema;
+import org.apache.avro.io.DecoderFactory;
 import org.apache.avro.io.Encoder;
 import org.apache.avro.io.EncoderFactory;
 import org.apache.avro.io.JsonDecoder;
@@ -46,7 +47,7 @@ public class TestGenericDatumWriter {
     e.flush();
     
     Object o = new GenericDatumReader<GenericRecord>(s).read(null,
-        new JsonDecoder(s, new ByteArrayInputStream(bao.toByteArray())));
+        DecoderFactory.get().jsonDecoder(s, new ByteArrayInputStream(bao.toByteArray())));
     assertEquals(r, o);
   }
 }

Modified: avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/Perf.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/Perf.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/Perf.java (original)
+++ avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/Perf.java Fri Feb 25 23:45:30 2011
@@ -332,7 +332,7 @@ public class Perf {
     }
 
     protected Decoder newDecoder() {
-      return decoder_factory.createBinaryDecoder(data, null);
+      return decoder_factory.binaryDecoder(data, null);
     }
     
     protected Encoder newEncoder() {

Modified: avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestBinaryDecoder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestBinaryDecoder.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestBinaryDecoder.java (original)
+++ avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestBinaryDecoder.java Fri Feb 25 23:45:30 2011
@@ -45,11 +45,11 @@ import org.junit.runners.Parameterized.P
 public class TestBinaryDecoder {
   // prime number buffer size so that looping tests hit the buffer edge
   // at different points in the loop.
-  DecoderFactory factory;
+  DecoderFactory factory = new DecoderFactory().configureDecoderBufferSize(521);
+  private boolean useDirect = false;
   static EncoderFactory e_factory = EncoderFactory.get();
   public TestBinaryDecoder(boolean useDirect) {
-    factory = new DecoderFactory().configureDecoderBufferSize(521);
-    factory.configureDirectDecoder(useDirect);
+    this.useDirect = useDirect;
   }
   
   @Parameters
@@ -66,16 +66,20 @@ public class TestBinaryDecoder {
 
   private Decoder newDecoder(byte[] bytes, int start, int len)
     throws IOException {
-    return factory.createBinaryDecoder(bytes, start, len, null);
+    return factory.binaryDecoder(bytes, start, len, null);
     
   }
 
   private Decoder newDecoder(InputStream in) {
-    return factory.createBinaryDecoder(in, null);
+    if (useDirect) {
+      return factory.directBinaryDecoder(in, null);
+    } else {
+      return factory.binaryDecoder(in, null);
+    }
   }
 
   private Decoder newDecoder(byte[] bytes) throws IOException {
-    return factory.createBinaryDecoder(bytes, null);
+    return factory.binaryDecoder(bytes, null);
   }
 
   /** Verify EOFException throw at EOF */
@@ -144,7 +148,7 @@ public class TestBinaryDecoder {
     ByteBuffer bb1 = d.readBytes(null);
     Assert.assertEquals(b1.length, bb1.limit() - bb1.position());
     
-    d.init(new ByteBufferInputStream(bbo2.getBufferList()));
+    d.configure(new ByteBufferInputStream(bbo2.getBufferList()));
     ByteBuffer bb2 = d.readBytes(null);
     Assert.assertEquals(b1.length, bb2.limit() - bb2.position());
     
@@ -202,11 +206,11 @@ public class TestBinaryDecoder {
 
     Decoder fromOffsetArray = newDecoder(data2, 15, data.length);
 
-    BinaryDecoder initOnInputStream = factory.createBinaryDecoder(
+    BinaryDecoder initOnInputStream = factory.binaryDecoder(
         new byte[50], 0, 30, null);
-    initOnInputStream = factory.createBinaryDecoder(is2, initOnInputStream);
-    BinaryDecoder initOnArray = factory.createBinaryDecoder(is3, null);
-    initOnArray = factory.createBinaryDecoder(
+    initOnInputStream = factory.binaryDecoder(is2, initOnInputStream);
+    BinaryDecoder initOnArray = factory.binaryDecoder(is3, null);
+    initOnArray = factory.binaryDecoder(
         data, 0, data.length, initOnArray);
     
     for (Object datum : records) {
@@ -236,16 +240,16 @@ public class TestBinaryDecoder {
       InputStream test = bd.inputStream();
       InputStream check = new ByteArrayInputStream(data);
       validateInputStreamReads(test, check);
-      bd = factory.createBinaryDecoder(data, bd);
+      bd = factory.binaryDecoder(data, bd);
       test = bd.inputStream();
       check = new ByteArrayInputStream(data);
       validateInputStreamSkips(test, check);
       // with input stream sources
-      bd = factory.createBinaryDecoder(new ByteArrayInputStream(data), bd);
+      bd = factory.binaryDecoder(new ByteArrayInputStream(data), bd);
       test = bd.inputStream();
       check = new ByteArrayInputStream(data);
       validateInputStreamReads(test, check);
-      bd = factory.createBinaryDecoder(new ByteArrayInputStream(data), bd);
+      bd = factory.binaryDecoder(new ByteArrayInputStream(data), bd);
       test = bd.inputStream();
       check = new ByteArrayInputStream(data);
       validateInputStreamSkips(test, check);
@@ -260,7 +264,7 @@ public class TestBinaryDecoder {
       InputStream test = bd.inputStream();
       InputStream check = new ByteArrayInputStream(data);
       // detach input stream and decoder from old source
-      factory.createBinaryDecoder(new byte[56], null);
+      factory.binaryDecoder(new byte[56], null);
       InputStream bad = bd.inputStream();
       InputStream check2 = new ByteArrayInputStream(data);
       validateInputStreamReads(test, check);
@@ -270,7 +274,7 @@ public class TestBinaryDecoder {
   
   @Test
   public void testInputStreamPartiallyUsed() throws IOException {
-    BinaryDecoder bd = factory.createBinaryDecoder(
+    BinaryDecoder bd = factory.binaryDecoder(
         new ByteArrayInputStream(data), null);
     InputStream test = bd.inputStream();
     InputStream check = new ByteArrayInputStream(data);
@@ -328,7 +332,7 @@ public class TestBinaryDecoder {
   public void testBadIntEncoding() throws IOException {
     byte[] badint = new byte[5];
     Arrays.fill(badint, (byte)0xff);
-    Decoder bd = factory.createBinaryDecoder(badint, null);
+    Decoder bd = factory.binaryDecoder(badint, null);
     String message = "";
     try {
       bd.readInt();
@@ -342,7 +346,7 @@ public class TestBinaryDecoder {
   public void testBadLongEncoding() throws IOException {
     byte[] badint = new byte[10];
     Arrays.fill(badint, (byte)0xff);
-    Decoder bd = factory.createBinaryDecoder(badint, null);
+    Decoder bd = factory.binaryDecoder(badint, null);
     String message = "";
     try {
       bd.readLong();
@@ -394,7 +398,7 @@ public class TestBinaryDecoder {
           throw e;
         }
       }
-      bd = factory.createBinaryDecoder(new ByteArrayInputStream(data), bd);
+      bd = factory.binaryDecoder(new ByteArrayInputStream(data), bd);
       skipGenerated(bd);
       try {
         Assert.assertTrue(bd.isEnd());

Modified: avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestBlockingIO.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestBlockingIO.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestBlockingIO.java (original)
+++ avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestBlockingIO.java Fri Feb 25 23:45:30 2011
@@ -70,7 +70,7 @@ public class TestBlockingIO {
       
       byte[] bb = os.toByteArray();
       // dump(bb);
-      this.input = DecoderFactory.defaultFactory().createBinaryDecoder(bb, null);
+      this.input = DecoderFactory.get().binaryDecoder(bb, null);
       this.parser =  f.createJsonParser(new ByteArrayInputStream(in));
     }
     

Modified: avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestBlockingIO2.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestBlockingIO2.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestBlockingIO2.java (original)
+++ avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestBlockingIO2.java Fri Feb 25 23:45:30 2011
@@ -53,7 +53,7 @@ public class TestBlockingIO2 {
     
     byte[] bb = os.toByteArray();
     
-    decoder = DecoderFactory.defaultFactory().createBinaryDecoder(bb, null);
+    decoder = DecoderFactory.get().binaryDecoder(bb, null);
     this.calls = calls;
   }
     

Modified: avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestResolvingIO.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestResolvingIO.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestResolvingIO.java (original)
+++ avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestResolvingIO.java Fri Feb 25 23:45:30 2011
@@ -101,7 +101,7 @@ public class TestResolvingIO {
     switch (encoding) {
     case BINARY:
     case BLOCKING_BINARY:
-      bvi = DecoderFactory.defaultFactory().createBinaryDecoder(bytes, null);
+      bvi = DecoderFactory.get().binaryDecoder(bytes, null);
       break;
     case JSON:
       InputStream in = new ByteArrayInputStream(bytes);

Modified: avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestValidatingIO.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestValidatingIO.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestValidatingIO.java (original)
+++ avro/trunk/lang/java/avro/src/test/java/org/apache/avro/io/TestValidatingIO.java Fri Feb 25 23:45:30 2011
@@ -303,7 +303,7 @@ public class TestValidatingIO {
     switch (encoding) {
     case BINARY:
     case BLOCKING_BINARY:
-      bvi = DecoderFactory.defaultFactory().createBinaryDecoder(bytes, null);
+      bvi = DecoderFactory.get().binaryDecoder(bytes, null);
       break;
     case JSON:
       InputStream in = new ByteArrayInputStream(bytes);

Modified: avro/trunk/lang/java/ipc/src/main/java/org/apache/avro/ipc/Requestor.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/ipc/src/main/java/org/apache/avro/ipc/Requestor.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/ipc/src/main/java/org/apache/avro/ipc/Requestor.java (original)
+++ avro/trunk/lang/java/ipc/src/main/java/org/apache/avro/ipc/Requestor.java Fri Feb 25 23:45:30 2011
@@ -133,7 +133,7 @@ public abstract class Requestor {
       } else {                                    // two-way message
         List<ByteBuffer> response = t.transceive(requestBytes);
         ByteBufferInputStream bbi = new ByteBufferInputStream(response);
-        in = DecoderFactory.defaultFactory().createBinaryDecoder(bbi, in);
+        in = DecoderFactory.get().binaryDecoder(bbi, in);
       }
     } while (!readHandshake(in));
 
@@ -265,7 +265,7 @@ public abstract class Requestor {
       getTransceiver().transceive(bbo.getBufferList());
     ByteBufferInputStream bbi = new ByteBufferInputStream(response);
     BinaryDecoder in =
-      DecoderFactory.defaultFactory().createBinaryDecoder(bbi, null);
+      DecoderFactory.get().binaryDecoder(bbi, null);
     readHandshake(in);
     return this.remote;
   }

Modified: avro/trunk/lang/java/ipc/src/main/java/org/apache/avro/ipc/Responder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/ipc/src/main/java/org/apache/avro/ipc/Responder.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/ipc/src/main/java/org/apache/avro/ipc/Responder.java (original)
+++ avro/trunk/lang/java/ipc/src/main/java/org/apache/avro/ipc/Responder.java Fri Feb 25 23:45:30 2011
@@ -105,7 +105,7 @@ public abstract class Responder {
    * track handshake status of connection. */
   public List<ByteBuffer> respond(List<ByteBuffer> buffers,
                                   Transceiver connection) throws IOException {
-    Decoder in = DecoderFactory.defaultFactory().createBinaryDecoder(
+    Decoder in = DecoderFactory.get().binaryDecoder(
         new ByteBufferInputStream(buffers), null);
     ByteBufferOutputStream bbo = new ByteBufferOutputStream();
     BinaryEncoder out = EncoderFactory.get().binaryEncoder(bbo, null);

Modified: avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestSchema.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestSchema.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestSchema.java (original)
+++ avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestSchema.java Fri Feb 25 23:45:30 2011
@@ -599,7 +599,7 @@ public class TestSchema {
     reader.setSchema(schema);
         
     Object decoded =
-      reader.read(null, DecoderFactory.defaultFactory().createBinaryDecoder(
+      reader.read(null, DecoderFactory.get().binaryDecoder(
           data, null));
       
     assertEquals("Decoded data does not match.", datum, decoded);
@@ -617,8 +617,8 @@ public class TestSchema {
 
     reader.setSchema(schema);
 
-    Object decoded = reader.read(null, DecoderFactory.defaultFactory()
-        .createBinaryDecoder(data, null));
+    Object decoded = reader.read(null, DecoderFactory.get()
+        .binaryDecoder(data, null));
 
     assertEquals("Decoded data does not match.", datum, decoded);
   }
@@ -635,8 +635,8 @@ public class TestSchema {
 
     reader.setSchema(schema);
 
-    Object decoded = reader.read(null, DecoderFactory.defaultFactory()
-        .createBinaryDecoder(data, null));
+    Object decoded = reader.read(null, DecoderFactory.get()
+        .binaryDecoder(data, null));
 
     assertEquals("Decoded data does not match.", datum, decoded);
   }
@@ -654,7 +654,8 @@ public class TestSchema {
     byte[] data = out.toByteArray();
 
     reader.setSchema(schema);
-    Decoder decoder = new JsonDecoder(schema, new ByteArrayInputStream(data));
+    Decoder decoder = DecoderFactory.get().jsonDecoder(schema,
+        new ByteArrayInputStream(data));
     Object decoded = reader.read(null, decoder);
     assertEquals("Decoded data does not match.", datum, decoded);
 
@@ -677,8 +678,8 @@ public class TestSchema {
 
     DatumReader<Object> reader = new GenericDatumReader<Object>();
     reader.setSchema(schema);
-    Object decoded =
-      reader.read(null, new JsonDecoder(schema,new ByteArrayInputStream(data)));
+    Object decoded = reader.read(null, DecoderFactory.get()
+        .jsonDecoder(schema, new ByteArrayInputStream(data)));
       
     assertEquals("Decoded data does not match.", datum, decoded);
   }
@@ -695,7 +696,7 @@ public class TestSchema {
     Schema expected = Schema.parse(recordJson);
     DatumReader<Object> in = new GenericDatumReader<Object>(ACTUAL, expected);
     GenericData.Record record = (GenericData.Record)
-      in.read(null, DecoderFactory.defaultFactory().createBinaryDecoder(
+      in.read(null, DecoderFactory.get().binaryDecoder(
           new byte[0], null));
     assertEquals("Wrong default.", defaultValue, record.get("f"));
     assertEquals("Wrong toString", expected, Schema.parse(expected.toString()));
@@ -707,7 +708,7 @@ public class TestSchema {
       Schema.parse("{\"type\":\"record\", \"name\":\"Foo\", \"fields\":"+
                    "[{\"name\":\"f\", \"type\": \"string\"}]}");
     DatumReader<Object> in = new GenericDatumReader<Object>(ACTUAL, expected);
-    in.read(null, DecoderFactory.defaultFactory().createBinaryDecoder(
+    in.read(null, DecoderFactory.get().binaryDecoder(
         new ByteArrayInputStream(new byte[0]), null));
   }
 
@@ -724,7 +725,7 @@ public class TestSchema {
     writer.write(new GenericData.EnumSymbol(actual, "X"), encoder);
     encoder.flush();
     byte[] data = out.toByteArray();
-    Decoder decoder = DecoderFactory.defaultFactory().createBinaryDecoder(
+    Decoder decoder = DecoderFactory.get().binaryDecoder(
         data, null);
     DatumReader<String> in = new GenericDatumReader<String>(actual, expected);
     assertEquals("Wrong value", new GenericData.EnumSymbol(expected, "Y"),

Modified: avro/trunk/lang/java/mapred/src/main/java/org/apache/avro/mapred/AvroSerialization.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/mapred/src/main/java/org/apache/avro/mapred/AvroSerialization.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/mapred/src/main/java/org/apache/avro/mapred/AvroSerialization.java (original)
+++ avro/trunk/lang/java/mapred/src/main/java/org/apache/avro/mapred/AvroSerialization.java Fri Feb 25 23:45:30 2011
@@ -63,8 +63,7 @@ public class AvroSerialization<T> extend
     return new AvroWrapperDeserializer(datumReader, isKey);
   }
   
-  private static final DecoderFactory FACTORY = new DecoderFactory();
-  static { FACTORY.configureDirectDecoder(true); }
+  private static final DecoderFactory FACTORY = DecoderFactory.get();
 
   private class AvroWrapperDeserializer
     implements Deserializer<AvroWrapper<T>> {
@@ -79,7 +78,7 @@ public class AvroSerialization<T> extend
     }
     
     public void open(InputStream in) {
-      this.decoder = FACTORY.createBinaryDecoder(in, decoder);
+      this.decoder = FACTORY.directBinaryDecoder(in, decoder);
     }
     
     public AvroWrapper<T> deserialize(AvroWrapper<T> wrapper)

Modified: avro/trunk/lang/java/mapred/src/main/java/org/apache/avro/mapred/tether/TetherKeySerialization.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/mapred/src/main/java/org/apache/avro/mapred/tether/TetherKeySerialization.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/mapred/src/main/java/org/apache/avro/mapred/tether/TetherKeySerialization.java (original)
+++ avro/trunk/lang/java/mapred/src/main/java/org/apache/avro/mapred/tether/TetherKeySerialization.java Fri Feb 25 23:45:30 2011
@@ -44,14 +44,13 @@ class TetherKeySerialization
     return new TetherDataDeserializer();
   }
   
-  private static final DecoderFactory FACTORY = new DecoderFactory();
-  static { FACTORY.configureDirectDecoder(true); }
+  private static final DecoderFactory FACTORY = DecoderFactory.get();
 
   private class TetherDataDeserializer implements Deserializer<TetherData> {
     private BinaryDecoder decoder;
     
     public void open(InputStream in) {
-      this.decoder = FACTORY.createBinaryDecoder(in, decoder);
+      this.decoder = FACTORY.directBinaryDecoder(in, decoder);
     }
     
     public TetherData deserialize(TetherData datum) throws IOException {

Modified: avro/trunk/lang/java/mapred/src/test/java/org/apache/avro/mapred/tether/TetherTask.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/mapred/src/test/java/org/apache/avro/mapred/tether/TetherTask.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/mapred/src/test/java/org/apache/avro/mapred/tether/TetherTask.java (original)
+++ avro/trunk/lang/java/mapred/src/test/java/org/apache/avro/mapred/tether/TetherTask.java Fri Feb 25 23:45:30 2011
@@ -51,7 +51,7 @@ public abstract class TetherTask<IN,MID,
   private TaskType taskType;
   private int partitions;
 
-  private DecoderFactory decoderFactory = DecoderFactory.defaultFactory();
+  private DecoderFactory decoderFactory = DecoderFactory.get();
   private BinaryDecoder decoder;
 
   private SpecificDatumReader<IN> inReader;
@@ -137,7 +137,7 @@ public abstract class TetherTask<IN,MID,
 
   void input(ByteBuffer data, long count) {
     try {
-      decoder = decoderFactory.createBinaryDecoder(data.array(), decoder);
+      decoder = decoderFactory.binaryDecoder(data.array(), decoder);
       for (long i = 0; i < count; i++) {
         switch (taskType) {
         case MAP:

Modified: avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/BinaryFragmentToJsonTool.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/BinaryFragmentToJsonTool.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/BinaryFragmentToJsonTool.java (original)
+++ avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/BinaryFragmentToJsonTool.java Fri Feb 25 23:45:30 2011
@@ -56,7 +56,7 @@ public class BinaryFragmentToJsonTool im
     try {
       DatumReader<Object> reader = new GenericDatumReader<Object>(schema);
       Object datum = reader.read(null,
-          DecoderFactory.defaultFactory().createBinaryDecoder(input, null));
+          DecoderFactory.get().binaryDecoder(input, null));
       DatumWriter<Object> writer = new GenericDatumWriter<Object>(schema);
       JsonGenerator g =
         new JsonFactory().createJsonGenerator(out, JsonEncoding.UTF8);

Modified: avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/DataFileWriteTool.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/DataFileWriteTool.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/DataFileWriteTool.java (original)
+++ avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/DataFileWriteTool.java Fri Feb 25 23:45:30 2011
@@ -36,7 +36,7 @@ import org.apache.avro.generic.GenericDa
 import org.apache.avro.generic.GenericDatumWriter;
 import org.apache.avro.io.DatumReader;
 import org.apache.avro.io.Decoder;
-import org.apache.avro.io.JsonDecoder;
+import org.apache.avro.io.DecoderFactory;
 
 /** Reads new-line delimited JSON records and writers an Avro data file. */
 public class DataFileWriteTool implements Tool {
@@ -98,7 +98,7 @@ public class DataFileWriteTool implement
         new DataFileWriter<Object>(new GenericDatumWriter<Object>());
       writer.setCodec(CodecFactory.fromString(codec.value(opts)));
       writer.create(schema, out);
-      Decoder decoder = new JsonDecoder(schema, din);
+      Decoder decoder = DecoderFactory.get().jsonDecoder(schema, din);
       Object datum;
       while (true) {
         try {

Modified: avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/JsonToBinaryFragmentTool.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/JsonToBinaryFragmentTool.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/JsonToBinaryFragmentTool.java (original)
+++ avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/JsonToBinaryFragmentTool.java Fri Feb 25 23:45:30 2011
@@ -27,7 +27,7 @@ import org.apache.avro.generic.GenericDa
 import org.apache.avro.generic.GenericDatumWriter;
 import org.apache.avro.io.Encoder;
 import org.apache.avro.io.EncoderFactory;
-import org.apache.avro.io.JsonDecoder;
+import org.apache.avro.io.DecoderFactory;
 
 /** Tool to convert JSON data into the binary form. */
 public class JsonToBinaryFragmentTool implements Tool {
@@ -53,7 +53,8 @@ public class JsonToBinaryFragmentTool im
     try {
     GenericDatumReader<Object> reader = 
         new GenericDatumReader<Object>(schema);
-    Object datum = reader.read(null, new JsonDecoder(schema, input));
+      Object datum = reader.read(null,
+          DecoderFactory.get().jsonDecoder(schema, input));
     
     GenericDatumWriter<Object> writer = 
         new GenericDatumWriter<Object>(schema);

Modified: avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/Util.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/Util.java?rev=1074751&r1=1074750&r2=1074751&view=diff
==============================================================================
--- avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/Util.java (original)
+++ avro/trunk/lang/java/tools/src/main/java/org/apache/avro/tool/Util.java Fri Feb 25 23:45:30 2011
@@ -30,7 +30,7 @@ import java.net.URI;
 
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.io.JsonDecoder;
+import org.apache.avro.io.DecoderFactory;
 import org.apache.avro.file.DataFileReader;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -97,10 +97,11 @@ class Util {
    * This is inefficient (creates extra objects), so should be used 
    * sparingly.
    */
-  static Object jsonToGenericDatum(Schema schema, String jsonData) throws IOException {
-    GenericDatumReader<Object> reader = 
-      new GenericDatumReader<Object>(schema);
-    Object datum = reader.read(null, new JsonDecoder(schema, jsonData));
+  static Object jsonToGenericDatum(Schema schema, String jsonData)
+      throws IOException {
+    GenericDatumReader<Object> reader = new GenericDatumReader<Object>(schema);
+    Object datum = reader.read(null,
+        DecoderFactory.get().jsonDecoder(schema, jsonData));
     return datum;
   }
 



Mime
View raw message