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 2009/04/27 22:18:07 UTC

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

Author: cutting
Date: Mon Apr 27 20:18:06 2009
New Revision: 769143

URL: http://svn.apache.org/viewvc?rev=769143&view=rev
Log:
AVRO-9.  Restrict map keys to strings.

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/generic/GenericData.java
    hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumReader.java
    hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumWriter.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/py/avro/generic.py
    hadoop/avro/trunk/src/py/avro/reflect.py
    hadoop/avro/trunk/src/py/avro/schema.py
    hadoop/avro/trunk/src/test/java/org/apache/avro/RandomData.java
    hadoop/avro/trunk/src/test/java/org/apache/avro/TestSchema.java
    hadoop/avro/trunk/src/test/py/testio.py

Modified: hadoop/avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/CHANGES.txt?rev=769143&r1=769142&r2=769143&view=diff
==============================================================================
--- hadoop/avro/trunk/CHANGES.txt (original)
+++ hadoop/avro/trunk/CHANGES.txt Mon Apr 27 20:18:06 2009
@@ -5,7 +5,9 @@
   INCOMPATIBLE CHANGES
 
     AVRO-1. Record fields are now defined with JSON arrays, rather
-    than JSON objects, since fields are ordered.  (cutting)
+    than JSON objects, since fields are ordered.  (cutting & sharad)
+
+    AVRO-9. Restrict map keys to strings.  (cutting & sharad)
 
   NEW FEATURES
 

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=769143&r1=769142&r2=769143&view=diff
==============================================================================
--- hadoop/avro/trunk/src/doc/content/xdocs/spec.xml (original)
+++ hadoop/avro/trunk/src/doc/content/xdocs/spec.xml Mon Apr 27 20:18:06 2009
@@ -135,14 +135,14 @@
         <section>
           <title>Maps</title>
           <p>Maps use the type name <code>"map"</code> and support
-          two attributes:</p>
+          one attribute:</p>
 	  <ul>
-            <li><code>keys</code>: the schema of the map's keys.</li>
             <li><code>values</code>: the schema of the map's values.</li>
 	  </ul>
+	  <p>Map keys are assumed to be strings.</p>
 	  <p>For example, a map from string to long is declared
 	  with:</p>
-	  <source>{"type": "map", "keys": "string", "values": "long"}</source>
+	  <source>{"type": "map", "values": "long"}</source>
 	</section>
 
         <section>
@@ -304,7 +304,7 @@
           block consists of a <code>long</code> <em>count</em> value,
           followed by that many key/value pairs.  A block with count
           zero indicates the end of the map.  Each item is serialized
-          per the map's key and value schema.</p>
+          per the map's value schema.</p>
 
 	  <p>If a block's count is negative, then the count is
 	  followed immediately by a <code>long</code>
@@ -646,7 +646,7 @@
 	  <p>To match, one of the following must hold:</p>
 	  <ul>
 	    <li>both schemas are arrays whose item types match</li>
-	    <li>both schemas are maps whose key and value types match</li>
+	    <li>both schemas are maps whose value types match</li>
 	    <li>both schemas are records with the same name</li>
 	    <li>either schema is a union</li>
 	    <li>both schemas have same primitive type</li>
@@ -682,7 +682,7 @@
 
 	<li><strong>if both are maps:</strong>
 	  <p>This resolution algorithm is applied recursively to the reader's and
-	    writer's key and value schemas.</p>
+	    writer's value schemas.</p>
 	</li>
 
 	<li><strong>if both are unions:</strong>

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=769143&r1=769142&r2=769143&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/Protocol.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/Protocol.java Mon Apr 27 20:18:06 2009
@@ -93,7 +93,7 @@
 
       List<Schema> errTypes = errors.getTypes();  // elide system error
       if (errTypes.size() > 1) {
-        Schema errs = Schema.create(errTypes.subList(1, errTypes.size()));
+        Schema errs = Schema.createUnion(errTypes.subList(1, errTypes.size()));
         buffer.append(", \"errors\": "+errs.toString(types));
       }
 
@@ -132,7 +132,7 @@
   static {
     List<Schema> errors = new ArrayList<Schema>();
     errors.add(SYSTEM_ERROR);
-    SYSTEM_ERRORS = Schema.create(errors);
+    SYSTEM_ERRORS = Schema.createUnion(errors);
   }
 
   private Protocol() {}
