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 2009/05/22 19:48:19 UTC

svn commit: r777638 - in /hadoop/avro/trunk: ./ lib/paranamer-1.3/ src/doc/content/xdocs/ src/java/org/apache/avro/ src/java/org/apache/avro/generic/ src/java/org/apache/avro/reflect/ src/java/org/apache/avro/specific/ src/test/java/org/apache/avro/ sr...

Author: cutting
Date: Fri May 22 17:48:18 2009
New Revision: 777638

URL: http://svn.apache.org/viewvc?rev=777638&view=rev
Log:
AVRO-10.  Add Java support for fixed-sized types.

Added:
    hadoop/avro/trunk/lib/paranamer-1.3/qdox-1.9.1.jar   (with props)
    hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericFixed.java
    hadoop/avro/trunk/src/java/org/apache/avro/reflect/FixedSize.java
    hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificFixed.java
Removed:
    hadoop/avro/trunk/lib/paranamer-1.3/qdox-1.9.jar
Modified:
    hadoop/avro/trunk/CHANGES.txt
    hadoop/avro/trunk/src/doc/content/xdocs/spec.xml
    hadoop/avro/trunk/src/java/org/apache/avro/Schema.java
    hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericData.java
    hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumReader.java
    hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumWriter.java
    hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectData.java
    hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectDatumReader.java
    hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectResponder.java
    hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificCompiler.java
    hadoop/avro/trunk/src/test/java/org/apache/avro/RandomData.java
    hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolGeneric.java
    hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolSpecific.java
    hadoop/avro/trunk/src/test/java/org/apache/avro/TestSchema.java
    hadoop/avro/trunk/src/test/schemata/interop.js
    hadoop/avro/trunk/src/test/schemata/simple.js

Modified: hadoop/avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/CHANGES.txt?rev=777638&r1=777637&r2=777638&view=diff
==============================================================================
--- hadoop/avro/trunk/CHANGES.txt (original)
+++ hadoop/avro/trunk/CHANGES.txt Fri May 22 17:48:18 2009
@@ -21,6 +21,8 @@
 
     AVRO-18.  Add support for enum types.  (cutting & sharad)
 
+    AVRO-10.  Add Java support for fixed-sized types. (cutting)
+
   IMPROVEMENTS
 
     AVRO-11.  Re-implement specific and reflect datum readers and

Added: hadoop/avro/trunk/lib/paranamer-1.3/qdox-1.9.1.jar
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lib/paranamer-1.3/qdox-1.9.1.jar?rev=777638&view=auto
==============================================================================
Binary file - no diff available.

Propchange: hadoop/avro/trunk/lib/paranamer-1.3/qdox-1.9.1.jar
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Modified: hadoop/avro/trunk/src/doc/content/xdocs/spec.xml
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/doc/content/xdocs/spec.xml?rev=777638&r1=777637&r2=777638&view=diff
==============================================================================
--- hadoop/avro/trunk/src/doc/content/xdocs/spec.xml (original)
+++ hadoop/avro/trunk/src/doc/content/xdocs/spec.xml Fri May 22 17:48:18 2009
@@ -85,8 +85,8 @@
       <section id="schema_complex">
         <title>Complex Types</title>
         
-        <p>Avro supports five kinds of complex types: records, enums,
-        arrays, maps and unions.</p>
+        <p>Avro supports six kinds of complex types: records, enums,
+        arrays, maps, unions and fixed.</p>
 
         <section>
           <title>Records</title>
@@ -122,7 +122,7 @@
 		    <tr><td>enum</td><td>string</td><td>"FOO"</td></tr>
 		    <tr><td>array</td><td>array</td><td>[1]</td></tr>
 		    <tr><td>map</td><td>object</td><td>{"a": 1}</td></tr>
-		  </table>
+		    <tr><td>fixed</td><td>&nbsp;base64 string</td><td>"c3VyZS4="		  </table>
 		</li>
 	      </ul>
 	    </li>
@@ -192,13 +192,28 @@
           arrays.  For example, <code>["string", "null"]</code>
           declares a schema which may be either a string or null.</p>
 	  <p>Unions may not contain more than one schema with the same
