Author: thiru
Date: Fri Jan 29 19:11:45 2010
New Revision: 904596
URL: http://svn.apache.org/viewvc?rev=904596&view=rev
Log:
AVRO-383. Optiminzing ResolvingDecoder for default values
Added:
hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/io/parsing/
hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/io/parsing/TestResolvingGrammarGenerator.java
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=904596&r1=904595&r2=904596&view=diff
==============================================================================
--- hadoop/avro/trunk/CHANGES.txt (original)
+++ hadoop/avro/trunk/CHANGES.txt Fri Jan 29 19:11:45 2010
@@ -291,6 +291,8 @@
AVRO-343. Minor fixes to Eclipse config after build re-org (philz)
+ AVRO-383. Optiminzing ResolvingDecoder for default values (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=904596&r1=904595&r2=904596&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
Fri Jan 29 19:11:45 2010
@@ -165,7 +165,7 @@
} else if (top instanceof Symbol.DefaultStartAction) {
Symbol.DefaultStartAction dsa = (Symbol.DefaultStartAction) top;
backup = in;
- in = (new JsonDecoder(dsa.root, new ByteArrayInputStream(dsa.contents)));
+ in = (new BinaryDecoder(new ByteArrayInputStream(dsa.contents)));
} else if (top == Symbol.DEFAULT_END_ACTION) {
in = backup;
} else {
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=904596&r1=904595&r2=904596&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
Fri Jan 29 19:11:45 2010
@@ -17,14 +17,20 @@
*/
package org.apache.avro.io.parsing;
+import java.io.ByteArrayOutputStream;
import java.io.IOException;
+import java.util.Arrays;
import java.util.HashMap;
+import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.avro.AvroTypeException;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.avro.io.Encoder;
+import org.codehaus.jackson.JsonNode;
/**
* The class that generates a resolving grammar to resolve between two
@@ -258,9 +264,9 @@
Field wf = wfields.get(fname);
if (wf == null) {
Field rf = rfe.getValue();
- production[--count] = new Symbol.DefaultStartAction(
- new JsonGrammarGenerator().generate(rf.schema()),
- rf.defaultValue());
+ Symbol r = new JsonGrammarGenerator().generate(rf.schema());
+ byte[] bb = getBinary(rf.schema(), rf.defaultValue());
+ production[--count] = new Symbol.DefaultStartAction(r, bb);
production[--count] = super.generate(rf.schema(), seen);
production[--count] = Symbol.DEFAULT_END_ACTION;
}
@@ -269,6 +275,110 @@
return result;
}
+ /**
+ * Returns the Avro binary encoded version of <tt>n</tt> according to
+ * the schema <tt>s</tt>.
+ * @param s The schema for encoding
+ * @param n The Json node that has the value to be encoded.
+ * @return The binary encoded version of <tt>n</tt>.
+ * @throws IOException
+ */
+ private static byte[] getBinary(Schema s, JsonNode n) throws IOException {
+ ByteArrayOutputStream out = new ByteArrayOutputStream();
+ Encoder e = new BinaryEncoder(out);
+ encode(e, s, n);
+ return out.toByteArray();
+ }
+
+ /**
+ * Encodes the given Json node <tt>n</tt> on to the encoder <tt>e</tt>
+ * according to the schema <tt>s</tt>.
+ * @param e The encoder to encode into.
+ * @param s The schema for the object being encoded.
+ * @param n The Json node to encode.
+ * @throws IOException
+ */
+
+ static void encode(Encoder e, Schema s, JsonNode n)
+ throws IOException {
+ switch (s.getType()) {
+ case RECORD:
+ for (Map.Entry<String, Field> entry : s.getFields().entrySet()) {
+ String name = entry.getKey();
+ Field f = entry.getValue();
+ JsonNode v = n.get(name);
+ if (v == null) {
+ v = f.defaultValue();
+ }
+ if (v == null) {
+ throw new AvroTypeException("No default value for: " + name);
+ }
+ encode(e, f.schema(), v);
+ }
+ break;
+ case ENUM:
+ e.writeEnum(s.getEnumOrdinal(n.getTextValue()));
+ break;
+ case ARRAY:
+ e.writeArrayStart();
+ e.setItemCount(n.size());
+ Schema i = s.getElementType();
+ for (JsonNode node : n) {
+ e.startItem();
+ encode(e, i, node);
+ }
+ e.writeArrayEnd();
+ break;
+ case MAP:
+ e.writeMapStart();
+ e.setItemCount(n.size());
+ Schema v = s.getValueType();
+ for (Iterator<String> it = n.getFieldNames(); it.hasNext();) {
+ e.startItem();
+ String key = it.next();
+ e.writeString(key);
+ encode(e, v, n.get(key));
+ }
+ e.writeMapEnd();
+ break;
+ case UNION:
+ e.writeIndex(0);
+ encode(e, s.getTypes().get(0), n);
+ break;
+ case FIXED:
+ byte[] bb = n.getTextValue().getBytes("ISO-8859-1");
+ if (bb.length != s.getFixedSize()) {
+ bb = Arrays.copyOf(bb, s.getFixedSize());
+ }
+ e.writeFixed(bb);
+ break;
+ case STRING:
+ e.writeString(n.getTextValue());
+ break;
+ case BYTES:
+ e.writeBytes(n.getTextValue().getBytes("ISO-8859-1"));
+ break;
+ case INT:
+ e.writeInt(n.getIntValue());
+ break;
+ case LONG:
+ e.writeLong(n.getLongValue());
+ break;
+ case FLOAT:
+ e.writeFloat((float) n.getDoubleValue());
+ break;
+ case DOUBLE:
+ e.writeDouble(n.getDoubleValue());
+ break;
+ case BOOLEAN:
+ e.writeBoolean(n.getBooleanValue());
+ break;
+ case NULL:
+ e.writeNull();
+ break;
+ }
+ }
+
private static Symbol mkEnumAdjust(List<String> rsymbols,
List<String> wsymbols){
Object[] adjustments = new Object[wsymbols.size()];
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=904596&r1=904595&r2=904596&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
Fri Jan 29 19:11:45 2010
@@ -17,8 +17,6 @@
*/
package org.apache.avro.io.parsing;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
@@ -27,11 +25,6 @@
import java.util.NoSuchElementException;
import org.apache.avro.Schema;
-import org.codehaus.jackson.JsonEncoding;
-import org.codehaus.jackson.JsonFactory;
-import org.codehaus.jackson.JsonGenerator;
-import org.codehaus.jackson.JsonNode;
-import org.codehaus.jackson.map.ObjectMapper;
/**
* Symbol is the base of all symbols (terminals and non-terminals) of
@@ -477,15 +470,9 @@
public static class DefaultStartAction extends ImplicitAction {
public final Symbol root;
public final byte[] contents;
- public DefaultStartAction(Symbol root, JsonNode defaultValue)
- throws IOException {
+ public DefaultStartAction(Symbol root, byte[] contents) {
this.root = root;
- ByteArrayOutputStream os = new ByteArrayOutputStream();
- JsonGenerator g = new JsonFactory().createJsonGenerator(os,
- JsonEncoding.UTF8);
- new ObjectMapper().writeTree(g, defaultValue);
- g.flush();
- this.contents = os.toByteArray();
+ this.contents = 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=904596&r1=904595&r2=904596&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 Fri
Jan 29 19:11:45 2010
@@ -26,6 +26,7 @@
import java.util.Random;
import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
/**
* Performance tests for various low level operations of
@@ -52,6 +53,8 @@
tests.add(new NestedRecordTest());
} else if (a.equals("-S")) {
tests.add(new ResolverTest());
+ } else if (a.equals("-M")) {
+ tests.add(new MigrationTest());
} else {
usage();
System.exit(1);
@@ -281,6 +284,49 @@
}
+ /**
+ * Tests the performance of introducing default values.
+ */
+ private static class MigrationTest extends RepeaterTest {
+ private final Schema readerSchema;
+ public MigrationTest() throws IOException {
+ super("MigrationTest");
+ readerSchema = Schema.parse( "{ \"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_1\", \"type\": \"string\", "
+ + "\"default\": \"undefined\" },\n"
+ + "{ \"name\": \"f3_2\", \"type\": \"string\","
+ + "\"default\": \"undefined\" },\n"
+ + "{ \"name\": \"f4\", \"type\": \"int\" },\n"
+ + "{ \"name\": \"f5\", \"type\": \"int\" },\n"
+ + "{ \"name\": \"f6\", \"type\": \"int\" }\n"
+ + "] } }");
+ }
+
+ @Override
+ protected Decoder getDecoder() throws IOException {
+ return new ResolvingDecoder(schema, readerSchema, newDecoder(data));
+ }
+
+ @Override
+ protected void readInternal(Decoder d) throws IOException {
+ ResolvingDecoder r = (ResolvingDecoder) d;
+ Field[] ff = r.readFieldOrder();
+ for (Field f : ff) {
+ if (f.pos() < 3) {
+ r.readDouble();
+ } else if (f.pos() >= 5) {
+ r.readInt();
+ } else {
+ r.readString(null);
+ }
+ }
+ }
+ }
+
private static class NestedRecordTest extends ReadInt {
public NestedRecordTest() throws IOException {
super("NestedRecordTest",
Added:
hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/io/parsing/TestResolvingGrammarGenerator.java
URL:
http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/io/parsing/TestResolvingGrammarGenerator.java?rev=904596&view=auto
==============================================================================
---
hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/io/parsing/TestResolvingGrammarGenerator.java
(added)
+++
hadoop/avro/trunk/lang/java/src/test/java/org/apache/avro/io/parsing/TestResolvingGrammarGenerator.java
Fri Jan 29 19:11:45 2010
@@ -0,0 +1,93 @@
+/**
+ * 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.io.parsing;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.apache.avro.Schema;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.avro.io.Encoder;
+import org.apache.avro.io.ValidatingEncoder;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+...@runwith(Parameterized.class)
+public class TestResolvingGrammarGenerator {
+ private final Schema schema;
+ private final JsonNode data;
+
+ public TestResolvingGrammarGenerator(String jsonSchema, String jsonData)
+ throws IOException {
+ this.schema = Schema.parse(jsonSchema);
+ JsonFactory factory = new JsonFactory();
+ ObjectMapper mapper = new ObjectMapper(factory);
+
+ this.data = mapper.readTree(new StringReader(jsonData));
+ }
+
+ @Test
+ public void test() throws IOException {
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ Encoder e = new ValidatingEncoder(schema, new BinaryEncoder(baos));
+
+ ResolvingGrammarGenerator.encode(e, schema, data);
+ }
+
+ @Parameterized.Parameters
+ public static Collection<Object[]> data() {
+ Collection<Object[]> ret = Arrays.asList(
+ new Object[][] {
+ { "{ \"type\": \"record\", \"name\": \"r\", \"fields\": [ "
+ + " { \"name\" : \"f1\", \"type\": \"int\" }, "
+ + " { \"name\" : \"f2\", \"type\": \"float\" } "
+ + "] } }",
+ "{ \"f2\": 10.4, \"f1\": 10 } " },
+ { "{ \"type\": \"enum\", \"name\": \"e\", \"symbols\": "
+ + "[ \"s1\", \"s2\"] } }", " \"s1\" " },
+ { "{ \"type\": \"enum\", \"name\": \"e\", \"symbols\": "
+ + "[ \"s1\", \"s2\"] } }", " \"s2\" " },
+ { "{ \"type\": \"fixed\", \"name\": \"f\", \"size\": 10 }",
+ "\"hello\"" },
+ { "{ \"type\": \"array\", \"items\": \"int\" }",
+ "[ 10, 20, 30 ]" },
+ { "{ \"type\": \"map\", \"values\": \"int\" }",
+ "{ \"k1\": 10, \"k3\": 20, \"k3\": 30 }" },
+ { "[ \"int\", \"long\" ]", "10" },
+ { "\"string\"", "\"hello\"" },
+ { "\"bytes\"", "\"hello\"" },
+ { "\"int\"", "10" },
+ { "\"long\"", "10" },
+ { "\"float\"", "10.0" },
+ { "\"double\"", "10.0" },
+ { "\"boolean\"", "true" },
+ { "\"boolean\"", "false" },
+ { "\"null\"", "null" },
+ }
+ );
+ return ret;
+ }
+
+}