Author: cutting
Date: Wed Oct 21 17:45:57 2009
New Revision: 828117
URL: http://svn.apache.org/viewvc?rev=828117&view=rev
Log:
AVRO-167. Refactor Java SpecificCompiler to simplify testing, and add some
tests. Contributed by Philip Zeyliger.
Added:
hadoop/avro/trunk/src/test/java/org/apache/avro/specific/
hadoop/avro/trunk/src/test/java/org/apache/avro/specific/TestSpecificCompiler.java
Modified:
hadoop/avro/trunk/CHANGES.txt
hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificCompiler.java
hadoop/avro/trunk/src/test/java/org/apache/avro/TestSchema.java
Modified: hadoop/avro/trunk/CHANGES.txt
URL:
http://svn.apache.org/viewvc/hadoop/avro/trunk/CHANGES.txt?rev=828117&r1=828116&r2=828117&view=diff
==============================================================================
--- hadoop/avro/trunk/CHANGES.txt (original)
+++ hadoop/avro/trunk/CHANGES.txt Wed Oct 21 17:45:57 2009
@@ -23,6 +23,9 @@
AVRO-166. Improve error checking in Java schema parser.
(Philip Zeyliger via cutting)
+ AVRO-167. Refactor Java SpecificCompiler to simplify testing, and
+ add some tests. (Philip Zeyliger via cutting)
+
OPTIMIZATIONS
BUG FIXES
Modified:
hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificCompiler.java
URL:
http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificCompiler.java?rev=828117&r1=828116&r2=828117&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificCompiler.java
(original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificCompiler.java
Wed Oct 21 17:45:57 2009
@@ -18,14 +18,14 @@
package org.apache.avro.specific;
import java.io.File;
-import java.io.FileOutputStream;
-import java.io.Writer;
-import java.io.OutputStreamWriter;
+import java.io.FileWriter;
import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
import java.util.Map;
import java.util.Set;
-import java.util.List;
-import java.util.HashSet;
import org.apache.avro.Protocol;
import org.apache.avro.Schema;
@@ -33,36 +33,65 @@
/** Generate specific Java interfaces and classes for protocols and schemas. */
public class SpecificCompiler {
- private File dest;
- private Writer out;
- private Set<Schema> queue = new HashSet<Schema>();
+ private final Set<Schema> queue = new HashSet<Schema>();
+ private final Protocol protocol;
+
+ public SpecificCompiler(Protocol protocol) {
+ // enqueue all types
+ for (Schema s : protocol.getTypes()) {
+ enqueue(s);
+ }
+ this.protocol = protocol;
+ }
- private SpecificCompiler(File dest) {
- this.dest = dest; // root directory for output
+ public SpecificCompiler(Schema schema) {
+ enqueue(schema);
+ this.protocol = null;
}
- /** Generates Java interface and classes for a protocol.
+ /**
+ * Captures output file path and contents.
+ */
+ static class OutputFile {
+ String path;
+ String contents;
+
+ /**
+ * Writes output to path destination directory, creating directories as
+ * necessary.
+ */
+ void writeToDestination(File dest) throws IOException {
+ File f = new File(dest, path);
+ f.getParentFile().mkdirs();
+ FileWriter fw = new FileWriter(f);
+ try {
+ fw.write(contents);
+ } finally {
+ fw.close();
+ }
+ }
+ }
+
+ /**
+ * Generates Java interface and classes for a protocol.
* @param src the source Avro protocol file
* @param dest the directory to place generated files in
*/
public static void compileProtocol(File src, File dest) throws IOException {
- SpecificCompiler compiler = new SpecificCompiler(dest);
Protocol protocol = Protocol.parse(src);
- for (Schema s : protocol.getTypes()) // enqueue types
- compiler.enqueue(s);
- compiler.compileInterface(protocol); // generate interface
- compiler.compile(); // generate classes for types
+ SpecificCompiler compiler = new SpecificCompiler(protocol);
+ compiler.compileToDestination(dest);
}
/** Generates Java classes for a schema. */
public static void compileSchema(File src, File dest) throws IOException {
- SpecificCompiler compiler = new SpecificCompiler(dest);
- compiler.enqueue(Schema.parse(src)); // enqueue types
- compiler.compile(); // generate classes for types
+ Schema schema = Schema.parse(src);
+ SpecificCompiler compiler = new SpecificCompiler(schema);
+ compiler.compileToDestination(dest);
}
/** Recursively enqueue schemas that need a class generated. */
- private void enqueue(Schema schema) throws IOException {
+ private void enqueue(Schema schema) {
if (queue.contains(schema)) return;
switch (schema.getType()) {
case RECORD:
@@ -94,69 +123,86 @@
}
/** Generate java classes for enqueued schemas. */
- private void compile() throws IOException {
- for (Schema schema : queue)
- compile(schema);
- }
-
- private void compileInterface(Protocol protocol) throws IOException {
- startFile(protocol.getName(), protocol.getNamespace());
- try {
- line(0, "public interface "+protocol.getName()+" {");
-
- out.append("\n");
- for (Map.Entry<String,Message> e : protocol.getMessages().entrySet()) {
- String name = e.getKey();
- Message message = e.getValue();
- Schema request = message.getRequest();
- Schema response = message.getResponse();
- line(1, unbox(response)+" "+name+"("+params(request)+")");
- line(2,"throws AvroRemoteException"+errors(message.getErrors())+";");
- }
- line(0, "}");
- } finally {
- out.close();
+ Collection<OutputFile> compile() {
+ List<OutputFile> out = new ArrayList<OutputFile>();
+ for (Schema schema : queue) {
+ out.add(compile(schema));
}
+ if (protocol != null) {
+ out.add(compileInterface(protocol));
+ }
+ return out;
}
- private void startFile(String name, String space) throws IOException {
- File dir = new File(dest, space.replace('.', File.separatorChar));
- if (!dir.exists())
- if (!dir.mkdirs())
- throw new IOException("Unable to create " + dir);
- name = cap(name) + ".java";
- out = new OutputStreamWriter(new FileOutputStream(new File(dir, name)));
- header(space);
+ private void compileToDestination(File dst) throws IOException {
+ for (Schema schema : queue) {
+ OutputFile o = compile(schema);
+ o.writeToDestination(dst);
+ }
+ if (protocol != null) {
+ compileInterface(protocol).writeToDestination(dst);
+ }
}
- private void header(String namespace) throws IOException {
+ private OutputFile compileInterface(Protocol protocol) {
+ OutputFile outputFile = new OutputFile();
+ outputFile.path = makePath(protocol.getName(), protocol.getNamespace());
+ StringBuilder out = new StringBuilder();
+ header(out, protocol.getNamespace());
+ line(out, 0, "public interface "+protocol.getName()+" {");
+
+ out.append("\n");
+ for (Map.Entry<String,Message> e : protocol.getMessages().entrySet()) {
+ String name = e.getKey();
+ Message message = e.getValue();
+ Schema request = message.getRequest();
+ Schema response = message.getResponse();
+ line(out, 1, unbox(response)+" "+name+"("+params(request)+")");
+ line(out, 2,"throws
AvroRemoteException"+errors(message.getErrors())+";");
+ }
+ line(out, 0, "}");
+
+ outputFile.contents = out.toString();
+ return outputFile;
+ }
+
+ static String makePath(String name, String space) {
+ if (space == null || space.isEmpty()) {
+ return cap(name) + ".java";
+ } else {
+ return space.replace('.', File.separatorChar) + File.separatorChar
+ + cap(name) + ".java";
+ }
+ }
+
+ private void header(StringBuilder out, String namespace) {
if(namespace != null) {
- line(0, "package "+namespace+";\n");
+ line(out, 0, "package "+namespace+";\n");
}
- line(0, "import java.nio.ByteBuffer;");
- line(0, "import java.util.Map;");
- line(0, "import org.apache.avro.Protocol;");
- line(0, "import org.apache.avro.Schema;");
- line(0, "import org.apache.avro.AvroRuntimeException;");
- line(0, "import org.apache.avro.Protocol;");
- line(0, "import org.apache.avro.util.Utf8;");
- line(0, "import org.apache.avro.ipc.AvroRemoteException;");
- line(0, "import org.apache.avro.generic.GenericArray;");
- line(0, "import org.apache.avro.specific.SpecificExceptionBase;");
- line(0, "import org.apache.avro.specific.SpecificRecordBase;");
- line(0, "import org.apache.avro.specific.SpecificRecord;");
- line(0, "import org.apache.avro.specific.SpecificFixed;");
- line(0, "import org.apache.avro.reflect.FixedSize;");
+ line(out, 0, "import java.nio.ByteBuffer;");
+ line(out, 0, "import java.util.Map;");
+ line(out, 0, "import org.apache.avro.Protocol;");
+ line(out, 0, "import org.apache.avro.Schema;");
+ line(out, 0, "import org.apache.avro.AvroRuntimeException;");
+ line(out, 0, "import org.apache.avro.Protocol;");
+ line(out, 0, "import org.apache.avro.util.Utf8;");
+ line(out, 0, "import org.apache.avro.ipc.AvroRemoteException;");
+ line(out, 0, "import org.apache.avro.generic.GenericArray;");
+ line(out, 0, "import org.apache.avro.specific.SpecificExceptionBase;");
+ line(out, 0, "import org.apache.avro.specific.SpecificRecordBase;");
+ line(out, 0, "import org.apache.avro.specific.SpecificRecord;");
+ line(out, 0, "import org.apache.avro.specific.SpecificFixed;");
+ line(out, 0, "import org.apache.avro.reflect.FixedSize;");
for (Schema s : queue)
if (namespace == null
? (s.getNamespace() != null)
: !namespace.equals(s.getNamespace()))
- line(0, "import "+SpecificData.get().getClassName(s)+";");
- line(0, "");
- line(0, "@SuppressWarnings(\"all\")");
+ line(out, 0, "import "+SpecificData.get().getClassName(s)+";");
+ line(out, 0, "");
+ line(out, 0, "@SuppressWarnings(\"all\")");
}
- private String params(Schema request) throws IOException {
+ private String params(Schema request) {
StringBuilder b = new StringBuilder();
int count = 0;
for (Map.Entry<String, Schema> param : request.getFieldSchemas()) {
@@ -170,7 +216,7 @@
return b.toString();
}
- private String errors(Schema errs) throws IOException {
+ private String errors(Schema errs) {
StringBuilder b = new StringBuilder();
for (Schema error : errs.getTypes().subList(1, errs.getTypes().size())) {
b.append(", ");
@@ -179,70 +225,73 @@
return b.toString();
}
- private void compile(Schema schema) throws IOException {
- startFile(schema.getName(), schema.getNamespace());
- try {
- switch (schema.getType()) {
- case RECORD:
- line(0, "public class "+type(schema)+
- (schema.isError()
- ? " extends SpecificExceptionBase"
- : " extends SpecificRecordBase")
- +" implements SpecificRecord {");
- // schema definition
- line(1, "public static final Schema _SCHEMA = Schema.parse(\""
- +esc(schema)+"\");");
- // field declations
- for (Map.Entry<String, Schema> field : schema.getFieldSchemas())
- line(1,"public "+unbox(field.getValue())+" "+field.getKey()+";");
- // schema method
- line(1, "public Schema getSchema() { return _SCHEMA; }");
- // get method
- line(1, "public Object get(int _field) {");
- line(2, "switch (_field) {");
- int i = 0;
- for (Map.Entry<String, Schema> field : schema.getFieldSchemas())
- line(2, "case "+(i++)+": return "+field.getKey()+";");
- line(2, "default: throw new AvroRuntimeException(\"Bad index\");");
- line(2, "}");
- line(1, "}");
- // set method
- line(1, "@SuppressWarnings(value=\"unchecked\")");
- line(1, "public void set(int _field, Object _value) {");
- line(2, "switch (_field) {");
- i = 0;
- for (Map.Entry<String, Schema> field : schema.getFieldSchemas())
- line(2, "case "+(i++)+": "+field.getKey()+" = ("+
- type(field.getValue())+")_value; break;");
- line(2, "default: throw new AvroRuntimeException(\"Bad index\");");
- line(2, "}");
- line(1, "}");
- line(0, "}");
- break;
- case ENUM:
- line(0, "public enum "+type(schema)+" { ");
- StringBuilder b = new StringBuilder();
- int count = 0;
- for (String symbol : schema.getEnumSymbols()) {
- b.append(symbol);
- if (++count < schema.getEnumSymbols().size())
- b.append(", ");
- }
- line(1, b.toString());
- line(0, "}");
- break;
- case FIXED:
- line(0, "@FixedSize("+schema.getFixedSize()+")");
- line(0, "public class "+type(schema)+" extends SpecificFixed {}");
- break;
- case MAP: case ARRAY: case UNION: case STRING: case BYTES:
- case INT: case LONG: case FLOAT: case DOUBLE: case BOOLEAN: case NULL:
- break;
- default: throw new RuntimeException("Unknown type: "+schema);
+ private OutputFile compile(Schema schema) {
+ OutputFile outputFile = new OutputFile();
+ outputFile.path = makePath(schema.getName(), schema.getNamespace());
+ StringBuilder out = new StringBuilder();
+ header(out, schema.getNamespace());
+ switch (schema.getType()) {
+ case RECORD:
+ line(out, 0, "public class "+type(schema)+
+ (schema.isError()
+ ? " extends SpecificExceptionBase"
+ : " extends SpecificRecordBase")
+ +" implements SpecificRecord {");
+ // schema definition
+ line(out, 1, "public static final Schema _SCHEMA = Schema.parse(\""
+ +esc(schema)+"\");");
+ // field declations
+ for (Map.Entry<String,Schema.Field> field: schema.getFields().entrySet())
+ line(out, 1,
+ "public "+unbox(field.getValue().schema())+"
"+field.getKey()+";");
+ // schema method
+ line(out, 1, "public Schema getSchema() { return _SCHEMA; }");
+ // get method
+ line(out, 1, "public 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 "+field.getKey()+";");
+ line(out, 2, "default: throw new AvroRuntimeException(\"Bad index\");");
+ line(out, 2, "}");
+ line(out, 1, "}");
+ // set method
+ line(out, 1, "@SuppressWarnings(value=\"unchecked\")");
+ line(out, 1, "public void set(int _field, Object _value) {");
+ line(out, 2, "switch (_field) {");
+ i = 0;
+ for (Map.Entry<String, Schema> field : schema.getFieldSchemas())
+ line(out, 2, "case "+(i++)+": "+field.getKey()+" = ("+
+ type(field.getValue())+")_value; break;");
+ line(out, 2, "default: throw new AvroRuntimeException(\"Bad index\");");
+ line(out, 2, "}");
+ line(out, 1, "}");
+ line(out, 0, "}");
+ break;
+ case ENUM:
+ line(out, 0, "public enum "+type(schema)+" { ");
+ StringBuilder b = new StringBuilder();
+ int count = 0;
+ for (String symbol : schema.getEnumSymbols()) {
+ b.append(symbol);
+ if (++count < schema.getEnumSymbols().size())
+ b.append(", ");
}
- } finally {
- out.close();
+ line(out, 1, b.toString());
+ line(out, 0, "}");
+ break;
+ case FIXED:
+ line(out, 0, "@FixedSize("+schema.getFixedSize()+")");
+ line(out, 0, "public class "+type(schema)+" extends SpecificFixed {}");
+ break;
+ case MAP: case ARRAY: case UNION: case STRING: case BYTES:
+ case INT: case LONG: case FLOAT: case DOUBLE: case BOOLEAN: case NULL:
+ break;
+ default: throw new RuntimeException("Unknown type: "+schema);
}
+
+ outputFile.contents = out.toString();
+ return outputFile;
}
private static final Schema NULL_SCHEMA = Schema.create(Schema.Type.NULL);
@@ -285,7 +334,7 @@
}
}
- private void line(int indent, String text) throws IOException {
+ private void line(StringBuilder out, int indent, String text) {
for (int i = 0; i < indent; i ++) {
out.append(" ");
}
@@ -297,7 +346,7 @@
return name.substring(0,1).toUpperCase()+name.substring(1,name.length());
}
- private static String esc(Object o) {
+ static String esc(Object o) {
return o.toString().replace("\"", "\\\"");
}
Modified: hadoop/avro/trunk/src/test/java/org/apache/avro/TestSchema.java
URL:
http://svn.apache.org/viewvc/hadoop/avro/trunk/src/test/java/org/apache/avro/TestSchema.java?rev=828117&r1=828116&r2=828117&view=diff
==============================================================================
--- hadoop/avro/trunk/src/test/java/org/apache/avro/TestSchema.java (original)
+++ hadoop/avro/trunk/src/test/java/org/apache/avro/TestSchema.java Wed Oct 21
17:45:57 2009
@@ -39,10 +39,14 @@
import org.apache.avro.io.BinaryEncoder;
import org.apache.avro.io.JsonDecoder;
import org.apache.avro.io.JsonEncoder;
+import org.apache.avro.specific.TestSpecificCompiler;
import org.apache.avro.util.Utf8;
public class TestSchema {
+ public static final String BASIC_ENUM_SCHEMA = "{\"type\":\"enum\",
\"name\":\"Test\","
+ +"\"symbols\": [\"A\", \"B\"]}";
+
private static final int COUNT =
Integer.parseInt(System.getProperty("test.count", "10"));
@@ -115,9 +119,7 @@
@Test
public void testEnum() throws Exception {
- check("{\"type\":\"enum\", \"name\":\"Test\","
- +"\"symbols\": [\"A\", \"B\"]}", "\"B\"", "B",
- false);
+ check(BASIC_ENUM_SCHEMA, "\"B\"", "B", false);
}
@Test
@@ -209,6 +211,10 @@
checkJson(schema, datum,
new GenericDatumWriter<Object>(),
new GenericDatumReader<Object>());
+
+ // Check that we can generate the code for every
+ // schema we see.
+ assertTrue(null !=
TestSpecificCompiler.compileWithSpecificCompiler(schema));
}
}
Added:
hadoop/avro/trunk/src/test/java/org/apache/avro/specific/TestSpecificCompiler.java
URL:
http://svn.apache.org/viewvc/hadoop/avro/trunk/src/test/java/org/apache/avro/specific/TestSpecificCompiler.java?rev=828117&view=auto
==============================================================================
---
hadoop/avro/trunk/src/test/java/org/apache/avro/specific/TestSpecificCompiler.java
(added)
+++
hadoop/avro/trunk/src/test/java/org/apache/avro/specific/TestSpecificCompiler.java
Wed Oct 21 17:45:57 2009
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro.specific;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.util.Collection;
+
+import org.apache.avro.Schema;
+import org.apache.avro.TestSchema;
+import org.apache.avro.specific.SpecificCompiler.OutputFile;
+import org.junit.Test;
+
+
+public class TestSpecificCompiler {
+ @Test
+ public void testEsc() {
+ assertEquals("\\\"", SpecificCompiler.esc("\""));
+ }
+
+ @Test
+ public void testCap() {
+ assertEquals("Foobar", SpecificCompiler.cap("foobar"));
+ assertEquals("F", SpecificCompiler.cap("f"));
+ assertEquals("F", SpecificCompiler.cap("F"));
+ }
+
+ @Test
+ public void testMakePath() {
+ assertEquals("foo/bar/Baz.java".replace("/", File.separator),
SpecificCompiler.makePath("baz", "foo.bar"));
+ assertEquals("Baz.java", SpecificCompiler.makePath("baz", ""));
+ }
+
+ @Test
+ public void testPrimitiveSchemaGeneratesNothing() {
+ assertEquals(0, new
SpecificCompiler(Schema.parse("\"double\"")).compile().size());
+ }
+
+ @Test
+ public void testSimpleEnumSchema() {
+ Collection<OutputFile> outputs = new
SpecificCompiler(Schema.parse(TestSchema.BASIC_ENUM_SCHEMA)).compile();
+ assertEquals(1, outputs.size());
+ OutputFile o = outputs.iterator().next();
+ assertEquals(o.path, "Test.java");
+ assertTrue(o.contents.contains("public enum Test"));
+ }
+
+ /**
+ * Called from TestSchema as part of its comprehensive checks.
+ */
+ public static Collection<OutputFile>
+ compileWithSpecificCompiler(Schema schema) {
+ return new SpecificCompiler(schema).compile();
+ }
+}