-	  type, except for named records.  For example, unions
-	  containing two array types or two map types are not
-	  permitted, but two record types with different names are
-	  permitted.  (Record names permit efficient resolution when
-	  reading and writing unions.)</p>
+	  type, except for the named types record and fixed.  For
+	  example, unions containing two array types or two map types
+	  are not permitted, but two types with different names are
+	  permitted.  (Names permit efficient resolution when reading
+	  and writing unions.)</p>
 	  <p>Unions may not immediately contain other unions.</p>
         </section>
+
+        <section>
+          <title>Fixed</title>
+          <p>Fixed uses the type name <code>"fixed"</code> and supports
+          two attributes:</p>
+	  <ul>
+	    <li><code>name</code>: the name of the fixed (required).</li>
+            <li><code>size</code>: an integer, specifying the number
+            of bytes per value (required).</li>
+	  </ul>
+	  <p>For example, 16-byte quantity may be declared with:</p>
+	  <source>{"type": "fixed", "size": 16, "name": "md5"}</source>
+	</section>
+
+
       </section> <!-- end complex types -->
 
       <section>
@@ -398,6 +413,13 @@
 	      <source>00 02 61</source></li>
           </ul>
         </section>
+
+        <section>
+          <title>Fixed</title>
+	  <p>Fixed instances are serialized using the number of bytes
+	  declared in the schema.</p>
+        </section>
+
       </section> <!-- end complex types -->
 
     </section>
@@ -703,6 +725,7 @@
 	    <li>both schemas are arrays whose item types match</li>
 	    <li>both schemas are maps whose value types match</li>
 	    <li>both schemas are enums whose names match</li>
+	    <li>both schemas are fixed whose sizes and names match</li>
 	    <li>both schemas are records with the same name</li>
 	    <li>either schema is a union</li>
 	    <li>both schemas have same primitive type</li>

Modified: hadoop/avro/trunk/src/java/org/apache/avro/Schema.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/Schema.java?rev=777638&r1=777637&r2=777638&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/Schema.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/Schema.java Fri May 22 17:48:18 2009
@@ -43,6 +43,7 @@
  * <li>An <i>array</i> of values, all of the same schema;
  * <li>A <i>map</i>, containing string/value pairs, of a declared schema;
  * <li>A <i>union</i> of other schemas;
+ * <li>A <i>fixed</i> sized binary object;
  * <li>A unicode <i>string</i>;
  * <li>A sequence of <i>bytes</i>;
  * <li>A 32-bit signed <i>int</i>;
@@ -63,7 +64,7 @@
 
   /** The type of a schema. */
   public enum Type
-  { RECORD, ENUM, ARRAY, MAP, UNION, STRING, BYTES,
+  { RECORD, ENUM, ARRAY, MAP, UNION, FIXED, STRING, BYTES,
       INT, LONG, FLOAT, DOUBLE, BOOLEAN, NULL };
 
   private final Type type;
@@ -119,6 +120,11 @@
     return new UnionSchema(types);
   }
 
+  /** Create a union schema. */
+  public static Schema createFixed(String name, String space, int size) {
+    return new FixedSchema(name, space, size);
+  }
+
   /** Return the type of this schema. */
   public Type getType() { return type; }
 
@@ -147,13 +153,12 @@
     throw new AvroRuntimeException("Not an enum: "+this);
   }    
 
-
-  /** If this is a record or enum, returns its name, if any. */
+  /** If this is a record, enum or fixed, returns its name, if any. */
   public String getName() {
     throw new AvroRuntimeException("Not a record or enum: "+this);
   }
 
-  /** If this is a record or enum, returns its namespace, if any. */
+  /** If this is a record, enum or fixed, returns its namespace, if any. */
   public String getNamespace() {
     throw new AvroRuntimeException("Not a record or enum: "+this);
   }
@@ -178,6 +183,11 @@
     throw new AvroRuntimeException("Not a union: "+this);
   }
 
+  /** If this is fixed, returns its size. */
+  public int getFixedSize() {
+    throw new AvroRuntimeException("Not fixed: "+this);
+  }
+
   /** Render this as <a href="http://json.org/">JSON</a>.*/
   public String toString() { return toString(new Names()); }
 
