You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@avro.apache.org by cu...@apache.org on 2010/11/01 19:24:36 UTC

svn commit: r1029784 - in /avro/trunk: ./ lang/java/ lang/java/src/java/org/apache/avro/specific/ lang/java/src/java/org/apache/avro/specific/templates/ lang/java/src/java/org/apache/avro/specific/templates/java/ lang/java/src/java/org/apache/avro/spec...

Author: cutting
Date: Mon Nov  1 18:24:36 2010
New Revision: 1029784

URL: http://svn.apache.org/viewvc?rev=1029784&view=rev
Log:
Java: Use Velocity templates to generate specific code.  Contributed by philz.

Added:
    avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/
    avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/
    avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/
    avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/enum.vm
    avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/fixed.vm
    avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/protocol.vm
    avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/record.vm
    avro/trunk/lang/java/src/test/java/org/apache/avro/specific/Avro14SpecificCompiler.java
    avro/trunk/lang/java/src/test/java/org/apache/avro/specific/TestTemplatedCompilerFidelity.java
Modified:
    avro/trunk/CHANGES.txt
    avro/trunk/lang/java/build.xml
    avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificCompiler.java
    avro/trunk/lang/java/src/test/java/org/apache/avro/TestSchema.java
    avro/trunk/lang/java/src/test/java/org/apache/avro/specific/TestSpecificCompiler.java

Modified: avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/avro/trunk/CHANGES.txt?rev=1029784&r1=1029783&r2=1029784&view=diff
==============================================================================
--- avro/trunk/CHANGES.txt (original)
+++ avro/trunk/CHANGES.txt Mon Nov  1 18:24:36 2010
@@ -25,6 +25,9 @@ Avro 1.5.0 (unreleased)
     AVRO-642. Java, Python: Pretty-print schemas in some validation
     error messages.  (Harsh J Chouraria via cutting)
 
+    AVRO-648. Java: Use Velocity templates to generate specific code.
+    (philz via cutting)
+
   BUG FIXES
 
     AVRO-675. C: Bytes and fixed setters don't update datum size.

Modified: avro/trunk/lang/java/build.xml
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/build.xml?rev=1029784&r1=1029783&r2=1029784&view=diff
==============================================================================
--- avro/trunk/lang/java/build.xml (original)
+++ avro/trunk/lang/java/build.xml Mon Nov  1 18:24:36 2010
@@ -42,6 +42,7 @@
   <property name="build.dir" value="${basedir}/build"/>
   <property name="lib.dir" value="${basedir}/lib"/>
   <property name="template.dir" value="${src.dir}/java/org/apache/avro/ipc/stats/templates"/>
+  <property name="specifictemplate.dir" value="${src.dir}/java/org/apache/avro/specific/templates"/>
   <property name="static.dir" value="${src.dir}/java/org/apache/avro/ipc/stats/static"/>
 
 
@@ -50,6 +51,7 @@
   <property name="build.javadoc" value="${build.doc}/api/java"/>
   <property name="build.javadoc.log" value="${build.dir}/javadoc.log"/>
   <property name="build.template.dir" value="${build.classes}/org/apache/avro/ipc/stats/templates"/>
+  <property name="build.specifictemplate.dir" value="${build.classes}/org/apache/avro/specific/templates"/>
   <property name="build.static.dir" value="${build.classes}/org/apache/avro/ipc/stats/static"/>
 
   <property name="test.count" value="100"/>
@@ -161,6 +163,9 @@
   </target>
 
   <target name="compile" depends="javacc,ivy-retrieve">
