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> 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"}
]
},