@@ -232,7 +242,8 @@
           ?"":"\"namespace\": \""+space+"\", ");
     }
     public boolean equalNames(NamedSchema that) {
-      return (name==null ? that.name==null : name.equals(that.name))
+      return that == null ? false
+        : (name==null ? that.name==null : name.equals(that.name))
         && (space==null ? that.space==null : space.equals(that.space));
     }
     public int hashCode() {
@@ -327,7 +338,7 @@
       else if (name != null) names.put(name, this);
       StringBuilder buffer = new StringBuilder();
       buffer.append("{\"type\": \"enum\", "
-                    +"\"name\": \""+name+"\", "
+                    +(name!=null?"\"name\": \""+name+"\", ":"")
                     +"\"symbols\": [");
       int count = 0;
       for (String symbol : symbols) {
@@ -427,6 +438,28 @@
     }
   }
 
+  private static class FixedSchema extends NamedSchema {
+    private final int size;
+    public FixedSchema(String name, String space, int size) {
+      super(Type.FIXED, name, space);
+      this.size = size;
+    }
+    public int getFixedSize() { return size; }
+    public boolean equals(Object o) {
+      if (o == this) return true;
+      FixedSchema that = (FixedSchema)o;
+      return equalNames(that) && size == that.size;
+    }
+    public int hashCode() { return super.hashCode() + size; }
+    public String toString(Names names) {
+      if (this.equals(names.get(name))) return "\""+name+"\"";
+      else if (name != null) names.put(name, this);
+      return "{\"type\": \"fixed\", "
+        +(name!=null?"\"name\": \""+name+"\", ":"")
+        +"\"size\": "+size+"}";
+    }
+  }
+
   private static class StringSchema extends Schema {
     public StringSchema() { super(Type.STRING); }
     public String toString() { return "\"string\""; }
@@ -545,12 +578,16 @@
       if (typeNode == null)
         throw new SchemaParseException("No type: "+schema);
       String type = typeNode.getTextValue();
-      if (type.equals("record") || type.equals("error")) { // record
-        LinkedHashMap<String,Field> fields = new LinkedHashMap<String,Field>();
+      String name = null, space = null;
+      if (type.equals("record") || type.equals("error")
+          || type.equals("enum") || type.equals("fixed")) {
         JsonNode nameNode = schema.getFieldValue("name");
-        String name = nameNode != null ? nameNode.getTextValue() : null;
+        name = nameNode != null ? nameNode.getTextValue() : null;
         JsonNode spaceNode = schema.getFieldValue("namespace");
-        String space = spaceNode!=null?spaceNode.getTextValue():names.space();
+        space = spaceNode!=null?spaceNode.getTextValue():names.space();
+      }
+      if (type.equals("record") || type.equals("error")) { // record
+        LinkedHashMap<String,Field> fields = new LinkedHashMap<String,Field>();
         RecordSchema result =
           new RecordSchema(name, space, type.equals("error"));
         if (name != null) names.put(name, result);
@@ -571,10 +608,6 @@
         result.setFields(fields);
         return result;
       } else if (type.equals("enum")) {           // enum
-        JsonNode nameNode = schema.getFieldValue("name");
-        String name = nameNode != null ? nameNode.getTextValue() : null;
-        JsonNode spaceNode = schema.getFieldValue("namespace");
-        String space = spaceNode!=null?spaceNode.getTextValue():names.space();
         JsonNode symbolsNode = schema.getFieldValue("symbols");
         if (symbolsNode == null || !symbolsNode.isArray())
           throw new SchemaParseException("Enum has no symbols: "+schema);
@@ -588,6 +621,11 @@
         return new ArraySchema(parse(schema.getFieldValue("items"), names));
       } else if (type.equals("map")) {            // map
         return new MapSchema(parse(schema.getFieldValue("values"), names));
+      } else if (type.equals("fixed")) {          // fixed
+        Schema result = new FixedSchema(name, space, schema
+                                        .getFieldValue("size").getIntValue());
+        if (name != null) names.put(name, result);
+        return result;
       } else
         throw new SchemaParseException("Type not yet supported: "+type);
     } else if (schema.isArray()) {                // union

Modified: hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericData.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericData.java?rev=777638&r1=777637&r2=777638&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericData.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericData.java Fri May 22 17:48:18 2009
@@ -21,6 +21,7 @@
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
+import java.util.Arrays;
 import java.util.Map;
 
 import org.apache.avro.AvroTypeException;
@@ -96,6 +97,28 @@
     }
   }
 
