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 2012/11/01 18:42:13 UTC

svn commit: r1404687 - in /avro/trunk: ./ lang/java/avro/src/main/java/org/apache/avro/ lang/java/compiler/src/main/java/org/apache/avro/compiler/specific/ lang/java/compiler/src/main/javacc/org/apache/avro/compiler/idl/ lang/java/compiler/src/test/idl...

Author: cutting
Date: Thu Nov  1 17:42:12 2012
New Revision: 1404687

URL: http://svn.apache.org/viewvc?rev=1404687&view=rev
Log:
AVRO-1157. Java: Extend schema and protocol property support from string-only to full JSON.

Added:
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/JsonProperties.java   (with props)
Modified:
    avro/trunk/CHANGES.txt
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/Protocol.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/Schema.java
    avro/trunk/lang/java/compiler/src/main/java/org/apache/avro/compiler/specific/SpecificCompiler.java
    avro/trunk/lang/java/compiler/src/main/javacc/org/apache/avro/compiler/idl/idl.jj
    avro/trunk/lang/java/compiler/src/test/idl/input/simple.avdl
    avro/trunk/lang/java/compiler/src/test/idl/output/simple.avpr
    avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestSchema.java

Modified: avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/avro/trunk/CHANGES.txt?rev=1404687&r1=1404686&r2=1404687&view=diff
==============================================================================
--- avro/trunk/CHANGES.txt (original)
+++ avro/trunk/CHANGES.txt Thu Nov  1 17:42:12 2012
@@ -6,6 +6,9 @@ Trunk (not yet released)
 
     AVRO-485.  JavaScript: Add validator. (Quinn Slack via cutting) 
 
+    AVRO-1157. Java: Extend schema and protocol property support from
+    string-only to full JSON. (cutting)
+
   IMPROVEMENTS
 
     AVRO-1169. Java: Reduce memory footprint of resolver.

Added: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/JsonProperties.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/JsonProperties.java?rev=1404687&view=auto
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/JsonProperties.java (added)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/JsonProperties.java Thu Nov  1 17:42:12 2012
@@ -0,0 +1,120 @@
+/**
+ * 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 java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import java.io.IOException;
+
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.JsonGenerator;
+import org.codehaus.jackson.node.TextNode;
+
+/** Base class for objects that have Json-valued properties. */
+public abstract class JsonProperties {
+  Map<String,JsonNode> props = new LinkedHashMap<String,JsonNode>(1);
+
+  private Set<String> reserved;
+
+  JsonProperties(Set<String> reserved) {
+    this.reserved = reserved;
+  }
+
+  /**
+   * Returns the value of the named, string-valued property in this schema.
+   * Returns <tt>null</tt> if there is no string-valued property with that name.
+   */
+  public String getProp(String name) {
+    JsonNode value = getJsonProp(name);
+    return value != null && value.isTextual() ? value.getTextValue() : null;
+  }
+
+  /**
+   * Returns the value of the named property in this schema.
+   * Returns <tt>null</tt> if there is no property with that name.
+   */
+  public synchronized JsonNode getJsonProp(String name) {
+    return props.get(name);
+  }
+
+  /**
+   * Adds a property with the given name <tt>name</tt> and
+   * value <tt>value</tt>. Neither <tt>name</tt> nor <tt>value</tt> can be
+   * <tt>null</tt>. It is illegal to add a property if another with
+   * the same name but different value already exists in this schema.
+   * 
+   * @param name The name of the property to add
+   * @param value The value for the property to add
+   */
+  public void addProp(String name, String value) {
+    addProp(name, TextNode.valueOf(value));
+  }
+
+  /**
+   * Adds a property with the given name <tt>name</tt> and
+   * value <tt>value</tt>. Neither <tt>name</tt> nor <tt>value</tt> can be
+   * <tt>null</tt>. It is illegal to add a property if another with
+   * the same name but different value already exists in this schema.
+   * 
+   * @param name The name of the property to add
+   * @param value The value for the property to add
+   */
+  public synchronized void addProp(String name, JsonNode value) {
+    if (reserved.contains(name))
+      throw new AvroRuntimeException("Can't set reserved property: " + name);
+      
+    if (value == null)
+      throw new AvroRuntimeException("Can't set a property to null: " + name);
+    
+    JsonNode old = props.get(name);
+    if (old == null)
+      props.put(name, value);
+    else if (!old.equals(value))
+      throw new AvroRuntimeException("Can't overwrite property: " + name);
+  }
+
+  /** Return the defined properties that have string values. */
+  @Deprecated public Map<String,String> getProps() {
+    Map<String,String> result = new LinkedHashMap<String,String>();
+    for (Map.Entry<String,JsonNode> e : props.entrySet())
+      if (e.getValue().isTextual())
+        result.put(e.getKey(), e.getValue().getTextValue());
+    return result;
+  }
+
+  /** Convert a map of string-valued properties to Json properties. */
+  Map<String,JsonNode> jsonProps(Map<String,String> stringProps) {
+    Map<String,JsonNode> result = new LinkedHashMap<String,JsonNode>();
+    for (Map.Entry<String,String> e : stringProps.entrySet())
+      result.put(e.getKey(), TextNode.valueOf(e.getValue()));
+    return result;
+  }
+
+  /** Return the defined properties as an unmodifieable Map. */
+  public Map<String,JsonNode> getJsonProps() {
+    return Collections.unmodifiableMap(props);
+  }
+
+  void writeProps(JsonGenerator gen) throws IOException {
+    for (Map.Entry<String,JsonNode> e : props.entrySet())
+      gen.writeObjectField(e.getKey(), e.getValue());
+  }
+
+}

