You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@avro.apache.org by cu...@apache.org on 2011/08/26 22:54:52 UTC

svn commit: r1162253 [1/2] - in /avro/trunk: ./ lang/java/avro/src/main/java/org/apache/avro/ lang/java/avro/src/main/java/org/apache/avro/data/ lang/java/avro/src/main/java/org/apache/avro/generic/ lang/java/avro/src/main/java/org/apache/avro/io/parsi...

Author: cutting
Date: Fri Aug 26 20:54:50 2011
New Revision: 1162253

URL: http://svn.apache.org/viewvc?rev=1162253&view=rev
Log:
AVRO-839. Java: Add accessor methods and builders to generated java classes.  Contributed by James Baldassari.

Added:
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/ErrorBuilder.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/RecordBuilder.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/RecordBuilderBase.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/package.html
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericRecordBuilder.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificErrorBuilderBase.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificRecordBuilderBase.java
    avro/trunk/lang/java/avro/src/test/java/org/apache/avro/data/
    avro/trunk/lang/java/avro/src/test/java/org/apache/avro/data/RecordBuilderBaseTest.java
    avro/trunk/lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericRecordBuilder.java
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/generic/
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/generic/TestDeepCopy.java
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/specific/TestSpecificErrorBuilder.java
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/specific/TestSpecificRecordBuilder.java
    avro/trunk/share/test/schemas/social.avdl
Modified:
    avro/trunk/CHANGES.txt
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/AvroRemoteException.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.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/parsing/ResolvingGrammarGenerator.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumReader.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificDatumReader.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificExceptionBase.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificFixed.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/util/Utf8.java
    avro/trunk/lang/java/compiler/src/main/java/org/apache/avro/compiler/specific/SpecificCompiler.java
    avro/trunk/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/fixed.vm
    avro/trunk/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/record.vm
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestCompare.java
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestNamespaceSpecific.java
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestProtocolSpecific.java
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/compiler/specific/TestSpecificCompiler.java
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/ipc/TestNettyServer.java
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/ipc/TestNettyServerWithCallbacks.java
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/ipc/TestRpcPluginOrdering.java
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/ipc/trace/TestBasicTracing.java
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/ipc/trace/TestFileSpanStorage.java
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/ipc/trace/TestSpanAggregation.java
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/ipc/trace/TestSpanTraceFormation.java
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/specific/TestSpecificData.java
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/specific/TestSpecificDatumWriter.java
    avro/trunk/lang/java/mapred/src/main/java/org/apache/avro/mapred/Pair.java
    avro/trunk/lang/java/tools/src/test/compiler/output/Player.java

Modified: avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/avro/trunk/CHANGES.txt?rev=1162253&r1=1162252&r2=1162253&view=diff
==============================================================================
--- avro/trunk/CHANGES.txt (original)
+++ avro/trunk/CHANGES.txt Fri Aug 26 20:54:50 2011
@@ -6,6 +6,12 @@ Avro 1.6.0 (unreleased)
 
   NEW FEATURES
 
+    AVRO-839. Java: Add accessor methods and builders to generated
+    Java classes.  Builders use default values from schemas for fields
+    that are not explicitly set.  Direct use of public fields is now
+    deprecated and may be removed in a future release.  (James
+    Baldassari via cutting)
+
   OPTIMIZATIONS
 
     AVRO-853: Java: Cache Schema hash codes. (cutting)

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/AvroRemoteException.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/AvroRemoteException.java?rev=1162253&r1=1162252&r2=1162253&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/AvroRemoteException.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/AvroRemoteException.java Fri Aug 26 20:54:50 2011
@@ -36,6 +36,11 @@ public class AvroRemoteException extends
     this.value = value;
   }
   
+  public AvroRemoteException(Object value, Throwable cause) {
+    super(value != null ? value.toString() : null, cause);
+    this.value = value;
+  }
+  
   public Object getValue() { return value; }
 }
 

Added: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/ErrorBuilder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/ErrorBuilder.java?rev=1162253&view=auto
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/ErrorBuilder.java (added)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/ErrorBuilder.java Fri Aug 26 20:54:50 2011
@@ -0,0 +1,47 @@
+/**
+ * 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.data;
+
+/** Interface for error builders */
+public interface ErrorBuilder<T> extends RecordBuilder<T> {
+  
+  /** Gets the value */
+  Object getValue();
+  
+  /** Sets the value */
+  ErrorBuilder<T> setValue(Object value);
+  
+  /** Checks whether the value has been set */
+  boolean hasValue();
+  
+  /** Clears the value */
+  ErrorBuilder<T> clearValue();
+  
+  /** Gets the error cause */
+  Throwable getCause();
+  
+  /** Sets the error cause */
+  ErrorBuilder<T> setCause(Throwable cause);
+  
+  /** Checks whether the cause has been set */
+  boolean hasCause();
+  
+  /** Clears the cause */
+  ErrorBuilder<T> clearCause();
+
+}

Added: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/RecordBuilder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/RecordBuilder.java?rev=1162253&view=auto
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/RecordBuilder.java (added)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/RecordBuilder.java Fri Aug 26 20:54:50 2011
@@ -0,0 +1,30 @@
+/**
+ * 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.data;
+
+
+/** Interface for record builders */
+public interface RecordBuilder<T> {
+  /**
+   * Constructs a new instance using the values set in the RecordBuilder.
+   * If a particular value was not set and the schema defines a default 
+   * value, the default value will be used.
+   * @return a new instance using values set in the RecordBuilder.
+   */
+  T build();
+}