@@ -276,7 +276,7 @@
         throw new SchemaParseException("No param type: "+field);
       fields.put(fieldNameNode.getTextValue(),Schema.parse(fieldTypeNode,types));
     }
-    Schema request = Schema.create(fields);
+    Schema request = Schema.createRecord(fields);
     
     JsonNode responseNode = json.getFieldValue("response");
     if (responseNode == null)
@@ -299,7 +299,8 @@
         errs.add(schema);
       }
     }
-    return new Message(messageName, request, response, Schema.create(errs));
+    return new Message(messageName, request, response,
+                       Schema.createUnion(errs));
   }
 
   public static void main(String args[]) throws Exception {

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=769143&r1=769142&r2=769143&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/Schema.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/Schema.java Mon Apr 27 20:18:06 2009
@@ -39,7 +39,7 @@
  * <ul>
  * <li>An <i>record</i>, mapping field names to field value data;
  * <li>An <i>array</i> of values, all of the same schema;
- * <li>A <i>map</i> containing key/value pairs, each of a declared schema;
+ * <li>A <i>map</i>, containing string/value pairs, of a declared schema;
  * <li>A <i>union</i> of other schemas;
  * <li>A unicode <i>string</i>;
  * <li>A sequence of <i>bytes</i>;
@@ -84,29 +84,30 @@
   }
 
   /** Create an anonymous record schema. */
