parquet-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From b...@apache.org
Subject parquet-mr git commit: PARQUET-335: Remove Avro check for MAP_KEY_VALUE.
Date Fri, 11 Sep 2015 17:32:00 GMT
Repository: parquet-mr
Updated Branches:
  refs/heads/master 04f524d5a -> 9962a0fd0


PARQUET-335: Remove Avro check for MAP_KEY_VALUE.

This is not required by the map type spec. This does not affect data
written by the Avro object model because this bug is in the conversion
from a Parquet schema to an Avro schema. Files written with parquet-avro
do not convert the underlying schema because they use the Avro schema.

Author: Ryan Blue <blue@apache.org>

Closes #241 from rdblue/PARQUET-335-remove-key-value-check and squashes the following commits:

1fd9541 [Ryan Blue] PARQUET-335: Test that MAP_KEY_VALUE is not required.
247cc76 [Ryan Blue] PARQUET-335: Remove Avro check for MAP_KEY_VALUE.


Project: http://git-wip-us.apache.org/repos/asf/parquet-mr/repo
Commit: http://git-wip-us.apache.org/repos/asf/parquet-mr/commit/9962a0fd
Tree: http://git-wip-us.apache.org/repos/asf/parquet-mr/tree/9962a0fd
Diff: http://git-wip-us.apache.org/repos/asf/parquet-mr/diff/9962a0fd

Branch: refs/heads/master
Commit: 9962a0fd02fe2ef06765271605b06729af8b2e59
Parents: 04f524d
Author: Ryan Blue <blue@apache.org>
Authored: Fri Sep 11 10:31:38 2015 -0700
Committer: Ryan Blue <blue@apache.org>
Committed: Fri Sep 11 10:31:38 2015 -0700

----------------------------------------------------------------------
 .../parquet/avro/AvroSchemaConverter.java       |  1 -
 .../parquet/avro/TestAvroSchemaConverter.java   | 20 ++++++++++++++++++++
 2 files changed, 20 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/9962a0fd/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java
----------------------------------------------------------------------
diff --git a/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java b/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java
index 8f7a1af..04fe3a7 100644
--- a/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java
+++ b/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java
@@ -303,7 +303,6 @@ public class AvroSchemaConverter {
             }
             GroupType mapKeyValType = parquetGroupType.getType(0).asGroupType();
             if (!mapKeyValType.isRepetition(Type.Repetition.REPEATED) ||
-                !mapKeyValType.getOriginalType().equals(OriginalType.MAP_KEY_VALUE) ||
                 mapKeyValType.getFieldCount()!=2) {
               throw new UnsupportedOperationException("Invalid map type " + parquetGroupType);
             }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/9962a0fd/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java
----------------------------------------------------------------------
diff --git a/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java
b/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java
index 2d8bc7e..6c802a6 100644
--- a/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java
+++ b/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java
@@ -21,6 +21,7 @@ package org.apache.parquet.avro;
 import com.google.common.collect.Lists;
 import com.google.common.io.Resources;
 import java.util.Arrays;
+import java.util.Collections;
 import org.apache.avro.Schema;
 import org.apache.hadoop.conf.Configuration;
 import org.codehaus.jackson.node.NullNode;
@@ -392,6 +393,25 @@ public class TestAvroSchemaConverter {
         "}\n");
   }
 
+  @Test
+  public void testParquetMapWithoutMapKeyValueAnnotation() throws Exception {
+    Schema schema = Schema.createRecord("myrecord", null, null, false);
+    Schema map = Schema.createMap(Schema.create(Schema.Type.INT));
+    schema.setFields(Collections.singletonList(new Schema.Field("mymap", map, null, null)));
+    String parquetSchema =
+        "message myrecord {\n" +
+            "  required group mymap (MAP) {\n" +
+            "    repeated group map {\n" +
+            "      required binary key (UTF8);\n" +
+            "      required int32 value;\n" +
+            "    }\n" +
+            "  }\n" +
+            "}\n";
+
+    testParquetToAvroConversion(schema, parquetSchema);
+    testParquetToAvroConversion(NEW_BEHAVIOR, schema, parquetSchema);
+  }
+
   public static Schema optional(Schema original) {
     return Schema.createUnion(Lists.newArrayList(
         Schema.create(Schema.Type.NULL),


Mime
View raw message