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/01/08 20:49:52 UTC

svn commit: r897311 - in /hadoop/avro/trunk: ./ src/doc/content/xdocs/ src/java/org/apache/avro/ src/java/org/apache/avro/genavro/ src/java/org/apache/avro/generic/ src/java/org/apache/avro/reflect/ src/java/org/apache/avro/specific/ src/test/java/org/...

Author: cutting
Date: Fri Jan  8 19:49:51 2010
New Revision: 897311

URL: http://svn.apache.org/viewvc?rev=897311&view=rev
Log:
AVRO-152.  Add support for documentation strings to schemas, protocols and messages.  Contributed by Philip Zeyliger.

Added:
    hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolParsing.java
Modified:
    hadoop/avro/trunk/CHANGES.txt
    hadoop/avro/trunk/src/doc/content/xdocs/spec.xml
    hadoop/avro/trunk/src/java/org/apache/avro/Protocol.java
    hadoop/avro/trunk/src/java/org/apache/avro/Schema.java
    hadoop/avro/trunk/src/java/org/apache/avro/genavro/genavro.jj
    hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericData.java
    hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectData.java
    hadoop/avro/trunk/src/java/org/apache/avro/specific/SpecificCompiler.java
    hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolGeneric.java
    hadoop/avro/trunk/src/test/java/org/apache/avro/TestSchema.java
    hadoop/avro/trunk/src/test/java/org/apache/avro/specific/TestSpecificCompiler.java
    hadoop/avro/trunk/src/test/schemata/simple.avpr

Modified: hadoop/avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/CHANGES.txt?rev=897311&r1=897310&r2=897311&view=diff
==============================================================================
--- hadoop/avro/trunk/CHANGES.txt (original)
+++ hadoop/avro/trunk/CHANGES.txt Fri Jan  8 19:49:51 2010
@@ -53,6 +53,9 @@
     AVRO-273, AVRO-275, & AVRO-279. Add Java RPC statistics collection
     and display. (Philip Zeyliger via cutting)
 
+    AVRO-152. Add support for documentation strings to schemas,
+    protocols, and messages. (Philip Zeyliger via cutting)
+
   IMPROVEMENTS
 
     AVRO-157. Changes from code review comments for C++. (sbanacho)

Modified: hadoop/avro/trunk/src/doc/content/xdocs/spec.xml
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/doc/content/xdocs/spec.xml?rev=897311&r1=897310&r2=897311&view=diff
==============================================================================
--- hadoop/avro/trunk/src/doc/content/xdocs/spec.xml (original)
+++ hadoop/avro/trunk/src/doc/content/xdocs/spec.xml Fri Jan  8 19:49:51 2010
@@ -86,16 +86,20 @@
         <section id="schema_record">
           <title>Records</title>
           
-	  <p>Records use the type name "record" and support two attributes:</p>
+	  <p>Records use the type name "record" and support three attributes:</p>
 	  <ul>
 	    <li><code>name</code>: a JSON string providing the name
 	    of the record (required).</li>
 	    <li><em>namespace</em>, a JSON string that qualifies the name;</li>
+	    <li><code>doc</code>: a JSON string providing documentation to the
+	    user of this schema (optional).</li>
 	    <li><code>fields</code>: a JSON array, listing fields (required).
 	    Each field is a JSON object with the following attributes:
 	      <ul>
 		<li><code>name</code>: a JSON string providing the name
 		  of the field (required), and </li>
+		<li><code>doc</code>: a JSON string describing this field
+                  for users (optional).</li>
 		<li><code>type:</code> A JSON object defining a schema, or
 		  a JSON string naming a record definition
 		  (required).</li>
@@ -153,6 +157,8 @@
 	    <li><code>name</code>: a JSON string providing the name
 	    of the enum (required).</li>
 	    <li><em>namespace</em>, a JSON string that qualifies the name;</li>
+	    <li><code>doc</code>: a JSON string providing documentation to the
+	    user of this schema (optional).</li>
 	    <li><code>symbols</code>: a JSON array, listing symbols,
 	    as JSON strings (required).</li>
 	  </ul>
