You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@avro.apache.org by th...@apache.org on 2010/02/09 04:34:44 UTC
svn commit: r907913 - in /hadoop/avro/trunk: ./
lang/java/src/java/org/apache/avro/
lang/java/src/java/org/apache/avro/genavro/
lang/java/src/java/org/apache/avro/generic/
lang/java/src/java/org/apache/avro/io/
lang/java/src/java/org/apache/avro/io/par...
Author: thiru
Date: Tue Feb 9 03:34:43 2010
New Revision: 907913
URL: http://svn.apache.org/viewvc?rev=907913&view=rev
Log:
AVRO-261. Allow Schemas to be immutable
Modified:
hadoop/avro/trunk/CHANGES.txt
hadoop/avro/trunk/lang/java/src/java/org/apache/avro/Protocol.java
hadoop/avro/trunk/lang/java/src/java/org/apache/avro/Schema.java
hadoop/avro/trunk/lang/java/src/java/org/apache/avro/genavro/genavro.jj
hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericData.java
hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericDatumReader.java
hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericDatumWriter.java
hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/BinaryData.java
hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/JsonGrammarGenerator.java
hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/ResolvingGrammarGenerator.java
hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/ValidatingGrammarGenerator.java
hadoop/avro/trunk/lang/java/src/java/org/apache/avro/reflect/ReflectData.java
hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificCompiler.java
hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificRequestor.java
hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificResponder.java
hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/RandomData.java
hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestProtocolGeneric.java
hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestReflect.java
hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestSchema.java
hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/generic/TestGenericData.java
Modified: hadoop/avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/CHANGES.txt?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/CHANGES.txt (original)
+++ hadoop/avro/trunk/CHANGES.txt Tue Feb 9 03:34:43 2010
@@ -318,6 +318,8 @@
AVRO-420. Add namespace support to C implementation (massie)
+ AVRO-261. Allow Schemas to be immutable (thiru)
+
OPTIMIZATIONS
AVRO-172. More efficient schema processing (massie)
Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/Protocol.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/Protocol.java?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/Protocol.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/Protocol.java Tue Feb 9 03:34:43 2010
@@ -339,7 +339,7 @@
JsonNode requestNode = json.get("request");
if (requestNode == null || !requestNode.isArray())
throw new SchemaParseException("No request specified: "+json);
- LinkedHashMap<String,Field> fields = new LinkedHashMap<String,Field>();
+ List<Field> fields = new ArrayList<Field>();
for (JsonNode field : requestNode) {
JsonNode fieldNameNode = field.get("name");
if (fieldNameNode == null)
@@ -348,8 +348,7 @@
if (fieldTypeNode == null)
throw new SchemaParseException("No param type: "+field);
String name = fieldNameNode.getTextValue();
- fields.put(name,
- new Field(name, Schema.parse(fieldTypeNode,types),
+ fields.add(new Field(name, Schema.parse(fieldTypeNode,types),
null /* message fields don't have docs */,
field.get("default")));
}
Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/Schema.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/Schema.java?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/Schema.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/Schema.java Tue Feb 9 03:34:43 2010
@@ -22,6 +22,7 @@
import java.io.InputStream;
import java.io.StringReader;
import java.io.StringWriter;
+import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.ArrayList;
@@ -58,6 +59,19 @@
* <li>A <i>boolean</i>; or
* <li><i>null</i>.
* </ul>
+ *
+ * A schema can be constructed using one of its static <tt>createXXX</tt>
+ * methods. The schema objects are <i>logically</i> immutable.
+ * There are only two mutating methods - {@link #setFields(List)} and
+ * {@link #addProp(String, String)}. The following restrictions apply on these
+ * two methods.
+ * <ul>
+ * <li> {@link #setFields(List)}, can be called at most once. This method exists
+ * in order to enable clients to build recursive schemas.
+ * <li> {@link #addProp(String, String)} can be called with property names
+ * that are not present already. It is not possible to change or delete an
+ * existing property.
+ * </ul>
*/
public abstract class Schema {
static final JsonFactory FACTORY = new JsonFactory();
@@ -103,23 +117,45 @@
"size", "symbols", "values", "type");
}
- /** Return the value of the named property in this schema. */
+ /**
+ * Returns the value of the named property in this schema.
+ * Returns <tt>null</tt> if there is no property with that name.
+ */
public synchronized String getProp(String name) {
return props.get(name);
}
- /** Set the value of the named property in this schema. */
- public synchronized void setProp(String name, String value) {
- if (RESERVED_PROPS.contains(name))
- throw new AvroRuntimeException("Can't set a reserved property: "+name);
- if (value == null)
- props.remove(name);
- else
+ /**
+ * Adds a property with the given name <tt>name</tt> and
+ * value <tt>value</tt>. Neither <tt>name</tt> nor <tt>value</tt> can be
+ * <tt>null</tt>. It is illegal to add a property if another with
+ * the same name but different value already exists in this schema.
+ *
+ * @param name The name of the property to add
+ * @param value The value for the property to add
+ */
+ public synchronized void addProp(String name, String value) {
+ if (RESERVED_PROPS.contains(name)) {
+ throw new AvroRuntimeException("Can't set a reserved property: " + name);
+ }
+
+ if (value == null) {
+ throw new AvroRuntimeException(
+ "Can't set a null value for property: " + name);
+ }
+
+ String v = props.get(name);
+ if (v != null) {
+ if (! v.equals(value)) {
+ throw new AvroRuntimeException("Can't overwrite property: " + name);
+ }
+ } else {
props.put(name, value);
+ }
}
/** Create an anonymous record schema. */
- public static Schema createRecord(LinkedHashMap<String,Field> fields) {
+ public static Schema createRecord(List<Field> fields) {
Schema result = createRecord(null, null, null, false);
result.setFields(fields);
return result;
@@ -134,7 +170,8 @@
/** Create an enum schema. */
public static Schema createEnum(String name, String doc, String namespace,
List<String> values) {
- return new EnumSchema(new Name(name, namespace), doc, values);
+ return new EnumSchema(new Name(name, namespace), doc,
+ new LockableArrayList<String>(values));
}
/** Create an array schema. */
@@ -149,7 +186,7 @@
/** Create a union schema. */
public static Schema createUnion(List<Schema> types) {
- return new UnionSchema(types);
+ return new UnionSchema(new LockableArrayList<Schema>(types));
}
/** Create a union schema. */
@@ -161,18 +198,28 @@
/** Return the type of this schema. */
public Type getType() { return type; }
- /** If this is a record, returns its fields. */
- public Map<String, Field> getFields() {
+ /**
+ * If this is a record, returns the Field with the
+ * given name <tt>fieldName</tt>. If there is no field by that name, a
+ * <tt>null</tt> is returned.
+ */
+ public Field getField(String fieldname) {
throw new AvroRuntimeException("Not a record: "+this);
}
- /** If this is a record, enumerate its field names and their schemas. */
- public Iterable<Map.Entry<String,Schema>> getFieldSchemas() {
+ /**
+ * If this is a record, returns the fields in it. The returned
+ * list is in the order of their positions.
+ */
+ public List<Field> getFields() {
throw new AvroRuntimeException("Not a record: "+this);
}
-
- /** If this is a record, set its fields. */
- public void setFields(LinkedHashMap<String,Field> fields) {
+
+ /**
+ * If this is a record, set its fields. The fields can be set
+ * only once in a schema.
+ */
+ public void setFields(List<Field> fields) {
throw new AvroRuntimeException("Not a record: "+this);
}
@@ -436,32 +483,41 @@
@SuppressWarnings(value="unchecked")
private static class RecordSchema extends NamedSchema {
- private Map<String,Field> fields;
- private Iterable<Map.Entry<String,Schema>> fieldSchemas;
+ private List<Field> fields;
+ private Map<String, Field> fieldMap;
private final boolean isError;
public RecordSchema(Name name, String doc, boolean isError) {
super(Type.RECORD, name, doc);
this.isError = isError;
}
public boolean isError() { return isError; }
- public Map<String, Field> getFields() { return fields; }
- public Iterable<Map.Entry<String, Schema>> getFieldSchemas() {
- return fieldSchemas;
+
+ @Override
+ public Field getField(String fieldname) {
+ return fieldMap.get(fieldname);
+ }
+
+ @Override
+ public List<Field> getFields() {
+ return fields;
}
- public void setFields(LinkedHashMap<String,Field> fields) {
- if (this.fields != null)
+
+ @Override
+ public void setFields(List<Field> fields) {
+ if (this.fields != null) {
throw new AvroRuntimeException("Fields are already set");
+ }
int i = 0;
- LinkedHashMap<String,Schema> schemas = new LinkedHashMap<String,Schema>();
- for (Map.Entry<String, Field> pair : fields.entrySet()) {
- Field f = pair.getValue();
+ fieldMap = new HashMap<String, Field>();
+ LockableArrayList ff = new LockableArrayList();
+ for (Field f : fields) {
if (f.position != -1)
- throw new AvroRuntimeException("Field already used: "+f);
+ throw new AvroRuntimeException("Field already used: " + f);
f.position = i++;
- schemas.put(pair.getKey(), f.schema());
+ fieldMap.put(f.name(), f);
+ ff.add(f);
}
- this.fields = fields;
- this.fieldSchemas = schemas.entrySet();
+ this.fields = ff.lock();
}
public boolean equals(Object o) {
if (o == this) return true;
@@ -502,17 +558,17 @@
void fieldsToJson(Names names, JsonGenerator gen) throws IOException {
gen.writeStartArray();
- for (Map.Entry<String, Field> entry : fields.entrySet()) {
+ for (Field f : fields) {
gen.writeStartObject();
- gen.writeStringField("name", entry.getKey());
+ gen.writeStringField("name", f.name());
gen.writeFieldName("type");
- entry.getValue().schema().toJson(names, gen);
- if (entry.getValue().defaultValue() != null) {
+ f.schema().toJson(names, gen);
+ if (f.defaultValue() != null) {
gen.writeFieldName("default");
- gen.writeTree(entry.getValue().defaultValue());
+ gen.writeTree(f.defaultValue());
}
- if (entry.getValue().order() != Field.Order.ASCENDING)
- gen.writeStringField("order", entry.getValue().order().name);
+ if (f.order() != Field.Order.ASCENDING)
+ gen.writeStringField("order", f.order().name);
gen.writeEndObject();
}
gen.writeEndArray();
@@ -522,9 +578,10 @@
private static class EnumSchema extends NamedSchema {
private final List<String> symbols;
private final Map<String,Integer> ordinals;
- public EnumSchema(Name name, String doc, List<String> symbols) {
+ public EnumSchema(Name name, String doc,
+ LockableArrayList<String> symbols) {
super(Type.ENUM, name, doc);
- this.symbols = symbols;
+ this.symbols = symbols.lock();
this.ordinals = new HashMap<String,Integer>();
int i = 0;
for (String symbol : symbols)
@@ -612,9 +669,9 @@
private static class UnionSchema extends Schema {
private final List<Schema> types;
- public UnionSchema(List<Schema> types) {
+ public UnionSchema(LockableArrayList<Schema> types) {
super(Type.UNION);
- this.types = types;
+ this.types = types.lock();
int seen = 0;
Set<String> seenNames = new HashSet<String>();
for (Schema type : types) { // check legality of union
@@ -652,9 +709,12 @@
public int hashCode() {
return getType().hashCode() + types.hashCode() + props.hashCode();
}
- public void setProp(String name, String value) {
+
+ @Override
+ public void addProp(String name, String value) {
throw new AvroRuntimeException("Can't set properties on a union: "+this);
}
+
void toJson(Names names, JsonGenerator gen) throws IOException {
gen.writeStartArray();
for (Schema type : types)
@@ -837,7 +897,7 @@
if (PRIMITIVES.containsKey(type)) { // primitive
result = create(PRIMITIVES.get(type));
} else if (type.equals("record") || type.equals("error")) { // record
- LinkedHashMap<String,Field> fields = new LinkedHashMap<String,Field>();
+ List<Field> fields = new ArrayList<Field>();
result = new RecordSchema(name, doc, type.equals("error"));
if (name != null) names.add(result);
JsonNode fieldsNode = schema.get("fields");
@@ -854,7 +914,7 @@
JsonNode orderNode = field.get("order");
if (orderNode != null)
order = Field.Order.valueOf(orderNode.getTextValue().toUpperCase());
- fields.put(fieldName, new Field(fieldName, fieldSchema,
+ fields.add(new Field(fieldName, fieldSchema,
fieldDoc, field.get("default"), order));
}
result.setFields(fields);
@@ -862,7 +922,7 @@
JsonNode symbolsNode = schema.get("symbols");
if (symbolsNode == null || !symbolsNode.isArray())
throw new SchemaParseException("Enum has no symbols: "+schema);
- List<String> symbols = new ArrayList<String>();
+ LockableArrayList<String> symbols = new LockableArrayList<String>();
for (JsonNode n : symbolsNode)
symbols.add(n.getTextValue());
result = new EnumSchema(name, doc, symbols);
@@ -888,14 +948,16 @@
Iterator<String> i = schema.getFieldNames();
while (i.hasNext()) { // add properties
String prop = i.next();
- if (!RESERVED_PROPS.contains(prop)) // ignore reserved
- result.setProp(prop, schema.get(prop).getTextValue());
+ String value = schema.get(prop).getTextValue();
+ if (!RESERVED_PROPS.contains(prop) && value != null) // ignore reserved
+ result.addProp(prop, value);
}
if (savedSpace != null)
names.space(savedSpace); // restore space
return result;
} else if (schema.isArray()) { // union
- List<Schema> types = new ArrayList<Schema>(schema.size());
+ LockableArrayList<Schema> types =
+ new LockableArrayList<Schema>(schema.size());
for (JsonNode typeNode : schema)
types.add(parse(typeNode, names));
return new UnionSchema(types);
@@ -937,5 +999,89 @@
}
}
+ /**
+ * No change is permitted on LockableArrayList once lock() has been
+ * called on it.
+ * @param <E>
+ */
+
+ /*
+ * This class keeps a boolean variable <tt>locked</tt> which is set
+ * to <tt>true</tt> in the lock() method. It's legal to call
+ * lock() any number of times. Any lock() other than the first one
+ * is a no-op.
+ *
+ * This class throws <tt>IllegalStateException</tt> if a mutating
+ * operation is performed after being locked. Since modifications through
+ * iterator also use the list's mutating operations, this effectively
+ * blocks all modifications.
+ */
+ static class LockableArrayList<E> extends ArrayList<E> {
+ private static final long serialVersionUID = 1L;
+ private boolean locked = false;
+
+ public LockableArrayList() {
+ }
+
+ public LockableArrayList(int size) {
+ super(size);
+ }
+
+ public LockableArrayList(List<E> types) {
+ super(types);
+ }
+
+ public List<E> lock() {
+ locked = true;
+ return this;
+ }
+ private void ensureUnlocked() {
+ if (locked) {
+ throw new IllegalStateException();
+ }
+ }
+
+ public boolean add(E e) {
+ ensureUnlocked();
+ return super.add(e);
+ }
+
+ public boolean remove(Object o) {
+ ensureUnlocked();
+ return super.remove(o);
+ }
+
+ public E remove(int index) {
+ ensureUnlocked();
+ return super.remove(index);
+ }
+
+ public boolean addAll(Collection<? extends E> c) {
+ ensureUnlocked();
+ return super.addAll(c);
+ }
+
+ public boolean addAll(int index, Collection<? extends E> c) {
+ ensureUnlocked();
+ return super.addAll(index, c);
+ }
+
+ public boolean removeAll(Collection<?> c) {
+ ensureUnlocked();
+ return super.removeAll(c);
+ }
+
+ public boolean retainAll(Collection<?> c) {
+ ensureUnlocked();
+ return super.retainAll(c);
+ }
+
+ public void clear() {
+ ensureUnlocked();
+ super.clear();
+ }
+
+ }
+
}
Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/genavro/genavro.jj
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/genavro/genavro.jj?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/genavro/genavro.jj (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/genavro/genavro.jj Tue Feb 9 03:34:43 2010
@@ -1074,7 +1074,7 @@
Schema RecordDeclaration():
{
String name;
- LinkedHashMap<String, Field> fields = new LinkedHashMap<String, Field>();
+ List<Field> fields = new ArrayList<Field>();
boolean isError;
}
{
@@ -1115,7 +1115,7 @@
}
-void FieldDeclaration(LinkedHashMap<String, Field> fields):
+void FieldDeclaration(List<Field> fields):
{
Schema type;
Map<String, String> props = new HashMap<String, String>();
@@ -1130,19 +1130,19 @@
";"
{
for (Map.Entry<String, String> propEntry : props.entrySet()) {
- type.setProp(propEntry.getKey(), propEntry.getValue());
+ type.addProp(propEntry.getKey(), propEntry.getValue());
}
}
}
-void VariableDeclarator(Schema type, LinkedHashMap<String, Field> fields):
+void VariableDeclarator(Schema type, List<Field> fields):
{
String name;
}
{
name = Identifier()
{
- fields.put(name, new Field(name, type, null, null));
+ fields.add(new Field(name, type, null, null));
}
}
@@ -1178,7 +1178,7 @@
Schema FormalParameters():
{
- LinkedHashMap<String, Field> fields = new LinkedHashMap<String, Field>();
+ List<Field> fields = new ArrayList<Field>();
}
{
(
@@ -1189,7 +1189,7 @@
}
}
-void FormalParameter(LinkedHashMap<String, Field> fields):
+void FormalParameter(List<Field> fields):
{
Schema type;
}
Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericData.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericData.java?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericData.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericData.java Tue Feb 9 03:34:43 2010
@@ -52,11 +52,11 @@
}
@Override public Schema getSchema() { return schema; }
@Override public void put(String key, Object value) {
- values[schema.getFields().get(key).pos()] = value;
+ values[schema.getField(key).pos()] = value;
}
@Override public void put(int i, Object v) { values[i] = v; }
@Override public Object get(String key) {
- Field field = schema.getFields().get(key);
+ Field field = schema.getField(key);
if (field == null) return null;
return values[field.pos()];
}
@@ -179,8 +179,7 @@
case RECORD:
if (!(datum instanceof IndexedRecord)) return false;
IndexedRecord fields = (IndexedRecord)datum;
- for (Map.Entry<String, Field> entry : schema.getFields().entrySet()) {
- Field f = entry.getValue();
+ for (Field f : schema.getFields()) {
if (!validate(f.schema(), fields.get(f.pos())))
return false;
}
@@ -233,11 +232,10 @@
buffer.append("{");
int count = 0;
IndexedRecord record = (IndexedRecord)datum;
- for (Map.Entry<String,Field> e :
- record.getSchema().getFields().entrySet()) {
- toString(e.getKey(), buffer);
+ for (Field f : record.getSchema().getFields()) {
+ toString(f.name(), buffer);
buffer.append(": ");
- toString(record.get(e.getValue().pos()), buffer);
+ toString(record.get(f.pos()), buffer);
if (++count < record.getSchema().getFields().size())
buffer.append(", ");
}
@@ -416,8 +414,7 @@
switch (s.getType()) {
case RECORD:
IndexedRecord r = (IndexedRecord)o;
- for (Map.Entry<String, Field> e : s.getFields().entrySet()) {
- Field f = e.getValue();
+ for (Field f : s.getFields()) {
if (f.order() == Field.Order.IGNORE)
continue;
hashCode = hashCodeAdd(hashCode, r.get(f.pos()), f.schema());
@@ -454,8 +451,7 @@
case RECORD:
IndexedRecord r1 = (IndexedRecord)o1;
IndexedRecord r2 = (IndexedRecord)o2;
- for (Map.Entry<String, Field> e : s.getFields().entrySet()) {
- Field f = e.getValue();
+ for (Field f : s.getFields()) {
if (f.order() == Field.Order.IGNORE)
continue; // ignore this field
int pos = f.pos();
Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericDatumReader.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericDatumReader.java?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericDatumReader.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericDatumReader.java Tue Feb 9 03:34:43 2010
@@ -306,8 +306,8 @@
public static void skip(Schema schema, Decoder in) throws IOException {
switch (schema.getType()) {
case RECORD:
- for (Map.Entry<String, Schema> entry : schema.getFieldSchemas())
- skip(entry.getValue(), in);
+ for (Field field : schema.getFields())
+ skip(field.schema(), in);
break;
case ENUM:
in.readInt();
Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericDatumWriter.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericDatumWriter.java?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericDatumWriter.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/generic/GenericDatumWriter.java Tue Feb 9 03:34:43 2010
@@ -85,9 +85,8 @@
* representations.*/
protected void writeRecord(Schema schema, Object datum, Encoder out)
throws IOException {
- for (Entry<String, Field> entry : schema.getFields().entrySet()) {
- Field field = entry.getValue();
- write(field.schema(), getField(datum, entry.getKey(), field.pos()), out);
+ for (Field field : schema.getFields()) {
+ write(field.schema(), getField(datum, field.name(), field.pos()), out);
}
}
Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/BinaryData.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/BinaryData.java?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/BinaryData.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/BinaryData.java Tue Feb 9 03:34:43 2010
@@ -17,7 +17,6 @@
*/
package org.apache.avro.io;
-import java.util.Map;
import java.io.ByteArrayInputStream;
import java.io.IOException;
@@ -81,8 +80,7 @@
Decoder d1 = d.d1; Decoder d2 = d.d2;
switch (schema.getType()) {
case RECORD: {
- for (Map.Entry<String, Field> entry : schema.getFields().entrySet()) {
- Field field = entry.getValue();
+ for (Field field : schema.getFields()) {
if (field.order() == Field.Order.IGNORE) {
GenericDatumReader.skip(field.schema(), d1);
GenericDatumReader.skip(field.schema(), d2);
Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/JsonGrammarGenerator.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/JsonGrammarGenerator.java?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/JsonGrammarGenerator.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/JsonGrammarGenerator.java Tue Feb 9 03:34:43 2010
@@ -83,9 +83,9 @@
int i = production.length;
int n = 0;
production[--i] = Symbol.RECORD_START;
- for (Map.Entry<String, Field> f : sc.getFields().entrySet()) {
- production[--i] = new Symbol.FieldAdjustAction(n, f.getKey());
- production[--i] = generate(f.getValue().schema(), seen);
+ for (Field f : sc.getFields()) {
+ production[--i] = new Symbol.FieldAdjustAction(n, f.name());
+ production[--i] = generate(f.schema(), seen);
n++;
}
production[--i] = Symbol.RECORD_END;
Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/ResolvingGrammarGenerator.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/ResolvingGrammarGenerator.java?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/ResolvingGrammarGenerator.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/ResolvingGrammarGenerator.java Tue Feb 9 03:34:43 2010
@@ -196,8 +196,8 @@
LitS wsc = new LitS2(writer, reader);
Symbol result = seen.get(wsc);
if (result == null) {
- Map<String, Field> wfields = writer.getFields();
- Map<String, Field> rfields = reader.getFields();
+ List<Field> wfields = writer.getFields();
+ List<Field> rfields = reader.getFields();
// First, compute reordering of reader fields, plus
// number elements in the result's production
@@ -205,17 +205,16 @@
int ridx = 0;
int count = 1 + wfields.size();
- for (String fn : wfields.keySet()) {
- Field rdrField = rfields.get(fn);
+ for (Field f : wfields) {
+ Field rdrField = reader.getField(f.name());
if (rdrField != null) {
reordered[ridx++] = rdrField;
}
}
- for (Map.Entry<String, Field> rfe : rfields.entrySet()) {
- String fname = rfe.getKey();
- if (wfields.get(fname) == null) {
- Field rf = rfe.getValue();
+ for (Field rf : rfields) {
+ String fname = rf.name();
+ if (writer.getField(fname) == null) {
if (rf.defaultValue() == null) {
result = Symbol.error("Found " + writer + ", expecting " + reader);
seen.put(wsc, result);
@@ -245,25 +244,24 @@
*/
// Handle all the writer's fields
- for (Map.Entry<String, Field> wfe : wfields.entrySet()) {
- String fname = wfe.getKey();
- Field rf = rfields.get(fname);
+ for (Field wf : wfields) {
+ String fname = wf.name();
+ Field rf = reader.getField(fname);
if (rf == null) {
production[--count] =
- new Symbol.SkipAction(super.generate(wfe.getValue().schema(),
+ new Symbol.SkipAction(super.generate(wf.schema(),
seen));
} else {
production[--count] =
- generate(wfe.getValue().schema(), rf.schema(), seen);
+ generate(wf.schema(), rf.schema(), seen);
}
}
// Add default values for fields missing from Writer
- for (Map.Entry<String, Field> rfe : rfields.entrySet()) {
- String fname = rfe.getKey();
- Field wf = wfields.get(fname);
+ for (Field rf : rfields) {
+ String fname = rf.name();
+ Field wf = writer.getField(fname);
if (wf == null) {
- Field rf = rfe.getValue();
byte[] bb = getBinary(rf.schema(), rf.defaultValue());
production[--count] = new Symbol.DefaultStartAction(bb);
production[--count] = generate(rf.schema(), rf.schema(), seen);
@@ -302,9 +300,8 @@
throws IOException {
switch (s.getType()) {
case RECORD:
- for (Map.Entry<String, Field> entry : s.getFields().entrySet()) {
- String name = entry.getKey();
- Field f = entry.getValue();
+ for (Field f : s.getFields()) {
+ String name = f.name();
JsonNode v = n.get(name);
if (v == null) {
v = f.defaultValue();
Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/ValidatingGrammarGenerator.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/ValidatingGrammarGenerator.java?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/ValidatingGrammarGenerator.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/ValidatingGrammarGenerator.java Tue Feb 9 03:34:43 2010
@@ -91,7 +91,7 @@
seen.put(wsc, rresult);
int i = production.length;
- for (Field f : sc.getFields().values()) {
+ for (Field f : sc.getFields()) {
production[--i] = generate(f.schema(), seen);
}
}
Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/reflect/ReflectData.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/reflect/ReflectData.java?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/reflect/ReflectData.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/reflect/ReflectData.java Tue Feb 9 03:34:43 2010
@@ -108,10 +108,10 @@
case RECORD:
if (datum == null) return false;
Class c = datum.getClass();
- for (Map.Entry<String, Schema> entry : schema.getFieldSchemas()) {
+ for (Schema.Field f : schema.getFields()) {
try {
- if (!validate(entry.getValue(),
- getField(c, entry.getKey()).get(datum)))
+ if (!validate(f.schema(),
+ getField(c, f.name()).get(datum)))
return false;
} catch (IllegalAccessException e) {
throw new AvroRuntimeException(e);
@@ -226,12 +226,12 @@
if (params.length != 1)
throw new AvroTypeException("No array type specified.");
Schema schema = Schema.createArray(createSchema(params[0], names));
- schema.setProp(CLASS_PROP, raw.getName());
+ schema.addProp(CLASS_PROP, raw.getName());
return schema;
}
} else if ((type == Short.class) || (type == Short.TYPE)) {
Schema result = Schema.create(Schema.Type.INT);
- result.setProp(CLASS_PROP, Short.class.getName());
+ result.addProp(CLASS_PROP, Short.class.getName());
return result;
} else if (type instanceof Class) { // Class
Class<?> c = (Class<?>)type;
@@ -260,7 +260,7 @@
return getAnnotatedUnion(union, names);
} else if (c.isAnnotationPresent(Stringable.class)){ // Stringable
Schema result = Schema.create(Schema.Type.STRING);
- result.setProp(CLASS_PROP, c.getName());
+ result.addProp(CLASS_PROP, c.getName());
return result;
} else if (c.isEnum()) { // Enum
List<String> symbols = new ArrayList<String>();
@@ -272,15 +272,14 @@
int size = c.getAnnotation(FixedSize.class).value();
schema = Schema.createFixed(name, null /* doc */, space, size);
} else { // record
- LinkedHashMap<String,Schema.Field> fields =
- new LinkedHashMap<String,Schema.Field>();
+ List<Schema.Field> fields = new ArrayList<Schema.Field>();
schema = Schema.createRecord(name, null /* doc */, space,
Throwable.class.isAssignableFrom(c));
names.put(c.getName(), schema);
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(field.getName(),
+ fields.add(new Schema.Field(field.getName(),
fieldSchema, null /* doc */, null));
}
schema.setFields(fields);
@@ -300,7 +299,7 @@
Class<?> c = (Class<?>)element;
Union union = c.getAnnotation(Union.class);
if (union != null) // element is annotated union
- schema.setProp(ELEMENT_PROP, c.getName());
+ schema.addProp(ELEMENT_PROP, c.getName());
}
// construct a schema from a union annotation
@@ -361,8 +360,7 @@
private Message getMessage(Method method, Protocol protocol,
Map<String,Schema> names) {
- LinkedHashMap<String,Schema.Field> fields =
- new LinkedHashMap<String,Schema.Field>();
+ List<Schema.Field> fields = new ArrayList<Schema.Field>();
String[] paramNames = paranamer.lookupParameterNames(method);
Type[] paramTypes = method.getGenericParameterTypes();
Annotation[][] annotations = method.getParameterAnnotations();
@@ -376,8 +374,8 @@
String paramName = paramNames.length == paramTypes.length
? paramNames[i]
: paramSchema.getName()+i;
- fields.put(paramName, new Schema.Field(paramName, paramSchema,
- null /* doc */, null));
+ fields.add(new Schema.Field(paramName, paramSchema,
+ null /* doc */, null));
}
Schema request = Schema.createRecord(fields);
Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificCompiler.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificCompiler.java?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificCompiler.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificCompiler.java Tue Feb 9 03:34:43 2010
@@ -126,8 +126,8 @@
switch (schema.getType()) {
case RECORD:
queue.add(schema);
- for (Map.Entry<String, Schema> field : schema.getFieldSchemas())
- enqueue(field.getValue());
+ for (Schema.Field field : schema.getFields())
+ enqueue(field.schema());
break;
case MAP:
enqueue(schema.getValueType());
@@ -219,9 +219,9 @@
private String params(Schema request) {
StringBuilder b = new StringBuilder();
int count = 0;
- for (Map.Entry<String, Schema> param : request.getFieldSchemas()) {
- String paramName = mangle(param.getKey());
- b.append(unbox(param.getValue()));
+ for (Schema.Field param : request.getFields()) {
+ String paramName = mangle(param.name());
+ b.append(unbox(param.schema()));
b.append(" ");
b.append(paramName);
if (++count < request.getFields().size())
@@ -257,10 +257,10 @@
line(out, 1, "public static final org.apache.avro.Schema SCHEMA$ = org.apache.avro.Schema.parse(\""
+esc(schema)+"\");");
// field declations
- for (Map.Entry<String,Schema.Field> field: schema.getFields().entrySet()) {
- doc(out, 1, field.getValue().doc());
- line(out, 1, "public " + unbox(field.getValue().schema()) + " "
- + mangle(field.getKey()) + ";");
+ for (Schema.Field field : schema.getFields()) {
+ doc(out, 1, field.doc());
+ line(out, 1, "public " + unbox(field.schema()) + " "
+ + mangle(field.name()) + ";");
}
// schema method
line(out, 1, "public org.apache.avro.Schema getSchema() { return SCHEMA$; }");
@@ -268,8 +268,8 @@
line(out, 1, "public java.lang.Object get(int field$) {");
line(out, 2, "switch (field$) {");
int i = 0;
- for (Map.Entry<String, Schema> field : schema.getFieldSchemas())
- line(out, 2, "case "+(i++)+": return "+mangle(field.getKey())+";");
+ for (Schema.Field field : schema.getFields())
+ line(out, 2, "case "+(i++)+": return "+mangle(field.name())+";");
line(out, 2, "default: throw new org.apache.avro.AvroRuntimeException(\"Bad index\");");
line(out, 2, "}");
line(out, 1, "}");
@@ -278,9 +278,9 @@
line(out, 1, "public void put(int field$, java.lang.Object value$) {");
line(out, 2, "switch (field$) {");
i = 0;
- for (Map.Entry<String, Schema> field : schema.getFieldSchemas())
- line(out, 2, "case "+(i++)+": "+mangle(field.getKey())+" = ("+
- type(field.getValue())+")value$; break;");
+ for (Schema.Field field : schema.getFields())
+ line(out, 2, "case "+(i++)+": "+mangle(field.name())+" = ("+
+ type(field.schema())+")value$; break;");
line(out, 2, "default: throw new org.apache.avro.AvroRuntimeException(\"Bad index\");");
line(out, 2, "}");
line(out, 1, "}");
Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificRequestor.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificRequestor.java?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificRequestor.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificRequestor.java Tue Feb 9 03:34:43 2010
@@ -22,7 +22,6 @@
import java.lang.reflect.Method;
import java.lang.reflect.Proxy;
import java.lang.reflect.InvocationHandler;
-import java.util.Map;
import org.apache.avro.Protocol;
import org.apache.avro.Schema;
@@ -66,8 +65,8 @@
throws IOException {
Object[] args = (Object[])request;
int i = 0;
- for (Map.Entry<String, Schema> param : schema.getFieldSchemas())
- getDatumWriter(param.getValue()).write(args[i++], out);
+ for (Schema.Field param : schema.getFields())
+ getDatumWriter(param.schema()).write(args[i++], out);
}
@Override
Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificResponder.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificResponder.java?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificResponder.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificResponder.java Tue Feb 9 03:34:43 2010
@@ -21,7 +21,6 @@
import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
-import java.util.Map;
import org.apache.avro.Schema;
import org.apache.avro.Protocol;
@@ -66,8 +65,8 @@
public Object readRequest(Schema schema, Decoder in) throws IOException {
Object[] args = new Object[schema.getFields().size()];
int i = 0;
- for (Map.Entry<String, Schema> param : schema.getFieldSchemas())
- args[i++] = getDatumReader(param.getValue()).read(null, in);
+ for (Schema.Field param : schema.getFields())
+ args[i++] = getDatumReader(param.schema()).read(null, in);
return args;
}
@@ -89,8 +88,8 @@
Class[] paramTypes = new Class[message.getRequest().getFields().size()];
int i = 0;
try {
- for (Map.Entry<String,Schema> param: message.getRequest().getFieldSchemas())
- paramTypes[i++] = data.getClass(param.getValue());
+ for (Schema.Field param: message.getRequest().getFields())
+ paramTypes[i++] = data.getClass(param.schema());
Method method = impl.getClass().getMethod(message.getName(), paramTypes);
return method.invoke(impl, (Object[])request);
} catch (InvocationTargetException e) {
Modified: hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/RandomData.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/RandomData.java?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/RandomData.java (original)
+++ hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/RandomData.java Tue Feb 9 03:34:43 2010
@@ -66,8 +66,8 @@
switch (schema.getType()) {
case RECORD:
GenericRecord record = new GenericData.Record(schema);
- for (Map.Entry<String, Schema> entry : schema.getFieldSchemas())
- record.put(entry.getKey(), generate(entry.getValue(), random, d+1));
+ for (Schema.Field field : schema.getFields())
+ record.put(field.name(), generate(field.schema(), random, d+1));
return record;
case ENUM:
List<String> symbols = schema.getEnumSymbols();
Modified: hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestProtocolGeneric.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestProtocolGeneric.java?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestProtocolGeneric.java (original)
+++ hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestProtocolGeneric.java Tue Feb 9 03:34:43 2010
@@ -42,7 +42,7 @@
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.util.ArrayList;
-import java.util.LinkedHashMap;
+import java.util.List;
import java.util.Random;
public class TestProtocolGeneric {
@@ -162,12 +162,10 @@
argument to check that schema is sent to parse request. */
public void testHandshake() throws IOException {
Protocol protocol = new Protocol("Simple", "org.apache.avro.test");
- LinkedHashMap<String,Field> fields = new LinkedHashMap<String,Field>();
- fields.put("extra",
- new Schema.Field("extra", Schema.create(Schema.Type.BOOLEAN),
+ List<Field> fields = new ArrayList<Field>();
+ fields.add(new Schema.Field("extra", Schema.create(Schema.Type.BOOLEAN),
null, null));
- fields.put("greeting",
- new Schema.Field("greeting", Schema.create(Schema.Type.STRING),
+ fields.add(new Schema.Field("greeting", Schema.create(Schema.Type.STRING),
null, null));
Protocol.Message message =
protocol.createMessage("hello",
Modified: hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestReflect.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestReflect.java?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestReflect.java (original)
+++ hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestReflect.java Tue Feb 9 03:34:43 2010
@@ -237,7 +237,7 @@
assertEquals(Schema.Type.STRING, response.getTypes().get(1).getType());
// check request schema is union
Schema request = message.getRequest();
- Schema param = request.getFields().get("s").schema();
+ Schema param = request.getField("s").schema();
assertEquals(Schema.Type.UNION, param.getType());
assertEquals(Schema.Type.NULL, param.getTypes().get(0).getType());
assertEquals(Schema.Type.STRING, param.getTypes().get(1).getType());
Modified: hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestSchema.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestSchema.java?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestSchema.java (original)
+++ hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/TestSchema.java Tue Feb 9 03:34:43 2010
@@ -30,6 +30,7 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
+import java.util.Iterator;
import java.util.List;
import org.apache.avro.Schema.Type;
@@ -315,9 +316,9 @@
+"{\"name\":\"g\",\"type\":"+z+"}"
+"]}";
Schema xs = Schema.parse(x);
- Schema ys = xs.getFields().get("f").schema();
- assertEquals("p.Z", xs.getFields().get("g").schema().getFullName());
- assertEquals("q.Z", ys.getFields().get("f").schema().getFullName());
+ Schema ys = xs.getField("f").schema();
+ assertEquals("p.Z", xs.getField("g").schema().getFullName());
+ assertEquals("q.Z", ys.getField("f").schema().getFullName());
}
private static void checkParseError(String json) {
@@ -353,12 +354,12 @@
public void testDocs() {
Schema schema = Schema.parse(SCHEMA_WITH_DOC_TAGS);
assertEquals("This is not a world record.", schema.getDoc());
- assertEquals("Inner Fixed", schema.getFields().get("inner_fixed").doc());
- assertEquals("Very Inner Fixed", schema.getFields().get("inner_fixed").schema().getDoc());
- assertEquals("Inner String", schema.getFields().get("inner_string").doc());
- assertEquals("Inner Enum", schema.getFields().get("inner_enum").doc());
- assertEquals("Very Inner Enum", schema.getFields().get("inner_enum").schema().getDoc());
- assertEquals("Inner Union", schema.getFields().get("inner_union").doc());
+ assertEquals("Inner Fixed", schema.getField("inner_fixed").doc());
+ assertEquals("Very Inner Fixed", schema.getField("inner_fixed").schema().getDoc());
+ assertEquals("Inner String", schema.getField("inner_string").doc());
+ assertEquals("Inner Enum", schema.getField("inner_enum").doc());
+ assertEquals("Very Inner Enum", schema.getField("inner_enum").schema().getDoc());
+ assertEquals("Inner Union", schema.getField("inner_union").doc());
}
private static void check(String schemaJson, String defaultJson,
@@ -402,7 +403,7 @@
if(s0.getType().equals(Schema.Type.UNION)) return; // unions have no props
assertEquals(null, s0.getProp("foo"));
Schema s1 = Schema.parse(s0.toString());
- s1.setProp("foo", "bar");
+ s1.addProp("foo", "bar");
assertEquals("bar", s1.getProp("foo"));
assertFalse(s0.equals(s1));
Schema s2 = Schema.parse(s1.toString());
@@ -529,4 +530,83 @@
public void testEnumWithPrimitiveName() {
Schema.parse("{\"type\":\"enum\", \"name\":\"null\", \"symbols\": [\"A\"]}");
}
+
+ private static Schema enumSchema() {
+ return Schema.parse("{ \"type\": \"enum\", \"name\": \"e\", "
+ + "\"symbols\": [\"a\", \"b\"]}");
+ }
+
+ @Test(expected=AvroRuntimeException.class)
+ public void testImmutability1() {
+ Schema s = enumSchema();
+ s.addProp("p1", "1");
+ s.addProp("p1", "2");
+ }
+
+ @Test(expected=AvroRuntimeException.class)
+ public void testImmutability2() {
+ Schema s = enumSchema();
+ s.addProp("p1", null);
+ }
+
+ private static List<String> lockedArrayList() {
+ return new Schema.LockableArrayList<String>(Arrays.asList(new String[] {
+ "a", "b", "c" })).lock();
+ }
+
+ @Test(expected=IllegalStateException.class)
+ public void testLockedArrayList1() {
+ lockedArrayList().add("p");
+ }
+
+ @Test(expected=IllegalStateException.class)
+ public void testLockedArrayList2() {
+ lockedArrayList().remove("a");
+ }
+
+ @Test(expected=IllegalStateException.class)
+ public void testLockedArrayList3() {
+ lockedArrayList().addAll(Arrays.asList(new String[] { "p" }));
+ }
+
+ @Test(expected=IllegalStateException.class)
+ public void testLockedArrayList4() {
+ lockedArrayList().addAll(0,
+ Arrays.asList(new String[] { "p" }));
+ }
+
+ @Test(expected=IllegalStateException.class)
+ public void testLockedArrayList5() {
+ lockedArrayList().
+ removeAll(Arrays.asList(new String[] { "a" }));
+ }
+
+ @Test(expected=IllegalStateException.class)
+ public void testLockedArrayList6() {
+ lockedArrayList().
+ retainAll(Arrays.asList(new String[] { "a" }));
+ }
+
+ @Test(expected=IllegalStateException.class)
+ public void testLockedArrayList7() {
+ lockedArrayList().clear();
+ }
+
+ @Test(expected=IllegalStateException.class)
+ public void testLockedArrayList8() {
+ lockedArrayList().iterator().remove();
+ }
+
+
+ @Test(expected=IllegalStateException.class)
+ public void testLockedArrayList9() {
+ Iterator<String> it = lockedArrayList().iterator();
+ it.next();
+ it.remove();
+ }
+
+ @Test(expected=IllegalStateException.class)
+ public void testLockedArrayList10() {
+ lockedArrayList().remove(1);
+ }
}
Modified: hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/generic/TestGenericData.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/generic/TestGenericData.java?rev=907913&r1=907912&r2=907913&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/generic/TestGenericData.java (original)
+++ hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/generic/TestGenericData.java Tue Feb 9 03:34:43 2010
@@ -17,7 +17,8 @@
*/
package org.apache.avro.generic;
-import java.util.LinkedHashMap;
+import java.util.ArrayList;
+import java.util.List;
import static org.junit.Assert.*;
@@ -62,7 +63,7 @@
@Test
public void testRecordGetFieldDoesntExist() throws Exception {
- LinkedHashMap<String, Field> fields = new LinkedHashMap<String, Field>();
+ List<Field> fields = new ArrayList<Field>();
Schema schema = Schema.createRecord(fields);
GenericData.Record record = new GenericData.Record(schema);
assertNull(record.get("does not exist"));