Added: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/RecordBuilderBase.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/RecordBuilderBase.java?rev=1162253&view=auto
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/RecordBuilderBase.java (added)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/RecordBuilderBase.java Fri Aug 26 20:54:50 2011
@@ -0,0 +1,215 @@
+/**
+ * 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.data;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.avro.AvroRuntimeException;
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.apache.avro.Schema.Type;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.avro.io.parsing.ResolvingGrammarGenerator;
+import org.codehaus.jackson.JsonNode;
+
+/** Abstract base class for RecordBuilder implementations.  Not thread-safe. */
+public abstract class RecordBuilderBase<T extends IndexedRecord> 
+  implements RecordBuilder<T> {
+  private static final ConcurrentMap<String, ConcurrentMap<Integer, Object>> 
+    DEFAULT_VALUE_CACHE = 
+      new ConcurrentHashMap<String, ConcurrentMap<Integer, Object>>();
+  private static final Field[] EMPTY_FIELDS = new Field[0];
+  protected final Schema schema;
+  protected final Field[] fields;
+  protected final boolean[] fieldSetFlags;
+  protected final GenericData data;
+  private BinaryEncoder encoder = null;
+  private BinaryDecoder decoder = null;
+  
+  /**
+   * Creates a RecordBuilderBase for building records of the given type.
+   * @param schema the schema associated with the record class.
+   */
+  protected RecordBuilderBase(Schema schema, GenericData data) {
+    this.schema = schema;
+    this.data = data;
+    fields = (Field[]) schema.getFields().toArray(EMPTY_FIELDS);
+    fieldSetFlags = new boolean[fields.length];
+  }
+  
+  /**
+   * RecordBuilderBase copy constructor.
+   * Makes a deep copy of the values in the other builder.
+   * @param other RecordBuilderBase instance to copy.
+   */
+  protected RecordBuilderBase(RecordBuilderBase<T> other, GenericData data) {
+    this.schema = other.schema;
+    this.data = data;
+    fields = (Field[]) schema.getFields().toArray(EMPTY_FIELDS);
+    fieldSetFlags = new boolean[other.fieldSetFlags.length];
+    System.arraycopy(
+        other.fieldSetFlags, 0, fieldSetFlags, 0, fieldSetFlags.length);
+  }
+  
+  /**
+   * Validates that a particular value for a given field is valid according to 
+   * the following algorithm:
+   * 1. If the value is not null, or the field type is null, or the field type 
+   * is a union which accepts nulls, returns.
+   * 2. Else, if the field has a default value, returns.
+   * 3. Otherwise throws NullPointerException 
+   * @param field the field to validate.
+   * @param value the value to validate.
+   * @throws NullPointerException if value is null and the given field does 
+   * not accept null values.
+   */
+  protected void validate(Field field, Object value) {
+    if (isValidValue(field, value)) {
+      return;
+    }
+    else if (field.defaultValue() != null) {
+      return;
+    }
+    else {
+      throw new AvroRuntimeException(
+          "Field " + field + " does not accept null values");
+    }
+  }
+
+  /**
+   * Tests whether a value is valid for a specified field. 
+   * @param f the field for which to test the value.
+   * @param value the value to test.
+   * @return true if the value is valid for the given field; false otherwise.
+   */
+  protected static boolean isValidValue(Field f, Object value) {
+    if (value != null) {
+      return true;
+    }
+    
+    Schema schema = f.schema();
+    Type type = schema.getType();
+    
+    // If the type is null, any value is valid
+    if (type == Type.NULL) {
+      return true;
+    }
+
+    // If the type is a union that allows nulls, any value is valid
+    if (type == Type.UNION) {
+      for (Schema s : schema.getTypes()) {
+        if (s.getType() == Type.NULL) {
+          return true;
+        }
+      }
+    }
+    
+    // The value is null but the type does not allow nulls
+    return false;
+  }
+  
+  /**
+   * Gets the default value of the given field, if any.
+   * @param field the field whose default value should be retrieved.
+   * @return the default value associated with the given field, 
+   * or null if none is specified in the schema.
+   * @throws IOException 
+   */
+  @SuppressWarnings({ "rawtypes", "unchecked" })
+  protected Object getDefaultValue(Field field) throws IOException {    
+    if (field.schema().getType() == Type.NULL) {
+      return null;
+    }
+    
+    JsonNode defaultJsonValue = field.defaultValue();
+    if (defaultJsonValue == null) {
+      return null;
+    }
+    
+    // Get the default value
+    Object defaultValue = null;
+    
+    // First try to get the default value from cache:
+    ConcurrentMap<Integer, Object> defaultSchemaValues = 
+      DEFAULT_VALUE_CACHE.get(schema.getFullName());
+    if (defaultSchemaValues == null) {
+      DEFAULT_VALUE_CACHE.putIfAbsent(schema.getFullName(), 
+          new ConcurrentHashMap<Integer, Object>(fields.length));
+      defaultSchemaValues = DEFAULT_VALUE_CACHE.get(schema.getFullName());
+    }
+    defaultValue = defaultSchemaValues.get(field.pos());
+    
+    // If not cached, get the default Java value by encoding the default JSON
+    // value and then decoding it:
+    if (defaultValue == null) {
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      encoder = EncoderFactory.get().binaryEncoder(baos, encoder);
+      ResolvingGrammarGenerator.encode(
+          encoder, field.schema(), defaultJsonValue);
+      encoder.flush();
+      decoder = DecoderFactory.get().binaryDecoder(
+          baos.toByteArray(), decoder);
+      defaultValue = new GenericDatumReader(
+          field.schema()).read(null, decoder);
+      defaultSchemaValues.putIfAbsent(field.pos(), defaultValue);
+    }
+    
+    // Make a deep copy of the default value so that subsequent mutations 
+    // will not affect the default value cache:
+    return data.deepCopy(field.schema(), defaultValue);
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + Arrays.hashCode(fieldSetFlags);
+    result = prime * result + ((schema == null) ? 0 : schema.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    @SuppressWarnings("rawtypes")
+    RecordBuilderBase other = (RecordBuilderBase) obj;
+    if (!Arrays.equals(fieldSetFlags, other.fieldSetFlags))
+      return false;
+    if (schema == null) {
+      if (other.schema != null)
+        return false;
+    } else if (!schema.equals(other.schema))
+      return false;
+    return true;
+  }
+}

Added: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/package.html
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/package.html?rev=1162253&view=auto
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/package.html (added)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/package.html Fri Aug 26 20:54:50 2011
@@ -0,0 +1,22 @@
+<html>
+
+<!--
+   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.
+-->
+
+<body>Interfaces and base classes shared by generic, specific and reflect.
+</body>
+</html>

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java?rev=1162253&r1=1162252&r2=1162253&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java Fri Aug 26 20:54:50 2011
@@ -21,7 +21,9 @@ import java.nio.ByteBuffer;
 import java.util.AbstractList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.avro.AvroRuntimeException;
@@ -53,6 +55,19 @@ public class GenericData {
       this.schema = schema;
       this.values = new Object[schema.getFields().size()];
     }
+    public Record(Record other, boolean deepCopy) {
+      schema = other.schema;
+      values = new Object[schema.getFields().size()];
+      if (deepCopy) {
+        for (int ii = 0; ii < values.length; ii++) {
+          values[ii] = INSTANCE.deepCopy(
+              schema.getFields().get(ii).schema(), other.values[ii]);
+        }
+      }
+      else {
+        System.arraycopy(other.values, 0, values, 0, other.values.length);
+      }
+    }
     @Override public Schema getSchema() { return schema; }
     @Override public void put(String key, Object value) {
       Schema.Field field = schema.getField(key);
@@ -102,6 +117,15 @@ public class GenericData {
       if (capacity != 0)
         elements = new Object[capacity];
     }
+    public Array(Schema schema, Collection<T> c) {
+      if (schema == null || !Type.ARRAY.equals(schema.getType()))
+        throw new AvroRuntimeException("Not an array schema: "+schema);
+      this.schema = schema;
+      if (c != null) {
+        elements = new Object[c.size()];
+        addAll(c);
+      }
+    }
     public Schema getSchema() { return schema; }
     @Override public int size() { return size; }
     @Override public void clear() { size = 0; }
@@ -683,4 +707,129 @@ public class GenericData {
     }
   }
 
+  /**
+   * Makes a deep copy of a value given its schema.
+   * @param schema the schema of the value to deep copy.
+   * @param value the value to deep copy.
+   * @return a deep copy of the given value.
+   */
+  @SuppressWarnings({ "rawtypes", "unchecked" })
+  public Object deepCopy(Schema schema, Object value) {
+    if (value == null) {
+      return null;
+    }
+    switch (schema.getType()) {
+      case ARRAY:
+        List<Object> arrayValue = (List) value;
+        List<Object> arrayCopy = new GenericData.Array<Object>(
+            arrayValue.size(), schema);
+        for (Object obj : arrayValue) {
+          arrayCopy.add(deepCopy(schema.getElementType(), obj));
+        }
+        return arrayCopy;
+      case BOOLEAN:
+        return new Boolean((Boolean)value);
+      case BYTES:
+        ByteBuffer byteBufferValue = (ByteBuffer) value;
+        byte[] bytesCopy = new byte[byteBufferValue.capacity()];
+        byteBufferValue.rewind();
+        byteBufferValue.get(bytesCopy);
+        byteBufferValue.rewind();
+        return ByteBuffer.wrap(bytesCopy);
+      case DOUBLE:
+        return new Double((Double)value);
+      case ENUM:
+        // Enums are immutable; shallow copy will suffice
+        return value;
+      case FIXED:
+        return createFixed(null, ((GenericFixed) value).bytes(), schema);
+      case FLOAT:
+        return new Float((Float)value);
+      case INT:
+        return new Integer((Integer)value);
+      case LONG:
+        return new Long((Long)value);
+      case MAP:
+        Map<CharSequence, Object> mapValue = (Map) value;
+        Map<CharSequence, Object> mapCopy = 
+          new HashMap<CharSequence, Object>(mapValue.size());
+        for (Map.Entry<CharSequence, Object> entry : mapValue.entrySet()) {
+          mapCopy.put(new Utf8(entry.getKey().toString()), 
+              deepCopy(schema.getValueType(), entry.getValue()));
+        }
+        return mapCopy;
+      case NULL:
+        return null;
+      case RECORD:
+        IndexedRecord recordValue = (IndexedRecord) value;
+        IndexedRecord recordCopy = (IndexedRecord) newRecord(null, schema);
+        for (Field field : schema.getFields()) {
+          recordCopy.put(field.pos(), 
+              deepCopy(field.schema(), recordValue.get(field.pos())));
+        }
+        return recordCopy;
+      case STRING:
+        // Strings are immutable
+        if (value instanceof String) {
+          return value;
+        }
+        
+        // Some CharSequence subclasses are mutable, so we still need to make 
+        // a copy
+        else if (value instanceof Utf8) {
+          // Utf8 copy constructor is more efficient than converting 
+          // to string and then back to Utf8
+          return new Utf8((Utf8)value);
+        }
+        return new Utf8(value.toString());
+      case UNION:
+        for (Schema type : schema.getTypes()) {
+          if (GenericData.get().validate(type, value)) {
+            return deepCopy(type, value);
+          }
+        }
+        throw new AvroRuntimeException(
+            "Deep copy failed for schema \"" + schema + "\" and value \"" +
+            value + "\"");
+      default:
+        throw new AvroRuntimeException(
+            "Deep copy failed for schema \"" + schema + "\" and value \"" +
+            value + "\"");
+    }
+  }
+  
+  /** Called to create an fixed value. May be overridden for alternate fixed
+   * representations.  By default, returns {@link GenericFixed}. */
+  public Object createFixed(Object old, Schema schema) {
+    if ((old instanceof GenericFixed)
+        && ((GenericFixed)old).bytes().length == schema.getFixedSize())
+      return old;
+    return new GenericData.Fixed(schema);
+  }
+  
+  /** Called to create an fixed value. May be overridden for alternate fixed
+   * representations.  By default, returns {@link GenericFixed}. */
+  public Object createFixed(Object old, byte[] bytes, Schema schema) {
+    GenericFixed fixed = (GenericFixed)createFixed(old, schema);
+    System.arraycopy(bytes, 0, fixed.bytes(), 0, schema.getFixedSize());
+    return fixed;
+  }
+  
+  /**
+   * Called to create new record instances. Subclasses may override to use a
+   * different record implementation. The returned instance must conform to the
+   * schema provided. If the old object contains fields not present in the
+   * schema, they should either be removed from the old object, or it should
+   * create a new instance that conforms to the schema. By default, this returns
+   * a {@link GenericData.Record}.
+   */
+  public Object newRecord(Object old, Schema schema) {
+    if (old instanceof IndexedRecord) {
+      IndexedRecord record = (IndexedRecord)old;
+      if (record.getSchema() == schema)
+        return record;
+    }
+    return new GenericData.Record(schema);
+  }
+  
 }

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=1162253&r1=1162252&r2=1162253&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 Aug 26 20:54:50 2011
@@ -157,7 +157,7 @@ public class GenericDatumReader<D> imple
    * representations.*/
   protected Object readRecord(Object old, Schema expected, 
       ResolvingDecoder in) throws IOException {
-    Object record = newRecord(old, expected);
+    Object record = data.newRecord(old, expected);
     
     for (Field f : in.readFieldOrder()) {
       int pos = f.pos();
@@ -247,44 +247,11 @@ public class GenericDatumReader<D> imple
    * representations.  By default, returns {@link GenericFixed}. */
   protected Object readFixed(Object old, Schema expected, Decoder in)
     throws IOException {
-    GenericFixed fixed = (GenericFixed)createFixed(old, expected);
+    GenericFixed fixed = (GenericFixed)data.createFixed(old, expected);
     in.readFixed(fixed.bytes(), 0, expected.getFixedSize());
     return fixed;
   }
 
-  /** Called to create an fixed value. May be overridden for alternate fixed
-   * representations.  By default, returns {@link GenericFixed}. */
-  protected Object createFixed(Object old, Schema schema) {
-    if ((old instanceof GenericFixed)
-        && ((GenericFixed)old).bytes().length == schema.getFixedSize())
-      return old;
-    return new GenericData.Fixed(schema);
-  }
-
-  /** Called to create an fixed value. May be overridden for alternate fixed
-   * representations.  By default, returns {@link GenericFixed}. */
-  protected Object createFixed(Object old, byte[] bytes, Schema schema) {
-    GenericFixed fixed = (GenericFixed)createFixed(old, schema);
-    System.arraycopy(bytes, 0, fixed.bytes(), 0, schema.getFixedSize());
-    return fixed;
-  }
-  /**
-   * Called to create new record instances. Subclasses may override to use a
-   * different record implementation. The returned instance must conform to the
-   * schema provided. If the old object contains fields not present in the
-   * schema, they should either be removed from the old object, or it should
-   * create a new instance that conforms to the schema. By default, this returns
-   * a {@link GenericData.Record}.
-   */
-  protected Object newRecord(Object old, Schema schema) {
-    if (old instanceof IndexedRecord) {
-      IndexedRecord record = (IndexedRecord)old;
-      if (record.getSchema() == schema)
-        return record;
-    }
-    return new GenericData.Record(schema);
-  }
-
   /** Called to create new array instances.  Subclasses may override to use a
    * different array implementation.  By default, this returns a {@link
    * GenericData.Array}.*/

Added: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericRecordBuilder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericRecordBuilder.java?rev=1162253&view=auto
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericRecordBuilder.java (added)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericRecordBuilder.java Fri Aug 26 20:54:50 2011
@@ -0,0 +1,260 @@
+/**
+ * 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.generic;
+
+import java.io.IOException;
+
+import org.apache.avro.AvroRuntimeException;
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.apache.avro.data.RecordBuilderBase;
+import org.apache.avro.generic.GenericData.Record;
+
+/** A RecordBuilder for generic records */
+public class GenericRecordBuilder extends RecordBuilderBase<Record> {
+  private final GenericData.Record record;
+  
+  /**
+   * Creates a GenericRecordBuilder for building Record instances.
+   * @param schema the schema associated with the record class.
+   */
+  public GenericRecordBuilder(Schema schema) {
+    super(schema, GenericData.get());
+    record = new GenericData.Record(schema);
+  }
+  
+  /**
+   * Creates a GenericRecordBuilder by copying an existing GenericRecordBuilder.
+   * @param other the GenericRecordBuilder to copy.
+   */
+  public GenericRecordBuilder(GenericRecordBuilder other) {
+    super(other, GenericData.get());
+    record = new GenericData.Record(other.record, /* deepCopy = */ true);
+  }
+  
+  /**
+   * Creates a GenericRecordBuilder by copying an existing record instance.
+   * @param other the record instance to copy.
+   */
+  public GenericRecordBuilder(Record other) {
+    super(other.getSchema(), GenericData.get());
+    record = new GenericData.Record(other, /* deepCopy = */ true);
+    
+    // Set all fields in the RecordBuilder that are set in the record
+    for (Field f : schema.getFields()) {
+      Object value = other.get(f.pos());
+      // Only set the value if it is not null, if the schema type is null, 
+      // or if the schema type is a union that accepts nulls.
+      if (isValidValue(f, value)) {
+        set(f, data.deepCopy(f.schema(), value));
+      }
+    }
+  }
+  
+  /**
+   * Gets the value of a field.
+   * @param fieldName the name of the field to get.
+   * @return the value of the field with the given name, or null if not set.
+   */
+  public Object get(String fieldName) {
+    return get(schema.getField(fieldName));
+  }
+  
+  /**
+   * Gets the value of a field.
+   * @param field the field to get.
+   * @return the value of the given field, or null if not set.
+   */
+  public Object get(Field field) {
+    return get(field.pos());
+  }
+  
+  /**
+   * Gets the value of a field.
+   * @param pos the position of the field to get.
+   * @return the value of the field with the given position, or null if not set.
+   */
+  protected Object get(int pos) {
+    return record.get(pos);
+  }
+  
+  /**
+   * Sets the value of a field.
+   * @param fieldName the name of the field to set.
+   * @param value the value to set.
+   * @return a reference to the RecordBuilder.
+   */
+  public GenericRecordBuilder set(String fieldName, Object value) {
+    return set(schema.getField(fieldName), value);
+  }
+  
+  /**
+   * Sets the value of a field.
+   * @param field the field to set.
+   * @param value the value to set.
+   * @return a reference to the RecordBuilder.
+   */
+  public GenericRecordBuilder set(Field field, Object value) {
+    return set(field, field.pos(), value);
+  }
+  
+  /**
+   * Sets the value of a field.
+   * @param field the field to set.
+   * @param value the value to set.
+   * @return a reference to the RecordBuilder.
+   */
+  protected GenericRecordBuilder set(int pos, Object value) {
+    return set(fields[pos], pos, value);
+  }
+  
+  /**
+   * Sets the value of a field.
+   * @param field the field to set.
+   * @param pos the position of the field.
+   * @param value the value to set.
+   * @return a reference to the RecordBuilder.
+   */
+  private GenericRecordBuilder set(Field field, int pos, Object value) {
+    validate(field, value);
+    record.put(pos, value);
+    fieldSetFlags[pos] = true;
+    return this;
+  }
+  
+  /**
+   * Checks whether a field has been set.
+   * @param fieldName the name of the field to check.
+   * @return true if the given field is non-null; false otherwise.
+   */
+  public boolean has(String fieldName) {
+    return has(schema.getField(fieldName));
+  }
+  
+  /**
+   * Checks whether a field has been set.
+   * @param field the field to check.
+   * @return true if the given field is non-null; false otherwise.
+   */
+  public boolean has(Field field) {
+    return has(field.pos());
+  }
+  
+  /**
+   * Checks whether a field has been set.
+   * @param pos the position of the field to check.
+   * @return true if the given field is non-null; false otherwise.
+   */
+  protected boolean has(int pos) {
+    return fieldSetFlags[pos];
+  }
+  
+  /**
+   * Clears the value of the given field.
+   * @param fieldName the name of the field to clear.
+   * @return a reference to the RecordBuilder.
+   */
+  public GenericRecordBuilder clear(String fieldName) {
+    return clear(schema.getField(fieldName));
+  }
+  
+  /**
+   * Clears the value of the given field.
+   * @param field the field to clear.
+   * @return a reference to the RecordBuilder.
+   */
+  public GenericRecordBuilder clear(Field field) {
+    return clear(field.pos());
+  }
+  
+  /**
+   * Clears the value of the given field.
+   * @param pos the position of the field to clear.
+   * @return a reference to the RecordBuilder.
+   */
+  protected GenericRecordBuilder clear(int pos) {
+    record.put(pos, null);
+    fieldSetFlags[pos] = false;
+    return this;
+  }
+  
+  @Override
+  public Record build() {
+    Record record;
+    try {
+      record = new GenericData.Record(schema);
+    } catch (Exception e) {
+      throw new AvroRuntimeException(e);
+    }
+    
+    for (Field field : fields) {
+      Object value;
+      try {
+        value = getWithDefault(field);
+      } catch(IOException e) {
+        throw new AvroRuntimeException(e);
+      }
+      if (value != null) {
+        record.put(field.pos(), value);
+      }
+    }
+    
+    return record;
+  }
+  
+  /**
+   * Gets the value of the given field.
+   * If the field has been set, the set value is returned (even if it's null).
+   * If the field hasn't been set and has a default value, the default value 
+   * is returned.
+   * Otherwise, null is returned.
+   * @param field the field whose value should be retrieved.
+   * @return the value set for the given field, the field's default value, 
+   * or null.
+   * @throws IOException
+   */
+  private Object getWithDefault(Field field) throws IOException {
+    return fieldSetFlags[field.pos()] ? 
+        record.get(field.pos()) : getDefaultValue(field);
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = super.hashCode();
+    result = prime * result + ((record == null) ? 0 : record.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (!super.equals(obj))
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    GenericRecordBuilder other = (GenericRecordBuilder) obj;
+    if (record == null) {
+      if (other.record != null)
+        return false;
+    } else if (!record.equals(other.record))
+      return false;
+    return true;
+  }
+}

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/parsing/ResolvingGrammarGenerator.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/parsing/ResolvingGrammarGenerator.java?rev=1162253&r1=1162252&r2=1162253&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/parsing/ResolvingGrammarGenerator.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/io/parsing/ResolvingGrammarGenerator.java Fri Aug 26 20:54:50 2011
@@ -304,7 +304,7 @@ public class ResolvingGrammarGenerator e
    * @throws IOException
    */
   
-  static void encode(Encoder e, Schema s, JsonNode n)
+  public static void encode(Encoder e, Schema s, JsonNode n)
     throws IOException {
     switch (s.getType()) {
     case RECORD:

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumReader.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumReader.java?rev=1162253&r1=1162252&r2=1162253&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumReader.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumReader.java Fri Aug 26 20:54:50 2011
@@ -68,7 +68,7 @@ public class ReflectDatumReader<T> exten
       }
       if (collectionClass.isAssignableFrom(ArrayList.class))
         return new ArrayList();
-      return newInstance(collectionClass, schema);
+      return data.newInstance(collectionClass, schema);
     }
     Class elementClass = ReflectData.getClassProp(schema, ReflectData.ELEMENT_PROP);
     if (elementClass == null)

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java?rev=1162253&r1=1162252&r2=1162253&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java Fri Aug 26 20:54:50 2011
@@ -24,6 +24,7 @@ import java.util.WeakHashMap;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.LinkedHashMap;
 import java.nio.ByteBuffer;
+import java.lang.reflect.Constructor;
 import java.lang.reflect.ParameterizedType;
 
 import org.apache.avro.Schema;
@@ -37,8 +38,13 @@ import org.apache.avro.generic.GenericDa
 public class SpecificData extends GenericData {
 
   private static final SpecificData INSTANCE = new SpecificData();
-
+  
   private final ClassLoader classLoader;
+  
+  private static final Class<?>[] NO_ARG = new Class[]{};
+  private static final Class<?>[] SCHEMA_ARG = new Class[]{Schema.class};
+  private static final Map<Class,Constructor> CTOR_CACHE =
+    new ConcurrentHashMap<Class,Constructor>();
 
   /** For subclasses.  Applications normally use {@link SpecificData#get()}. */
   protected SpecificData() { this(SpecificData.class.getClassLoader()); }
@@ -206,5 +212,46 @@ public class SpecificData extends Generi
       return super.compare(o1, o2, s, eq);
     }
   }
+  
+  /** Create an instance of a class.  If the class implements {@link
+   * SchemaConstructable}, call a constructor with a {@link
+   * org.apache.avro.Schema} parameter, otherwise use a no-arg constructor. */
+  @SuppressWarnings("unchecked")
+  public static Object newInstance(Class c, Schema s) {
+    boolean useSchema = SchemaConstructable.class.isAssignableFrom(c);
+    Object result;
+    try {
+      Constructor meth = (Constructor)CTOR_CACHE.get(c);
+      if (meth == null) {
+        meth = c.getDeclaredConstructor(useSchema ? SCHEMA_ARG : NO_ARG);
+        meth.setAccessible(true);
+        CTOR_CACHE.put(c, meth);
+      }
+      result = meth.newInstance(useSchema ? new Object[]{s} : (Object[])null);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    return result;
+  }
+  
+  @Override
+  public Object createFixed(Object old, Schema schema) {
+    Class c = SpecificData.get().getClass(schema);
+    if (c == null) return super.createFixed(old, schema); // punt to generic
+    return c.isInstance(old) ? old : newInstance(c, schema);
+  }
+  
+  @Override
+  public Object newRecord(Object old, Schema schema) {
+    Class c = SpecificData.get().getClass(schema);
+    if (c == null) return super.newRecord(old, schema); // punt to generic
+    return (c.isInstance(old) ? old : newInstance(c, schema));
+  }
 
+  /** Tag interface that indicates that a class has a one-argument constructor
+   * that accepts a Schema.
+   * @see SpecificDatumReader#newInstance
+   */
+  public interface SchemaConstructable {}
+  
 }

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificDatumReader.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificDatumReader.java?rev=1162253&r1=1162252&r2=1162253&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificDatumReader.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificDatumReader.java Fri Aug 26 20:54:50 2011
@@ -17,10 +17,6 @@
  */
 package org.apache.avro.specific;
 
-import java.lang.reflect.Constructor;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericDatumReader;
 
@@ -55,13 +51,6 @@ public class SpecificDatumReader<T> exte
   public SpecificData getSpecificData() { return (SpecificData)getData(); }
 
   @Override
-  protected Object newRecord(Object old, Schema schema) {
-    Class c = getSpecificData().getClass(schema);
-    if (c == null) return super.newRecord(old, schema); // punt to generic
-    return (c.isInstance(old) ? old : newInstance(c, schema));
-  }
-
-  @Override
   @SuppressWarnings("unchecked")
   protected Object createEnum(String symbol, Schema schema) {
     Class c = getSpecificData().getClass(schema);
@@ -69,44 +58,5 @@ public class SpecificDatumReader<T> exte
     return Enum.valueOf(c, symbol);
   }
 
-  @Override
-  protected Object createFixed(Object old, Schema schema) {
-    Class c = getSpecificData().getClass(schema);
-    if (c == null) return super.createFixed(old, schema); // punt to generic
-    return c.isInstance(old) ? old : newInstance(c, schema);
-  }
-
-  private static final Class<?>[] NO_ARG = new Class[]{};
-  private static final Class<?>[] SCHEMA_ARG = new Class[]{Schema.class};
-  private static final Map<Class,Constructor> CTOR_CACHE =
-    new ConcurrentHashMap<Class,Constructor>();
-
-  /** Tag interface that indicates that a class has a one-argument constructor
-   * that accepts a Schema.
-   * @see SpecificDatumReader#newInstance
-   */
-  public interface SchemaConstructable {}
-
-  /** Create an instance of a class.  If the class implements {@link
-   * SchemaConstructable}, call a constructor with a {@link
-   * org.apache.avro.Schema} parameter, otherwise use a no-arg constructor. */
-  @SuppressWarnings("unchecked")
-  protected static Object newInstance(Class c, Schema s) {
-    boolean useSchema = SchemaConstructable.class.isAssignableFrom(c);
-    Object result;
-    try {
-      Constructor meth = (Constructor)CTOR_CACHE.get(c);
-      if (meth == null) {
-        meth = c.getDeclaredConstructor(useSchema ? SCHEMA_ARG : NO_ARG);
-        meth.setAccessible(true);
-        CTOR_CACHE.put(c, meth);
-      }
-      result = meth.newInstance(useSchema ? new Object[]{s} : (Object[])null);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-    return result;
-  }
-
 }
 

Added: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificErrorBuilderBase.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificErrorBuilderBase.java?rev=1162253&view=auto
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificErrorBuilderBase.java (added)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificErrorBuilderBase.java Fri Aug 26 20:54:50 2011
@@ -0,0 +1,124 @@
+/**
+ * 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.specific;
+
+import java.lang.reflect.Constructor;
+
+import org.apache.avro.Schema;
+import org.apache.avro.data.ErrorBuilder;
+import org.apache.avro.data.RecordBuilderBase;
+
+/** 
+ * Abstract base class for specific ErrorBuilder implementations.
+ * Not thread-safe.
+ */
+abstract public class SpecificErrorBuilderBase<T extends SpecificExceptionBase> 
+  extends RecordBuilderBase<T> implements ErrorBuilder<T> {
+  private Constructor<T> errorConstructor;
+  private Object value;
+  private boolean hasValue;
+  private Throwable cause;
+  private boolean hasCause;
+  
+  /**
+   * Creates a SpecificErrorBuilderBase for building errors of the given type.
+   * @param schema the schema associated with the error class.
+   */
+  protected SpecificErrorBuilderBase(Schema schema) {
+    super(schema, SpecificData.get());
+  }
+  
+  /**
+   * SpecificErrorBuilderBase copy constructor.
+   * @param other SpecificErrorBuilderBase instance to copy.
+   */
+  protected SpecificErrorBuilderBase(SpecificErrorBuilderBase<T> other) {
+    super(other, SpecificData.get());
+    this.errorConstructor = other.errorConstructor;
+    this.value = other.value;
+    this.hasValue = other.hasValue;
+    this.cause = other.cause;
+    this.hasCause = other.hasCause;
+  }
+  
+  /**
+   * Creates a SpecificErrorBuilderBase by copying an existing error instance.
+   * @param other the error instance to copy.
+   */
+  protected SpecificErrorBuilderBase(T other) {
+    super(other.getSchema(), SpecificData.get());
+    
+    Object otherValue = other.getValue();
+    if (otherValue != null) {
+      setValue(otherValue);
+    }
+    
+    Throwable otherCause = other.getCause();
+    if (otherCause != null) {
+      setCause(otherCause);
+    }
+  }
+
+  @Override
+  public Object getValue() {
+    return value;
+  }
+
+  @Override
+  public SpecificErrorBuilderBase<T> setValue(Object value) {
+    this.value = value;
+    hasValue = true;
+    return this;
+  }
+  
+  @Override
+  public boolean hasValue() {
+    return hasValue;
+  }
+  
+  @Override
+  public SpecificErrorBuilderBase<T> clearValue() {
+    value = null;
+    hasValue = false;
+    return this;
+  }
+
+  @Override
+  public Throwable getCause() {
+    return cause;
+  }
+
+  @Override
+  public SpecificErrorBuilderBase<T> setCause(Throwable cause) {
+    this.cause = cause;
+    hasCause = true;
+    return this;
+  }
+  
+  @Override
+  public boolean hasCause() {
+    return hasCause;
+  }
+  
+  @Override
+  public SpecificErrorBuilderBase<T> clearCause() {
+    cause = null;
+    hasCause = false;
+    return this;
+  }
+}

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificExceptionBase.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificExceptionBase.java?rev=1162253&r1=1162252&r2=1162253&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificExceptionBase.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificExceptionBase.java Fri Aug 26 20:54:50 2011
@@ -25,6 +25,22 @@ import org.apache.avro.Schema;
 public abstract class SpecificExceptionBase extends AvroRemoteException
   implements SpecificRecord {
 
+  public SpecificExceptionBase() {
+    super();
+  }
+  
+  public SpecificExceptionBase(Throwable value) {
+    super(value);
+  }
+
+  public SpecificExceptionBase(Object value) {
+    super(value);
+  }
+  
+  public SpecificExceptionBase(Object value, Throwable cause) {
+    super(value, cause);
+  }
+
   public abstract Schema getSchema();
   public abstract Object get(int field);
   public abstract void put(int field, Object value);

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificFixed.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificFixed.java?rev=1162253&r1=1162252&r2=1162253&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificFixed.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificFixed.java Fri Aug 26 20:54:50 2011
@@ -24,5 +24,10 @@ public abstract class SpecificFixed exte
   public SpecificFixed() {
     setSchema(SpecificData.get().getSchema(getClass()));
   }
+  
+  public SpecificFixed(byte[] bytes) {
+    this();
+    bytes(bytes); 
+  }
 }
 

Added: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificRecordBuilderBase.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificRecordBuilderBase.java?rev=1162253&view=auto
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificRecordBuilderBase.java (added)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificRecordBuilderBase.java Fri Aug 26 20:54:50 2011
@@ -0,0 +1,53 @@
+/**
+ * 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.specific;
+
+import org.apache.avro.Schema;
+import org.apache.avro.data.RecordBuilderBase;
+
+/**
+ * Abstract base class for specific RecordBuilder implementations.
+ * Not thread-safe.
+ */
+abstract public class SpecificRecordBuilderBase<T extends SpecificRecord> 
+  extends RecordBuilderBase<T> {
+  
+  /**
+   * Creates a SpecificRecordBuilderBase for building records of the given type.
+   * @param schema the schema associated with the record class.
+   */
+  protected SpecificRecordBuilderBase(Schema schema) {
+    super(schema, SpecificData.get());
+  }
+  
+  /**
+   * SpecificRecordBuilderBase copy constructor.
+   * @param other SpecificRecordBuilderBase instance to copy.
+   */
+  protected SpecificRecordBuilderBase(SpecificRecordBuilderBase<T> other) {
+    super(other, SpecificData.get());
+  }
+  
+  /**
+   * Creates a SpecificRecordBuilderBase by copying an existing record instance.
+   * @param other the record instance to copy.
+   */
+  protected SpecificRecordBuilderBase(T other) {
+    super(other.getSchema(), SpecificData.get());
+  }
+}

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/util/Utf8.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/util/Utf8.java?rev=1162253&r1=1162252&r2=1162253&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/util/Utf8.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/util/Utf8.java Fri Aug 26 20:54:50 2011
@@ -17,7 +17,7 @@
  */
 package org.apache.avro.util;
 
-import java.io.UnsupportedEncodingException;
+import java.nio.charset.Charset;
 
 import org.apache.avro.io.BinaryData;
 
@@ -26,6 +26,7 @@ import org.apache.avro.io.BinaryData;
  * as a single instance may be reused. */
 public class Utf8 implements Comparable<Utf8>, CharSequence {
   private static final byte[] EMPTY = new byte[0];
+  private static final Charset UTF8 = Charset.forName("UTF-8");
 
   private byte[] bytes = EMPTY;
   private int length;
@@ -38,6 +39,13 @@ public class Utf8 implements Comparable<
     this.length = bytes.length;
     this.string = string;
   }
+  
+  public Utf8(Utf8 other) {
+    this.length = other.length;
+    this.bytes = new byte[other.length];
+    System.arraycopy(other.bytes, 0, this.bytes, 0, this.length);
+    this.string = other.string;
+  }
 
   public Utf8(byte[] bytes) {
     this.bytes = bytes;
@@ -77,12 +85,9 @@ public class Utf8 implements Comparable<
 
   @Override
   public String toString() {
-    if (this.string == null)
-      try {
-        this.string = new String(bytes, 0, length, "UTF-8");
-      } catch (UnsupportedEncodingException e) {
-        throw new RuntimeException(e);
-      }
+    if (this.string == null) {
+      this.string = new String(bytes, 0, length, UTF8);
+    }
     return this.string;
   }
 
@@ -122,11 +127,7 @@ public class Utf8 implements Comparable<
 
   /** Gets the UTF-8 bytes for a String */
   public static final byte[] getBytesFor(String str) {
-    try {
-      return str.getBytes("UTF-8");
-    } catch (UnsupportedEncodingException e) {
-      throw new RuntimeException(e);
-    }
+    return str.getBytes(UTF8);
   }
 
 }

Added: avro/trunk/lang/java/avro/src/test/java/org/apache/avro/data/RecordBuilderBaseTest.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/test/java/org/apache/avro/data/RecordBuilderBaseTest.java?rev=1162253&view=auto
==============================================================================
--- avro/trunk/lang/java/avro/src/test/java/org/apache/avro/data/RecordBuilderBaseTest.java (added)
+++ avro/trunk/lang/java/avro/src/test/java/org/apache/avro/data/RecordBuilderBaseTest.java Fri Aug 26 20:54:50 2011
@@ -0,0 +1,76 @@
+package org.apache.avro.data;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.apache.avro.Schema.Type;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Unit test for RecordBuilderBase.
+ */
+public class RecordBuilderBaseTest {
+  private static Set<Type> primitives;
+  private static Set<Type> nonNullPrimitives;
+  
+  @BeforeClass()
+  public static void setUpBeforeClass() {
+    primitives = new HashSet<Type>(Arrays.asList(Type.values()));
+    primitives.removeAll(Arrays.asList(new Type[] { 
+        Type.RECORD, Type.ENUM, Type.ARRAY, Type.MAP, Type.UNION, Type.FIXED 
+    }));
+    
+    nonNullPrimitives = new HashSet<Type>(primitives);
+    nonNullPrimitives.remove(Type.NULL);
+  }
+  
+  @Test
+  public void testIsValidValueWithPrimitives() { 
+    // Verify that a non-null value is valid for all primitives:
+    for (Type type : primitives) {
+      Field f = new Field("f", Schema.create(type), null, null);
+      Assert.assertTrue(RecordBuilderBase.isValidValue(f, new Object()));
+    }
+    
+    // Verify that null is not valid for all non-null primitives:
+    for (Type type : nonNullPrimitives) {
+      Field f = new Field("f", Schema.create(type), null, null);
+      Assert.assertFalse(RecordBuilderBase.isValidValue(f, null));
+    }
+  }
+  
+  @Test
+  public void testIsValidValueWithNullField() {
+    // Verify that null is a valid value for null fields:
+    Assert.assertTrue(RecordBuilderBase.isValidValue(
+        new Field("f", Schema.create(Type.NULL), null, null), null));
+  }
+  
+  @Test
+  public void testIsValidValueWithUnion() {
+    // Verify that null values are not valid for a union with no null type:
+    Schema unionWithoutNull = Schema.createUnion(Arrays.asList(new Schema[] { 
+        Schema.create(Type.STRING), Schema.create(Type.BOOLEAN)
+    }));
+    
+    Assert.assertTrue(RecordBuilderBase.isValidValue(
+        new Field("f", unionWithoutNull, null, null), new Object()));
+    Assert.assertFalse(RecordBuilderBase.isValidValue(
+        new Field("f", unionWithoutNull, null, null), null));
+    
+    // Verify that null values are valid for a union with a null type:
+    Schema unionWithNull = Schema.createUnion(Arrays.asList(new Schema[] { 
+        Schema.create(Type.STRING), Schema.create(Type.NULL)
+    }));
+    
+    Assert.assertTrue(RecordBuilderBase.isValidValue(
+        new Field("f", unionWithNull, null, null), new Object()));
+    Assert.assertTrue(RecordBuilderBase.isValidValue(
+        new Field("f", unionWithNull, null, null), null));
+  }
+}

Added: avro/trunk/lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericRecordBuilder.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericRecordBuilder.java?rev=1162253&view=auto
==============================================================================
--- avro/trunk/lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericRecordBuilder.java (added)
+++ avro/trunk/lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericRecordBuilder.java Fri Aug 26 20:54:50 2011
@@ -0,0 +1,90 @@
+/**
+ * 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.generic;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.apache.avro.Schema.Type;
+import org.apache.avro.generic.GenericData.Record;
+import org.codehaus.jackson.node.TextNode;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Unit test for the GenericRecordBuilder class.
+ */
+public class TestGenericRecordBuilder {
+  @Test
+  public void testGenericBuilder() {
+    Schema schema = recordSchema();
+    GenericRecordBuilder builder = new GenericRecordBuilder(schema);
+    
+    // Verify that builder has no fields set after initialization:
+    for (Field field : schema.getFields()) {
+      Assert.assertFalse("RecordBuilder should not have field " + field.name(), 
+          builder.has(field.name()));
+      Assert.assertNull("Field " + field.name() + " should be null",
+          builder.get(field.name()));
+    }
+    
+    // Set field in builder:
+    builder.set("intField", 1);
+    List<String> anArray = Arrays.asList(new String[] { "one", "two", "three" });
+    builder.set("anArray", anArray);
+    Assert.assertTrue("anArray should be set", builder.has("anArray"));
+    Assert.assertEquals(anArray, builder.get("anArray"));
+    Assert.assertFalse("id should not be set", builder.has("id"));
+    Assert.assertNull(builder.get("id"));
+    
+    // Build the record, and verify that fields are set:
+    Record record = builder.build();
+    Assert.assertEquals(new Integer(1), record.get("intField"));
+    Assert.assertEquals(anArray, record.get("anArray"));
+    Assert.assertNotNull(record.get("id"));
+    Assert.assertEquals("0", record.get("id").toString());
+    
+    // Test copy constructors:
+    Assert.assertEquals(builder, new GenericRecordBuilder(builder));
+    Assert.assertEquals(record, new GenericRecordBuilder(record).build());
+    
+    // Test clear:
+    builder.clear("intField");
+    Assert.assertFalse(builder.has("intField"));
+    Assert.assertNull(builder.get("intField"));
+  }
+  
+  @Test(expected=org.apache.avro.AvroRuntimeException.class)
+  public void attemptToSetNonNullableFieldToNull() {
+    new GenericRecordBuilder(recordSchema()).set("intField", null);
+  }
+  
+  /** Creates a test record schema */
+  private static Schema recordSchema() {
+    List<Field> fields = new ArrayList<Field>();
+    fields.add(new Field("id", Schema.create(Type.STRING), null, new TextNode("0")));
+    fields.add(new Field("intField", Schema.create(Type.INT), null, null));
+    fields.add(new Field("anArray", Schema.createArray(Schema.create(Type.STRING)), null, null));
+    Schema schema = Schema.createRecord("Foo", "test", "mytest", false);
+    schema.setFields(fields);
+    return schema;
+  }
+}

Modified: avro/trunk/lang/java/compiler/src/main/java/org/apache/avro/compiler/specific/SpecificCompiler.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/compiler/src/main/java/org/apache/avro/compiler/specific/SpecificCompiler.java?rev=1162253&r1=1162252&r2=1162253&view=diff
==============================================================================
--- avro/trunk/lang/java/compiler/src/main/java/org/apache/avro/compiler/specific/SpecificCompiler.java (original)
+++ avro/trunk/lang/java/compiler/src/main/java/org/apache/avro/compiler/specific/SpecificCompiler.java Fri Aug 26 20:54:50 2011
@@ -30,6 +30,7 @@ import java.util.Set;
 
 import org.apache.avro.Protocol;
 import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
 import org.apache.velocity.Template;
 import org.apache.velocity.VelocityContext;
 import org.apache.velocity.app.VelocityEngine;
@@ -62,6 +63,23 @@ public class SpecificCompiler {
           "super", "switch", "synchronized", "this", "throw", "throws",
           "transient", "true", "try", "void", "volatile", "while"
         }));