@@ -650,6 +656,7 @@
       <ul>
 	<li><em>name</em>, string, to distinguish it from other protocols;</li>
 	<li><em>namespace</em>, a string that qualifies the name;</li>
+	<li><em>doc</em>, an optional string, describing this protocol;</li>
 	<li><em>types</em>, a list of record, enum and error
 	  definitions.  An error definition is just like a record
 	  definition except it uses "error" instead of "record".  Note
@@ -664,6 +671,7 @@
 	<title>Messages</title>
 	<p>A message has attributes:</p>
 	<ul>
+          <li>a <em>doc</em>, an optional description of the message,</li>
 	  <li>a <em>request</em>, a list of named,
 	    typed <em>parameter</em> schemas (this has the same form
 	    as the fields of a record declaration);</li>
@@ -683,6 +691,7 @@
 {
   "namespace": "com.acme",
   "protocol": "HelloWorld",
+  "doc": "Protocol Greetings",
 
   "types": [
     {"name": "Greeting", "type": "record", "fields": [
@@ -693,6 +702,7 @@
 
   "messages": {
     "hello": {
+      "doc": "Say hello.",
       "request": [{"name": "greeting", "type": "Greeting" }],
       "response": "Greeting",
       "errors": ["Curse"]
@@ -1000,6 +1010,10 @@
 	</li>
 	  
       </ul>
+
+      <p>A schema's "doc" fields are ignored for the purposes of schema resolution.  Hence,
+        the "doc" portion of a schema may be dropped at serialization.</p>
+
     </section>
   </body>
 </document>

Modified: hadoop/avro/trunk/src/java/org/apache/avro/Protocol.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/Protocol.java?rev=897311&r1=897310&r2=897311&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/Protocol.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/Protocol.java Fri Jan  8 19:49:51 2010
@@ -56,14 +56,16 @@
   /** A protocol message. */
   public class Message {
     private String name;
+    private String doc;
     private Schema request;
     private Schema response;
     private Schema errors;
     
     /** Construct a message. */
-    private Message(String name, Schema request,
+    private Message(String name, String doc, Schema request,
                     Schema response, Schema errors) {
       this.name = name;
+      this.doc = doc;
       this.request = request;
       this.response = response;
       this.errors = errors;
@@ -123,10 +125,15 @@
         + request.hashCode() + response.hashCode() + errors.hashCode();
     }
 
+    public String getDoc() {
+      return doc;
+    }
+
   }
 
   private String name;
   private String namespace;
+  private String doc;
 
   private Schema.Names types = new Schema.Names();
   private Map<String,Message> messages = new LinkedHashMap<String,Message>();
@@ -155,6 +162,9 @@
 
   /** The namespace of this protocol.  Qualifies its name. */
   public String getNamespace() { return namespace; }
+  
+  /** Doc string for this protocol. */
+  public String getDoc() { return doc; }
 
   /** The types of this protocol. */
   public Collection<Schema> getTypes() { return types.values(); }
@@ -172,9 +182,9 @@
   /** The messages of this protocol. */
   public Map<String,Message> getMessages() { return messages; }
 
-  public Message createMessage(String name, Schema request,
+  public Message createMessage(String name, String doc, Schema request,
                                Schema response, Schema errors) {
-    return new Message(name, request, response, errors);
+    return new Message(name, doc, request, response, errors);
   }
 
 
@@ -277,6 +287,7 @@
     parseName(json);
     parseTypes(json);
     parseMessages(json);
+    parseDoc(json);
   }
 
   private void parseNamespace(JsonNode json) {
@@ -285,6 +296,16 @@
     this.namespace = nameNode.getTextValue();
     types.space(this.namespace);
   }
+  
+  private void parseDoc(JsonNode json) {
+    this.doc = parseDocNode(json);
+  }
+
+  private String parseDocNode(JsonNode json) {
+    JsonNode nameNode = json.get("doc");
+    if (nameNode == null) return null;                 // no doc defined
+    return nameNode.getTextValue();
+  }
 
   private void parseName(JsonNode json) {
     JsonNode nameNode = json.get("protocol");
@@ -328,6 +349,7 @@
         throw new SchemaParseException("No param type: "+field);
       fields.put(fieldNameNode.getTextValue(),
                  new Field(Schema.parse(fieldTypeNode,types),
+                           null /* message fields don't have docs */,
                            field.get("default")));
     }
     Schema request = Schema.createRecord(fields);
@@ -353,7 +375,8 @@
         errs.add(schema);
       }
     }
-    return new Message(messageName, request, response,
+    String doc = parseDocNode(json);
+    return new Message(messageName, doc, request, response,
                        Schema.createUnion(errs));
   }
 

Modified: hadoop/avro/trunk/src/java/org/apache/avro/Schema.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/Schema.java?rev=897311&r1=897310&r2=897311&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/Schema.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/Schema.java Fri Jan  8 19:49:51 2010
@@ -120,21 +120,21 @@
 
   /** Create an anonymous record schema. */
   public static Schema createRecord(LinkedHashMap<String,Field> fields) {
-    Schema result = createRecord(null, null, false);
+    Schema result = createRecord(null, null, null, false);
     result.setFields(fields);
     return result;
   }
 
   /** Create a named record schema. */
-  public static Schema createRecord(String name, String namespace,
+  public static Schema createRecord(String name, String doc, String namespace,
                                     boolean isError) {
-    return new RecordSchema(new Name(name, namespace), isError);
+    return new RecordSchema(new Name(name, namespace), doc, isError);
   }
 
   /** Create an enum schema. */
-  public static Schema createEnum(String name, String namespace,
+  public static Schema createEnum(String name, String doc, String namespace,
                                   List<String> values) {
-    return new EnumSchema(new Name(name, namespace), values);
+    return new EnumSchema(new Name(name, namespace), doc, values);
   }
 
   /** Create an array schema. */
@@ -153,8 +153,9 @@
   }
 
   /** Create a union schema. */
-  public static Schema createFixed(String name, String space, int size) {
-    return new FixedSchema(new Name(name, space), size);
+  public static Schema createFixed(String name, String doc, String space,
+      int size) {
+    return new FixedSchema(new Name(name, space), doc, size);
   }
 
   /** Return the type of this schema. */
@@ -189,6 +190,12 @@
    * of the primitive type. */
   public String getName() { return type.name; }
 
+  /** If this is a record, enum, or fixed, returns its docstring,
+   * if available.  Otherwise, returns null. */
+  public String getDoc() {
+    return null;
+  }
+
   /** If this is a record, enum or fixed, returns its namespace, if any. */
   public String getNamespace() {
     throw new AvroRuntimeException("Not a named type: "+this);
@@ -286,14 +293,16 @@
 
     private int position = -1;
     private final Schema schema;
+    private final String doc;
     private final JsonNode defaultValue;
     private final Order order;
 
-    public Field(Schema schema, JsonNode defaultValue) {
-      this(schema, defaultValue, Order.ASCENDING);
+    public Field(Schema schema, String doc, JsonNode defaultValue) {
+      this(schema, doc, defaultValue, Order.ASCENDING);
     }
-    public Field(Schema schema, JsonNode defaultValue, Order order) {
+    public Field(Schema schema, String doc, JsonNode defaultValue, Order order) {
       this.schema = schema;
+      this.doc = doc;
       this.defaultValue = defaultValue;
       this.order = order;
     }
@@ -301,6 +310,8 @@
     public int pos() { return position; }
     /** This field's {@link Schema}. */
     public Schema schema() { return schema; }
+    /** Field's documentation within the record, if set. May return null. */
+    public String doc() { return doc; }
     public JsonNode defaultValue() { return defaultValue; }
     public Order order() { return order; }
     public boolean equals(Object other) {
@@ -358,11 +369,14 @@
 
   private static abstract class NamedSchema extends Schema {
     private final Name name;
-    public NamedSchema(Type type, Name name) {
+    private final String doc;
+    public NamedSchema(Type type, Name name, String doc) {
       super(type);
       this.name = name;
+      this.doc = doc;
     }
     public String getName() { return name.name; }
+    public String getDoc() { return doc; }
     public String getNamespace() { return name.space; }
     public String getFullName() { return name.full; }
     public boolean writeNameRef(Names names, JsonGenerator gen)
@@ -412,8 +426,8 @@
     private Map<String,Field> fields;
     private Iterable<Map.Entry<String,Schema>> fieldSchemas;
     private final boolean isError;
-    public RecordSchema(Name name, boolean isError) {
-      super(Type.RECORD, name);
+    public RecordSchema(Name name, String doc, boolean isError) {
+      super(Type.RECORD, name, doc);
       this.isError = isError;
     }
     public boolean isError() { return isError; }
@@ -495,8 +509,8 @@
   private static class EnumSchema extends NamedSchema {
     private final List<String> symbols;
     private final Map<String,Integer> ordinals;
-    public EnumSchema(Name name, List<String> symbols) {
-      super(Type.ENUM, name);
+    public EnumSchema(Name name, String doc, List<String> symbols) {
+      super(Type.ENUM, name, doc);
       this.symbols = symbols;
       this.ordinals = new HashMap<String,Integer>();
       int i = 0;
@@ -624,8 +638,8 @@
 
   private static class FixedSchema extends NamedSchema {
     private final int size;
-    public FixedSchema(Name name, int size) {
-      super(Type.FIXED, name);
+    public FixedSchema(Name name, String doc, int size) {
+      super(Type.FIXED, name, doc);
       if (size < 0)
         throw new IllegalArgumentException("Invalid fixed size: "+size);
       this.size = size;
@@ -779,9 +793,11 @@
       String type = getRequiredText(schema, "type", "No type");
       Name name = null;
       String savedSpace = null;
+      String doc = null;
       if (type.equals("record") || type.equals("error")
           || type.equals("enum") || type.equals("fixed")) {
         String space = getOptionalText(schema, "namespace");
+        doc = getOptionalText(schema, "doc");
         if (space == null)
           space = names.space();
         name = new Name(getRequiredText(schema, "name", "No name in schema"),
@@ -795,15 +811,14 @@
         result = create(PRIMITIVES.get(type));
       } else if (type.equals("record") || type.equals("error")) { // record
         LinkedHashMap<String,Field> fields = new LinkedHashMap<String,Field>();
-        result = new RecordSchema(name, type.equals("error"));
+        result = new RecordSchema(name, doc, type.equals("error"));
         if (name != null) names.add(result);
         JsonNode fieldsNode = schema.get("fields");
         if (fieldsNode == null || !fieldsNode.isArray())
           throw new SchemaParseException("Record has no fields: "+schema);
         for (JsonNode field : fieldsNode) {
-          JsonNode fieldNameNode = field.get("name");
-          if (fieldNameNode == null)
-            throw new SchemaParseException("No field name: "+field);
+          String fieldName = getRequiredText(field, "name", "No field name");
+          String fieldDoc = getOptionalText(field, "doc");
           JsonNode fieldTypeNode = field.get("type");
           if (fieldTypeNode == null)
             throw new SchemaParseException("No field type: "+field);
@@ -812,8 +827,8 @@
           JsonNode orderNode = field.get("order");
           if (orderNode != null)
             order = Field.Order.valueOf(orderNode.getTextValue().toUpperCase());
-          fields.put(fieldNameNode.getTextValue(),
-                     new Field(fieldSchema, field.get("default"), order));
+          fields.put(fieldName,
+                     new Field(fieldSchema, fieldDoc, field.get("default"), order));
         }
         result.setFields(fields);
       } else if (type.equals("enum")) {           // enum
@@ -823,7 +838,7 @@
         List<String> symbols = new ArrayList<String>();
         for (JsonNode n : symbolsNode)
           symbols.add(n.getTextValue());
-        result = new EnumSchema(name, symbols);
+        result = new EnumSchema(name, doc, symbols);
         if (name != null) names.add(result);
       } else if (type.equals("array")) {          // array
         JsonNode itemsNode = schema.get("items");
@@ -839,7 +854,7 @@
         JsonNode sizeNode = schema.get("size");
         if (sizeNode == null || !sizeNode.isInt())
           throw new SchemaParseException("Invalid or no size: "+schema);
-        result = new FixedSchema(name, sizeNode.getIntValue());
+        result = new FixedSchema(name, doc, sizeNode.getIntValue());
         if (name != null) names.add(result);
       } else
         throw new SchemaParseException("Type not supported: "+type);

Modified: hadoop/avro/trunk/src/java/org/apache/avro/genavro/genavro.jj
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/genavro/genavro.jj?rev=897311&r1=897310&r2=897311&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/genavro/genavro.jj (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/genavro/genavro.jj Fri Jan  8 19:49:51 2010
@@ -1000,7 +1000,7 @@
   name = Identifier()
   symbols = EnumBody()
   {
-    Schema s = Schema.createEnum(name, this.namespace, symbols);
+    Schema s = Schema.createEnum(name, null, this.namespace, symbols);
     names.put(s.getFullName(), s);
     return s;
   }
@@ -1064,7 +1064,7 @@
   "fixed" name = Identifier() "(" sizeTok = <INTEGER_LITERAL> ")"
   ";"
   {
-    Schema s = Schema.createFixed(name, this.namespace,
+    Schema s = Schema.createFixed(name, null, this.namespace,
                                   Integer.parseInt(sizeTok.image));
     names.put(s.getFullName(), s);
     return s;
@@ -1085,7 +1085,7 @@
   name = Identifier()
   {
     Schema result = Schema.createRecord(
-      name, this.namespace, isError);
+      name, null, this.namespace, isError);
     names.put(result.getFullName(), result);
   }
   "{"
@@ -1142,7 +1142,7 @@
 {
   name = Identifier()
   {
-    fields.put(name, new Field(type, null));
+    fields.put(name, new Field(type, null, null));
   }
 }
 
@@ -1163,7 +1163,7 @@
   ";"
   {
     Schema errors = Schema.createUnion(errorSchemata);
-    return p.createMessage(name, request, response, errors);
+    return p.createMessage(name, null, request, response, errors);
   }
 }
 

Modified: hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericData.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericData.java?rev=897311&r1=897310&r2=897311&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericData.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericData.java Fri Jan  8 19:49:51 2010
@@ -270,7 +270,7 @@
       GenericRecord record = (GenericRecord)datum;
       LinkedHashMap<String,Field> fields = new LinkedHashMap<String,Field>();
       for (Map.Entry<String,Object> entry : record.entrySet())
-        fields.put(entry.getKey(), new Field(induce(entry.getValue()), null));
+        fields.put(entry.getKey(), new Field(induce(entry.getValue()), null, null));
       return Schema.createRecord(fields);
     } else if (datum instanceof GenericArray) {
       Schema elementType = null;
@@ -302,7 +302,7 @@
       }
       return Schema.createMap(value);
     } else if (datum instanceof GenericFixed) {
-      return Schema.createFixed(null, null,
+      return Schema.createFixed(null, null, null,
                                 ((GenericFixed)datum).bytes().length);
     }
     else if (datum instanceof Utf8)       return Schema.create(Type.STRING);

Modified: hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectData.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectData.java?rev=897311&r1=897310&r2=897311&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectData.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/reflect/ReflectData.java Fri Jan  8 19:49:51 2010
@@ -267,20 +267,21 @@
           Enum[] constants = (Enum[])c.getEnumConstants();
           for (int i = 0; i < constants.length; i++)
             symbols.add(constants[i].name());
-          schema = Schema.createEnum(name, space, symbols);
+          schema = Schema.createEnum(name, null /* doc */, space, symbols);
         } else if (GenericFixed.class.isAssignableFrom(c)) { // fixed
           int size = c.getAnnotation(FixedSize.class).value();
-          schema = Schema.createFixed(name, space, size);
+          schema = Schema.createFixed(name, null /* doc */, space, size);
         } else {                                             // record
           LinkedHashMap<String,Schema.Field> fields =
             new LinkedHashMap<String,Schema.Field>();
-          schema = Schema.createRecord(name, space,
+          schema = Schema.createRecord(name, null /* doc */, space, 
                                        Throwable.class.isAssignableFrom(c));
           names.put(c.getName(), schema);
           for (Field field : getFields(c))
             if ((field.getModifiers()&(Modifier.TRANSIENT|Modifier.STATIC))==0){
               Schema fieldSchema = createFieldSchema(field, names);
-              fields.put(field.getName(), new Schema.Field(fieldSchema, null));
+              fields.put(field.getName(), new Schema.Field(fieldSchema, 
+                  null /* doc */, null));
             }
           schema.setFields(fields);
         }
@@ -375,7 +376,7 @@
       String paramName =  paramNames.length == paramTypes.length
         ? paramNames[i]
         : paramSchema.getName()+i;
-      fields.put(paramName, new Schema.Field(paramSchema, null));
+      fields.put(paramName, new Schema.Field(paramSchema, null /* doc */, null));
     }
     Schema request = Schema.createRecord(fields);
 
@@ -391,7 +392,7 @@
         errs.add(getSchema(err, names));
     Schema errors = Schema.createUnion(errs);
 
-    return protocol.createMessage(method.getName(), request, response, errors);
+    return protocol.createMessage(method.getName(), null /* doc */, request, response, errors);
   }
 
   private Schema getSchema(Type type, Map<String,Schema> names) {

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=897311&r1=897310&r2=897311&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 Fri Jan  8 19:49:51 2010
@@ -180,6 +180,7 @@
     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)+"\");");
@@ -188,6 +189,7 @@
       Message message = e.getValue();
       Schema request = message.getRequest();
       Schema response = message.getResponse();
+      doc(out, 1, e.getValue().getDoc());
       line(out, 1, unbox(response)+" "+ mangle(name)+"("+params(request)+")");
       line(out, 2,"throws org.apache.avro.ipc.AvroRemoteException"+errors(message.getErrors())+";");
     }
@@ -197,6 +199,7 @@
     return outputFile;
   }
 
+
   static String makePath(String name, String space) {
     if (space == null || space.isEmpty()) {
       return name + ".java";
@@ -244,6 +247,7 @@
     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"
@@ -253,9 +257,11 @@
       line(out, 1, "public static final org.apache.avro.Schema SCHEMA$ = org.apache.avro.Schema.parse(\""
            +esc(schema)+"\");");
       // field declations
-      for (Map.Entry<String,Schema.Field> field: schema.getFields().entrySet())
+      for (Map.Entry<String,Schema.Field> field: schema.getFields().entrySet()) {
+        doc(out, 1, field.getValue().doc());
         line(out, 1, "public " + unbox(field.getValue().schema()) + " "
              + mangle(field.getKey()) + ";");
+      }
       // schema method
       line(out, 1, "public org.apache.avro.Schema getSchema() { return SCHEMA$; }");
       // get method
@@ -281,6 +287,7 @@
       line(out, 0, "}");
       break;
     case ENUM:
+      doc(out, 0, schema.getDoc());
       line(out, 0, "public enum "+name+" { ");
       StringBuilder b = new StringBuilder();
       int count = 0;
@@ -293,6 +300,7 @@
       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;
@@ -306,6 +314,18 @@
     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) {

Modified: hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolGeneric.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolGeneric.java?rev=897311&r1=897310&r2=897311&view=diff
==============================================================================
--- hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolGeneric.java (original)
+++ hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolGeneric.java Fri Jan  8 19:49:51 2010
@@ -165,11 +165,12 @@
     Protocol protocol = new Protocol("Simple", "org.apache.avro.test");
     LinkedHashMap<String,Field> fields = new LinkedHashMap<String,Field>();
     fields.put("extra",
-               new Schema.Field(Schema.create(Schema.Type.BOOLEAN), null));
+               new Schema.Field(Schema.create(Schema.Type.BOOLEAN), null, null));
     fields.put("greeting",
-               new Schema.Field(Schema.create(Schema.Type.STRING), null));
+               new Schema.Field(Schema.create(Schema.Type.STRING), null, null));
     Protocol.Message message =
       protocol.createMessage("hello",
+                             null /* doc */,
                              Schema.createRecord(fields),
                              Schema.create(Schema.Type.STRING),
                              Schema.createUnion(new ArrayList<Schema>()));

Added: hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolParsing.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolParsing.java?rev=897311&view=auto
==============================================================================
--- hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolParsing.java (added)
+++ hadoop/avro/trunk/src/test/java/org/apache/avro/TestProtocolParsing.java Fri Jan  8 19:49:51 2010
@@ -0,0 +1,42 @@
+/**
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.junit.Test;
+
+public class TestProtocolParsing {
+  public static Protocol getSimpleProtocol() throws IOException {
+    File file = new File("src/test/schemata/simple.avpr");
+    Protocol protocol = Protocol.parse(file);
+    return protocol;
+  }
+  
+  @Test
+  public void testParsing() throws IOException {
+    Protocol protocol = getSimpleProtocol();
+    
+    assertEquals(protocol.getDoc(), "Protocol used for testing.");
+    assertEquals(5, protocol.getMessages().size());
+    assertEquals("Pretend you're in a cave!", protocol.getMessages().get("echo").getDoc());    
+  }
+}

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=897311&r1=897310&r2=897311&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 Fri Jan  8 19:49:51 2010
@@ -49,6 +49,24 @@
   public static final String BASIC_ENUM_SCHEMA = "{\"type\":\"enum\", \"name\":\"Test\","
             +"\"symbols\": [\"A\", \"B\"]}";
 
+  public static final String SCHEMA_WITH_DOC_TAGS = "{\n"
+      + "  \"type\": \"record\",\n"
+      + "  \"name\": \"outer_record\",\n"
+      + "  \"doc\": \"This is not a world record.\",\n"
+      + "  \"fields\": [\n"
+      + "    { \"type\": { \"type\": \"fixed\", \"doc\": \"Very Inner Fixed\", "
+      + "                  \"name\": \"very_inner_fixed\", \"size\": 1 },\n"
+      + "      \"doc\": \"Inner Fixed\", \"name\": \"inner_fixed\" },\n"
+      + "    { \"type\": \"string\",\n"
+      + "      \"name\": \"inner_string\",\n"
+      + "      \"doc\": \"Inner String\" },\n"
+      + "    { \"type\": { \"type\": \"enum\", \"doc\": \"Very Inner Enum\", \n"
+      + "                  \"name\": \"very_inner_enum\", \n"
+      + "                  \"symbols\": [ \"A\", \"B\", \"C\" ] },\n"
+      + "      \"doc\": \"Inner Enum\", \"name\": \"inner_enum\" },\n"
+      + "    { \"type\": [\"string\", \"int\"], \"doc\": \"Inner Union\", \n"
+      + "      \"name\": \"inner_union\" }\n" + "  ]\n" + "}\n";
+
   private static final int COUNT =
     Integer.parseInt(System.getProperty("test.count", "10"));
 
@@ -230,6 +248,23 @@
     fail("Should not have parsed: "+json);
   }
 
+  /**
+   * Makes sure that "doc" tags are transcribed in the schemas.
+   * Note that there are docs both for fields and for the records
+   * themselves.
+   */
+  @Test
+  public void testDocs() {
+    Schema schema = Schema.parse(SCHEMA_WITH_DOC_TAGS);
+    assertEquals("This is not a world record.", schema.getDoc());
+    assertEquals("Inner Fixed", schema.getFields().get("inner_fixed").doc());
+    assertEquals("Very Inner Fixed", schema.getFields().get("inner_fixed").schema().getDoc());
+    assertEquals("Inner String", schema.getFields().get("inner_string").doc());
+    assertEquals("Inner Enum", schema.getFields().get("inner_enum").doc());
+    assertEquals("Very Inner Enum", schema.getFields().get("inner_enum").schema().getDoc());
+    assertEquals("Inner Union", schema.getFields().get("inner_union").doc());
+  }
+
   private static void check(String schemaJson, String defaultJson,
                             Object defaultValue) throws Exception {
     check(schemaJson, defaultJson, defaultValue, true);

Modified: 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=897311&r1=897310&r2=897311&view=diff
==============================================================================
--- hadoop/avro/trunk/src/test/java/org/apache/avro/specific/TestSpecificCompiler.java (original)
+++ hadoop/avro/trunk/src/test/java/org/apache/avro/specific/TestSpecificCompiler.java Fri Jan  8 19:49:51 2010
@@ -35,6 +35,7 @@
 import org.apache.avro.AvroTestUtil;
 import org.apache.avro.Protocol;
 import org.apache.avro.Schema;
+import org.apache.avro.TestProtocolParsing;
 import org.apache.avro.TestSchema;
 import org.apache.avro.specific.SpecificCompiler.OutputFile;
 import org.junit.Test;
@@ -142,6 +143,50 @@
     assertCompilesWithJavaCompiler(c);
   }
 
+  @Test
+  public void testSchemaWithDocs() {
+    Collection<OutputFile> outputs = new SpecificCompiler(
+        Schema.parse(TestSchema.SCHEMA_WITH_DOC_TAGS)).compile();
+    assertEquals(3, outputs.size());
+    int count = 0;
+    for (OutputFile o : outputs) {
+      if (o.path.endsWith("outer_record.java")) {
+        count++;
+        assertTrue(o.contents.contains("/** This is not a world record. */"));
+        assertTrue(o.contents.contains("/** Inner Fixed */"));
+        assertTrue(o.contents.contains("/** Inner Enum */"));
+        assertTrue(o.contents.contains("/** Inner String */"));
+      }
+      if (o.path.endsWith("very_inner_fixed.java")) {
+        count++;
+        assertTrue(o.contents.contains("/** Very Inner Fixed */"));
+        assertTrue(o.contents.contains("@org.apache.avro.specific.FixedSize(1)"));
+      }
+      if (o.path.endsWith("very_inner_enum.java")) {
+        count++;
+        assertTrue(o.contents.contains("/** Very Inner Enum */"));
+      }
+    }
+ 
+    assertEquals(3, count);
+  }
+  
+  @Test
+  public void testProtocolWithDocs() throws IOException {
+    Protocol protocol = TestProtocolParsing.getSimpleProtocol();
+    Collection<OutputFile> out = new SpecificCompiler(protocol).compile();
+    assertEquals(5, out.size());
+    int count = 0;
+    for (OutputFile o : out) {
+      if (o.path.endsWith("Simple.java")) {
+        count++;
+        assertTrue(o.contents.contains("/** Protocol used for testing. */"));
+        assertTrue(o.contents.contains("/** Send a greeting */"));
+      }
+    }
+    assertEquals("Missed generated protocol!", 1, count);
+  }
+
   /**
    * Checks that a schema passes through the SpecificCompiler, and,
    * optionally, uses the system's Java compiler to check

Modified: hadoop/avro/trunk/src/test/schemata/simple.avpr
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/test/schemata/simple.avpr?rev=897311&r1=897310&r2=897311&view=diff
==============================================================================
--- hadoop/avro/trunk/src/test/schemata/simple.avpr (original)
+++ hadoop/avro/trunk/src/test/schemata/simple.avpr Fri Jan  8 19:49:51 2010
@@ -1,5 +1,6 @@
 {"namespace": "org.apache.avro.test",
  "protocol": "Simple",
+ "doc": "Protocol used for testing.",
 
  "types": [
      {"name": "Kind", "type": "enum", "symbols": ["FOO","BAR","BAZ"]},
@@ -24,11 +25,13 @@
  "messages": {
 
      "hello": {
+         "doc": "Send a greeting",
          "request": [{"name": "greeting", "type": "string"}],
          "response": "string"
      },
 
      "echo": {
+         "doc": "Pretend you're in a cave!",
          "request": [{"name": "record", "type": "TestRecord"}],
          "response": "TestRecord"
      },
@@ -44,6 +47,7 @@
      },
 
      "error": {
+         "doc": "Always throws an error.",
          "request": [],
          "response": "null",
          "errors": ["TestError"]