Propchange: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/JsonProperties.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/Protocol.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/Protocol.java?rev=1404687&r1=1404686&r2=1404687&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/Protocol.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/Protocol.java Thu Nov  1 17:42:12 2012
@@ -37,6 +37,7 @@ import org.apache.avro.Schema.Field;
 import org.codehaus.jackson.JsonNode;
 import org.codehaus.jackson.JsonParser;
 import org.codehaus.jackson.JsonGenerator;
+import org.codehaus.jackson.node.TextNode;
 
 /** A set of messages forming an application protocol.
  * <p> A protocol consists of:
@@ -58,7 +59,7 @@ import org.codehaus.jackson.JsonGenerato
  *   </ul>
  * </ul>
  */
-public class Protocol {
+public class Protocol extends JsonProperties {
   /** The version of the protocol specification implemented here. */
   public static final long VERSION = 1;
 
@@ -70,22 +71,27 @@ public class Protocol {
   }
 
   /** A protocol message. */
-  public class Message {
+  public class Message extends JsonProperties {
     private String name;
     private String doc;
     private Schema request;
-    private final Schema.Props props = new Schema.Props(MESSAGE_RESERVED);
 
     /** Construct a message. */
     private Message(String name, String doc,
-                    Map<String,String> propMap, Schema request) {
+                    Map<String,?> propMap, Schema request) {
+      super(MESSAGE_RESERVED);
       this.name = name;
       this.doc = doc;
       this.request = request;
 
       if (propMap != null)                        // copy props
-        for (Map.Entry<String,String> prop : propMap.entrySet())
-          this.addProp(prop.getKey(), prop.getValue());
+        for (Map.Entry<String,?> prop : propMap.entrySet()) {
+          Object value = prop.getValue();
+          this.addProp(prop.getKey(),
+                       value instanceof String
+                       ? TextNode.valueOf((String)value)
+                       : (JsonNode)value);
+        }
     }
 
     /** The name of this message. */
@@ -102,17 +108,6 @@ public class Protocol {
     /** Returns true if this is a one-way message, with no response or errors.*/
     public boolean isOneWay() { return true; }
 
-    /** Return the value of the named property in this field or null. */
-    public synchronized String getProp(String name) { return props.get(name); }
-    /** Add a property with the given name to this field. */
-    public synchronized void addProp(String name, String value) {
-      props.add(name, value);
-    }
-    /** Return the defined properties as an unmodifieable Map. */
-    public Map<String,String> getProps() {
-      return Collections.unmodifiableMap(props);
-    }
-
     public String toString() {
       try {
         StringWriter writer = new StringWriter();
@@ -127,7 +122,7 @@ public class Protocol {
     void toJson(JsonGenerator gen) throws IOException {
       gen.writeStartObject();
       if (doc != null) gen.writeStringField("doc", doc);
-      props.write(gen);                           // write out properties
+      writeProps(gen);                           // write out properties
       gen.writeFieldName("request");
       request.fieldsToJson(types, gen);
 
@@ -162,7 +157,7 @@ public class Protocol {
     private Schema errors;
     
     /** Construct a message. */
-    private TwoWayMessage(String name, String doc, Map<String,String> propMap,
+    private TwoWayMessage(String name, String doc, Map<String,?> propMap,
                           Schema request, Schema response, Schema errors) {
       super(name, doc, propMap, request);
       this.response = response;
@@ -224,11 +219,13 @@ public class Protocol {
        "namespace", "protocol", "doc",
        "messages","types", "errors");
   }
-  Schema.Props props = new Schema.Props(PROTOCOL_RESERVED);
 
-  private Protocol() {}
+  private Protocol() {
+    super(PROTOCOL_RESERVED);
+  }
 
   public Protocol(String name, String doc, String namespace) {
+    super(PROTOCOL_RESERVED);
     this.name = name;
     this.doc = doc;
     this.namespace = namespace;
@@ -268,8 +265,8 @@ public class Protocol {
     return createMessage(name, doc, new LinkedHashMap<String,String>(),request);
   }
   /** Create a one-way message. */
-  public Message createMessage(String name, String doc,
-                               Map<String,String> propMap, Schema request) {
+  public <T> Message createMessage(String name, String doc,
+                                   Map<String,T> propMap, Schema request) {
     return new Message(name, doc, propMap, request);
   }
 
@@ -281,39 +278,12 @@ public class Protocol {
                          request, response, errors);
   }
   /** Create a two-way message. */
-  public Message createMessage(String name, String doc,
-                               Map<String,String> propMap, Schema request,
-                               Schema response, Schema errors) {
+  public <T> Message createMessage(String name, String doc,
+                                   Map<String,T> propMap, Schema request,
+                                   Schema response, Schema errors) {
     return new TwoWayMessage(name, doc, propMap, request, response, errors);
   }
 
-  /**
-   * Returns the value of the named property in this schema.
-   * Returns <tt>null</tt> if there is no property with that name.
-   * @param name
-   */
-  public synchronized String getProp(String name) {
-    return props.get(name);
-  }
-
-  /**
-   * Adds a property with the given name <tt>name</tt> and
-   * value <tt>value</tt>. Neither <tt>name</tt> nor <tt>value</tt> can be
-   * <tt>null</tt>. It is illegal to add a property if another with
-   * the same name but different value already exists in this schema.
-   *
-   * @param name The name of the property to add
-   * @param value The value for the property to add
-   */
-  public synchronized void addProp(String name, String value) {
-    props.add(name, value);
-  }
-
-  /** Return the defined properties as an unmodifieable Map. */
-  public Map<String,String> getProps() {
-    return Collections.unmodifiableMap(props);
-  }
-
   public boolean equals(Object o) {
     if (o == this) return true;
     if (!(o instanceof Protocol)) return false;
@@ -357,7 +327,7 @@ public class Protocol {
     gen.writeStringField("namespace", namespace);
 
     if (doc != null) gen.writeStringField("doc", doc);
-    props.write(gen);
+    writeProps(gen);
     gen.writeArrayFieldStart("types");
     Schema.Names resolved = new Schema.Names(namespace);
     for (Schema type : types.values())
@@ -464,11 +434,8 @@ public class Protocol {
   private void parseProps(JsonNode json) {
     for (Iterator<String> i = json.getFieldNames(); i.hasNext();) {
       String p = i.next();                        // add non-reserved as props
-      if (!PROTOCOL_RESERVED.contains(p)) {
-        JsonNode prop = json.get(p);
-        if (prop.isValueNode() && prop.isTextual())
-          this.addProp(p,prop.getTextValue());
-      }
+      if (!PROTOCOL_RESERVED.contains(p))
+        this.addProp(p, json.get(p));
     }
   }
 
@@ -484,14 +451,11 @@ public class Protocol {
   private Message parseMessage(String messageName, JsonNode json) {
     String doc = parseDocNode(json);
 
-    Map<String,String> mProps = new LinkedHashMap<String,String>();
+    Map<String,JsonNode> mProps = new LinkedHashMap<String,JsonNode>();
     for (Iterator<String> i = json.getFieldNames(); i.hasNext();) {
       String p = i.next();                        // add non-reserved as props
-      if (!MESSAGE_RESERVED.contains(p)) {
-        JsonNode prop = json.get(p);
-        if (prop.isValueNode() && prop.isTextual())
-          mProps.put(p,prop.getTextValue());
-      }
+      if (!MESSAGE_RESERVED.contains(p))
+        mProps.put(p, json.get(p));
     }
 
     JsonNode requestNode = json.get("request");

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/Schema.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/Schema.java?rev=1404687&r1=1404686&r2=1404687&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/Schema.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/Schema.java Thu Nov  1 17:42:12 2012
@@ -75,7 +75,7 @@ import org.codehaus.jackson.node.DoubleN
  * existing property.
  * </ul>
  */
-public abstract class Schema {
+public abstract class Schema extends JsonProperties {
   static final JsonFactory FACTORY = new JsonFactory();
   static final ObjectMapper MAPPER = new ObjectMapper(FACTORY);
 
@@ -97,7 +97,10 @@ public abstract class Schema {
 
   private final Type type;
 
-  Schema(Type type) { this.type = type; }
+  Schema(Type type) {
+    super(SCHEMA_RESERVED);
+    this.type = type;
+  }
 
   /** Create a schema for a primitive type. */
   public static Schema create(Type type) {
@@ -114,69 +117,20 @@ public abstract class Schema {
     }
   }
 
-  static final class Props extends LinkedHashMap<String,String> {
-    private Set<String> reserved;
-    public Props(Set<String> reserved) {
-      super(1);
-      this.reserved = reserved;
-    }
-    public void add(String name, String value) {
-      if (reserved.contains(name))
-        throw new AvroRuntimeException("Can't set reserved property: " + name);
-      
-      if (value == null)
-        throw new AvroRuntimeException("Can't set a property to null: " + name);
-    
-      String old = get(name);
-      if (old == null)
-        put(name, value);
-      else if (!old.equals(value))
-        throw new AvroRuntimeException("Can't overwrite property: " + name);
-    }
-
-    public void write(JsonGenerator gen) throws IOException {
-      for (Map.Entry<String,String> e : entrySet())
-        gen.writeStringField(e.getKey(), e.getValue());
-    }
-  }
-
   private static final Set<String> SCHEMA_RESERVED = new HashSet<String>();
   static {
     Collections.addAll(SCHEMA_RESERVED,
                        "doc", "fields", "items", "name", "namespace",
-                       "size", "symbols", "values", "type");
+                       "size", "symbols", "values", "type", "aliases");
   }
 
-  Props props = new Props(SCHEMA_RESERVED);
   int hashCode = NO_HASHCODE;
 
-  /**
-   * Returns the value of the named property in this schema.
-   * Returns <tt>null</tt> if there is no property with that name.
-   */
-  public synchronized String getProp(String name) {
-    return props.get(name);
-  }
-
-  /**
-   * Adds a property with the given name <tt>name</tt> and
-   * value <tt>value</tt>. Neither <tt>name</tt> nor <tt>value</tt> can be
-   * <tt>null</tt>. It is illegal to add a property if another with
-   * the same name but different value already exists in this schema.
-   * 
-   * @param name The name of the property to add
-   * @param value The value for the property to add
-   */
-  public synchronized void addProp(String name, String value) {
-    props.add(name, value);
+  @Override public void addProp(String name, JsonNode value) {
+    super.addProp(name, value);
     hashCode = NO_HASHCODE;
   }
 
-  /** Return the defined properties as an unmodifieable Map. */
-  public Map<String,String> getProps() {
-    return Collections.unmodifiableMap(props);
-  }
-
   /** Create an anonymous record schema. */
   public static Schema createRecord(List<Field> fields) {
     Schema result = createRecord(null, null, null, false);
@@ -348,7 +302,7 @@ public abstract class Schema {
     } else {
       gen.writeStartObject();
       gen.writeStringField("type", getName());
-      props.write(gen);
+      writeProps(gen);
       gen.writeEndObject();
     }
   }
@@ -380,11 +334,12 @@ public abstract class Schema {
 
   private static final Set<String> FIELD_RESERVED = new HashSet<String>();
   static {
-    Collections.addAll(FIELD_RESERVED, "default","doc","name","order","type");
+    Collections.addAll(FIELD_RESERVED,
+                       "default","doc","name","order","type","aliases");
   }
 
   /** A field within a record. */
-  public static class Field {
+  public static class Field extends JsonProperties {
 
     /** How values of this field should be ordered when sorting records. */
     public enum Order {
@@ -400,7 +355,6 @@ public abstract class Schema {
     private final JsonNode defaultValue;
     private final Order order;
     private Set<String> aliases;
-    private final Props props = new Props(FIELD_RESERVED);
 
     public Field(String name, Schema schema, String doc,
         JsonNode defaultValue) {
@@ -408,6 +362,7 @@ public abstract class Schema {
     }
     public Field(String name, Schema schema, String doc,
         JsonNode defaultValue, Order order) {
+      super(FIELD_RESERVED);
       this.name = validateName(name);
       this.schema = schema;
       this.doc = doc;
@@ -423,16 +378,7 @@ public abstract class Schema {
     public String doc() { return doc; }
     public JsonNode defaultValue() { return defaultValue; }
     public Order order() { return order; }
-    /** Return the value of the named property in this field or null. */
-    public synchronized String getProp(String name) { return props.get(name); }
-    /** Add a property with the given name to this field. */
-    public synchronized void addProp(String name, String value) {
-      props.add(name, value);
-    }
-    /** Return the defined properties as an unmodifieable Map. */
-    public Map<String,String> props() {
-      return Collections.unmodifiableMap(props);
-    }
+    @Deprecated public Map<String,String> props() { return getProps(); }
     public void addAlias(String alias) {
       if (aliases == null)
         this.aliases = new LinkedHashSet<String>();
@@ -672,7 +618,7 @@ public abstract class Schema {
         gen.writeStringField("doc", getDoc());
       gen.writeFieldName("fields");
       fieldsToJson(names, gen);
-      props.write(gen);
+      writeProps(gen);
       aliasesToJson(gen);
       gen.writeEndObject();
       names.space = savedSpace;                   // restore namespace
@@ -700,7 +646,7 @@ public abstract class Schema {
             gen.writeString(alias);
           gen.writeEndArray();
         }
-        f.props.write(gen);
+        f.writeProps(gen);
         gen.writeEndObject();
       }
       gen.writeEndArray();
@@ -745,7 +691,7 @@ public abstract class Schema {
       for (String symbol : symbols)
         gen.writeString(symbol);
       gen.writeEndArray();
-      props.write(gen);
+      writeProps(gen);
       aliasesToJson(gen);
       gen.writeEndObject();
     }
@@ -774,7 +720,7 @@ public abstract class Schema {
       gen.writeStringField("type", "array");
       gen.writeFieldName("items");
       elementType.toJson(names, gen);
-      props.write(gen);
+      writeProps(gen);
       gen.writeEndObject();
     }
   }
@@ -802,7 +748,7 @@ public abstract class Schema {
       gen.writeStringField("type", "map");
       gen.writeFieldName("values");
       valueType.toJson(names, gen);
-      props.write(gen);
+      writeProps(gen);
       gen.writeEndObject();
     }
   }
@@ -882,7 +828,7 @@ public abstract class Schema {
       if (getDoc() != null)
         gen.writeStringField("doc", getDoc());
       gen.writeNumberField("size", size);
-      props.write(gen);
+      writeProps(gen);
       aliasesToJson(gen);
       gen.writeEndObject();
     }
@@ -1162,9 +1108,8 @@ public abstract class Schema {
           Iterator<String> i = field.getFieldNames();
           while (i.hasNext()) {                       // add field props
             String prop = i.next();
-            String value = field.get(prop).getTextValue();
-            if (!FIELD_RESERVED.contains(prop) && value != null)
-              f.addProp(prop, value);
+            if (!FIELD_RESERVED.contains(prop))
+              f.addProp(prop, field.get(prop));
           }
           f.aliases = parseAliases(field);
           fields.add(f);
@@ -1200,9 +1145,8 @@ public abstract class Schema {
       Iterator<String> i = schema.getFieldNames();
       while (i.hasNext()) {                       // add properties
         String prop = i.next();
-        String value = schema.get(prop).getTextValue();
-        if (!SCHEMA_RESERVED.contains(prop) && value != null) // ignore reserved
-          result.addProp(prop, value);
+        if (!SCHEMA_RESERVED.contains(prop))      // ignore reserved
+          result.addProp(prop, schema.get(prop));
       }
       if (savedSpace != null)
         names.space(savedSpace);                  // restore space

Modified: avro/trunk/lang/java/compiler/src/main/java/org/apache/avro/compiler/specific/SpecificCompiler.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/compiler/src/main/java/org/apache/avro/compiler/specific/SpecificCompiler.java?rev=1404687&r1=1404686&r2=1404687&view=diff
==============================================================================
--- avro/trunk/lang/java/compiler/src/main/java/org/apache/avro/compiler/specific/SpecificCompiler.java (original)
+++ avro/trunk/lang/java/compiler/src/main/java/org/apache/avro/compiler/specific/SpecificCompiler.java Thu Nov  1 17:42:12 2012
@@ -31,6 +31,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.codehaus.jackson.JsonNode;
+
 import org.apache.avro.Protocol;
 import org.apache.avro.Protocol.Message;
 import org.apache.avro.Schema;
@@ -349,7 +351,7 @@ public class SpecificCompiler {
     Map<Schema,Schema> types = new LinkedHashMap<Schema,Schema>();
 
     // Copy properties
-    for (Map.Entry<String,String> prop : p.getProps().entrySet())
+    for (Map.Entry<String,JsonNode> prop : p.getJsonProps().entrySet())
       newP.addProp(prop.getKey(), prop.getValue());   // copy props
 
     // annotate types
@@ -362,9 +364,9 @@ public class SpecificCompiler {
     Map<String,Message> newM = newP.getMessages();
     for (Message m : p.getMessages().values())
       newM.put(m.getName(), m.isOneWay()
-               ? newP.createMessage(m.getName(), m.getDoc(), m.getProps(),
+               ? newP.createMessage(m.getName(), m.getDoc(), m.getJsonProps(),
                                     addStringType(m.getRequest(), types))
-               : newP.createMessage(m.getName(), m.getDoc(), m.getProps(),
+               : newP.createMessage(m.getName(), m.getDoc(), m.getJsonProps(),
                                     addStringType(m.getRequest(), types),
                                     addStringType(m.getResponse(), types),
                                     addStringType(m.getErrors(), types)));
@@ -395,7 +397,7 @@ public class SpecificCompiler {
         Schema fSchema = addStringType(f.schema(), seen);
         Field newF =
           new Field(f.name(), fSchema, f.doc(), f.defaultValue(), f.order());
-        for (Map.Entry<String,String> p : f.props().entrySet())
+        for (Map.Entry<String,JsonNode> p : f.getJsonProps().entrySet())
           newF.addProp(p.getKey(), p.getValue()); // copy props
         newFields.add(newF);
       }
@@ -417,7 +419,7 @@ public class SpecificCompiler {
       result = Schema.createUnion(types);
       break;
     }
-    for (Map.Entry<String,String> p : s.getProps().entrySet())
+    for (Map.Entry<String,JsonNode> p : s.getJsonProps().entrySet())
       result.addProp(p.getKey(), p.getValue());   // copy props
     seen.put(s, result);
     return result;

Modified: avro/trunk/lang/java/compiler/src/main/javacc/org/apache/avro/compiler/idl/idl.jj
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/compiler/src/main/javacc/org/apache/avro/compiler/idl/idl.jj?rev=1404687&r1=1404686&r2=1404687&view=diff
==============================================================================
--- avro/trunk/lang/java/compiler/src/main/javacc/org/apache/avro/compiler/idl/idl.jj (original)
+++ avro/trunk/lang/java/compiler/src/main/javacc/org/apache/avro/compiler/idl/idl.jj Thu Nov  1 17:42:12 2012
@@ -1027,8 +1027,8 @@ Schema NamedSchemaDeclaration(Map<String
      } else if ("aliases".equals(key)) {          // aliases
        for (String alias : getTextProps("aliases", props, token))
          s.addAlias(alias);
-     } else if (props.get(key).isTextual()) {     // ignore other non-textual
-       s.addProp(key, getTextProp(key, props, token));
+     } else {                                     // add all other props
+       s.addProp(key, props.get(key));
      }
 
    return s;
@@ -1078,8 +1078,8 @@ Protocol ProtocolDeclaration():
    p = new Protocol(name, getDoc(), namespace);
    for (String key : props.keySet())
      if ("namespace".equals(key)) {               // already handled: ignore
-     } else if (props.get(key).isTextual()) {     // ignore other non-textual
-       p.addProp(key, getTextProp(key, props, token));
+     } else {                                     // add all other props
+       p.addProp(key, props.get(key));
      }
  }
  ProtocolBody(p)
@@ -1296,7 +1296,7 @@ void FieldDeclaration(List<Field> fields
   ";"
   {
     for (String key : props.keySet())
-      type.addProp(key, getTextProp(key, props, token));
+      type.addProp(key, props.get(key));
   }
 }
 
@@ -1324,8 +1324,8 @@ void VariableDeclarator(Schema type, Lis
       } else if ("aliases".equals(key)) {         // aliases
         for (String alias : getTextProps("aliases", props, token))
           field.addAlias(alias);
-      } else if (props.get(key).isTextual()) {    // ignore other non-textual
-        field.addProp(key, getTextProp(key, props, token));
+      } else {                                    // add all other props
+        field.addProp(key, props.get(key));
       }
     fields.add(field);
   }
@@ -1350,7 +1350,6 @@ Message MessageDeclaration(Protocol p, M
   boolean oneWay = false;
   List<Schema> errorSchemata = new ArrayList<Schema>();
   errorSchemata.add(Protocol.SYSTEM_ERROR);
-  Map<String, String> propMap = new LinkedHashMap<String, String>();
 }
 {
   msgDoc = MessageDocumentation()
@@ -1360,17 +1359,12 @@ Message MessageDeclaration(Protocol p, M
   [ "oneway" {oneWay = true; } | "throws" ErrorList(errorSchemata) ]
   ";"
   {
-    for (String key : props.keySet())
-      if (props.get(key).isTextual()) {    // ignore other non-textual
-        propMap.put(key, getTextProp(key, props, token));
-      }
-
     Schema errors = Schema.createUnion(errorSchemata);
     if (oneWay && response.getType() != Type.NULL)
       throw error("One-way message'"+name+"' must return void", token);
     return oneWay
-    ? p.createMessage(name, msgDoc, propMap, request)
-    : p.createMessage(name, msgDoc, propMap, request, response, errors);
+    ? p.createMessage(name, msgDoc, props, request)
+    : p.createMessage(name, msgDoc, props, request, response, errors);
     
   }
 }

Modified: avro/trunk/lang/java/compiler/src/test/idl/input/simple.avdl
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/compiler/src/test/idl/input/simple.avdl?rev=1404687&r1=1404686&r2=1404687&view=diff
==============================================================================
--- avro/trunk/lang/java/compiler/src/test/idl/input/simple.avdl (original)
+++ avro/trunk/lang/java/compiler/src/test/idl/input/simple.avdl Thu Nov  1 17:42:12 2012
@@ -34,6 +34,7 @@ protocol Simple {
   fixed MD5(16);
 
   /** A TestRecord. */
+  @myProperty({"key":3})
   record TestRecord {
     string @order("ignore") name = "foo";
 

Modified: avro/trunk/lang/java/compiler/src/test/idl/output/simple.avpr
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/compiler/src/test/idl/output/simple.avpr?rev=1404687&r1=1404686&r2=1404687&view=diff
==============================================================================
--- avro/trunk/lang/java/compiler/src/test/idl/output/simple.avpr (original)
+++ avro/trunk/lang/java/compiler/src/test/idl/output/simple.avpr Thu Nov  1 17:42:12 2012
@@ -44,7 +44,10 @@
       "name" : "average",
       "type" : "float",
       "default" : "-Infinity"
-    } ]
+    } ],
+    "myProperty" : {
+      "key" : 3
+    }
   }, {
     "type" : "error",
     "name" : "TestError",

Modified: avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestSchema.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestSchema.java?rev=1404687&r1=1404686&r2=1404687&view=diff
==============================================================================
--- avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestSchema.java (original)
+++ avro/trunk/lang/java/ipc/src/test/java/org/apache/avro/TestSchema.java Thu Nov  1 17:42:12 2012
@@ -850,7 +850,7 @@ public class TestSchema {
   @Test(expected=AvroRuntimeException.class)
   public void testImmutability2() {
     Schema s = enumSchema();
-    s.addProp("p1", null);
+    s.addProp("p1", (String)null);
   }
 
   private static List<String> lockedArrayList() {