+  public static class Fixed implements GenericFixed {
+    private byte[] bytes;
+
+    public Fixed(Schema schema) { bytes(new byte[schema.getFixedSize()]); }
+    public Fixed(byte[] bytes) { bytes(bytes); }
+
+    protected Fixed() {}
+    protected void bytes(byte[] bytes) { this.bytes = bytes; }
+
+    public byte[] bytes() { return bytes; }
+
+    public boolean equals(Object o) {
+      if (o == this) return true;
+      return o instanceof GenericFixed
+        && Arrays.equals(bytes, ((GenericFixed)o).bytes());
+    }
+
+    public int hashCode() { return Arrays.hashCode(bytes); }
+
+  }
+
+
   /** Returns true if a Java datum matches a schema. */
   public static boolean validate(Schema schema, Object datum) {
     switch (schema.getType()) {
@@ -128,6 +151,9 @@
         if (validate(type, datum))
           return true;
       return false;
+    case FIXED:
+      return datum instanceof GenericFixed
+        && ((GenericFixed)datum).bytes().length==schema.getFixedSize();
     case STRING:  return datum instanceof Utf8;
     case BYTES:   return datum instanceof ByteBuffer;
     case INT:     return datum instanceof Integer;
@@ -236,6 +262,9 @@
         throw new AvroTypeException("Empty map: "+datum);
       }
       return Schema.createMap(value);
+    } else if (datum instanceof GenericFixed) {
+      return Schema.createFixed(null, null,
+                                ((GenericFixed)datum).bytes().length);
     }
     else if (datum instanceof Utf8)       return Schema.create(Type.STRING);
     else if (datum instanceof ByteBuffer) return Schema.create(Type.BYTES);

Modified: hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumReader.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumReader.java?rev=777638&r1=777637&r2=777638&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumReader.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumReader.java Fri May 22 17:48:18 2009
@@ -70,6 +70,7 @@
     case ENUM:    return readEnum(actual, expected, in);
     case ARRAY:   return readArray(old, actual, expected, in);
     case MAP:     return readMap(old, actual, expected, in);
+    case FIXED:   return readFixed(old, actual, expected, in);
     case STRING:  return readString(old, in);
     case BYTES:   return readBytes(old, in);
     case INT:     return in.readInt();
@@ -88,6 +89,7 @@
       if (branch.getType() == actual.getType())
         switch (branch.getType()) {
         case RECORD:
+        case FIXED:
           String name = branch.getName();
           if (name == null || name.equals(actual.getName()))
             return branch;
@@ -225,6 +227,7 @@
       }
       return map;
     case UNION:   return defaultFieldValue(old, schema.getTypes().get(0), json);
+    case FIXED:   return createFixed(old,json.getTextValue().getBytes(),schema);
     case STRING:  return createString(json.getTextValue());
     case BYTES:   return createBytes(json.getTextValue().getBytes());
     case INT:     return json.getIntValue();
@@ -305,6 +308,34 @@
     ((Map) map).put(key, value);
   }
   
+  /** Called to read a fixed value. May be overridden for alternate fixed
+   * representations.  By default, returns {@link GenericFixed}. */
+  protected Object readFixed(Object old, Schema actual, Schema expected,
+                             ValueReader in)
+    throws IOException {
+    if (!actual.equals(expected))
+      throw new AvroTypeException("Expected "+expected+", found "+actual);
+    GenericFixed fixed = (GenericFixed)createFixed(old, expected);
+    in.readBytes(fixed.bytes(), 0, actual.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
@@ -395,6 +426,9 @@
     case UNION:
       skip(schema.getTypes().get((int)in.readLong()), in);
       break;
+    case FIXED:
+      in.skip(schema.getFixedSize());
+      break;
     case STRING:
     case BYTES:
       long length = in.readLong();

Modified: hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumWriter.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumWriter.java?rev=777638&r1=777637&r2=777638&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumWriter.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumWriter.java Fri May 22 17:48:18 2009
@@ -61,6 +61,7 @@
       out.writeLong(index);
       write(schema.getTypes().get(index), datum, out);
       break;
+    case FIXED:   writeFixed(schema, datum, out);   break;
     case STRING:  writeString(datum, out);          break;
     case BYTES:   writeBytes(datum, out);           break;
     case INT:     out.writeInt((Integer)datum);     break;
@@ -189,6 +190,7 @@
     case ENUM:    return isEnum(datum);
     case ARRAY:   return isArray(datum);
     case MAP:     return isMap(datum);
+    case FIXED:   return isFixed(datum);
     case STRING:  return isString(datum);
     case BYTES:   return isBytes(datum);
     case INT:     return datum instanceof Integer;
@@ -201,6 +203,13 @@
     }
   }
 
