Author: daijy
Date: Mon Dec  8 17:56:43 2014
New Revision: 1643872

URL: http://svn.apache.org/r1643872
Log:
PIG-4326: AvroStorageSchemaConversionUtilities does not properly convert schema 
for maps of arrays of records

Added:
    
pig/trunk/test/org/apache/pig/builtin/avro/data/json/recordsWithMapsOfArrayOfRecords.json
    
pig/trunk/test/org/apache/pig/builtin/avro/schema/recordsWithMapsOfArrayOfRecords.avsc
Modified:
    pig/trunk/CHANGES.txt
    
pig/trunk/src/org/apache/pig/impl/util/avro/AvroStorageSchemaConversionUtilities.java
    pig/trunk/test/org/apache/pig/builtin/TestAvroStorage.java

Modified: pig/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1643872&r1=1643871&r2=1643872&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Mon Dec  8 17:56:43 2014
@@ -28,6 +28,8 @@ PIG-4333: Split BigData tests into multi
  
 BUG FIXES
 
+PIG-4326: AvroStorageSchemaConversionUtilities does not properly convert 
schema for maps of arrays of records (mprim via daijy)
+
 PIG-4345: e2e test "RubyUDFs_13" fails because of the different result of 
"group a all" in different engines like "spark", "mapreduce" (kellyzly via 
rohini)
 
 PIG-4332: Remove redundant jars packaged into pig-withouthadoop.jar for hadoop 
2 (rohini)

Modified: 
pig/trunk/src/org/apache/pig/impl/util/avro/AvroStorageSchemaConversionUtilities.java
URL: 
http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/impl/util/avro/AvroStorageSchemaConversionUtilities.java?rev=1643872&r1=1643871&r2=1643872&view=diff
==============================================================================
--- 
pig/trunk/src/org/apache/pig/impl/util/avro/AvroStorageSchemaConversionUtilities.java
 (original)
+++ 
pig/trunk/src/org/apache/pig/impl/util/avro/AvroStorageSchemaConversionUtilities.java
 Mon Dec  8 17:56:43 2014
@@ -219,21 +219,32 @@ public class AvroStorageSchemaConversion
       ResourceSchema mapSchema = new ResourceSchema();
       ResourceSchema.ResourceFieldSchema[] mapSchemaFields =
           new ResourceSchema.ResourceFieldSchema[1];
-      if (mapAvroSchema.getType() == Type.RECORD) {
-        ResourceSchema innerResourceSchema =
-            avroSchemaToResourceSchema(fieldSchema.getValueType(), 
schemasInStack,
-            alreadyDefinedSchemas, allowRecursiveSchema);
+      switch(mapAvroSchema.getType()) {
+      case RECORD:
+        ResourceSchema innerResourceSchemaRecord =
+          avroSchemaToResourceSchema(fieldSchema.getValueType(), 
schemasInStack,
+          alreadyDefinedSchemas, allowRecursiveSchema);
         mapSchemaFields[0] = new ResourceSchema.ResourceFieldSchema();
         mapSchemaFields[0].setType(DataType.TUPLE);
         mapSchemaFields[0].setName(mapAvroSchema.getName());
-        mapSchemaFields[0].setSchema(innerResourceSchema);
+        mapSchemaFields[0].setSchema(innerResourceSchemaRecord);
         mapSchemaFields[0].setDescription(fieldSchema.getDoc());
-      } else {
+        mapSchema.setFields(mapSchemaFields);
+        rf.setSchema(mapSchema);
+        break;
+      case MAP:
+      case ARRAY:
+        ResourceSchema innerResourceSchema =
+            avroSchemaToResourceSchema(fieldSchema.getValueType(), 
schemasInStack,
+            alreadyDefinedSchemas, allowRecursiveSchema);
+        rf.setSchema(innerResourceSchema);
+        break;
+      default:
         mapSchemaFields[0] = new ResourceSchema.ResourceFieldSchema();
         mapSchemaFields[0].setType(getPigType(mapAvroSchema));
+        mapSchema.setFields(mapSchemaFields);
+        rf.setSchema(mapSchema);
       }
-      mapSchema.setFields(mapSchemaFields);
-      rf.setSchema(mapSchema);
     }
       break;
     case DataType.TUPLE:

