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/11/25 21:22:10 UTC
svn commit: r884249 - in /hadoop/avro/trunk: ./ src/java/org/apache/avro/
src/java/org/apache/avro/ipc/ src/java/org/apache/avro/reflect/
src/java/org/apache/avro/specific/ src/test/java/org/apache/avro/
Author: cutting
Date: Wed Nov 25 20:22:09 2009
New Revision: 884249
URL: http://svn.apache.org/viewvc?rev=884249&view=rev
Log:
AVRO-237. More improvements to Java reflection.
Modified:
hadoop/avro/trunk/CHANGES.txt
hadoop/avro/trunk/src/java/org/apache/avro/AvroRuntimeException.java
hadoop/avro/trunk/src/java/org/apache/avro/AvroTypeException.java
hadoop/avro/trunk/src/java/org/apache/avro/ipc/Responder.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/ReflectDatumWriter.java
hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificData.java
hadoop/avro/trunk/src/test/java/org/apache/avro/TestReflect.java
Modified: hadoop/avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/CHANGES.txt?rev=884249&r1=884248&r2=884249&view=diff
==============================================================================
--- hadoop/avro/trunk/CHANGES.txt (original)
+++ hadoop/avro/trunk/CHANGES.txt Wed Nov 25 20:22:09 2009
@@ -15,6 +15,11 @@
for string and array. Instead now Java strings and arrays or
Lists are used. (cutting)
+ AVRO-237. Reflect API now represents any Java Collection as an
+ Avro array. Also inherited fields are included in records, and
+ inherited methods in protocols. Finally, Java shorts are
+ supported as integers. (cutting)
+
NEW FEATURES
AVRO-151. Validating Avro schema parser for C (massie)
Modified: hadoop/avro/trunk/src/java/org/apache/avro/AvroRuntimeException.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/AvroRuntimeException.java?rev=884249&r1=884248&r2=884249&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/AvroRuntimeException.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/AvroRuntimeException.java Wed Nov 25 20:22:09 2009
@@ -22,5 +22,8 @@
public class AvroRuntimeException extends RuntimeException {
public AvroRuntimeException(Throwable cause) { super(cause); }
public AvroRuntimeException(String message) { super(message); }
+ public AvroRuntimeException(String message, Throwable cause) {
+ super(message, cause);
+ }
}
Modified: hadoop/avro/trunk/src/java/org/apache/avro/AvroTypeException.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/AvroTypeException.java?rev=884249&r1=884248&r2=884249&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/AvroTypeException.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/AvroTypeException.java Wed Nov 25 20:22:09 2009
@@ -22,5 +22,8 @@
/** Thrown when an illegal type is used. */
public class AvroTypeException extends AvroRuntimeException {
public AvroTypeException(String message) { super(message); }
+ public AvroTypeException(String message, Throwable cause) {
+ super(message, cause);
+ }
}
Modified: hadoop/avro/trunk/src/java/org/apache/avro/ipc/Responder.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/ipc/Responder.java?rev=884249&r1=884248&r2=884249&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/ipc/Responder.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/ipc/Responder.java Wed Nov 25 20:22:09 2009
@@ -111,7 +111,8 @@
// create response using local protocol specification
m = getLocal().getMessages().get(messageName);
if (m == null)
- throw new AvroRuntimeException("No such local message: "+messageName);
+ throw new AvroRuntimeException("No message named "+messageName
+ +" in "+getLocal());
Object response = null;
try {
response = respond(m, request);
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=884249&r1=884248&r2=884249&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 Wed Nov 25 20:22:09 2009
@@ -23,10 +23,12 @@
import java.lang.reflect.Type;
import java.lang.reflect.ParameterizedType;
import java.lang.reflect.GenericArrayType;
+import java.util.Collection;
import java.util.List;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.LinkedHashMap;
import java.util.Map;
@@ -46,13 +48,11 @@
/** Utilities to use existing Java classes and interfaces via reflection.
*
- * <p><b>Records</b> When creating a record schema, only fields of the direct
- * class, not it's super classes, are used. Fields are not permitted to be
- * null. {@link Class#getDeclaredFields() declared fields} (not inherited)
- * which are not static or transient are used.
+ * <p><b>Records</b>Fields are not permitted to be null. Fields which are not
+ * static or transient are used.
*
- * <p><b>Arrays</b>Both Java arrays and implementations of {@link List} are
- * mapped to Avro arrays.
+ * <p><b>Arrays</b>Both Java arrays and implementations of {@link Collection}
+ * are mapped to Avro arrays.
*
* <p><b>{@link String}</b> is mapped to Avro string.
* <p><b>byte[]</b> is mapped to Avro bytes.
@@ -92,7 +92,7 @@
@Override
protected boolean isArray(Object datum) {
- return datum instanceof List || datum.getClass().isArray();
+ return (datum instanceof Collection) || datum.getClass().isArray();
}
@Override
@@ -129,8 +129,8 @@
}
return true;
case ARRAY:
- if (datum instanceof List) { // list
- for (Object element : (List)datum)
+ if (datum instanceof Collection) { // collection
+ for (Object element : (Collection)datum)
if (!validate(schema.getElementType(), element))
return false;
return true;
@@ -148,28 +148,49 @@
}
}
- static Field getField(Class c, String name) {
- try {
- Field f = c.getDeclaredField(name);
- f.setAccessible(true);
- return f;
- } catch (NoSuchFieldException e) {
- throw new AvroRuntimeException(e);
+ private static final Map<Class,Map<String,Field>> FIELD_CACHE =
+ new ConcurrentHashMap<Class,Map<String,Field>>();
+
+ /** Return the named field of the provided class. Implementation caches
+ * values, since this is used at runtime to get and set fields. */
+ protected static Field getField(Class c, String name) {
+ Map<String,Field> fields = FIELD_CACHE.get(c);
+ if (fields == null) {
+ fields = new ConcurrentHashMap<String,Field>();
+ FIELD_CACHE.put(c, fields);
}
+ Field f = fields.get(name);
+ if (f == null) {
+ f = findField(c, name);
+ fields.put(name, f);
+ }
+ return f;
+ }
+
+ private static Field findField(Class c, String name) {
+ do {
+ try {
+ Field f = c.getDeclaredField(name);
+ f.setAccessible(true);
+ return f;
+ } catch (NoSuchFieldException e) {}
+ c = c.getSuperclass();
+ } while (c != null);
+ throw new AvroRuntimeException("No field named "+name+" in: "+c);
}
// Indicates the Java representation for an array schema. If an entry is
- // present, it contains the Java List class of this array. If no entry is
- // present, then a Java array should be used to implement this array.
- private static final Map<Schema,Class> LIST_CLASSES =
+ // present, it contains the Java Collection class of this array. If no entry
+ // is present, then a Java array should be used to implement this array.
+ private static final Map<Schema,Class> COLLECTION_CLASSES =
new WeakIdentityHashMap<Schema,Class>();
- private static synchronized void setListClass(Schema schema, Class c) {
- LIST_CLASSES.put(schema, c);
+ private static synchronized void setCollectionClass(Schema schema, Class c) {
+ COLLECTION_CLASSES.put(schema, c);
}
- /** Return the {@link List} subclass that implements this schema.*/
- public static synchronized Class getListClass(Schema schema) {
- return LIST_CLASSES.get(schema);
+ /** Return the {@link Collection} subclass that implements this schema.*/
+ public static synchronized Class getCollectionClass(Schema schema) {
+ return COLLECTION_CLASSES.get(schema);
}
private static final Class BYTES_CLASS = new byte[0].getClass();
@@ -178,9 +199,9 @@
public Class getClass(Schema schema) {
switch (schema.getType()) {
case ARRAY:
- Class listClass = getListClass(schema);
- if (listClass != null)
- return listClass;
+ Class collectionClass = getCollectionClass(schema);
+ if (collectionClass != null)
+ return collectionClass;
return java.lang.reflect.Array.newInstance(getClass(schema.getElementType()),0).getClass();
case STRING: return String.class;
case BYTES: return BYTES_CLASS;
@@ -201,20 +222,19 @@
ParameterizedType ptype = (ParameterizedType)type;
Class raw = (Class)ptype.getRawType();
Type[] params = ptype.getActualTypeArguments();
- for (int i = 0; i < params.length; i++)
- if (List.class.isAssignableFrom(raw)) { // List
- if (params.length != 1)
- throw new AvroTypeException("No array type specified.");
- Schema schema = Schema.createArray(createSchema(params[0], names));
- setListClass(schema, raw);
- return schema;
- } else if (Map.class.isAssignableFrom(raw)) { // Map
- Type key = params[0];
- Type value = params[1];
- if (!(key == String.class))
- throw new AvroTypeException("Map key class not String: "+key);
- return Schema.createMap(createSchema(value, names));
- }
+ if (Map.class.isAssignableFrom(raw)) { // Map
+ Type key = params[0];
+ Type value = params[1];
+ if (!(key == String.class))
+ throw new AvroTypeException("Map key class not String: "+key);
+ return Schema.createMap(createSchema(value, names));
+ } else if (Collection.class.isAssignableFrom(raw)) { // Collection
+ if (params.length != 1)
+ throw new AvroTypeException("No array type specified.");
+ Schema schema = Schema.createArray(createSchema(params[0], names));
+ setCollectionClass(schema, raw);
+ return schema;
+ }
} else if (type instanceof Class) { // Class
Class c = (Class)type;
if (c.isPrimitive() || Number.class.isAssignableFrom(c)
@@ -250,7 +270,7 @@
schema = Schema.createRecord(name, space,
Throwable.class.isAssignableFrom(c));
names.put(c.getName(), schema);
- for (Field field : c.getDeclaredFields())
+ for (Field field : getFields(c))
if ((field.getModifiers()&(Modifier.TRANSIENT|Modifier.STATIC))==0){
Schema fieldSchema = createFieldSchema(field, names);
fields.put(field.getName(), new Schema.Field(fieldSchema, null));
@@ -264,6 +284,23 @@
return super.createSchema(type, names);
}
+ // Return of this class and its superclasses to serialize.
+ // Not cached, since this is only used to create schemas, which are cached.
+ private Collection<Field> getFields(Class recordClass) {
+ Map<String,Field> fields = new LinkedHashMap<String,Field>();
+ Class c = recordClass;
+ do {
+ if (c.getPackage().getName().startsWith("java."))
+ break; // skip java built-in classes
+ for (Field field : c.getDeclaredFields())
+ if ((field.getModifiers() & (Modifier.TRANSIENT|Modifier.STATIC)) == 0)
+ if (fields.put(field.getName(), field) != null)
+ throw new AvroTypeException(c+" contains two fields named: "+field);
+ c = c.getSuperclass();
+ } while (c != null);
+ return fields.values();
+ }
+
/** Create a schema for a field. */
protected Schema createFieldSchema(Field field, Map<String, Schema> names) {
return createSchema(field.getGenericType(), names);
@@ -279,7 +316,7 @@
Protocol protocol =
new Protocol(iface.getSimpleName(), iface.getPackage().getName());
Map<String,Schema> names = new LinkedHashMap<String,Schema>();
- for (Method method : iface.getDeclaredMethods())
+ for (Method method : iface.getMethods())
if ((method.getModifiers() & Modifier.STATIC) == 0)
protocol.getMessages().put(method.getName(),
getMessage(method, protocol, names));
@@ -302,7 +339,7 @@
String[] paramNames = paranamer.lookupParameterNames(method);
Type[] paramTypes = method.getGenericParameterTypes();
for (int i = 0; i < paramTypes.length; i++) {
- Schema paramSchema = createSchema(paramTypes[i], names);
+ Schema paramSchema = getSchema(paramTypes[i], names);
String paramName = paramNames.length == paramTypes.length
? paramNames[i]
: paramSchema.getName()+i;
@@ -310,18 +347,27 @@
}
Schema request = Schema.createRecord(fields);
- Schema response = createSchema(method.getGenericReturnType(), names);
+ Schema response = getSchema(method.getGenericReturnType(), names);
List<Schema> errs = new ArrayList<Schema>();
errs.add(Protocol.SYSTEM_ERROR); // every method can throw
for (Type err : method.getGenericExceptionTypes())
if (err != AvroRemoteException.class)
- errs.add(createSchema(err, names));
+ errs.add(getSchema(err, names));
Schema errors = Schema.createUnion(errs);
return protocol.createMessage(method.getName(), request, response, errors);
}
+ private Schema getSchema(Type type, Map<String,Schema> names) {
+ try {
+ return createSchema(type, names);
+ } catch (AvroTypeException e) { // friendly exception
+ throw new AvroTypeException("Error getting schema for "+type+": "
+ +e.getMessage(), e);
+ }
+ }
+
@Override
public int compare(Object o1, Object o2, Schema s) {
throw new UnsupportedOperationException();
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=884249&r1=884248&r2=884249&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 Wed Nov 25 20:22:09 2009
@@ -18,9 +18,10 @@
package org.apache.avro.reflect;
import java.io.IOException;
-import java.util.List;
+import java.util.Collection;
import java.util.ArrayList;
import java.lang.reflect.Array;
+import java.lang.reflect.Field;
import java.nio.ByteBuffer;
import org.apache.avro.AvroRuntimeException;
@@ -46,7 +47,10 @@
@Override
protected void addField(Object record, String name, int position, Object o) {
try {
- ReflectData.getField(record.getClass(), name).set(record, o);
+ Field field = ReflectData.getField(record.getClass(), name);
+ if (field.getType() == Short.TYPE)
+ o = ((Integer)o).shortValue(); // downgrade int to short
+ field.set(record, o);
} catch (IllegalAccessException e) {
throw new AvroRuntimeException(e);
}
@@ -69,15 +73,15 @@
@Override
@SuppressWarnings(value="unchecked")
protected Object newArray(Object old, int size, Schema schema) {
- Class listClass = ReflectData.get().getListClass(schema);
- if (listClass != null) {
- if (old instanceof List) {
- ((List)old).clear();
+ Class collectionClass = ReflectData.get().getCollectionClass(schema);
+ if (collectionClass != null) {
+ if (old instanceof Collection) {
+ ((Collection)old).clear();
return old;
}
- if (listClass.isAssignableFrom(ArrayList.class))
+ if (collectionClass.isAssignableFrom(ArrayList.class))
return new ArrayList();
- return newInstance(listClass);
+ return newInstance(collectionClass);
}
Class elementClass = ReflectData.get().getClass(schema.getElementType());
return Array.newInstance(elementClass, size);
@@ -91,8 +95,8 @@
@Override
@SuppressWarnings(value="unchecked")
protected void addToArray(Object array, long pos, Object e) {
- if (array instanceof List) {
- ((List)array).add(e);
+ if (array instanceof Collection) {
+ ((Collection)array).add(e);
} else {
Array.set(array, (int)pos, e);
}
Modified: hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectDatumWriter.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectDatumWriter.java?rev=884249&r1=884248&r2=884249&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectDatumWriter.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectDatumWriter.java Wed Nov 25 20:22:09 2009
@@ -20,7 +20,7 @@
import java.lang.reflect.Array;
import java.io.IOException;
import java.util.Iterator;
-import java.util.List;
+import java.util.Collection;
import org.apache.avro.AvroRuntimeException;
import org.apache.avro.Schema;
@@ -60,7 +60,10 @@
@Override
protected Object getField(Object record, String name, int position) {
try {
- return ReflectData.getField(record.getClass(), name).get(record);
+ Object value = ReflectData.getField(record.getClass(), name).get(record);
+ if (value instanceof Short)
+ return ((Short)value).intValue(); // upgrade short to int
+ return value;
} catch (IllegalAccessException e) {
throw new AvroRuntimeException(e);
}
@@ -69,8 +72,8 @@
@Override
@SuppressWarnings("unchecked")
protected long getArraySize(Object array) {
- if (array instanceof List)
- return ((List)array).size();
+ if (array instanceof Collection)
+ return ((Collection)array).size();
return Array.getLength(array);
}
@@ -78,8 +81,8 @@
@Override
@SuppressWarnings("unchecked")
protected Iterator<Object> getArrayElements(final Object array) {
- if (array instanceof List)
- return ((List<Object>)array).iterator();
+ if (array instanceof Collection)
+ return ((Collection<Object>)array).iterator();
return new Iterator<Object>() {
private int i = 0;
private final int length = Array.getLength(array);
Modified: hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificData.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificData.java?rev=884249&r1=884248&r2=884249&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificData.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificData.java Wed Nov 25 20:22:09 2009
@@ -145,7 +145,8 @@
return Schema.create(Type.STRING);
else if (type == ByteBuffer.class)
return Schema.create(Type.BYTES);
- else if ((type == Integer.class) || (type == Integer.TYPE))
+ else if ((type == Integer.class) || (type == Integer.TYPE)
+ || (type == Short.TYPE))
return Schema.create(Type.INT);
else if ((type == Long.class) || (type == Long.TYPE))
return Schema.create(Type.LONG);
Modified: hadoop/avro/trunk/src/test/java/org/apache/avro/TestReflect.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/test/java/org/apache/avro/TestReflect.java?rev=884249&r1=884248&r2=884249&view=diff
==============================================================================
--- hadoop/avro/trunk/src/test/java/org/apache/avro/TestReflect.java (original)
+++ hadoop/avro/trunk/src/test/java/org/apache/avro/TestReflect.java Wed Nov 25 20:22:09 2009
@@ -23,6 +23,7 @@
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.lang.reflect.Type;
+import java.util.Collection;
import java.util.Map;
import java.util.List;
import java.util.HashMap;
@@ -117,21 +118,21 @@
public static class R2 {
private String[] arrayField;
- private List<String> listField;
+ private Collection<String> collectionField;
public boolean equals(Object o) {
if (!(o instanceof R2)) return false;
R2 that = (R2)o;
return Arrays.equals(this.arrayField, that.arrayField)
- && listField.equals(that.listField);
+ && collectionField.equals(that.collectionField);
}
}
@Test public void testR2() throws Exception {
R2 r2 = new R2();
r2.arrayField = new String[] {"foo"};
- r2.listField = new ArrayList<String>();
- r2.listField.add("foo");
+ r2.collectionField = new ArrayList<String>();
+ r2.collectionField.add("foo");
checkReadWrite(r2);
}
@@ -151,6 +152,23 @@
checkReadWrite(r3);
}
+ public static class R4 {
+ public short value;
+
+ public boolean equals(Object o) {
+ if (!(o instanceof R4)) return false;
+ return this.value == ((R4)o).value;
+ }
+ }
+
+ public static class R5 extends R4 {}
+
+ @Test public void testR5() throws Exception {
+ R5 r5 = new R5();
+ r5.value = 1;
+ checkReadWrite(r5);
+ }
+
void checkReadWrite(Object object) throws Exception {
Schema s = ReflectData.get().getSchema(object.getClass());
ReflectDatumWriter writer = new ReflectDatumWriter(s);