+  /** Called to write a fixed value.  May be overridden for alternate fixed
+   * representations.*/
+  protected void writeFixed(Schema schema, Object datum, ValueWriter out)
+    throws IOException {
+    out.write(((GenericFixed)datum).bytes(), 0, schema.getFixedSize());
+  }
+
   /** Called by the default implementation of {@link #instanceOf}.*/
   protected boolean isArray(Object datum) {
     return datum instanceof GenericArray;
@@ -220,6 +229,11 @@
   protected boolean isMap(Object datum) {
     return (datum instanceof Map) && (!(datum instanceof GenericRecord));
   }
+  
+  /** Called by the default implementation of {@link #instanceOf}.*/
+  protected boolean isFixed(Object datum) {
+    return datum instanceof GenericFixed;
+  }
 
   /** Called by the default implementation of {@link #instanceOf}.*/
   protected boolean isString(Object datum) {

Added: hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericFixed.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericFixed.java?rev=777638&view=auto
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericFixed.java (added)
+++ hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericFixed.java Fri May 22 17:48:18 2009
@@ -0,0 +1,24 @@
+/**
+ * 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;
+
+/** Fixed-size data. */
+public interface GenericFixed {
+  /** Return the data. */
+  public byte[] bytes();
+}

Added: hadoop/avro/trunk/src/java/org/apache/avro/reflect/FixedSize.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/reflect/FixedSize.java?rev=777638&view=auto
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/reflect/FixedSize.java (added)
+++ hadoop/avro/trunk/src/java/org/apache/avro/reflect/FixedSize.java Fri May 22 17:48:18 2009
@@ -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.reflect;
+
+import org.apache.avro.generic.GenericFixed;
+import java.lang.annotation.*;
+
+/** Declares the size of implementations of {@link GenericFixed}. */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.TYPE})
+@Documented
+public @interface FixedSize {
+  /** The declared size of instances of classes with this annotation. */
+  int value();
+}

Modified: hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectData.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectData.java?rev=777638&r1=777637&r2=777638&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectData.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectData.java Fri May 22 17:48:18 2009
@@ -37,6 +37,7 @@
 import org.apache.avro.Protocol.Message;
 import org.apache.avro.Schema.Type;
 import org.apache.avro.generic.GenericArray;
+import org.apache.avro.generic.GenericFixed;
 import org.apache.avro.ipc.AvroRemoteException;
 import org.apache.avro.util.Utf8;
 
@@ -79,6 +80,7 @@
         if (validate(type, datum))
           return true;
       return false;
+    case FIXED:   return datum instanceof GenericFixed;
     case STRING:  return datum instanceof Utf8;
     case BYTES:   return datum instanceof ByteBuffer;
     case INT:     return datum instanceof Integer;
@@ -163,6 +165,13 @@
           names.put(name, schema);
           return schema;
         }
+                                                  // fixed
+        if (GenericFixed.class.isAssignableFrom(c)) {
+          int size = ((FixedSize)c.getAnnotation(FixedSize.class)).value();
+          schema = Schema.createFixed(name, space, size);
+          names.put(name, schema);
+          return schema;
+        }
                                                   // record
         LinkedHashMap<String,Schema.Field> fields =
           new LinkedHashMap<String,Schema.Field>();

Modified: hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectDatumReader.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectDatumReader.java?rev=777638&r1=777637&r2=777638&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectDatumReader.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectDatumReader.java Fri May 22 17:48:18 2009
@@ -44,7 +44,7 @@
 
   protected Object newRecord(Object old, Schema schema) {
     Class c = getClass(schema);
-    return(c.isInstance(old) ? old : newInstance(c));
+    return (c.isInstance(old) ? old : newInstance(c));
   }
 
   protected void addField(Object record, String name, int position, Object o) {
@@ -74,6 +74,11 @@
     return Enum.valueOf(getClass(schema), symbol);
   }
 