+    <copy todir="${build.specifictemplate.dir}">
+      <fileset dir="${specifictemplate.dir}" includes="**/**"/>
+    </copy>
     <property name="depends.on.generated"
 	      value="**/ipc/** **/*Requestor.java **/*Responder.java
 		     **/tool/** **/mapred/**"/>
@@ -246,6 +251,7 @@
 	 optimize="${javac.optimize}"
 	 target="${javac.version}"
 	 source="${javac.version}"
+	 includeantruntime="false"
 	 deprecation="${javac.deprecation}">
 	<compilerarg line="${javac.args} ${javac.args.warnings}" />
 	<classpath refid="@{classpath}"/>

Modified: avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificCompiler.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificCompiler.java?rev=1029784&r1=1029783&r2=1029784&view=diff
==============================================================================
--- avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificCompiler.java (original)
+++ avro/trunk/lang/java/src/java/org/apache/avro/specific/SpecificCompiler.java Mon Nov  1 18:24:36 2010
@@ -22,18 +22,20 @@ import java.io.FileWriter;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintStream;
+import java.io.StringWriter;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 
 import org.apache.avro.Protocol;
 import org.apache.avro.Schema;
-import org.apache.avro.Protocol.Message;
 import org.apache.avro.tool.Tool;
+import org.apache.velocity.Template;
+import org.apache.velocity.VelocityContext;
+import org.apache.velocity.app.VelocityEngine;
 
 /**
  * Generate specific Java interfaces and classes for protocols and schemas.
@@ -42,7 +44,9 @@ import org.apache.avro.tool.Tool;
  */
 public class SpecificCompiler {
   private final Set<Schema> queue = new HashSet<Schema>();
-  private final Protocol protocol;
+  private Protocol protocol;
+  private VelocityEngine velocityEngine;
+  private String templateDir;
 
   /* List of Java reserved words from
    * http://java.sun.com/docs/books/jls/third_edition/html/lexical.html. */
@@ -65,6 +69,7 @@ public class SpecificCompiler {
       " */\n";
   
   public SpecificCompiler(Protocol protocol) {
+    this();
     // enqueue all types
     for (Schema s : protocol.getTypes()) {
       enqueue(s);
@@ -73,9 +78,33 @@ public class SpecificCompiler {
   }
 
   public SpecificCompiler(Schema schema) {
+    this();
     enqueue(schema);
     this.protocol = null;
   }
+  
+  SpecificCompiler() {
+    this.templateDir =
+      System.getProperty("org.apache.avro.specific.templates",
+                         "org/apache/avro/specific/templates/java/classic/");
+    initializeVelocity();
+  }
+
+  /** Set the CLASSPATH resource directory where templates reside. */
+  public void setTemplateDir(String templateDir) {
+    this.templateDir = templateDir;
+  }
+
+  private void initializeVelocity() {
+    this.velocityEngine = new VelocityEngine();
+
+    // These two properties tell Velocity to use its own classpath-based
+    // loader
+    velocityEngine.addProperty("resource.loader", "class");
+    velocityEngine.addProperty("class.resource.loader.class",
+        "org.apache.velocity.runtime.resource.loader.ClasspathResourceLoader");
+    velocityEngine.setProperty("runtime.references.strict", true);
+  }
 
   /**
    * Captures output file path and contents.
@@ -122,13 +151,6 @@ public class SpecificCompiler {
     compiler.compileToDestination(src, dest);
   }
 
-  static String mangle(String word) {
-    if (RESERVED_WORDS.contains(word)) {
-      return word + "$";
-    }
-    return word;
-  }
-
   /** Recursively enqueue schemas that need a class generated. */
   private void enqueue(Schema schema) {
     if (queue.contains(schema)) return;
@@ -173,10 +195,10 @@ public class SpecificCompiler {
     return out;
   }
 
-  private void compileToDestination(File src, File dst) throws IOException {
+  /** Generate output under dst, unless existing file is newer than src. */
+  public void compileToDestination(File src, File dst) throws IOException {
     for (Schema schema : queue) {
       OutputFile o = compile(schema);
-      File outputFile = new File(dst, o.path);
       o.writeToDestination(src, dst);
     }
     if (protocol != null) {
@@ -184,180 +206,91 @@ public class SpecificCompiler {
     }
   }
 
-  private OutputFile compileInterface(Protocol protocol) {
+  private String renderTemplate(String templateName, VelocityContext context) {
+    Template template;
+    try {
+      template = this.velocityEngine.getTemplate(templateName);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    StringWriter writer = new StringWriter();
+    try {
+      template.merge(context, writer);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    return writer.toString();
+  }
+
+  OutputFile compileInterface(Protocol protocol) {
+    VelocityContext context = new VelocityContext();
+    context.put("protocol", protocol);
+    context.put("this", this);
+    String out = renderTemplate(templateDir+"protocol.vm", context);
+
     OutputFile outputFile = new OutputFile();
     String mangledName = mangle(protocol.getName());
     outputFile.path = makePath(mangledName, protocol.getNamespace());
-    StringBuilder out = new StringBuilder();
-    header(out, protocol.getNamespace());
-    doc(out, 1, protocol.getDoc());
-    line(out, 0, "public interface " + mangledName + " {");
-    line(out, 1, "public static final org.apache.avro.Protocol PROTOCOL = org.apache.avro.Protocol.parse(\""
-           +esc(protocol)+"\");");
-    for (Map.Entry<String,Message> e : protocol.getMessages().entrySet()) {
-      String name = e.getKey();
-      Message message = e.getValue();
-      Schema request = message.getRequest();
-      String response = message.isOneWay() ? "void"
-        : unbox(message.getResponse());
-      doc(out, 1, e.getValue().getDoc());
-      line(out, 1, response+" "+ mangle(name)+"("+params(request)+")"
-           + (message.isOneWay() ? ""
-              : (" throws org.apache.avro.ipc.AvroRemoteException"
-                 +errors(message.getErrors())))
-           +";");
-    }
-    line(out, 0, "}");
-
-    outputFile.contents = out.toString();
+    outputFile.contents = out;
     return outputFile;
   }
 
-
   static String makePath(String name, String space) {
     if (space == null || space.isEmpty()) {
       return name + ".java";
     } else {
-      return space.replace('.', File.separatorChar) + File.separatorChar
-        + name + ".java";
+      return space.replace('.', File.separatorChar) + File.separatorChar + name
+          + ".java";
     }
   }
 
-  private void header(StringBuilder out, String namespace) {
-    if(namespace != null) {
-      line(out, 0, "package "+namespace+";\n");
-    }
-    line(out, 0, "@SuppressWarnings(\"all\")");
-  }
-
-  private String params(Schema request) {
-    StringBuilder b = new StringBuilder();
-    int count = 0;
-    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())
-        b.append(", ");
-    }
-    return b.toString();
-  }
+  OutputFile compile(Schema schema) {
+    String output = "";
+    VelocityContext context = new VelocityContext();
+    context.put("this", this);
+    context.put("schema", schema);
 
-  private String errors(Schema errs) {
-    StringBuilder b = new StringBuilder();
-    for (Schema error : errs.getTypes().subList(1, errs.getTypes().size())) {
-      b.append(", ");
-      b.append(mangle(error.getFullName()));
-    }
-    return b.toString();
-  }
-
-  private OutputFile compile(Schema schema) {
-    OutputFile outputFile = new OutputFile();
-    String name = mangle(schema.getName());
-    outputFile.path = makePath(name, schema.getNamespace());
-    StringBuilder out = new StringBuilder();
-    header(out, schema.getNamespace());
     switch (schema.getType()) {
     case RECORD:
-      doc(out, 0, schema.getDoc());
-      line(out, 0, "public class "+name+
-           (schema.isError()
-            ? " extends org.apache.avro.specific.SpecificExceptionBase"
-             : " extends org.apache.avro.specific.SpecificRecordBase")
-           +" implements org.apache.avro.specific.SpecificRecord {");
-      // schema definition
-      line(out, 1, "public static final org.apache.avro.Schema SCHEMA$ = org.apache.avro.Schema.parse(\""
-           +esc(schema)+"\");");
-      // field declations
-      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$; }");
-      // get method
-      line(out, 1, "// Used by DatumWriter.  Applications should not call. ");
-      line(out, 1, "public java.lang.Object get(int field$) {");
-      line(out, 2, "switch (field$) {");
-      int i = 0;
-      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, "}");
-      // put method
-      line(out, 1, "// Used by DatumReader.  Applications should not call. ");
-      line(out, 1, "@SuppressWarnings(value=\"unchecked\")");
-      line(out, 1, "public void put(int field$, java.lang.Object value$) {");
-      line(out, 2, "switch (field$) {");
-      i = 0;
-      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, "}");
-      line(out, 0, "}");
+      output = renderTemplate(templateDir+"record.vm", context);
       break;
     case ENUM:
-      doc(out, 0, schema.getDoc());
-      line(out, 0, "public enum "+name+" { ");
-      StringBuilder b = new StringBuilder();
-      int count = 0;
-      for (String symbol : schema.getEnumSymbols()) {
-        b.append(mangle(symbol));
-        if (++count < schema.getEnumSymbols().size())
-          b.append(", ");
-      }
-      line(out, 1, b.toString());
-      line(out, 0, "}");
+      output = renderTemplate(templateDir+"enum.vm", context);
       break;
     case FIXED:
-      doc(out, 0, schema.getDoc());
-      line(out, 0, "@org.apache.avro.specific.FixedSize("+schema.getFixedSize()+")");
-      line(out, 0, "public class "+name+" extends org.apache.avro.specific.SpecificFixed {}");
+      output = renderTemplate(templateDir+"fixed.vm", context);
       break;
-    case MAP: case ARRAY: case UNION: case STRING: case BYTES:
-    case INT: case LONG: case FLOAT: case DOUBLE: case BOOLEAN: case NULL:
+    case BOOLEAN:
+    case NULL:
       break;
     default: throw new RuntimeException("Unknown type: "+schema);
     }
 
-    outputFile.contents = out.toString();
+    OutputFile outputFile = new OutputFile();
+    String name = mangle(schema.getName());
+    outputFile.path = makePath(name, schema.getNamespace());
+    outputFile.contents = output;
     return outputFile;
   }
 
-  private void doc(StringBuilder out, int indent, String doc) {
-    if (doc != null) {
-      line(out, indent, "/** " + escapeForJavaDoc(doc) + " */");
-    }
-  }
-
-  /** Be sure that generated code will compile by replacing
-   * end-comment markers with the appropriate HTML entity. */
-  private String escapeForJavaDoc(String doc) {
-    return doc.replace("*/", "*&#47;");
-  }
-
   private static final Schema NULL_SCHEMA = Schema.create(Schema.Type.NULL);
 
-  private String type(Schema schema) {
+  /** Utility for template use.  Returns the java type for a Schema. */
+  public static String javaType(Schema schema) {
     switch (schema.getType()) {
     case RECORD:
     case ENUM:
     case FIXED:
       return mangle(schema.getFullName());
     case ARRAY:
-      return "java.util.List<"+type(schema.getElementType())+">";
+      return "java.util.List<" + javaType(schema.getElementType()) + ">";
     case MAP:
-      return "java.util.Map<java.lang.CharSequence,"+type(schema.getValueType())+">";
+      return "java.util.Map<java.lang.CharSequence,"
+          + javaType(schema.getValueType()) + ">";
     case UNION:
-      List<Schema> types = schema.getTypes();     // elide unions with null
+      List<Schema> types = schema.getTypes(); // elide unions with null
       if ((types.size() == 2) && types.contains(NULL_SCHEMA))
-        return type(types.get(types.get(0).equals(NULL_SCHEMA) ? 1 : 0));
+        return javaType(types.get(types.get(0).equals(NULL_SCHEMA) ? 1 : 0));
       return "java.lang.Object";
     case STRING:  return "java.lang.CharSequence";
     case BYTES:   return "java.nio.ByteBuffer";
@@ -371,27 +304,39 @@ public class SpecificCompiler {
     }
   }
 
-  private String unbox(Schema schema) {
+  /** Utility for template use.  Returns the unboxed java type for a Schema. */
+  public static String javaUnbox(Schema schema) {
     switch (schema.getType()) {
     case INT:     return "int";
     case LONG:    return "long";
     case FLOAT:   return "float";
     case DOUBLE:  return "double";
     case BOOLEAN: return "boolean";
-    default:      return type(schema);
+    default:      return javaType(schema);
     }
   }
 
-  private void line(StringBuilder out, int indent, String text) {
-    for (int i = 0; i < indent; i ++) {
-      out.append("  ");
-    }
-    out.append(text);
-    out.append("\n");
+  /** Utility for template use.  Escapes quotes in java strings. */
+  public static String javaEscape(Object o) {
+    return o.toString().replace("\"", "\\\"");
   }
 
-  static String esc(Object o) {
-    return o.toString().replace("\"", "\\\"");
+  /** Utility for template use.  Escapes comment end with HTML entities. */
+  public static String escapeForJavadoc(String s) {
+      return s.replace("*/", "*&#47;");
+  }
+  
+  /** Utility for template use.  Returns empty string for null. */
+  public static String nullToEmpty(String x) {
+    return x == null ? "" : x;
+  }
+
+  /** Utility for template use.  Adds a dollar sign to reserved words. */
+  public static String mangle(String word) {
+    if (RESERVED_WORDS.contains(word)) {
+      return word + "$";
+    }
+    return word;
   }
 
   public static void main(String[] args) throws Exception {

Added: avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/enum.vm
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/enum.vm?rev=1029784&view=auto
==============================================================================
--- avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/enum.vm (added)
+++ avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/enum.vm Mon Nov  1 18:24:36 2010
@@ -0,0 +1,28 @@
+##
+## 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.
+##
+#if ($schema.getNamespace())
+package $schema.getNamespace();  
+#end  
+@SuppressWarnings("all")
+#if ($schema.getDoc())
+/** $schema.getDoc() */
+#end
+public enum ${this.mangle($schema.getName())} { 
+  #foreach ($symbol in ${schema.getEnumSymbols()})${this.mangle($symbol)}#if ($velocityHasNext), #end#end
+
+}

Added: avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/fixed.vm
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/fixed.vm?rev=1029784&view=auto
==============================================================================
--- avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/fixed.vm (added)
+++ avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/fixed.vm Mon Nov  1 18:24:36 2010
@@ -0,0 +1,26 @@
+##
+## 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.
+##
+#if ($schema.getNamespace())
+package $schema.getNamespace();  
+#end
+@SuppressWarnings("all")
+#if ($schema.getDoc())
+/** $schema.getDoc() */
+#end
+@org.apache.avro.specific.FixedSize($schema.getFixedSize())
+public class ${this.mangle($schema.getName())} extends org.apache.avro.specific.SpecificFixed {}

Added: avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/protocol.vm
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/protocol.vm?rev=1029784&view=auto
==============================================================================
--- avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/protocol.vm (added)
+++ avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/protocol.vm Mon Nov  1 18:24:36 2010
@@ -0,0 +1,49 @@
+##
+## 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.
+##
+#if ($protocol.getNamespace())
+package $protocol.getNamespace();
+#end
+
+@SuppressWarnings("all")
+#if ($protocol.getDoc())
+  /** $protocol.getDoc() */
+#end
+public interface $this.mangle($protocol.getName()) {
+  public static final org.apache.avro.Protocol PROTOCOL = org.apache.avro.Protocol.parse("${this.javaEscape($protocol.toString())}");
+#foreach ($e in $protocol.getMessages().entrySet())
+#set ($name = $e.getKey())
+#set ($message = $e.getValue())
+#set ($response = $message.getResponse())
+#if ($message.getDoc())
+  /** $this.escapeForJavadoc($message.getDoc()) */
+#end
+  #if ($message.isOneWay())void#else${this.javaUnbox($response)}#end
+ ${this.mangle($name)}(##
+#foreach ($p in $message.getRequest().getFields())##
+#*      *#${this.javaUnbox($p.schema())} ${this.mangle($p.name())}#if ($velocityHasNext), #end
+#end
+)#if (! $message.isOneWay())
+ throws org.apache.avro.ipc.AvroRemoteException##
+## The first error is always "string", so we skip it.
+#foreach ($error in $message.getErrors().getTypes().subList(1, $message.getErrors().getTypes().size()))
+, ${this.mangle($error.getFullName())}##
+#end##    (error list)
+#end##    (one way)
+;
+#end## (requests)
+}

Added: avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/record.vm
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/record.vm?rev=1029784&view=auto
==============================================================================
--- avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/record.vm (added)
+++ avro/trunk/lang/java/src/java/org/apache/avro/specific/templates/java/classic/record.vm Mon Nov  1 18:24:36 2010
@@ -0,0 +1,57 @@
+##
+## 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.
+##
+#if ($schema.getNamespace())
+package $schema.getNamespace();  
+#end
+@SuppressWarnings("all")
+#if ($schema.getDoc())
+/** $schema.getDoc() */
+#end
+public class ${this.mangle($schema.getName())}#if ($schema.isError()) extends org.apache.avro.specific.SpecificExceptionBase#else extends org.apache.avro.specific.SpecificRecordBase#end implements org.apache.avro.specific.SpecificRecord {
+  public static final org.apache.avro.Schema SCHEMA$ = org.apache.avro.Schema.parse("${this.javaEscape($schema.toString())}");
+#foreach ($field in $schema.getFields())
+#if ($field.doc())
+  /** $field.doc() */
+#end
+  public ${this.javaUnbox($field.schema())} ${this.mangle($field.name())};
+#end
+  public org.apache.avro.Schema getSchema() { return SCHEMA$; }
+  // Used by DatumWriter.  Applications should not call. 
+  public java.lang.Object get(int field$) {
+    switch (field$) {
+#set ($i = 0)
+#foreach ($field in $schema.getFields())
+    case $i: return ${this.mangle($field.name())};
+#set ($i = $i + 1)
+#end
+    default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+    }
+  }
+  // Used by DatumReader.  Applications should not call. 
+  @SuppressWarnings(value="unchecked")
+  public void put(int field$, java.lang.Object value$) {
+    switch (field$) {
+#set ($i = 0)
+#foreach ($field in $schema.getFields())
+    case $i: ${this.mangle($field.name())} = (${this.javaType($field.schema())})value$; break;
+#set ($i = $i + 1)
+#end
+    default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+    }
+  }
+}

Modified: avro/trunk/lang/java/src/test/java/org/apache/avro/TestSchema.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/test/java/org/apache/avro/TestSchema.java?rev=1029784&r1=1029783&r2=1029784&view=diff
==============================================================================
--- avro/trunk/lang/java/src/test/java/org/apache/avro/TestSchema.java (original)
+++ avro/trunk/lang/java/src/test/java/org/apache/avro/TestSchema.java Mon Nov  1 18:24:36 2010
@@ -52,6 +52,12 @@ import org.apache.avro.util.Utf8;
 
 public class TestSchema {
 
+  public static final String LISP_SCHEMA = "{\"type\": \"record\", \"name\": \"Lisp\", \"fields\": ["
+            +"{\"name\":\"value\", \"type\":[\"null\", \"string\","
+            +"{\"type\": \"record\", \"name\": \"Cons\", \"fields\": ["
+            +"{\"name\":\"car\", \"type\":\"Lisp\"},"
+            +"{\"name\":\"cdr\", \"type\":\"Lisp\"}]}]}]}";
+
   public static final String BASIC_ENUM_SCHEMA = "{\"type\":\"enum\", \"name\":\"Test\","
             +"\"symbols\": [\"A\", \"B\"]}";
 
@@ -262,12 +268,7 @@ public class TestSchema {
 
   @Test
   public void testLisp() throws Exception {
-    check("{\"type\": \"record\", \"name\": \"Lisp\", \"fields\": ["
-          +"{\"name\":\"value\", \"type\":[\"null\", \"string\","
-          +"{\"type\": \"record\", \"name\": \"Cons\", \"fields\": ["
-          +"{\"name\":\"car\", \"type\":\"Lisp\"},"
-          +"{\"name\":\"cdr\", \"type\":\"Lisp\"}]}]}]}",
-          false);
+    check(LISP_SCHEMA, false);
   }
 
   @Test

Added: avro/trunk/lang/java/src/test/java/org/apache/avro/specific/Avro14SpecificCompiler.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/test/java/org/apache/avro/specific/Avro14SpecificCompiler.java?rev=1029784&view=auto
==============================================================================
--- avro/trunk/lang/java/src/test/java/org/apache/avro/specific/Avro14SpecificCompiler.java (added)
+++ avro/trunk/lang/java/src/test/java/org/apache/avro/specific/Avro14SpecificCompiler.java Mon Nov  1 18:24:36 2010
@@ -0,0 +1,277 @@
+/**
+ * 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 java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.avro.Protocol;
+import org.apache.avro.Protocol.Message;
+import org.apache.avro.Schema;
+import org.apache.avro.specific.SpecificCompiler.OutputFile;
+
+/**
+ * Copy of "legacy" Avro 1.4 Specific Compiler.
+ * This is used by a test to ensure that the
+ * templated version corresponds to the 
+ * new version.  After the template version proves
+ * itself out, it would be reasonable to remove both
+ * this class and the comparison.
+ * 
+ * The logic to handle multiple schemas has
+ * been stripped out, since this is used
+ * exclusively for comparing the outputs.
+ */
+public class Avro14SpecificCompiler {
+  /* List of Java reserved words from
+   * http://java.sun.com/docs/books/jls/third_edition/html/lexical.html. */
+  private static final Set<String> RESERVED_WORDS = new HashSet<String>(
+      Arrays.asList(new String[] {
+          "abstract", "assert", "boolean", "break", "byte", "case", "catch",
+          "char", "class", "const", "continue", "default", "do", "double",
+          "else", "enum", "extends", "false", "final", "finally", "float",
+          "for", "goto", "if", "implements", "import", "instanceof", "int",
+          "interface", "long", "native", "new", "null", "package", "private",
+          "protected", "public", "return", "short", "static", "strictfp",
+          "super", "switch", "synchronized", "this", "throw", "throws",
+          "transient", "true", "try", "void", "volatile", "while"
+        }));
+
+  static String mangle(String word) {
+    if (RESERVED_WORDS.contains(word)) {
+      return word + "$";
+    }
+    return word;
+  }
+
+  OutputFile compileInterface(Protocol protocol) {
+    OutputFile outputFile = new OutputFile();
+    String mangledName = mangle(protocol.getName());
+    outputFile.path = makePath(mangledName, protocol.getNamespace());
+    StringBuilder out = new StringBuilder();
+    header(out, protocol.getNamespace());
+    doc(out, 1, protocol.getDoc());
+    line(out, 0, "public interface " + mangledName + " {");
+    line(out, 1, "public static final org.apache.avro.Protocol PROTOCOL = org.apache.avro.Protocol.parse(\""
+           +esc(protocol)+"\");");
+    for (Map.Entry<String,Message> e : protocol.getMessages().entrySet()) {
+      String name = e.getKey();
+      Message message = e.getValue();
+      Schema request = message.getRequest();
+      String response = message.isOneWay() ? "void"
+        : unbox(message.getResponse());
+      doc(out, 1, e.getValue().getDoc());
+      line(out, 1, response+" "+ mangle(name)+"("+params(request)+")"
+           + (message.isOneWay() ? ""
+              : (" throws org.apache.avro.ipc.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 name + ".java";
+    } else {
+      return space.replace('.', File.separatorChar) + File.separatorChar
+        + name + ".java";
+    }
+  }
+
+  private void header(StringBuilder out, String namespace) {
+    if(namespace != null) {
+      line(out, 0, "package "+namespace+";\n");
+    }
+    line(out, 0, "@SuppressWarnings(\"all\")");
+  }
+
+  private String params(Schema request) {
+    StringBuilder b = new StringBuilder();
+    int count = 0;
+    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())
+        b.append(", ");
+    }
+    return b.toString();
+  }
+
+  private String errors(Schema errs) {
+    StringBuilder b = new StringBuilder();
+    for (Schema error : errs.getTypes().subList(1, errs.getTypes().size())) {
+      b.append(", ");
+      b.append(mangle(error.getFullName()));
+    }
+    return b.toString();
+  }
+
+  OutputFile compile(Schema schema) {
+    OutputFile outputFile = new OutputFile();
+    String name = mangle(schema.getName());
+    outputFile.path = makePath(name, schema.getNamespace());
+    StringBuilder out = new StringBuilder();
+    header(out, schema.getNamespace());
+    switch (schema.getType()) {
+    case RECORD:
+      doc(out, 0, schema.getDoc());
+      line(out, 0, "public class "+name+
+           (schema.isError()
+            ? " extends org.apache.avro.specific.SpecificExceptionBase"
+             : " extends org.apache.avro.specific.SpecificRecordBase")
+           +" implements org.apache.avro.specific.SpecificRecord {");
+      // schema definition
+      line(out, 1, "public static final org.apache.avro.Schema SCHEMA$ = org.apache.avro.Schema.parse(\""
+           +esc(schema)+"\");");
+      // field declations
+      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$; }");
+      // get method
+      line(out, 1, "// Used by DatumWriter.  Applications should not call. ");
+      line(out, 1, "public java.lang.Object get(int field$) {");
+      line(out, 2, "switch (field$) {");
+      int i = 0;
+      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, "}");
+      // put method
+      line(out, 1, "// Used by DatumReader.  Applications should not call. ");
+      line(out, 1, "@SuppressWarnings(value=\"unchecked\")");
+      line(out, 1, "public void put(int field$, java.lang.Object value$) {");
+      line(out, 2, "switch (field$) {");
+      i = 0;
+      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, "}");
+      line(out, 0, "}");
+      break;
+    case ENUM:
+      doc(out, 0, schema.getDoc());
+      line(out, 0, "public enum "+name+" { ");
+      StringBuilder b = new StringBuilder();
+      int count = 0;
+      for (String symbol : schema.getEnumSymbols()) {
+        b.append(mangle(symbol));
+        if (++count < schema.getEnumSymbols().size())
+          b.append(", ");
+      }
+      line(out, 1, b.toString());
+      line(out, 0, "}");
+      break;
+    case FIXED:
+      doc(out, 0, schema.getDoc());
+      line(out, 0, "@org.apache.avro.specific.FixedSize("+schema.getFixedSize()+")");
+      line(out, 0, "public class "+name+" extends org.apache.avro.specific.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 void doc(StringBuilder out, int indent, String doc) {
+    if (doc != null) {
+      line(out, indent, "/** " + escapeForJavaDoc(doc) + " */");
+    }
+  }
+
+  /** Be sure that generated code will compile by replacing
+   * end-comment markers with the appropriate HTML entity. */
+  private String escapeForJavaDoc(String doc) {
+    return doc.replace("*/", "*&#47;");
+  }
+
+  private static final Schema NULL_SCHEMA = Schema.create(Schema.Type.NULL);
+
+  private String type(Schema schema) {
+    switch (schema.getType()) {
+    case RECORD:
+    case ENUM:
+    case FIXED:
+      return mangle(schema.getFullName());
+    case ARRAY:
+      return "java.util.List<"+type(schema.getElementType())+">";
+    case MAP:
+      return "java.util.Map<java.lang.CharSequence,"+type(schema.getValueType())+">";
+    case UNION:
+      List<Schema> types = schema.getTypes();     // elide unions with null
+      if ((types.size() == 2) && types.contains(NULL_SCHEMA))
+        return type(types.get(types.get(0).equals(NULL_SCHEMA) ? 1 : 0));
+      return "java.lang.Object";
+    case STRING:  return "java.lang.CharSequence";
+    case BYTES:   return "java.nio.ByteBuffer";
+    case INT:     return "java.lang.Integer";
+    case LONG:    return "java.lang.Long";
+    case FLOAT:   return "java.lang.Float";
+    case DOUBLE:  return "java.lang.Double";
+    case BOOLEAN: return "java.lang.Boolean";
+    case NULL:    return "java.lang.Void";
+    default: throw new RuntimeException("Unknown type: "+schema);
+    }
+  }
+
+  private String unbox(Schema schema) {
+    switch (schema.getType()) {
+    case INT:     return "int";
+    case LONG:    return "long";
+    case FLOAT:   return "float";
+    case DOUBLE:  return "double";
+    case BOOLEAN: return "boolean";
+    default:      return type(schema);
+    }
+  }
+
+  private void line(StringBuilder out, int indent, String text) {
+    for (int i = 0; i < indent; i ++) {
+      out.append("  ");
+    }
+    out.append(text);
+    out.append("\n");
+  }
+
+  static String esc(Object o) {
+    return o.toString().replace("\"", "\\\"");
+  }
+
+}
+

Modified: avro/trunk/lang/java/src/test/java/org/apache/avro/specific/TestSpecificCompiler.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/test/java/org/apache/avro/specific/TestSpecificCompiler.java?rev=1029784&r1=1029783&r2=1029784&view=diff
==============================================================================
--- avro/trunk/lang/java/src/test/java/org/apache/avro/specific/TestSpecificCompiler.java (original)
+++ avro/trunk/lang/java/src/test/java/org/apache/avro/specific/TestSpecificCompiler.java Mon Nov  1 18:24:36 2010
@@ -43,9 +43,27 @@ import org.junit.Test;
 
 
 public class TestSpecificCompiler {
+  static final String PROTOCOL = "" +
+        "{ \"protocol\": \"default\",\n" +
+        "  \"types\":\n" +
+        "    [\n" +
+        "      {\n" +
+        "       \"name\": \"finally\",\n" +
+        "       \"type\": \"error\",\n" +
+        "       \"fields\": [{\"name\": \"catch\", \"type\": \"boolean\"}]\n" +
+        "      }\n" +
+        "    ],\n" +
+        "  \"messages\": { \"goto\":\n" +
+        "    { \"request\": [{\"name\": \"break\", \"type\": \"string\"}],\n" +
+        "      \"response\": \"string\",\n" +
+        "      \"errors\": [\"finally\"]\n" +
+        "    }" +
+        "   }\n" +
+        "}\n";
+
   @Test
   public void testEsc() {
-    assertEquals("\\\"", SpecificCompiler.esc("\""));
+    assertEquals("\\\"", new SpecificCompiler().javaEscape("\""));
   }
 
   @Test
@@ -77,23 +95,7 @@ public class TestSpecificCompiler {
 
   @Test
   public void testManglingForProtocols() throws IOException {
-    String protocolDef = "" +
-      "{ \"protocol\": \"default\",\n" +
-      "  \"types\":\n" +
-      "    [\n" +
-      "      {\n" +
-      "       \"name\": \"finally\",\n" +
-      "       \"type\": \"error\",\n" +
-      "       \"fields\": [{\"name\": \"catch\", \"type\": \"boolean\"}]\n" +
-      "      }\n" +
-      "    ],\n" +
-      "  \"messages\": { \"goto\":\n" +
-      "    { \"request\": [{\"name\": \"break\", \"type\": \"string\"}],\n" +
-      "      \"response\": \"string\",\n" +
-      "      \"errors\": [\"finally\"]\n" +
-      "    }" +
-      "   }\n" +
-      "}\n";
+    String protocolDef = PROTOCOL;
     Collection<OutputFile> c =
       new SpecificCompiler(Protocol.parse(protocolDef)).compile();
     Iterator<OutputFile> i = c.iterator();

Added: avro/trunk/lang/java/src/test/java/org/apache/avro/specific/TestTemplatedCompilerFidelity.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/test/java/org/apache/avro/specific/TestTemplatedCompilerFidelity.java?rev=1029784&view=auto
==============================================================================
--- avro/trunk/lang/java/src/test/java/org/apache/avro/specific/TestTemplatedCompilerFidelity.java (added)
+++ avro/trunk/lang/java/src/test/java/org/apache/avro/specific/TestTemplatedCompilerFidelity.java Mon Nov  1 18:24:36 2010
@@ -0,0 +1,85 @@
+/**
+ * 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 java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+
+import org.apache.avro.Protocol;
+import org.apache.avro.Schema;
+import org.apache.avro.TestProtocolParsing;
+import org.apache.avro.TestSchema;
+import org.junit.Test;
+
+public class TestTemplatedCompilerFidelity {
+  SpecificCompiler templated = new SpecificCompiler();
+  Avro14SpecificCompiler legacy = new Avro14SpecificCompiler();
+  /** If true, records output to /tmp, for diffing with real tools. */
+  private final static boolean DEBUG = true;
+
+  @Test
+  public void testSchemas() throws IOException {
+    Schema[] schemas = new Schema[] {
+        Schema.parse(TestSchema.BASIC_ENUM_SCHEMA),
+        Schema.parse(TestSchema.SCHEMA_WITH_DOC_TAGS),
+        Schema.parse(TestSchema.SCHEMA_WITH_DOC_TAGS).getField("inner_fixed").schema(),
+        Schema.parse(TestSchema.SCHEMA_WITH_DOC_TAGS).getField("inner_enum").schema(),
+        Schema.parse(TestSchema.LISP_SCHEMA),
+        // The error schema in the following protocol
+        Protocol.parse(TestSpecificCompiler.PROTOCOL).getMessages().get("goto").getErrors().getTypes().get(1)
+    };
+    for (Schema s : schemas) {
+      String expected = legacy.compile(s).contents;
+      String actual = templated.compile(s).contents;
+      debug(expected, actual);
+      assertEquals(expected, actual);
+    }
+    
+  }
+  
+  @Test
+  public void testProtocols() throws IOException {
+    Protocol p = TestProtocolParsing.getSimpleProtocol();
+    String expected = legacy.compileInterface(p).contents;
+    String actual = templated.compileInterface(p).contents;
+    debug(expected, actual);
+    assertEquals(expected, actual);
+  }
+
+  /** Because asserts are pretty poor for debugging, optionally write to
+   * temporary files. 
+   */
+  @SuppressWarnings("unused")
+  private void debug(String expected, String actual) throws IOException {
+    if (expected.equals(actual)) return;
+    
+    if (DEBUG) {
+      writeToFile(new File("/tmp/a"), expected);
+      writeToFile(new File("/tmp/b"), actual);
+    }
+  }
+
+  private void writeToFile(File file, String expected) throws IOException {
+    FileWriter fw = new FileWriter(file);
+    fw.write(expected);
+    fw.close();
+  }
+}