-  public static Schema create(Map<String,Schema> fields) {
-    Schema result = create(null, null, false);
+  public static Schema createRecord(Map<String,Schema> fields) {
+    Schema result = createRecord(null, null, false);
     result.setFields(fields);
     return result;
   }
 
   /** Create a named record schema. */
-  public static Schema create(String name, String namespace, boolean isError) {
+  public static Schema createRecord(String name, String namespace,
+                                    boolean isError) {
      return new RecordSchema(name, namespace, isError);
   }
 
   /** Create an array schema. */
-  public static Schema create(Schema elementType) {
+  public static Schema createArray(Schema elementType) {
     return new ArraySchema(elementType);
   }
 
   /** Create a map schema. */
-  public static Schema create(Schema keyType, Schema valueType) {
-    return new MapSchema(keyType, valueType);
+  public static Schema createMap(Schema valueType) {
+    return new MapSchema(valueType);
   }
 
   /** Create a union schema. */
-  public static Schema create(List<Schema> types) {
+  public static Schema createUnion(List<Schema> types) {
     return new UnionSchema(types);
   }
 
@@ -148,11 +149,6 @@
     throw new AvroRuntimeException("Not an array: "+this);
   }
 
-  /** If this is a map, returns its key type. */
-  public Schema getKeyType() {
-    throw new AvroRuntimeException("Not a map: "+this);
-  }
-
   /** If this is a map, returns its value type. */
   public Schema getValueType() {
     throw new AvroRuntimeException("Not a map: "+this);
@@ -274,29 +270,23 @@
   }
 
   static class MapSchema extends Schema {
-    private final Schema keyType;
     private final Schema valueType;
-    public MapSchema(Schema keyType, Schema valueType) {
+    public MapSchema(Schema valueType) {
       super(Type.MAP);
-      this.keyType = keyType;
       this.valueType = valueType;
     }
-    public Schema getKeyType() { return keyType; }
     public Schema getValueType() { return valueType; }
     public boolean equals(Object o) {
       if (o == this) return true;
       return o instanceof MapSchema
-        && keyType.equals(((MapSchema)o).keyType)
         && valueType.equals(((MapSchema)o).valueType);
     }
     public int hashCode() {
-      return getType().hashCode()+keyType.hashCode()+valueType.hashCode();
+      return getType().hashCode()+valueType.hashCode();
     }
     public String toString(Map<String,Schema> names) {
       StringBuilder buffer = new StringBuilder();
-      buffer.append("{\"type\": \"map\", \"keys\":  ");
-      buffer.append(keyType.toString(names));
-      buffer.append(", \"values\": ");
+      buffer.append("{\"type\": \"map\", \"values\": ");
       buffer.append(valueType.toString(names));
       buffer.append("}");
       return buffer.toString();
@@ -481,8 +471,7 @@
       } else if (type.equals("array")) {          // array
         return new ArraySchema(parse(schema.getFieldValue("items"), names));
       } else if (type.equals("map")) {            // map
-        return new MapSchema(parse(schema.getFieldValue("keys"), names),
-                             parse(schema.getFieldValue("values"), names));
+        return new MapSchema(parse(schema.getFieldValue("values"), names));
       } else
         throw new SchemaParseException("Type not yet supported: "+type);
     } else if (schema.isArray()) {                // union

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=769143&r1=769142&r2=769143&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 Mon Apr 27 20:18:06 2009
@@ -117,8 +117,7 @@
       @SuppressWarnings(value="unchecked")
       Map<Object,Object> map = (Map<Object,Object>)datum;
       for (Map.Entry<Object,Object> entry : map.entrySet())
-        if (!(validate(schema.getKeyType(), entry.getKey()) &&
-              validate(schema.getValueType(), entry.getValue()))) 
+        if (!validate(schema.getValueType(), entry.getValue()))
           return false;
       return true;
     case UNION:
@@ -204,7 +203,7 @@
       Map<String,Schema> fields = new LinkedHashMap<String,Schema>();
       for (Map.Entry<String,Object> entry : record.entrySet())
         fields.put(entry.getKey(), induce(entry.getValue()));
-      return Schema.create(fields);
+      return Schema.createRecord(fields);
     } else if (datum instanceof GenericArray) {
       Schema elementType = null;
       for (Object element : (GenericArray)datum) {
@@ -217,27 +216,23 @@
       if (elementType == null) {
         throw new AvroTypeException("Empty array: "+datum);
       }
-      return Schema.create(elementType);
+      return Schema.createArray(elementType);
 
     } else if (datum instanceof Map) {
       @SuppressWarnings(value="unchecked")
       Map<Object,Object> map = (Map<Object,Object>)datum;
-      Schema key = null;
       Schema value = null;
       for (Map.Entry<Object,Object> entry : map.entrySet()) {
-        if (key == null) {
-          key = induce(entry.getKey());
+        if (value == null) {
           value = induce(entry.getValue());
-        } else if (!key.equals(induce(entry.getKey()))) {
-          throw new AvroTypeException("No mixed type map keys.");
         } else if (!value.equals(induce(entry.getValue()))) {
           throw new AvroTypeException("No mixed type map values.");
         }
       }
-      if (key == null) {
+      if (value == null) {
         throw new AvroTypeException("Empty map: "+datum);
       }
-      return Schema.create(key, value);
+      return Schema.createMap(value);
     }
     else if (datum instanceof Utf8)       return Schema.create(Type.STRING);
     else if (datum instanceof ByteBuffer) return Schema.create(Type.BYTES);

Modified: hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumReader.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumReader.java?rev=769143&r1=769142&r2=769143&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumReader.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumReader.java Mon Apr 27 20:18:06 2009
@@ -208,16 +208,15 @@
   @SuppressWarnings(value="unchecked")
   protected Object readMap(Object old, Schema actual, Schema expected,
                            ValueReader in) throws IOException {
-    Schema aKey = actual.getKeyType();
     Schema aValue = actual.getValueType();
-    Schema eKey = expected.getKeyType();
     Schema eValue = expected.getValueType();
     int firstBlockSize = (int)in.readLong();
     Object map = newMap(old, firstBlockSize);
     for (long l = firstBlockSize; l > 0; l = in.readLong())
       for (long i = 0; i < l; i++)
-        addToMap(map, read(null, aKey, eKey, in),
-            read(null, aValue, eValue, in));
+        addToMap(map,
+                 readString(null, in),
+                 read(null, aValue, eValue, in));
     return map;
   }
 
@@ -279,6 +278,8 @@
     return in.readBuffer(old);
   }
 
+  private static final Schema STRING_SCHEMA = Schema.create(Type.STRING);
+
   /** Skip an instance of a schema. */
   public static void skip(Schema schema, ValueReader in) throws IOException {
     switch (schema.getType()) {
@@ -293,11 +294,10 @@
           skip(elementType, in);
       break;
     case MAP:
-      Schema key = schema.getKeyType();
       Schema value = schema.getValueType();
       for (int l = (int)in.readLong(); l > 0; l = (int)in.readLong())
         for (int i = 0; i < l; i++) {
-          skip(key, in);
+          skip(STRING_SCHEMA, in);
           skip(value, in);
         }
       break;

Modified: hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumWriter.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumWriter.java?rev=769143&r1=769142&r2=769143&view=diff
==============================================================================
--- hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumWriter.java (original)
+++ hadoop/avro/trunk/src/java/org/apache/avro/generic/GenericDatumWriter.java Mon Apr 27 20:18:06 2009
@@ -122,13 +122,12 @@
    * representations.*/
   protected void writeMap(Schema schema, Object datum, ValueWriter out)
     throws IOException {
-    Schema key = schema.getKeyType();
     Schema value = schema.getValueType();
     int size = getMapSize(datum);
     if (size > 0) {
       out.writeLong(size);                // write a single block
       for (Map.Entry<Object,Object> entry : getMapEntries(datum)) {
-        write(key, entry.getKey(), out);
+        writeString(entry.getKey(), out);
         write(value, entry.getValue(), out);
       }
     }

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=769143&r1=769142&r2=769143&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 Mon Apr 27 20:18:06 2009
@@ -134,12 +134,13 @@
       if (GenericArray.class.isAssignableFrom(raw)) { // array
         if (params.length != 1)
           throw new AvroTypeException("No array type specified.");
-        return Schema.create(createSchema(params[0], names));
+        return Schema.createArray(createSchema(params[0], names));
       } else if (Map.class.isAssignableFrom(raw)) { // map
         java.lang.reflect.Type key = params[0];
         java.lang.reflect.Type value = params[1];
-        return Schema.create(createSchema(key, names),
-                             createSchema(value, names));
+        if (!(key == Utf8.class))
+          throw new AvroTypeException("Map key class not Utf8: "+key);
+        return Schema.createMap(createSchema(value, names));
       }
     } else if (type instanceof Class) {             // record
       Class c = (Class)type;
@@ -147,8 +148,8 @@
       Schema schema = names.get(name);
       if (schema == null) {
         Map<String,Schema> fields = new LinkedHashMap<String,Schema>();
-        schema = Schema.create(name, c.getPackage().getName(),
-                               Throwable.class.isAssignableFrom(c));
+        schema = Schema.createRecord(name, c.getPackage().getName(),
+                                     Throwable.class.isAssignableFrom(c));
         if (!names.containsKey(name))
           names.put(name, schema);
         for (Field field : c.getDeclaredFields())
@@ -187,7 +188,7 @@
     java.lang.reflect.Type[] paramTypes = method.getGenericParameterTypes();
     for (int i = 0; i < paramTypes.length; i++)
       fields.put(paramNames[i], createSchema(paramTypes[i], names));
-    Schema request = Schema.create(fields);
+    Schema request = Schema.createRecord(fields);
 
     Schema response = createSchema(method.getGenericReturnType(), names);
 
@@ -196,7 +197,7 @@
     for (java.lang.reflect.Type err : method.getGenericExceptionTypes())
       if (err != AvroRemoteException.class) 
         errs.add(createSchema(err, names));
-    Schema errors = Schema.create(errs);
+    Schema errors = Schema.createUnion(errs);
 
     return protocol.createMessage(method.getName(), request, response, errors);
   }

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=769143&r1=769142&r2=769143&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 Mon Apr 27 20:18:06 2009
@@ -170,7 +170,6 @@
       compile(schema.getElementType(), name+"Element", d);
       break;
     case MAP:
-      compile(schema.getKeyType(), name+"Key", d);
       compile(schema.getValueType(), name+"Value", d);
       break;
     case UNION:
@@ -195,9 +194,7 @@
     case ARRAY:
       return "GenericArray<"+type(schema.getElementType(),name+"Element")+">";
     case MAP:
-      return "Map<"
-        +type(schema.getKeyType(),name+"Key")+","
-        +type(schema.getValueType(),name+"Value")+">";
+      return "Map<Utf8,"+type(schema.getValueType(),name+"Value")+">";
     case UNION:   return "Object";
     case STRING:  return "Utf8";
     case BYTES:   return "ByteBuffer";

Modified: hadoop/avro/trunk/src/py/avro/generic.py
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/py/avro/generic.py?rev=769143&r1=769142&r2=769143&view=diff
==============================================================================
--- hadoop/avro/trunk/src/py/avro/generic.py (original)
+++ hadoop/avro/trunk/src/py/avro/generic.py Mon Apr 27 20:18:06 2009
@@ -46,8 +46,7 @@
   if not isinstance(object, dict):
     return False
   for k,v in object.items():
-    if not (validate(schm.getkeytype(), k) and 
-            validate(schm.getvaluetype(), v)):
+    if not validate(schm.getvaluetype(), v):
       return False
   return True
 
@@ -131,7 +130,7 @@
     size = valuereader.readlong()
     if size != 0:
       for i in range(0, size):
-        key = self.readdata(schm.getkeytype(), valuereader)
+        key = valuereader.readutf8()
         result[key] = self.readdata(schm.getvaluetype(), valuereader)
       valuereader.readlong()
     return result
@@ -161,20 +160,20 @@
   def __init__(self, schm=None):
     self.setschema(schm)
     self.__writefn = {
-     schema.BOOLEAN : lambda schm, datum, valuereader: 
-                  valuereader.writeboolean(datum),
-     schema.STRING : lambda schm, datum, valuereader: 
-                  valuereader.writeutf8(datum),
-     schema.INT : lambda schm, datum, valuereader: 
-                  valuereader.writeint(datum),
-     schema.LONG : lambda schm, datum, valuereader: 
-                  valuereader.writelong(datum),
-     schema.FLOAT : lambda schm, datum, valuereader: 
-                  valuereader.writefloat(datum),
-     schema.DOUBLE : lambda schm, datum, valuereader: 
-                  valuereader.writedouble(datum),
-     schema.BYTES : lambda schm, datum, valuereader: 
-                  valuereader.writebytes(datum),
+     schema.BOOLEAN : lambda schm, datum, valuewriter: 
+                  valuewriter.writeboolean(datum),
+     schema.STRING : lambda schm, datum, valuewriter: 
+                  valuewriter.writeutf8(datum),
+     schema.INT : lambda schm, datum, valuewriter: 
+                  valuewriter.writeint(datum),
+     schema.LONG : lambda schm, datum, valuewriter: 
+                  valuewriter.writelong(datum),
+     schema.FLOAT : lambda schm, datum, valuewriter: 
+                  valuewriter.writefloat(datum),
+     schema.DOUBLE : lambda schm, datum, valuewriter: 
+                  valuewriter.writedouble(datum),
+     schema.BYTES : lambda schm, datum, valuewriter: 
+                  valuewriter.writebytes(datum),
      schema.ARRAY : self.writearray,
      schema.MAP : self.writemap,
      schema.RECORD : self.writerecord,
@@ -204,7 +203,7 @@
     if len(datum) > 0:
       valuewriter.writelong(len(datum))
       for k,v in datum.items():
-        self.writedata(schm.getkeytype(), k, valuewriter)
+        valuewriter.writeutf8(k)
         self.writedata(schm.getvaluetype(), v, valuewriter)
     valuewriter.writelong(0)
 

Modified: hadoop/avro/trunk/src/py/avro/reflect.py
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/py/avro/reflect.py?rev=769143&r1=769142&r2=769143&view=diff
==============================================================================
--- hadoop/avro/trunk/src/py/avro/reflect.py (original)
+++ hadoop/avro/trunk/src/py/avro/reflect.py Mon Apr 27 20:18:06 2009
@@ -37,8 +37,7 @@
   if not isinstance(object, dict):
     return False
   for k,v in object.items():
-    if not (validate(schm.getkeytype(), pkgname, k) and 
-            validate(schm.getvaluetype(), pkgname, v)):
+    if not validate(schm.getvaluetype(), pkgname, v):
       return False
   return True
 

Modified: hadoop/avro/trunk/src/py/avro/schema.py
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/py/avro/schema.py?rev=769143&r1=769142&r2=769143&view=diff
==============================================================================
--- hadoop/avro/trunk/src/py/avro/schema.py (original)
+++ hadoop/avro/trunk/src/py/avro/schema.py Mon Apr 27 20:18:06 2009
@@ -18,7 +18,7 @@
 A schema may be one of:
   An record, mapping field names to field value data;
   An array of values, all of the same schema;
-  A map containing key/value pairs, each of a declared schema;
+  A map containing string/value pairs, each of a declared schema;
   A union of other schemas;
   A unicode string;
   A sequence of bytes;
@@ -210,22 +210,16 @@
     return self.gettype().__hash__() + self.__elemtype.__hash__(seen)
 
 class _MapSchema(Schema):
-  def __init__(self, keytype, valuetype):
+  def __init__(self, valuetype):
     Schema.__init__(self, MAP)
-    self.__ktype = keytype
     self.__vtype = valuetype
 
-  def getkeytype(self):
-    return self.__ktype
-
   def getvaluetype(self):
     return self.__vtype
 
   def str(self, names):
     str = cStringIO.StringIO()
-    str.write("{\"type\": \"map\", \"keys\":  ")
-    str.write(self.__ktype.str(names))
-    str.write(", \"values\": ");
+    str.write("{\"type\": \"map\", \"values\":  ")
     str.write(self.__vtype.str(names));
     str.write("}")
     return str.getvalue()
@@ -234,16 +228,14 @@
     if self is other or seen.get(id(self)) is other:
       return True
     seen[id(self)]= other
-    return (isinstance(other, _MapSchema) and 
-            self.__ktype.__eq__(other.__ktype, seen) and 
+    return (isinstance(other, _MapSchema) and
             self.__vtype.__eq__(other.__vtype), seen)
 
   def __hash__(self, seen=set()):
     if seen.__contains__(id(self)):
       return 0
     seen.add(id(self))
-    return (self.gettype().__hash__() + 
-            self.__ktype.__hash__(seen) +
+    return (self.gettype().__hash__() +
             self.__vtype.__hash__(seen))
 
 class _UnionSchema(Schema):
@@ -353,8 +345,7 @@
     elif type == "array":
       return _ArraySchema(_parse(obj.get("items"), names))
     elif type == "map":
-      return _MapSchema(_parse(obj.get("keys"), names), 
-                       _parse(obj.get("values"), names))
+      return _MapSchema(_parse(obj.get("values"), names))
     else:
       raise SchemaParseException("Type not yet supported: "+type.__str__())
   elif isinstance(obj, list):

Modified: hadoop/avro/trunk/src/test/java/org/apache/avro/RandomData.java
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/test/java/org/apache/avro/RandomData.java?rev=769143&r1=769142&r2=769143&view=diff
==============================================================================
--- hadoop/avro/trunk/src/test/java/org/apache/avro/RandomData.java (original)
+++ hadoop/avro/trunk/src/test/java/org/apache/avro/RandomData.java Mon Apr 27 20:18:06 2009
@@ -80,7 +80,7 @@
       length = (random.nextInt(5)+2)-d;
       Map<Object,Object> map = new HashMap<Object,Object>(length<=0?0:length);
       for (int i = 0; i < length; i++) {
-        map.put(generate(schema.getKeyType(), random, d+1),
+        map.put(randomUtf8(random, 40),
                 generate(schema.getValueType(), random, d+1));
       }
       return map;

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=769143&r1=769142&r2=769143&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 Mon Apr 27 20:18:06 2009
@@ -66,7 +66,7 @@
   }
 
   public void testMap() throws Exception {
-    check("{\"type\":\"map\", \"keys\": \"long\", \"values\": \"string\"}");
+    check("{\"type\":\"map\", \"values\": \"string\"}");
   }
 
   public void testRecord() throws Exception {

Modified: hadoop/avro/trunk/src/test/py/testio.py
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/src/test/py/testio.py?rev=769143&r1=769142&r2=769143&view=diff
==============================================================================
--- hadoop/avro/trunk/src/test/py/testio.py (original)
+++ hadoop/avro/trunk/src/test/py/testio.py Mon Apr 27 20:18:06 2009
@@ -70,7 +70,7 @@
       if len < 0:
         len = 0
       for i in range(1, len):
-        map[self.nextdata(schm.getkeytype(), d+1)] = self.nextdata(
+        map[self.nextdata(schema._StringSchema())] = self.nextdata(
                                                     schm.getvaluetype(), d+1)
       return map
     elif schm.gettype() == schema.RECORD:
@@ -123,8 +123,7 @@
     self.check("{\"type\":\"array\", \"items\": \"long\"}")
 
   def testMap(self):
-    self.check("{\"type\":\"map\", \"keys\": \"long\", \"values\": "+
-          "\"string\"}")
+    self.check("{\"type\":\"map\", \"values\": \"string\"}")
 
   def testRecord(self):
     self.check("{\"type\":\"record\",\"fields\":[{\"name\":\"f\", \"type\":" +