+  protected Object createFixed(Object old, Schema schema) {
+    Class c = getClass(schema);
+    return c.isInstance(old) ? old : newInstance(c);
+  }
+
   private static final Class<?>[] EMPTY_ARRAY = new Class[]{};
   private static final Map<Class,Constructor> CTOR_CACHE =
     new ConcurrentHashMap<Class,Constructor>();
@@ -113,4 +118,3 @@
   }
 
 }
-

Modified: hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectResponder.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectResponder.java?rev=777638&r1=777637&r2=777638&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectResponder.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectResponder.java Fri May 22 17:48:18 2009
@@ -102,6 +102,7 @@
 
   private Class paramType(Schema schema) throws ClassNotFoundException {
     switch (schema.getType()) {
+    case FIXED:
     case RECORD:
     case ENUM:    return Class.forName(packageName+schema.getName());
     case ARRAY:   return GenericArray.class;

Modified: hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificCompiler.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificCompiler.java?rev=777638&r1=777637&r2=777638&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificCompiler.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificCompiler.java Fri May 22 17:48:18 2009
@@ -98,7 +98,10 @@
     line(0, "import org.apache.avro.util.Utf8;");
     line(0, "import org.apache.avro.ipc.AvroRemoteException;");
     line(0, "import org.apache.avro.generic.GenericArray;");
-    line(0, "import org.apache.avro.specific.SpecificRecord;\n");
+    line(0, "import org.apache.avro.specific.SpecificRecord;");
+    line(0, "import org.apache.avro.specific.SpecificFixed;");
+    line(0, "import org.apache.avro.reflect.FixedSize;");
+    buffer.append("\n");
   }
 
   private String params(Schema request) {
@@ -190,6 +193,13 @@
     case MAP:
       compile(schema.getValueType(), name+"Value", d);
       break;
+    case FIXED:
+      buffer.append("\n");
+      line(d, "@FixedSize("+schema.getFixedSize()+")");
+      line(d, ((d==0)?"public ":"")
+           +((d>1)?"static ":"")+"class "+type
+           +" extends SpecificFixed {}");
+      break;
     case UNION:
       int choice = 0;
       for (Schema t : schema.getTypes())
@@ -209,6 +219,7 @@
     switch (schema.getType()) {
     case RECORD:
     case ENUM:
+    case FIXED:
       return schema.getName() == null ? cap(name) : schema.getName();
     case ARRAY:
       return "GenericArray<"+type(schema.getElementType(),name+"Element")+">";

Added: hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificFixed.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificFixed.java?rev=777638&view=auto
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificFixed.java (added)
+++ hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificFixed.java Fri May 22 17:48:18 2009
@@ -0,0 +1,28 @@
+/**
+ * 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.generic.GenericData;
+import org.apache.avro.reflect.FixedSize;
+
+/** Base class for generated fixed-sized data classes. */
+public abstract class SpecificFixed extends GenericData.Fixed {
+  public SpecificFixed() {
+    bytes(new byte[getClass().getAnnotation(FixedSize.class).value()]);
+  }
+}

Modified: hadoop/avro/trunk/src/test/java/org/apache/avro/RandomData.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/test/java/org/apache/avro/RandomData.java?rev=777638&r1=777637&r2=777638&view=diff
==============================================================================
--- hadoop/avro/trunk/src/test/java/org/apache/avro/RandomData.java (original)
+++ hadoop/avro/trunk/src/test/java/org/apache/avro/RandomData.java Fri May 22 17:48:18 2009
@@ -86,6 +86,10 @@
     case UNION:
       List<Schema> types = schema.getTypes();
       return generate(types.get(random.nextInt(types.size())), random, d);
+    case FIXED:
+      byte[] bytes = new byte[schema.getFixedSize()];
+      random.nextBytes(bytes);
+      return new GenericData.Fixed(bytes);
     case STRING:  return randomUtf8(random, 40);
     case BYTES:   return randomBytes(random, 40);
     case INT:     return random.nextInt();

Modified: hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolGeneric.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolGeneric.java?rev=777638&r1=777637&r2=777638&view=diff
==============================================================================
--- hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolGeneric.java (original)
+++ hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolGeneric.java Fri May 22 17:48:18 2009
@@ -115,6 +115,8 @@
       new GenericData.Record(PROTOCOL.getTypes().get("TestRecord"));
     record.put("name", new Utf8("foo"));
     record.put("kind", "BAR");
+    record.put("hash", new GenericData.Fixed
+               (new byte[]{0,1,2,3,4,5,6,7,8,9,0,1,2,3,4,5}));
     GenericRecord params =
       new GenericData.Record(PROTOCOL.getMessages().get("echo").getRequest());
     params.put("record", record);

Modified: hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolSpecific.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolSpecific.java?rev=777638&r1=777637&r2=777638&view=diff
==============================================================================
--- hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolSpecific.java (original)
+++ hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolSpecific.java Fri May 22 17:48:18 2009
@@ -25,6 +25,7 @@
 import org.apache.avro.specific.SpecificResponder;
 import org.apache.avro.test.Simple;
 import org.apache.avro.test.Simple.Kind;
+import org.apache.avro.test.Simple.MD5;
 import org.apache.avro.test.Simple.TestError;
 import org.apache.avro.test.Simple.TestRecord;
 import org.apache.avro.util.Utf8;
@@ -93,6 +94,9 @@
     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);
     TestRecord echoed = proxy.echo(record);
     assertEquals(record.name, echoed.name);
   }

Modified: hadoop/avro/trunk/src/test/java/org/apache/avro/TestSchema.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/test/java/org/apache/avro/TestSchema.java?rev=777638&r1=777637&r2=777638&view=diff
==============================================================================
--- hadoop/avro/trunk/src/test/java/org/apache/avro/TestSchema.java (original)
+++ hadoop/avro/trunk/src/test/java/org/apache/avro/TestSchema.java Fri May 22 17:48:18 2009
@@ -114,6 +114,12 @@
   }
 
   @Test
