avro-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From b...@apache.org
Subject svn commit: r1666774 - in /avro/trunk: ./ lang/java/avro/src/main/java/org/apache/avro/generic/ lang/java/avro/src/main/java/org/apache/avro/reflect/ lang/java/avro/src/test/java/org/apache/avro/reflect/
Date Sun, 15 Mar 2015 03:26:58 GMT
Author: blue
Date: Sun Mar 15 03:26:57 2015
New Revision: 1666774

URL: http://svn.apache.org/r1666774
Log:
AVRO-680. Java: Support non-string map keys. Contributed by Sachin Goyal.

Added:
    avro/trunk/lang/java/avro/src/test/java/org/apache/avro/reflect/TestNonStringMapKeys.java
Modified:
    avro/trunk/CHANGES.txt
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectData.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumReader.java
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumWriter.java

Modified: avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/avro/trunk/CHANGES.txt?rev=1666774&r1=1666773&r2=1666774&view=diff
==============================================================================
--- avro/trunk/CHANGES.txt (original)
+++ avro/trunk/CHANGES.txt Sun Mar 15 03:26:57 2015
@@ -40,6 +40,8 @@ Trunk (not yet released)
     Support for running a Docker container with all build dependencies.
     (tomwhite)
 
+    AVRO-680. Java: Support non-string map keys. (Sachin Goyal via Ryan Blue).
+
   OPTIMIZATIONS
 
   IMPROVEMENTS

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java?rev=1666774&r1=1666773&r2=1666774&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java Sun Mar
15 03:26:57 2015
@@ -387,7 +387,7 @@ public class GenericData {
       return schema.getEnumSymbols().contains(datum.toString());
     case ARRAY:
       if (!(isArray(datum))) return false;
-      for (Object element : (Collection<?>)datum)
+      for (Object element : getArrayAsCollection(datum))
         if (!validate(schema.getElementType(), element))
           return false;
       return true;
@@ -442,7 +442,7 @@ public class GenericData {
       }
       buffer.append("}");
     } else if (isArray(datum)) {
-      Collection<?> array = (Collection<?>)datum;
+      Collection<?> array = getArrayAsCollection(datum);
       buffer.append("[");
       long last = array.size()-1;
       int i = 0;
@@ -534,7 +534,7 @@ public class GenericData {
       return getRecordSchema(datum);
     } else if (isArray(datum)) {
       Schema elementType = null;
-      for (Object element : (Collection<?>)datum) {
+      for (Object element : getArrayAsCollection(datum)) {
         if (elementType == null) {
           elementType = induce(element);
         } else if (!elementType.equals(induce(element))) {
@@ -683,6 +683,11 @@ public class GenericData {
     return datum instanceof Collection;
   }
 
+  /** Called to access an array as a collection. */
+  protected Collection getArrayAsCollection(Object datum) {
+    return (Collection)datum;
+  }
+
   /** Called by the default implementation of {@link #instanceOf}.*/
   protected boolean isRecord(Object datum) {
     return datum instanceof IndexedRecord;

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectData.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectData.java?rev=1666774&r1=1666773&r2=1666774&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectData.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectData.java Sun Mar
15 03:26:57 2015
@@ -18,6 +18,7 @@
 package org.apache.avro.reflect;
 
 import java.io.IOException;
+import java.io.UnsupportedEncodingException;
 import java.lang.annotation.Annotation;
 import java.lang.reflect.Field;
 import java.lang.reflect.GenericArrayType;
@@ -52,6 +53,7 @@ import org.apache.avro.io.DatumReader;
 import org.apache.avro.io.DatumWriter;
 import org.apache.avro.specific.FixedSize;
 import org.apache.avro.specific.SpecificData;
+import org.apache.avro.SchemaNormalization;
 import org.codehaus.jackson.JsonNode;
 import org.codehaus.jackson.node.NullNode;
 
@@ -173,10 +175,21 @@ public class ReflectData extends Specifi
     return getSchema(datum.getClass()).getType() == Schema.Type.RECORD;
   }
 
+  /**
+   * Returns true also for non-string-keyed maps, which are written as an array
+   * of key/value pair records.
+   */
   @Override
   protected boolean isArray(Object datum) {
     if (datum == null) return false;
-    return (datum instanceof Collection) || datum.getClass().isArray();
+    return (datum instanceof Collection)
+      || datum.getClass().isArray()
+      || isNonStringMap(datum);
+  }
+
+  @Override
+  protected Collection getArrayAsCollection(Object datum) {
+    return (datum instanceof Map) ? ((Map)datum).entrySet() : (Collection)datum;
   }
 
   @Override
@@ -337,6 +350,33 @@ public class ReflectData extends Specifi
     ARRAY_CLASSES.put(boolean.class, boolean[].class);
   }
 
+  /**
+   * It returns false for non-string-maps because Avro writes out such maps
+   * as an array of records. Even their JSON representation is an array.
+   */
+  protected boolean isMap(Object datum) {
+    return (datum instanceof Map) && !isNonStringMap(datum);
+  }
+
+  /* Without the Field or Schema corresponding to the datum, it is
+   * not possible to accurately find out the non-stringable nature
+   * of the key. So we check the class of the keys.
+   * If the map is empty, then it doesn't matter whether its considered
+   * a string-key map or a non-string-key map
+   */
+  private boolean isNonStringMap(Object datum) {
+    if (datum instanceof Map) {
+      Map m = (Map)datum;
+      if (m.size() > 0) {
+        Class keyClass = m.keySet().iterator().next().getClass();
+        if (isStringable(keyClass) || keyClass == String.class)
+          return false;
+        return true;
+      }
+    }
+    return false;
+  }
+
   @Override
   public Class getClass(Schema schema) {
     switch (schema.getType()) {
@@ -367,6 +407,78 @@ public class ReflectData extends Specifi
     }
   }
 
+  static final String NS_MAP_ARRAY_RECORD =   // record name prefix
+    "org.apache.avro.reflect.Pair";
+  static final String NS_MAP_KEY = "key";     // name of key field
+  static final int NS_MAP_KEY_INDEX = 0;      // index of key field
+  static final String NS_MAP_VALUE = "value"; // name of value field
+  static final int NS_MAP_VALUE_INDEX = 1;    // index of value field
+
+  /*
+   * Non-string map-keys need special handling and we convert it to an
+   * array of records as: [{"key":{...}, "value":{...}}]
+   */
+  Schema createNonStringMapSchema(Type keyType, Type valueType,
+                                  Map<String, Schema> names) {
+    Schema keySchema = createSchema(keyType, names);
+    Schema valueSchema = createSchema(valueType, names);
+    Schema.Field keyField = 
+      new Schema.Field(NS_MAP_KEY, keySchema, null, null);
+    Schema.Field valueField = 
+      new Schema.Field(NS_MAP_VALUE, valueSchema, null, null);
+    String name = getNameForNonStringMapRecord(keyType, valueType,
+      keySchema, valueSchema);
+    Schema elementSchema = Schema.createRecord(name, null, null, false);
+    elementSchema.setFields(Arrays.asList(keyField, valueField));
+    Schema arraySchema = Schema.createArray(elementSchema);
+    return arraySchema;
+  }
+
+  /*
+   * Gets a unique and consistent name per key-value pair. So if the same
+   * key-value are seen in another map, the same name is generated again.
+   */
+  private String getNameForNonStringMapRecord(Type keyType, Type valueType,
+                                  Schema keySchema, Schema valueSchema) {
+
+    // Generate a nice name for classes in java* package
+    if (keyType instanceof Class && valueType instanceof Class) {
+
+      Class keyClass = (Class)keyType;
+      Class valueClass = (Class)valueType;
+      Package pkg1 = keyClass.getPackage();
+      Package pkg2 = valueClass.getPackage();
+
+      if (pkg1 != null && pkg1.getName().startsWith("java") &&
+        pkg2 != null && pkg2.getName().startsWith("java")) {
+        return NS_MAP_ARRAY_RECORD +
+          keyClass.getSimpleName() + valueClass.getSimpleName();
+      }
+    }
+
+    String name = keySchema.getFullName() + valueSchema.getFullName();
+    long fingerprint = 0;
+    try {
+      fingerprint = SchemaNormalization.fingerprint64(name.getBytes("UTF-8"));
+    } catch (UnsupportedEncodingException e) {
+      String msg = "Unable to create fingerprint for ("
+                   + keyType + ", "  + valueType + ") pair";
+      throw new AvroRuntimeException(msg, e);
+    }
+    if (fingerprint < 0) fingerprint = -fingerprint;  // ignore sign
+    String fpString = Long.toString(fingerprint, 16); // hex
+    return NS_MAP_ARRAY_RECORD + fpString;
+  }
+
+  static boolean isNonStringMapSchema(Schema s) {
+    if (s != null && s.getType() == Schema.Type.ARRAY) {
+      Class c = getClassProp(s, CLASS_PROP);
+      if (c != null && Map.class.isAssignableFrom (c))
+        return true;
+    }
+    return false;
+  }
+
   @Override
   protected Schema createSchema(Type type, Map<String,Schema> names) {
     if (type instanceof GenericArrayType) {                  // generic array
@@ -381,14 +493,16 @@ public class ReflectData extends Specifi
       Class raw = (Class)ptype.getRawType();
       Type[] params = ptype.getActualTypeArguments();
       if (Map.class.isAssignableFrom(raw)) {                 // Map
-        Schema schema = Schema.createMap(createSchema(params[1], names));
         Class key = (Class)params[0];
         if (isStringable(key)) {                             // Stringable key
+          Schema schema = Schema.createMap(createSchema(params[1], names));
           schema.addProp(KEY_CLASS_PROP, key.getName());
+          return schema;
         } else if (key != String.class) {
-          throw new AvroTypeException("Map key class not String: "+key);
+          Schema schema = createNonStringMapSchema(params[0], params[1], names);
+          schema.addProp(CLASS_PROP, raw.getName());
+          return schema;
         }
-        return schema;
       } else if (Collection.class.isAssignableFrom(raw)) {   // Collection
         if (params.length != 1)
           throw new AvroTypeException("No array type specified.");

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumReader.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumReader.java?rev=1666774&r1=1666773&r2=1666774&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumReader.java
(original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumReader.java
Sun Mar 15 03:26:57 2015
@@ -22,10 +22,12 @@ import java.lang.reflect.Array;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Map;
 
 import org.apache.avro.AvroRuntimeException;
 import org.apache.avro.Schema;
 import org.apache.avro.Schema.Field;
+import org.apache.avro.generic.IndexedRecord;
 import org.apache.avro.io.Decoder;
 import org.apache.avro.io.ResolvingDecoder;
 import org.apache.avro.specific.SpecificData;
@@ -121,6 +123,24 @@ public class ReflectDatumReader<T> exten
       @SuppressWarnings("unchecked")
       Collection<Object> c = (Collection<Object>) array;
       return readCollection(c, expectedType, l, in);
+    } else if (array instanceof Map) {
+      // Only for non-string keys, we can use NS_MAP_* fields
+      // So we check the samee explicitly here
+      if (ReflectData.isNonStringMapSchema(expected)) {
+        Collection<Object> c = new ArrayList<Object> ();
+        readCollection(c, expectedType, l, in);
+        Map m = (Map)array;
+        for (Object ele: c) {
+          IndexedRecord rec = ((IndexedRecord)ele);
+          Object key = rec.get(ReflectData.NS_MAP_KEY_INDEX);
+          Object value = rec.get(ReflectData.NS_MAP_VALUE_INDEX);
+          m.put (key, value);
+        }
+        return array;
+      } else {
+        String msg = "Expected a schema of map with non-string keys but got " + expected;
+        throw new AvroRuntimeException(msg);
+      }
     } else {
       return readJavaArray(array, expectedType, l, in);
     }

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumWriter.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumWriter.java?rev=1666774&r1=1666773&r2=1666774&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumWriter.java
(original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumWriter.java
Sun Mar 15 03:26:57 2015
@@ -19,6 +19,7 @@ package org.apache.avro.reflect;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.Map;
 
 import org.apache.avro.AvroRuntimeException;
 import org.apache.avro.Schema;
@@ -139,6 +140,12 @@ public class ReflectDatumWriter<T> exten
       datum = ((Short)datum).intValue();
     else if (datum instanceof Character)
         datum = (int)(char)(Character)datum;
+    else if (datum instanceof Map && ReflectData.isNonStringMapSchema(schema)) {
+        // Maps with non-string keys are written as arrays.
+        // Schema for such maps is already changed. Here we
+        // just switch the map to a similar form too.
+        datum = ((Map)datum).entrySet();
+      }
     try {
       super.write(schema, datum, out);
     } catch (NullPointerException e) {            // improve error message

Added: avro/trunk/lang/java/avro/src/test/java/org/apache/avro/reflect/TestNonStringMapKeys.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/test/java/org/apache/avro/reflect/TestNonStringMapKeys.java?rev=1666774&view=auto
==============================================================================
--- avro/trunk/lang/java/avro/src/test/java/org/apache/avro/reflect/TestNonStringMapKeys.java
(added)
+++ avro/trunk/lang/java/avro/src/test/java/org/apache/avro/reflect/TestNonStringMapKeys.java
Sun Mar 15 03:26:57 2015
@@ -0,0 +1,509 @@
+/**
+ * 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.reflect;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+
+import static org.junit.Assert.*;
+
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.file.SeekableByteArrayInput;
+import org.apache.avro.generic.GenericArray;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.reflect.ReflectData;
+import org.apache.avro.reflect.ReflectDatumReader;
+import org.apache.avro.reflect.ReflectDatumWriter;
+import org.apache.avro.reflect.ReflectData;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.io.Encoder;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.avro.util.Utf8;
+import org.junit.Test;
+
+/**
+ * Test serialization and de-serialization of non-string map-keys
+ */
+public class TestNonStringMapKeys {
+
+  @Test
+  public void testNonStringMapKeys() throws Exception {
+
+    Company entityObj1 = buildCompany();
+    Company entityObj2 = buildCompany();
+
+    String testType = "NonStringKeysTest";
+    Company [] entityObjs = {entityObj1, entityObj2};
+    byte[] bytes = testSerialization(testType, entityObj1, entityObj2);
+    List<GenericRecord> records = 
+      (List<GenericRecord>) testGenericDatumRead(testType, bytes, entityObjs);
+
+    GenericRecord record = records.get(0);
+    Object employees = record.get("employees");
+    assertTrue ("Unable to read 'employees' map", employees instanceof GenericArray);
+    GenericArray arrayEmployees = ((GenericArray)employees);
+    Object employeeRecord = arrayEmployees.get(0);
+    assertTrue (employeeRecord instanceof GenericRecord);
+    Object key = ((GenericRecord)employeeRecord).get(ReflectData.NS_MAP_KEY);
+    Object value = ((GenericRecord)employeeRecord).get(ReflectData.NS_MAP_VALUE);
+    assertTrue (key instanceof GenericRecord);
+    assertTrue (value instanceof GenericRecord);
+    //Map stored: 1:foo, 2:bar
+    Object id = ((GenericRecord)key).get("id");
+    Object name = ((GenericRecord)value).get("name").toString();
+    assertTrue (
+      (id.equals(1) && name.equals("Foo")) || 
+      (id.equals(2) && name.equals("Bar"))
+    );
+
+    List<Company> records2 =
+      (List<Company>) testReflectDatumRead(testType, bytes, entityObjs);
+    Company co = records2.get(0);
+    log ("Read: " + co);
+    assertNotNull (co.getEmployees());
+    assertEquals (2, co.getEmployees().size());
+    Iterator<Entry<EmployeeId, EmployeeInfo>> itr = co.getEmployees().entrySet().iterator();
+    while (itr.hasNext()) {
+      Entry<EmployeeId, EmployeeInfo> e = itr.next();
+      id = e.getKey().getId();
+      name = e.getValue().getName();
+      assertTrue (
+        (id.equals(1) && name.equals("Foo")) || 
+        (id.equals(2) && name.equals("Bar"))
+      );
+    }
+
+
+    byte[] jsonBytes = testJsonEncoder (testType, entityObj1);
+    assertNotNull ("Unable to serialize using jsonEncoder", jsonBytes);
+    GenericRecord jsonRecord = testJsonDecoder(testType, jsonBytes, entityObj1);
+    assertEquals ("JSON decoder output not same as Binary Decoder", record, jsonRecord);
+  }
+  
+  @Test
+  public void testNonStringMapKeysInNestedMaps() throws Exception {
+
+    Company2 entityObj1 = buildCompany2();
+
+    String testType = "NestedMapsTest";
+    Company2 [] entityObjs = {entityObj1};
+    byte[] bytes = testSerialization(testType, entityObj1);
+    List<GenericRecord> records =
+      (List<GenericRecord>) testGenericDatumRead(testType, bytes, entityObjs);
+
+    GenericRecord record = records.get(0);
+    Object employees = record.get("employees");
+    assertTrue ("Unable to read 'employees' map", employees instanceof GenericArray);
+    GenericArray employeesMapArray = ((GenericArray)employees);
+    
+    Object employeeMapElement = employeesMapArray.get(0);
+    assertTrue (employeeMapElement instanceof GenericRecord);
+    Object key = ((GenericRecord)employeeMapElement).get(ReflectData.NS_MAP_KEY);
+    Object value = ((GenericRecord)employeeMapElement).get(ReflectData.NS_MAP_VALUE);
+    assertEquals (11, key);
+    assertTrue (value instanceof GenericRecord);
+    GenericRecord employeeInfo = (GenericRecord)value;
+    Object name = employeeInfo.get("name").toString();
+    assertEquals ("Foo", name);
+    
+    Object companyMap = employeeInfo.get("companyMap");
+    assertTrue (companyMap instanceof GenericArray);
+    GenericArray companyMapArray = (GenericArray)companyMap;
+    
+    Object companyMapElement = companyMapArray.get(0);
+    assertTrue (companyMapElement instanceof GenericRecord);
+    key = ((GenericRecord)companyMapElement).get(ReflectData.NS_MAP_KEY);
+    value = ((GenericRecord)companyMapElement).get(ReflectData.NS_MAP_VALUE);
+    assertEquals (14, key);
+    if (value instanceof Utf8)
+      value = ((Utf8)value).toString();
+    assertEquals ("CompanyFoo", value);
+    
+    List<Company2> records2 =
+      (List<Company2>) testReflectDatumRead(testType, bytes, entityObjs);
+    Company2 co = records2.get(0);
+    log ("Read: " + co);
+    assertNotNull (co.getEmployees());
+    assertEquals (1, co.getEmployees().size());
+    Iterator<Entry<Integer, EmployeeInfo2>> itr = co.getEmployees().entrySet().iterator();
+    while (itr.hasNext()) {
+      Entry<Integer, EmployeeInfo2> e = itr.next();
+      Integer id = e.getKey();
+      name = e.getValue().getName();
+      assertTrue (id.equals(11) && name.equals("Foo"));
+      assertEquals ("CompanyFoo", e.getValue().companyMap.values().iterator().next());
+    }
+
+
+    byte[] jsonBytes = testJsonEncoder (testType, entityObj1);
+    assertNotNull ("Unable to serialize using jsonEncoder", jsonBytes);
+    GenericRecord jsonRecord = testJsonDecoder(testType, jsonBytes, entityObj1);
+    assertEquals ("JSON decoder output not same as Binary Decoder", record, jsonRecord);
+  }
+
+  @Test
+  public void testRecordNameInvariance() throws Exception {
+
+    SameMapSignature entityObj1 = buildSameMapSignature();
+
+    String testType = "RecordNameInvariance";
+    SameMapSignature [] entityObjs = {entityObj1};
+    byte[] bytes = testSerialization(testType, entityObj1);
+    List<GenericRecord> records =
+      (List<GenericRecord>) testGenericDatumRead(testType, bytes, entityObjs);
+
+    GenericRecord record = records.get(0);
+    Object map1obj = record.get("map1");
+    assertTrue ("Unable to read map1", map1obj instanceof GenericArray);
+    GenericArray map1array = ((GenericArray)map1obj);
+    
+    Object map1element = map1array.get(0);
+    assertTrue (map1element instanceof GenericRecord);
+    Object key = ((GenericRecord)map1element).get(ReflectData.NS_MAP_KEY);
+    Object value = ((GenericRecord)map1element).get(ReflectData.NS_MAP_VALUE);
+    assertEquals (1, key);
+    assertEquals ("Foo", value.toString());
+
+    Object map2obj = record.get("map2");
+    assertEquals (map1obj, map2obj);
+    
+    List<SameMapSignature> records2 =
+      (List<SameMapSignature>) testReflectDatumRead(testType, bytes, entityObjs);
+    SameMapSignature entity = records2.get(0);
+    log ("Read: " + entity);
+    assertNotNull (entity.getMap1());
+    assertEquals (1, entity.getMap1().size());
+    Iterator<Entry<Integer, String>> itr = entity.getMap1().entrySet().iterator();
+    while (itr.hasNext()) {
+      Entry<Integer, String> e = itr.next();
+      key = e.getKey();
+      value = e.getValue();
+      assertEquals (1, key);
+      assertEquals ("Foo", value.toString());
+    }
+    assertEquals (entity.getMap1(), entity.getMap2());
+
+
+    ReflectData rdata = ReflectData.get();
+    Schema schema = rdata.getSchema(SameMapSignature.class);
+    Schema map1schema = schema.getField("map1").schema().getElementType();
+    Schema map2schema = schema.getField("map2").schema().getElementType();
+    log ("Schema for map1 = " + map1schema);
+    log ("Schema for map2 = " + map2schema);
+    assertEquals (map1schema.getFullName(), "org.apache.avro.reflect.PairIntegerString");
+    assertEquals (map1schema, map2schema);
+
+
+    byte[] jsonBytes = testJsonEncoder (testType, entityObj1);
+    assertNotNull ("Unable to serialize using jsonEncoder", jsonBytes);
+    GenericRecord jsonRecord = testJsonDecoder(testType, jsonBytes, entityObj1);
+    assertEquals ("JSON decoder output not same as Binary Decoder", 
+      record.get("map1"), jsonRecord.get("map1"));
+    assertEquals ("JSON decoder output not same as Binary Decoder", 
+      record.get("map2"), jsonRecord.get("map2"));
+  }
+
+  /**
+   * Test serialization of non-string map-key POJOs
+   */
+  public <T> byte[] testSerialization(String testType, T ... entityObjs) throws Exception
{
+
+    log ("---- Beginning " + testType + " ----");
+    T entityObj1 = entityObjs[0];
+    ReflectData rdata = ReflectData.AllowNull.get();
+
+    Schema schema = rdata.getSchema(entityObj1.getClass());
+    assertNotNull("Unable to get schema for " + testType, schema);
+    log (schema.toString(true));
+
+    ReflectDatumWriter<T> datumWriter =
+      new ReflectDatumWriter (entityObj1.getClass(), rdata);
+    DataFileWriter<T> fileWriter = new DataFileWriter<T> (datumWriter);
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    fileWriter.create(schema, baos);
+    for (T entityObj : entityObjs) {
+      fileWriter.append(entityObj);
+    }
+    fileWriter.close();
+
+    byte[] bytes = baos.toByteArray();
+    return bytes;
+  }
+
+  /**
+   * Test that non-string map-keys are readable through GenericDatumReader
+   * This methoud should read as array of {key, value} and not as a map
+   */
+  private <T> List<GenericRecord> testGenericDatumRead
+    (String testType, byte[] bytes, T ... entityObjs) throws IOException {
+
+    GenericDatumReader<GenericRecord> datumReader =
+      new GenericDatumReader<GenericRecord> ();
+    SeekableByteArrayInput avroInputStream = new SeekableByteArrayInput(bytes);
+    DataFileReader<GenericRecord> fileReader =
+      new DataFileReader<GenericRecord>(avroInputStream, datumReader);
+
+    Schema schema = fileReader.getSchema();
+    assertNotNull("Unable to get schema for " + testType, schema);
+    GenericRecord record = null;
+    List<GenericRecord> records = new ArrayList<GenericRecord> ();
+    while (fileReader.hasNext()) {
+      records.add (fileReader.next(record));
+    }
+    return records;
+  }
+
+  /**
+   * Test that non-string map-keys are readable through ReflectDatumReader
+   * This methoud should form the original map and should not return any
+   * array of {key, value} as done by {@link #testGenericDatumRead()} 
+   */
+  private <T> List<T> testReflectDatumRead
+    (String testType, byte[] bytes, T ... entityObjs) throws IOException {
+
+    ReflectDatumReader<T> datumReader = new ReflectDatumReader<T> ();
+    SeekableByteArrayInput avroInputStream = new SeekableByteArrayInput(bytes);
+    DataFileReader<T> fileReader = new DataFileReader<T>(avroInputStream, datumReader);
+
+    Schema schema = fileReader.getSchema();
+    T record = null;
+    List<T> records = new ArrayList<T> ();
+    while (fileReader.hasNext()) {
+      records.add (fileReader.next(record));
+    }
+    return records;
+  }
+
+  private <T> byte[] testJsonEncoder
+    (String testType, T entityObj) throws IOException {
+
+    ReflectData rdata = ReflectData.AllowNull.get();
+
+    Schema schema = rdata.getSchema(entityObj.getClass());
+    ByteArrayOutputStream os = new ByteArrayOutputStream();
+    Encoder encoder = EncoderFactory.get().jsonEncoder(schema, os);
+    ReflectDatumWriter<T> datumWriter = new ReflectDatumWriter<T>(schema, rdata);
+    datumWriter.write(entityObj, encoder);
+    encoder.flush();
+
+    byte[] bytes = os.toByteArray();
+    System.out.println ("JSON encoder output:\n" + new String(bytes));
+    return bytes;
+  }
+
+  private <T> GenericRecord testJsonDecoder
+    (String testType, byte[] bytes, T entityObj) throws IOException {
+
+    ReflectData rdata = ReflectData.AllowNull.get();
+
+    Schema schema = rdata.getSchema(entityObj.getClass());
+    GenericDatumReader<GenericRecord> datumReader =
+      new GenericDatumReader<GenericRecord>(schema);
+
+    Decoder decoder = DecoderFactory.get().jsonDecoder(schema, new String(bytes));
+    GenericRecord r = datumReader.read(null, decoder);
+    return r;
+  }
+
+  /**
+   * Create a POJO having non-string map-keys
+   */
+  private Company buildCompany () {
+    Company co = new Company ();
+    HashMap<EmployeeId, EmployeeInfo> employees = new HashMap<EmployeeId, EmployeeInfo>();
+    co.setEmployees(employees);
+    employees.put(new EmployeeId(1), new EmployeeInfo("Foo"));
+    employees.put(new EmployeeId(2), new EmployeeInfo("Bar"));
+    return co;
+  }
+
+  /**
+   * Create a POJO having non-string map-keys
+   * The objects inside that map should also have non-string map-keys
+   */
+  private Company2 buildCompany2 () {
+    Company2 co = new Company2 ();
+    HashMap<Integer, EmployeeInfo2> employees = new HashMap<Integer, EmployeeInfo2>();
+    co.setEmployees(employees);
+    
+    EmployeeId2 empId = new EmployeeId2(1);
+    EmployeeInfo2 empInfo = new EmployeeInfo2("Foo");
+    HashMap<Integer, String> companyMap = new HashMap<Integer, String>();
+    empInfo.setCompanyMap(companyMap);
+    companyMap.put(14, "CompanyFoo");
+    
+    employees.put(11, empInfo);
+    
+    return co;
+  }
+
+  private SameMapSignature buildSameMapSignature () {
+    SameMapSignature obj = new SameMapSignature();
+    obj.setMap1(new HashMap<Integer, String>());
+    obj.getMap1().put(1, "Foo");
+    obj.setMap2(new HashMap<Integer, String>());
+    obj.getMap2().put(1, "Foo");
+    return obj;
+  }
+
+  private void log (String msg) {
+    System.out.println (msg);
+  }
+}
+
+class Company {
+  HashMap <EmployeeId, EmployeeInfo> employees;
+
+  public HashMap<EmployeeId, EmployeeInfo> getEmployees() {
+    return employees;
+  }
+  public void setEmployees(HashMap<EmployeeId, EmployeeInfo> employees) {
+    this.employees = employees;
+  }
+  @Override
+  public String toString() {
+    return "Company [employees=" + employees + "]";
+  }
+}
+
+class EmployeeId {
+  Integer id;
+
+  public EmployeeId() {}
+  public EmployeeId(Integer id) {
+    this.id = id;
+  }
+  public Integer getId() {
+    return id;
+  }
+  public void setId(Integer zip) {
+    this.id = zip;
+  }
+  @Override
+  public String toString() {
+    return "EmployeeId [id=" + id + "]";
+  }
+}
+
+class EmployeeInfo {
+  String name;
+
+  public EmployeeInfo() {}
+  public EmployeeInfo(String name) {
+    this.name = name;
+  }
+  public String getName() {
+    return name;
+  }
+  public void setName(String name) {
+    this.name = name;
+  }
+  @Override
+  public String toString() {
+    return "EmployeeInfo [name=" + name + "]";
+  }
+}
+
+class Company2 {
+  HashMap <Integer, EmployeeInfo2> employees;
+
+  public Company2() {}
+  public HashMap<Integer, EmployeeInfo2> getEmployees() {
+    return employees;
+  }
+  public void setEmployees(HashMap<Integer, EmployeeInfo2> employees) {
+    this.employees = employees;
+  }
+  @Override
+  public String toString() {
+    return "Company2 [employees=" + employees + "]";
+  }
+}
+
+class EmployeeId2 {
+  Integer id;
+
+  public EmployeeId2() {}
+  public EmployeeId2(Integer id) {
+    this.id = id;
+  }
+  public Integer getId() {
+    return id;
+  }
+  public void setId(Integer zip) {
+    this.id = zip;
+  }
+  @Override
+  public String toString() {
+    return "EmployeeId2 [id=" + id + "]";
+  }
+}
+
+class EmployeeInfo2 {
+  String name;
+  HashMap<Integer, String> companyMap;
+
+  public EmployeeInfo2() {}
+  public EmployeeInfo2(String name) {
+    this.name = name;
+  }
+  public String getName() {
+    return name;
+  }
+  public void setName(String name) {
+    this.name = name;
+  }
+  public HashMap<Integer, String> getCompanyMap() {
+    return companyMap;
+  }
+  public void setCompanyMap(HashMap<Integer, String> companyMap) {
+    this.companyMap = companyMap;
+  }
+  @Override
+  public String toString() {
+    return "EmployeeInfo2 [name=" + name + "]";
+  }
+}
+
+class SameMapSignature {
+
+  HashMap<Integer, String> map1;
+  HashMap<Integer, String> map2;
+
+  public HashMap<Integer, String> getMap1() {
+    return map1;
+  }
+  public void setMap1(HashMap<Integer, String> map1) {
+    this.map1 = map1;
+  }
+  public HashMap<Integer, String> getMap2() {
+    return map2;
+  }
+  public void setMap2(HashMap<Integer, String> map2) {
+    this.map2 = map2;
+  }
+}



Mime
View raw message