Modified: pig/trunk/test/org/apache/pig/builtin/TestAvroStorage.java
URL: 
http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/builtin/TestAvroStorage.java?rev=1643872&r1=1643871&r2=1643872&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/builtin/TestAvroStorage.java (original)
+++ pig/trunk/test/org/apache/pig/builtin/TestAvroStorage.java Mon Dec  8 
17:56:43 2014
@@ -103,6 +103,7 @@ public class TestAvroStorage {
         "recordsWithFixed",
         "recordsWithMaps",
         "recordsWithMapsOfRecords",
+        "recordsWithMapsOfArrayOfRecords",
         "recordsWithNullableUnions",
         "recordWithRepeatedSubRecords",
         "recursiveRecord",
@@ -543,6 +544,18 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
+    @Test public void testLoadRecordsWithMapsOfArrayOfRecords() throws 
Exception {
+        final String input = basedir + 
"data/avro/uncompressed/recordsWithMapsOfArrayOfRecords.avro";
+        final String check = input;
+        testAvroStorage(true, basedir + "code/pig/identity_just_ao2.pig",
+            ImmutableMap.of(
+                "INFILE",             input,
+                "AVROSTORAGE_OUT_2", "-f " + basedir + 
"schema/recordsWithMapsOfArrayOfRecords.avsc",
+                "OUTFILE",            createOutputName())
+          );
+        verifyResults(createOutputName(),check);
+      }
+
     @Test public void testLoadRecordsWithNullableUnions() throws Exception {
       final String input = basedir + 
"data/avro/uncompressed/recordsWithNullableUnions.avro";
       final String check = input;

Added: 
pig/trunk/test/org/apache/pig/builtin/avro/data/json/recordsWithMapsOfArrayOfRecords.json
URL: 
http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/builtin/avro/data/json/recordsWithMapsOfArrayOfRecords.json?rev=1643872&view=auto
==============================================================================
--- 
pig/trunk/test/org/apache/pig/builtin/avro/data/json/recordsWithMapsOfArrayOfRecords.json
 (added)
+++ 
pig/trunk/test/org/apache/pig/builtin/avro/data/json/recordsWithMapsOfArrayOfRecords.json
 Mon Dec  8 17:56:43 2014
@@ -0,0 +1,25 @@
+{"key" : "A",
+ "value" : 1,
+ "parameters" : {
+   "firstParameter" : []
+  }
+}
+{"key" : "B",
+ "value" : 2,
+ "parameters" : {}
+}
+{"key" : "C",
+ "value" : 3,
+ "parameters" : {
+     "firstParameter" : [
+       {"k" : "c", "v" : 3},
+       {"k" : "c1", "v" : 4},
+       {"k" : "c2", "v" : 5}
+     ],
+     "secondParameter" : [
+       {"k" : "c3", "v" : 6},
+       {"k" : "c4", "v" : 7},
+       {"k" : "c5", "v" : 8}
+     ]
+  }
+}

Added: 
pig/trunk/test/org/apache/pig/builtin/avro/schema/recordsWithMapsOfArrayOfRecords.avsc
URL: 
http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/builtin/avro/schema/recordsWithMapsOfArrayOfRecords.avsc?rev=1643872&view=auto
==============================================================================
--- 
pig/trunk/test/org/apache/pig/builtin/avro/schema/recordsWithMapsOfArrayOfRecords.avsc
 (added)
+++ 
pig/trunk/test/org/apache/pig/builtin/avro/schema/recordsWithMapsOfArrayOfRecords.avsc
 Mon Dec  8 17:56:43 2014
@@ -0,0 +1,24 @@
+{
+  "name" : "recordsWithMapsOfArrayOfRecords",
+  "namespace" : "org.apache.pig.test.builtin",
+  "type" : "record",
+  "fields" : [
+    {"name" : "key", "type" : "string"},
+    {"name" : "value", "type" : "int"},
+    {"name" : "parameters", "type" :
+      {"type" : "map",
+       "values" : {
+         "type" : "array",
+         "items" : {
+            "name" : "innerRecord",
+            "type" : "record",
+            "fields" : [
+              {"name" : "k", "type" : "string"},
+              {"name" : "v", "type" : "int"}
+            ]
+          }
+        }
+      }  
+    }    
+  ]
+}


Reply via email to