+  public void testFixed() throws Exception {
+    check("{\"type\": \"fixed\", \"size\": 1}", "\"a\"",
+          new GenericData.Fixed(new byte[]{(byte)'a'}));
+  }
+
+  @Test
   public void testRecursive() throws Exception {
     check("{\"type\": \"record\", \"name\": \"Node\", \"fields\": ["
           +"{\"name\":\"label\", \"type\":\"string\"},"
@@ -209,6 +215,4 @@
     assertEquals("Wrong default.", defaultValue, record.get("f"));
   }
 
-
-
 }

Modified: hadoop/avro/trunk/src/test/schemata/interop.js
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/test/schemata/interop.js?rev=777638&r1=777637&r2=777638&view=diff
==============================================================================
--- hadoop/avro/trunk/src/test/schemata/interop.js (original)
+++ hadoop/avro/trunk/src/test/schemata/interop.js Fri May 22 17:48:18 2009
@@ -17,6 +17,8 @@
        ["boolean", "double", {"type": "array", "items": "bytes"}]},
       {"name": "enumField", "type":
        {"type": "enum", "name": "Kind", "symbols": ["A","B","C"]}},
+      {"name": "fixedField", "type":
+       {"type": "fixed", "name": "MD5", "size": 16}},
       {"name": "recordField", "type":
        {"type": "record", "name": "Node",
         "fields": [

Modified: hadoop/avro/trunk/src/test/schemata/simple.js
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/test/schemata/simple.js?rev=777638&r1=777637&r2=777638&view=diff
==============================================================================
--- hadoop/avro/trunk/src/test/schemata/simple.js (original)
+++ hadoop/avro/trunk/src/test/schemata/simple.js Fri May 22 17:48:18 2009
@@ -4,10 +4,13 @@
  "types": [
      {"name": "Kind", "type": "enum", "symbols": ["FOO","BAR","BAZ"]},
 
+     {"name": "MD5", "type": "fixed", "size": 16},
+
      {"name": "TestRecord", "type": "record",
       "fields": [
           {"name": "name", "type": "string"},
-          {"name": "kind", "type": "Kind"}
+          {"name": "kind", "type": "Kind"},
+          {"name": "hash", "type": "MD5"}
       ]
      },