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/01/19 15:25:06 UTC
svn commit: r900790 - in /hadoop/avro/trunk: ./
lang/java/src/java/org/apache/avro/io/
lang/java/src/java/org/apache/avro/io/parsing/
lang/java/src/test/java/org/apache/avro/io/
Author: thiru
Date: Tue Jan 19 14:25:06 2010
New Revision: 900790
URL: http://svn.apache.org/viewvc?rev=900790&view=rev
Log:
AVRO-345. Optimization for ResolvingDecoder
Modified:
hadoop/avro/trunk/CHANGES.txt
hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/ResolvingDecoder.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/Symbol.java
hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/io/Perf.java
Modified: hadoop/avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/CHANGES.txt?rev=900790&r1=900789&r2=900790&view=diff
==============================================================================
--- hadoop/avro/trunk/CHANGES.txt (original)
+++ hadoop/avro/trunk/CHANGES.txt Tue Jan 19 14:25:06 2010
@@ -240,6 +240,8 @@
AVRO-328. Performance improvements Validating encoder/decoder for nested records (thiru)
+ AVRO-345. Optimization for ResolvingDecoder (thiru)
+
BUG FIXES
AVRO-176. Safeguard against bad istreams before reading. (sbanacho)
Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/ResolvingDecoder.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/ResolvingDecoder.java?rev=900790&r1=900789&r2=900790&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/ResolvingDecoder.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/ResolvingDecoder.java Tue Jan 19 14:25:06 2010
@@ -26,20 +26,18 @@
import org.apache.avro.io.parsing.Symbol;
/**
- * {@link Decoder} that peforms type-resolution between the reader's and
+ * {@link Decoder} that performs type-resolution between the reader's and
* writer's schemas.
*
* <p>When resolving schemas, this class will return the values of fields in
* _writer's_ order, not the reader's order. (However, it returns _only_ the
* reader's fields, not any extra fields the writer may have written.) To help
* clients handle fields that appear to be coming out of order, this class
- * defines the methods {@link #readFieldName} and {@link #readFieldIndex}.
- * When called just before reading the value of a record-field, they return the
- * name/index of the field about to be read. See {@link #readFieldIndex} for
- * usage details.
+ * defines the method {@link #readFieldOrder}.
*
* <p>See the <a href="doc-files/parsing.html">parser documentation</a> for
- * information on how this works.*/
+ * information on how this works.
+ */
public class ResolvingDecoder extends ValidatingDecoder {
private Decoder backup;
@@ -49,60 +47,50 @@
super(new ResolvingGrammarGenerator().generate(writer, reader), in);
}
- /** Returns the name of the next field of the record we're reading.
- * Similar to {@link #readFieldIndex} -- see that method for
- * details.
- *
- * @throws IllegalStateExcpetion If we're not about to read a record-field
- */
- public String readFieldName() throws IOException {
- return ((Symbol.FieldAdjustAction) parser.advance(Symbol.FIELD_ACTION)).
- fname;
- }
-
- /** Returns the (zero-based) index of the next field of the record
- * we're reading.
- *
- * This method is useful because {@link ResolvingDecoder}
- * returns values in the order written by the writer, rather than
- * the order expected by the reader. This method allows reader's
- * to figure out what fields to expect. Let's say the reader is
- * expecting a three-field record, the first field is a long, the
- * second a string, and the third an array. In this case, a
- * typical usage might be as follows:
- * <pre>
- * for (int i = 0; i < 3; i++) {
- * switch (in.readFieldIndex()) {
- * case 1:
- * foo(in.readLong());
- * break;
- * case 2:
- * someVariable = in.readString();
- * break;
- * case 3:
- * bar(in); // The code of "bar" will read an array-of-int
- * break;
- * }
- * </pre>
- * Note that {@link ResolvingDecoder} will return only the
- * fields expected by the reader, not other fields that may have
- * been written by the writer. Thus, the iteration-count of "3" in
- * the above loop will always be correct.
- *
- * Throws a runtime exception if we're not just about to read the
- * field of a record. Also, this method (and {@link
- * #readFieldName}) will <i>consume</i> the field information, and
- * thus may only be called <i>once</i> before reading the field
- * value. (However, if the client knows the order of incoming
- * fields and does not need to reorder them, then the client does
- * <i>not</i> need to call this or {@link #readFieldName}.)
- *
- * @throws IllegalStateExcpetion If we're not about to read a record-field
- *
- */
- public int readFieldIndex() throws IOException {
- return ((Symbol.FieldAdjustAction) parser.advance(Symbol.FIELD_ACTION)).
- rindex;
+ /** Returns the actual order in which the reader's fields will be
+ * returned to the reader.
+ *
+ * This method is useful because {@link ResolvingDecoder}
+ * returns values in the order written by the writer, rather than
+ * the order expected by the reader. This method allows readers
+ * to figure out what fields to expect. Let's say the reader is
+ * expecting a three-field record, the first field is a long, the
+ * second a string, and the third an array. In this case, a
+ * typical usage might be as follows:
+ * <pre>
+ * Schema.Fields[] fieldOrder = in.readFieldOrder();
+ * for (int i = 0; i < 3; i++) {
+ * switch (fieldOrder[i].pos()) {
+ * case 1:
+ * foo(in.readLong());
+ * break;
+ * case 2:
+ * someVariable = in.readString();
+ * break;
+ * case 3:
+ * bar(in); // The code of "bar" will read an array-of-int
+ * break;
+ * }
+ * </pre>
+ * Note that {@link ResolvingDecoder} will return only the
+ * fields expected by the reader, not other fields that may have
+ * been written by the writer. Thus, the iteration-count of "3" in
+ * the above loop will always be correct.
+ *
+ * Throws a runtime exception if we're not just about to read the
+ * field of a record. Also, this method will consume the field
+ * information, and thus may only be called <em>once</em> before
+ * reading the field value. (However, if the client knows the
+ * order of incoming fields, then the client does <em>not</em>
+ * need to call this method but rather can just start reading the
+ * field values.)
+ *
+ * @throws AvroTypeException If we're not starting a new record
+ *
+ */
+ public final Schema.Field[] readFieldOrder() throws IOException {
+ return ((Symbol.FieldOrderAction) parser.advance(Symbol.FIELD_ACTION)).
+ fields;
}
@Override
@@ -156,7 +144,7 @@
@Override
public Symbol doAction(Symbol input, Symbol top) throws IOException {
- if (top instanceof Symbol.FieldAdjustAction) {
+ if (top instanceof Symbol.FieldOrderAction) {
return input == Symbol.FIELD_ACTION ? top : null;
} if (top instanceof Symbol.ResolvingAction) {
Symbol.ResolvingAction t = (Symbol.ResolvingAction) top;
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=900790&r1=900789&r2=900790&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 Jan 19 14:25:06 2010
@@ -190,13 +190,20 @@
LitS wsc = new LitS2(writer, reader);
Symbol result = seen.get(wsc);
if (result == null) {
- // first count the number of entries in the production;
- int count = 0;
-
Map<String, Field> wfields = writer.getFields();
Map<String, Field> rfields = reader.getFields();
+
+ // First, compute reordering of reader fields, plus
+ // number elements in the result's production
+ Field[] reordered = new Field[rfields.size()];
+ int ridx = 0;
+ int count = 1 + wfields.size();
+
for (String fn : wfields.keySet()) {
- count += (rfields.get(fn) == null) ? 1 : 2;
+ Field rdrField = rfields.get(fn);
+ if (rdrField != null) {
+ reordered[ridx++] = rdrField;
+ }
}
for (Map.Entry<String, Field> rfe : rfields.entrySet()) {
@@ -208,12 +215,14 @@
seen.put(wsc, result);
return result;
} else {
- count += 4;
+ reordered[ridx++] = rf;
+ count += 3;
}
}
}
Symbol[] production = new Symbol[count];
+ production[--count] = new Symbol.FieldOrderAction(reordered);
/**
* We construct a symbol without filling the array. Please see
@@ -239,8 +248,6 @@
seen));
} else {
production[--count] =
- new Symbol.FieldAdjustAction(rf.pos(), fname);
- production[--count] =
generate(wfe.getValue().schema(), rf.schema(), seen);
}
}
@@ -251,7 +258,6 @@
Field wf = wfields.get(fname);
if (wf == null) {
Field rf = rfe.getValue();
- production[--count] = new Symbol.FieldAdjustAction(rf.pos(), fname);
production[--count] = new Symbol.DefaultStartAction(
new JsonGrammarGenerator().generate(rf.schema()),
rf.defaultValue());
Modified: hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/Symbol.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/Symbol.java?rev=900790&r1=900789&r2=900790&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/Symbol.java (original)
+++ hadoop/avro/trunk/lang/java/src/java/org/apache/avro/io/parsing/Symbol.java Tue Jan 19 14:25:06 2010
@@ -26,6 +26,7 @@
import java.util.Map;
import java.util.NoSuchElementException;
+import org.apache.avro.Schema;
import org.codehaus.jackson.JsonEncoding;
import org.codehaus.jackson.JsonFactory;
import org.codehaus.jackson.JsonGenerator;
@@ -466,6 +467,13 @@
}
}
+ public static final class FieldOrderAction extends ImplicitAction {
+ public final Schema.Field[] fields;
+ public FieldOrderAction(Schema.Field[] fields) {
+ this.fields = fields;
+ }
+ }
+
public static class DefaultStartAction extends ImplicitAction {
public final Symbol root;
public final byte[] contents;
Modified: hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/io/Perf.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/io/Perf.java?rev=900790&r1=900789&r2=900790&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/io/Perf.java (original)
+++ hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/io/Perf.java Tue Jan 19 14:25:06 2010
@@ -51,6 +51,8 @@
tests.add(new RepeaterTest());
} else if (a.equals("-N")) {
tests.add(new NestedRecordTest());
+ } else if (a.equals("-S")) {
+ tests.add(new ResolverTest());
} else {
usage();
System.exit(1);
@@ -107,6 +109,10 @@
}
protected Decoder getDecoder() throws IOException {
+ return newDecoder(data);
+ }
+
+ protected static Decoder newDecoder(byte[] data) {
return new BinaryDecoder(new ByteArrayInputStream(data));
}
@@ -205,11 +211,18 @@
private static class RepeaterTest extends Test {
public RepeaterTest() throws IOException {
- super("RepeaterTest", "{ \"type\": \"array\", \"items\":\n"
+ this("RepeaterTest");
+ }
+
+ public RepeaterTest(String name) throws IOException {
+ super(name, "{ \"type\": \"array\", \"items\":\n"
+ "{ \"type\": \"record\", \"name\": \"R\", \"fields\": [\n"
+ "{ \"name\": \"f1\", \"type\": \"double\" },\n"
+ "{ \"name\": \"f2\", \"type\": \"double\" },\n"
- + "{ \"name\": \"f3\", \"type\": \"double\" }\n"
+ + "{ \"name\": \"f3\", \"type\": \"double\" },\n"
+ + "{ \"name\": \"f4\", \"type\": \"int\" },\n"
+ + "{ \"name\": \"f5\", \"type\": \"int\" },\n"
+ + "{ \"name\": \"f6\", \"type\": \"int\" }\n"
+ "] } }");
}
@@ -222,6 +235,9 @@
e.writeDouble(r.nextDouble());
e.writeDouble(r.nextDouble());
e.writeDouble(r.nextDouble());
+ e.writeInt(r.nextInt());
+ e.writeInt(r.nextInt());
+ e.writeInt(r.nextInt());
}
e.writeArrayEnd();
}
@@ -231,6 +247,9 @@
d.readDouble();
d.readDouble();
d.readDouble();
+ d.readInt();
+ d.readInt();
+ d.readInt();
}
@Override
@@ -239,10 +258,23 @@
}
}
+
+ private static class ResolverTest extends RepeaterTest {
+
+ public ResolverTest() throws IOException {
+ super("ResolverTest");
+ }
+
+ @Override
+ protected Decoder getDecoder() throws IOException {
+ return new ResolvingDecoder(schema, schema, newDecoder(data));
+ }
+
+ }
private static class NestedRecordTest extends ReadInt {
public NestedRecordTest() throws IOException {
- super("RepeaterTest",
+ super("NestedRecordTest",
"{ \"type\": \"array\", \"items\": \n"
+ "{ \"type\": \"record\", \"name\": \"r1\", \n"
+ "\"fields\": \n"
@@ -262,5 +294,7 @@
System.out.println(" -f readFloat() performance");
System.out.println(" -d readDouble() performance");
System.out.println(" -R repeater performance in validating decoder");
+ System.out.println(" -N nested record performance in validating decoder");
+ System.out.println(" -S resolving decoder performance");
}
}