+  
+  /* Reserved words for accessor/mutator methods */
+  private static final Set<String> ACCESSOR_MUTATOR_RESERVED_WORDS = 
+      new HashSet<String>(Arrays.asList(new String[] { "class", "schema" }));
+  static {
+    // Add reserved words to accessor/mutator reserved words
+    ACCESSOR_MUTATOR_RESERVED_WORDS.addAll(RESERVED_WORDS);
+  }
+  
+  /* Reserved words for error types */
+  private static final Set<String> ERROR_RESERVED_WORDS = new HashSet<String>(
+      Arrays.asList(new String[] { "message", "cause" }));
+  static {
+    // Add accessor/mutator reserved words to error reserved words
+    ERROR_RESERVED_WORDS.addAll(ACCESSOR_MUTATOR_RESERVED_WORDS);
+  }
+  
   private static final String FILE_HEADER = 
       "/**\n" +
       " * Autogenerated by Avro\n" +
@@ -367,11 +385,131 @@ public class SpecificCompiler {
 
   /** Utility for template use.  Adds a dollar sign to reserved words. */
   public static String mangle(String word) {
-    if (RESERVED_WORDS.contains(word)) {
+    return mangle(word, false);
+  }
+  
+  /** Utility for template use.  Adds a dollar sign to reserved words. */
+  public static String mangle(String word, boolean isError) {
+    return mangle(word, isError ? ERROR_RESERVED_WORDS : RESERVED_WORDS);
+  }
+  
+  /** Utility for template use.  Adds a dollar sign to reserved words. */
+  public static String mangle(String word, Set<String> reservedWords) {
+    return mangle(word, reservedWords, false);
+  }
+  
+  /** Utility for template use.  Adds a dollar sign to reserved words. */
+  public static String mangle(String word, Set<String> reservedWords, 
+      boolean isMethod) {
+    if (reservedWords.contains(word) || 
+        (isMethod && reservedWords.contains(
+            Character.toLowerCase(word.charAt(0)) + 
+            ((word.length() > 1) ? word.substring(1) : "")))) {
       return word + "$";
     }
     return word;
   }
+  
+  /**
+   * Generates the name of a field accessor method.
+   * @param schema the schema in which the field is defined.
+   * @param field the field for which to generate the accessor name.
+   * @return the name of the accessor method for the given field.
+   */
+  public static String generateGetMethod(Schema schema, Field field) {
+    return generateMethodName(schema, field, "get");
+  }
+  
+  /**
+   * Generates the name of a field mutator method.
+   * @param schema the schema in which the field is defined.
+   * @param field the field for which to generate the accessor name.
+   * @return the name of the mutator method for the given field.
+   */
+  public static String generateSetMethod(Schema schema, Field field) {
+    return generateMethodName(schema, field, "set");
+  }
+  
+  /**
+   * Generates the name of a field "has" method.
+   * @param schema the schema in which the field is defined.
+   * @param field the field for which to generate the accessor name.
+   * @return the name of the has method for the given field.
+   */
+  public static String generateHasMethod(Schema schema, Field field) {
+    return generateMethodName(schema, field, "has");
+  }
+  
+  /**
+   * Generates the name of a field "clear" method.
+   * @param schema the schema in which the field is defined.
+   * @param field the field for which to generate the accessor name.
+   * @return the name of the has method for the given field.
+   */
+  public static String generateClearMethod(Schema schema, Field field) {
+    return generateMethodName(schema, field, "clear");
+  }
+  
+  /**
+   * Generates a method name from a field name.
+   * @param schema the schema in which the field is defined.
+   * @param field the field for which to generate the accessor name.
+   * @param prefix method name prefix, e.g. "get" or "set".
+   * @return the generated method name.
+   */
+  private static String generateMethodName(Schema schema, Field field, 
+      String prefix) {
+
+    // Check for the special case in which the schema defines two fields whose 
+    // names are identical except for the case of the first character:
+    char firstChar = field.name().charAt(0);
+    String conflictingFieldName = (Character.isLowerCase(firstChar) ?
+        Character.toUpperCase(firstChar) : Character.toLowerCase(firstChar)) +
+        (field.name().length() > 1 ? field.name().substring(1) : "");
+    boolean fieldNameConflict = schema.getField(conflictingFieldName) != null;
+    
+    StringBuilder methodBuilder = new StringBuilder(prefix);
+    String fieldName = mangle(field.name(), 
+        schema.isError() ? ERROR_RESERVED_WORDS : 
+          ACCESSOR_MUTATOR_RESERVED_WORDS, true);
+
+    boolean nextCharToUpper = true;
+    for (int ii = 0; ii < fieldName.length(); ii++) {
+      if (fieldName.charAt(ii) == '_') {
+        nextCharToUpper = true;
+      }
+      else if (nextCharToUpper) {
+        methodBuilder.append(Character.toUpperCase(fieldName.charAt(ii)));
+        nextCharToUpper = false;
+      }
+      else {
+        methodBuilder.append(fieldName.charAt(ii));
+      }
+    }
+    
+    // If there is a field name conflict append $0 or $1
+    if (fieldNameConflict) {
+      if (methodBuilder.charAt(methodBuilder.length() - 1) != '$') {
+        methodBuilder.append('$');
+      }
+      methodBuilder.append(Character.isLowerCase(firstChar) ? '0' : '1');
+    }
+
+    return methodBuilder.toString();
+  }
+  
+  /** Tests whether an unboxed Java type can be set to null */
+  public static boolean isUnboxedJavaTypeNullable(Schema schema) {
+    switch (schema.getType()) {
+    // Primitives can't be null; assume anything else can
+    case INT:
+    case LONG:
+    case FLOAT:
+    case DOUBLE:
+    case BOOLEAN: return false;
+    default: return true;
+    }
+  }
 
   public static void main(String[] args) throws Exception {
     //compileSchema(new File(args[0]), new File(args[1]));

Modified: avro/trunk/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/fixed.vm
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/fixed.vm?rev=1162253&r1=1162252&r2=1162253&view=diff
==============================================================================
--- avro/trunk/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/fixed.vm (original)
+++ avro/trunk/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/fixed.vm Fri Aug 26 20:54:50 2011
@@ -25,4 +25,14 @@ package $schema.getNamespace();  
 @org.apache.avro.specific.FixedSize($schema.getFixedSize())
 public class ${this.mangle($schema.getName())} extends org.apache.avro.specific.SpecificFixed {
   public static final org.apache.avro.Schema SCHEMA$ = org.apache.avro.Schema.parse("${this.javaEscape($schema.toString())}");
+  
+  /** Creates a new ${this.mangle($schema.getName())} */
+  public ${this.mangle($schema.getName())}() {
+    super();
+  }
+  
+  /** Creates a new ${this.mangle($schema.getName())} with the given bytes */
+  public ${this.mangle($schema.getName())}(byte[] bytes) {
+    super(bytes);
+  }
 }

Modified: avro/trunk/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/record.vm
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/record.vm?rev=1162253&r1=1162252&r2=1162253&view=diff
==============================================================================
--- avro/trunk/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/record.vm (original)
+++ avro/trunk/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/record.vm Fri Aug 26 20:54:50 2011
@@ -28,7 +28,26 @@ public class ${this.mangle($schema.getNa
 #if ($field.doc())
   /** $field.doc() */
 #end
-  public ${this.javaUnbox($field.schema())} ${this.mangle($field.name())};
+  @Deprecated public ${this.javaUnbox($field.schema())} ${this.mangle($field.name(), $schema.isError())};
+#end
+#if ($schema.isError())
+
+  public ${this.mangle($schema.getName())}() {
+    super();
+  }
+  
+  public ${this.mangle($schema.getName())}(Object value) {
+    super(value);
+  }
+
+  public ${this.mangle($schema.getName())}(Throwable cause) {
+    super(cause);
+  }
+
+  public ${this.mangle($schema.getName())}(Object value, Throwable cause) {
+    super(value, cause);
+  }
+
 #end
   public org.apache.avro.Schema getSchema() { return SCHEMA$; }
   // Used by DatumWriter.  Applications should not call. 
@@ -36,7 +55,7 @@ public class ${this.mangle($schema.getNa
     switch (field$) {
 #set ($i = 0)
 #foreach ($field in $schema.getFields())
-    case $i: return ${this.mangle($field.name())};
+    case $i: return ${this.mangle($field.name(), $schema.isError())};
 #set ($i = $i + 1)
 #end
     default: throw new org.apache.avro.AvroRuntimeException("Bad index");
@@ -48,10 +67,146 @@ public class ${this.mangle($schema.getNa
     switch (field$) {
 #set ($i = 0)
 #foreach ($field in $schema.getFields())
-    case $i: ${this.mangle($field.name())} = (${this.javaType($field.schema())})value$; break;
+    case $i: ${this.mangle($field.name(), $schema.isError())} = (${this.javaType($field.schema())})value$; break;
 #set ($i = $i + 1)
 #end
     default: throw new org.apache.avro.AvroRuntimeException("Bad index");
     }
   }
+
+#foreach ($field in $schema.getFields())
+  /**
+   * Gets the value of the '${this.mangle($field.name(), $schema.isError())}' field.
+#if ($field.doc())   * $field.doc()#end
+   */
+  public ${this.javaType($field.schema())} ${this.generateGetMethod($schema, $field)}() {
+    return ${this.mangle($field.name(), $schema.isError())};
+  }
+
+  /**
+   * Sets the value of the '${this.mangle($field.name(), $schema.isError())}' field.
+#if ($field.doc())   * $field.doc()#end
+   * @param value the value to set.
+   */
+  public void ${this.generateSetMethod($schema, $field)}(${this.javaType($field.schema())} value) {
+    this.${this.mangle($field.name(), $schema.isError())} = value;
+  }
+
+#end
+  /** Creates a new ${this.mangle($schema.getName())} RecordBuilder */
+  public static #if ($schema.getNamespace())$schema.getNamespace().#end${this.mangle($schema.getName())}.Builder newBuilder() {
+    return new #if ($schema.getNamespace())$schema.getNamespace().#end${this.mangle($schema.getName())}.Builder();
+  }
+  
+  /** Creates a new ${this.mangle($schema.getName())} RecordBuilder by copying an existing Builder */
+  public static #if ($schema.getNamespace())$schema.getNamespace().#end${this.mangle($schema.getName())}.Builder newBuilder(#if ($schema.getNamespace())$schema.getNamespace().#end${this.mangle($schema.getName())}.Builder other) {
+    return new #if ($schema.getNamespace())$schema.getNamespace().#end${this.mangle($schema.getName())}.Builder(other);
+  }
+  
+  /** Creates a new ${this.mangle($schema.getName())} RecordBuilder by copying an existing $this.mangle($schema.getName()) instance */
+  public static #if ($schema.getNamespace())$schema.getNamespace().#end${this.mangle($schema.getName())}.Builder newBuilder(#if ($schema.getNamespace())$schema.getNamespace().#end${this.mangle($schema.getName())} other) {
+    return new #if ($schema.getNamespace())$schema.getNamespace().#end${this.mangle($schema.getName())}.Builder(other);
+  }
+  
+  /**
+   * RecordBuilder for ${this.mangle($schema.getName())} instances.
+   */
+  public static class Builder extends#if ($schema.isError()) org.apache.avro.specific.SpecificErrorBuilderBase<${this.mangle($schema.getName())}>#else org.apache.avro.specific.SpecificRecordBuilderBase<${this.mangle($schema.getName())}>#end
+
+    implements#if ($schema.isError()) org.apache.avro.data.ErrorBuilder<${this.mangle($schema.getName())}>#else org.apache.avro.data.RecordBuilder<${this.mangle($schema.getName())}>#end {
+
+#foreach ($field in $schema.getFields())
+    private ${this.javaUnbox($field.schema())} ${this.mangle($field.name(), $schema.isError())};
+#end
+
+    /** Creates a new Builder */
+    private Builder() {
+      super(#if ($schema.getNamespace())$schema.getNamespace().#end${this.mangle($schema.getName())}.SCHEMA$);
+    }
+    
+    /** Creates a Builder by copying an existing Builder */
+    private Builder(#if ($schema.getNamespace())$schema.getNamespace().#end${this.mangle($schema.getName())}.Builder other) {
+      super(other);
+    }
+    
+    /** Creates a Builder by copying an existing $this.mangle($schema.getName()) instance */
+    private Builder(#if ($schema.getNamespace())$schema.getNamespace().#end${this.mangle($schema.getName())} other) {
+      #if ($schema.isError())super(other)#else
+      super(#if ($schema.getNamespace())$schema.getNamespace().#end${this.mangle($schema.getName())}.SCHEMA$)#end;
+#foreach ($field in $schema.getFields())
+      if (isValidValue(fields[$field.pos()], other.${this.mangle($field.name(), $schema.isError())})) {
+        ${this.mangle($field.name(), $schema.isError())} = (${this.javaType($field.schema())}) data.deepCopy(fields[$field.pos()].schema(), other.${this.mangle($field.name(), $schema.isError())});
+        fieldSetFlags[$field.pos()] = true;
+      }
+#end
+    }
+#if ($schema.isError())
+
+    @Override
+    public #if ($schema.getNamespace())$schema.getNamespace().#end${this.mangle($schema.getName())}.Builder setValue(Object value) {
+      super.setValue(value);
+      return this;
+    }
+    
+    @Override
+    public #if ($schema.getNamespace())$schema.getNamespace().#end${this.mangle($schema.getName())}.Builder clearValue() {
+      super.clearValue();
+      return this;
+    }
+
+    @Override
+    public #if ($schema.getNamespace())$schema.getNamespace().#end${this.mangle($schema.getName())}.Builder setCause(Throwable cause) {
+      super.setCause(cause);
+      return this;
+    }
+    
+    @Override
+    public #if ($schema.getNamespace())$schema.getNamespace().#end${this.mangle($schema.getName())}.Builder clearCause() {
+      super.clearCause();
+      return this;
+    }
+#end
+
+#foreach ($field in $schema.getFields())
+    /** Gets the value of the '${this.mangle($field.name(), $schema.isError())}' field */
+    public ${this.javaType($field.schema())} ${this.generateGetMethod($schema, $field)}() {
+      return ${this.mangle($field.name(), $schema.isError())};
+    }
+    
+    /** Sets the value of the '${this.mangle($field.name(), $schema.isError())}' field */
+    public #if ($schema.getNamespace())$schema.getNamespace().#end${this.mangle($schema.getName())}.Builder ${this.generateSetMethod($schema, $field)}(${this.javaUnbox($field.schema())} value) {
+      validate(fields[$field.pos()], value);
+      this.${this.mangle($field.name(), $schema.isError())} = value;
+      fieldSetFlags[$field.pos()] = true;
+      return this; 
+    }
+    
+    /** Checks whether the '${this.mangle($field.name(), $schema.isError())}' field has been set */
+    public boolean ${this.generateHasMethod($schema, $field)}() {
+      return fieldSetFlags[$field.pos()];
+    }
+    
+    /** Clears the value of the '${this.mangle($field.name(), $schema.isError())}' field */
+    public #if ($schema.getNamespace())$schema.getNamespace().#end${this.mangle($schema.getName())}.Builder ${this.generateClearMethod($schema, $field)}() {
+#if (${this.isUnboxedJavaTypeNullable($field.schema())})
+      ${this.mangle($field.name(), $schema.isError())} = null;
+#end
+      fieldSetFlags[$field.pos()] = false;
+      return this;
+    }
+
+#end
+    @Override
+    public ${this.mangle($schema.getName())} build() {
+      try {
+        ${this.mangle($schema.getName())} record = new ${this.mangle($schema.getName())}(#if ($schema.isError())getValue(), getCause()#end);
+#foreach ($field in $schema.getFields())
+        record.${this.mangle($field.name(), $schema.isError())} = fieldSetFlags[$field.pos()] ? this.${this.mangle($field.name(), $schema.isError())} : (${this.javaType($field.schema())}) getDefaultValue(fields[$field.pos()]);
+#end
+        return record;
+      } catch (Exception e) {
+        throw new org.apache.avro.AvroRuntimeException(e);
+      }
+    }
+  }
 }

Modified: avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestCompare.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestCompare.java?rev=1162253&r1=1162252&r2=1162253&view=diff
==============================================================================
--- avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestCompare.java (original)
+++ avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestCompare.java Fri Aug 26 20:54:50 2011
@@ -167,19 +167,17 @@ public class TestCompare {
   public void testSpecificRecord() throws Exception {
     TestRecord s1 = new TestRecord();
     TestRecord s2 = new TestRecord();
-    s1.name = new Utf8("foo");
-    s1.kind = Kind.BAZ;
-    s1.hash = new MD5();
-    s1.hash.bytes(new byte[] {0,1,2,3,4,5,6,7,8,9,0,1,2,3,4,5});
-    s2.name = new Utf8("bar");
-    s2.kind = Kind.BAR;
-    s2.hash = new MD5();
-    s2.hash.bytes(new byte[] {0,1,2,3,4,5,6,7,8,9,0,1,2,3,4,6});
+    s1.setName(new Utf8("foo"));
+    s1.setKind(Kind.BAZ);
+    s1.setHash(new MD5(new byte[] {0,1,2,3,4,5,6,7,8,9,0,1,2,3,4,5}));
+    s2.setName(new Utf8("bar"));
+    s2.setKind(Kind.BAR);
+    s2.setHash(new MD5(new byte[] {0,1,2,3,4,5,6,7,8,9,0,1,2,3,4,6}));
     Schema schema = SpecificData.get().getSchema(TestRecord.class);
 
     check(schema, s1, s2, true, new SpecificDatumWriter<TestRecord>(schema),
           SpecificData.get());
-    s2.kind = Kind.BAZ;
+    s2.setKind(Kind.BAZ);
     check(schema, s1, s2, true, new SpecificDatumWriter<TestRecord>(schema),
           SpecificData.get());
   }  

Modified: avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestNamespaceSpecific.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestNamespaceSpecific.java?rev=1162253&r1=1162252&r2=1162253&view=diff
==============================================================================
--- avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestNamespaceSpecific.java (original)
+++ avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestNamespaceSpecific.java Fri Aug 26 20:54:50 2011
@@ -41,9 +41,7 @@ public class TestNamespaceSpecific {
   public static class TestImpl implements TestNamespace {
     public TestRecord echo(TestRecord record) { return record; }
     public Void error() throws AvroRemoteException {
-      TestError error = new TestError();
-      error.message = new Utf8("an error");
-      throw error;
+      throw TestError.newBuilder().setMessage$(new Utf8("an error")).build();
     }
   }
 
@@ -64,9 +62,7 @@ public class TestNamespaceSpecific {
   @Test
   public void testEcho() throws IOException {
     TestRecord record = new TestRecord();
-    record.hash = new MD5();
-    System.arraycopy(new byte[]{0,1,2,3,4,5,6,7,8,9,0,1,2,3,4,5}, 0,
-                     record.hash.bytes(), 0, 16);
+    record.setHash(new MD5(new byte[]{0,1,2,3,4,5,6,7,8,9,0,1,2,3,4,5}));
     TestRecord echoed = proxy.echo(record);
     assertEquals(record, echoed);
     assertEquals(record.hashCode(), echoed.hashCode());
@@ -81,7 +77,7 @@ public class TestNamespaceSpecific {
       error = e;
     }
     assertNotNull(error);
-    assertEquals("an error", error.message.toString());
+    assertEquals("an error", error.getMessage$().toString());
   }
 
   @AfterClass

Modified: avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestProtocolSpecific.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestProtocolSpecific.java?rev=1162253&r1=1162252&r2=1162253&view=diff
==============================================================================
--- avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestProtocolSpecific.java (original)
+++ avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestProtocolSpecific.java Fri Aug 26 20:54:50 2011
@@ -76,9 +76,7 @@ public class TestProtocolSpecific {
     public ByteBuffer echoBytes(ByteBuffer data) { return data; }
     public Void error() throws AvroRemoteException {
       if (throwUndeclaredError) throw new RuntimeException("foo");
-      TestError error = new TestError();
-      error.message = new Utf8("an error");
-      throw error;
+      throw TestError.newBuilder().setMessage$(new Utf8("an error")).build();
     }
     public void ack() { ackCount++; }
   }
@@ -137,11 +135,9 @@ public class TestProtocolSpecific {
   @Test
   public void testEcho() throws IOException {
     TestRecord record = new TestRecord();
-    record.name = new Utf8("foo");
-    record.kind = Kind.BAR;
-    record.hash = new MD5();
-    System.arraycopy(new byte[]{0,1,2,3,4,5,6,7,8,9,0,1,2,3,4,5}, 0,
-                     record.hash.bytes(), 0, 16);
+    record.setName(new Utf8("foo"));
+    record.setKind(Kind.BAR);
+    record.setHash(new MD5(new byte[]{0,1,2,3,4,5,6,7,8,9,0,1,2,3,4,5}));
     TestRecord echoed = proxy.echo(record);
     assertEquals(record, echoed);
     assertEquals(record.hashCode(), echoed.hashCode());
@@ -181,7 +177,7 @@ public class TestProtocolSpecific {
       error = e;
     }
     assertNotNull(error);
-    assertEquals("an error", error.message.toString());
+    assertEquals("an error", error.getMessage$().toString());
   }
 
   @Test