parquet-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jul...@apache.org
Subject parquet-mr git commit: PARQUET-753: Fixed GroupType.union() to handle original type
Date Wed, 26 Oct 2016 16:47:52 GMT
Repository: parquet-mr
Updated Branches:
  refs/heads/master df9d8e415 -> e5cd652ae


PARQUET-753: Fixed GroupType.union() to handle original type

also fixed GroupType.equals() to compare the original type and 2 unit tests that weren't setting
the original type properly on the expected results

Author: adeneche <adeneche@apache.org>
Author: adeneche <adeneche@gmail.com>

Closes #380 from adeneche/fix-grouptype-union and squashes the following commits:

b04af7d [adeneche] reverted unnecessary formatting changes
5461a57 [adeneche] Fixed unit tests in TestPigSchemaConverter that were failing because of
my fix to GroupType.equals()
ec91315 [adeneche] fixed expected error message in TestMessageType#testMergeSchema
a1d7f63 [adeneche] Fixed GroupType.union() to handle original type


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

Branch: refs/heads/master
Commit: e5cd652aeb3305ef2b82a7925cce3a132bf6f5ae
Parents: df9d8e4
Author: adeneche <adeneche@apache.org>
Authored: Wed Oct 26 09:47:44 2016 -0700
Committer: Julien Le Dem <julien@dremio.com>
Committed: Wed Oct 26 09:47:44 2016 -0700

----------------------------------------------------------------------
 .../org/apache/parquet/schema/GroupType.java    |  9 +++-
 .../parquet/parser/TestParquetParser.java       |  6 +--
 .../apache/parquet/schema/TestMessageType.java  | 29 ++++++++++-
 .../parquet/pig/TestPigSchemaConverter.java     | 54 ++++++++++----------
 4 files changed, 65 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/e5cd652a/parquet-column/src/main/java/org/apache/parquet/schema/GroupType.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/schema/GroupType.java b/parquet-column/src/main/java/org/apache/parquet/schema/GroupType.java
index f8404a1..68dba97 100644
--- a/parquet-column/src/main/java/org/apache/parquet/schema/GroupType.java
+++ b/parquet-column/src/main/java/org/apache/parquet/schema/GroupType.java
@@ -26,6 +26,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
+import java.util.Objects;
 
 import org.apache.parquet.io.InvalidRecordException;
 
@@ -243,7 +244,7 @@ public class GroupType extends Type {
    */
   @Override
   public int hashCode() {
-    return super.hashCode() * 31 + getFields().hashCode();
+    return Objects.hash(getOriginalType(), getFields());
   }
 
   /**
@@ -254,6 +255,7 @@ public class GroupType extends Type {
     return
         !otherType.isPrimitive()
         && super.equals(otherType)
+        && getOriginalType() == otherType.getOriginalType()
         && getFields().equals(otherType.asGroupType().getFields());
   }
 
@@ -347,7 +349,7 @@ public class GroupType extends Type {
     if (toMerge.isPrimitive()) {
       throw new IncompatibleSchemaModificationException("can not merge primitive type " +
toMerge + " into group type " + this);
     }
-    return new GroupType(toMerge.getRepetition(), getName(), mergeFields(toMerge.asGroupType()));
+    return new GroupType(toMerge.getRepetition(), getName(), toMerge.getOriginalType(), mergeFields(toMerge.asGroupType()),
getId());
   }
 
   /**
@@ -375,6 +377,9 @@ public class GroupType extends Type {
         if (fieldToMerge.getRepetition().isMoreRestrictiveThan(type.getRepetition())) {
           throw new IncompatibleSchemaModificationException("repetition constraint is more
restrictive: can not merge type " + fieldToMerge + " into " + type);
         }
+        if (type.getOriginalType() != null && fieldToMerge.getOriginalType() != type.getOriginalType())
{
+          throw new IncompatibleSchemaModificationException("cannot merge original type "
+ fieldToMerge.getOriginalType() + " into " + type.getOriginalType());
+        }
         merged = type.union(fieldToMerge, strict);
       } else {
         merged = type;

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/e5cd652a/parquet-column/src/test/java/org/apache/parquet/parser/TestParquetParser.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/parser/TestParquetParser.java
b/parquet-column/src/test/java/org/apache/parquet/parser/TestParquetParser.java
index dc9369b..e2f737a 100644
--- a/parquet-column/src/test/java/org/apache/parquet/parser/TestParquetParser.java
+++ b/parquet-column/src/test/java/org/apache/parquet/parser/TestParquetParser.java
@@ -169,8 +169,8 @@ public class TestParquetParser {
 
     MessageType parsed = parseMessageType(message);
     MessageType expected = buildMessage()
-        .optionalGroup()
-        .repeatedGroup()
+        .optionalGroup().as(MAP)
+        .repeatedGroup().as(MAP_KEY_VALUE)
         .required(BINARY).as(UTF8).named("key")
         .required(INT32).named("value")
         .named("map")
@@ -194,7 +194,7 @@ public class TestParquetParser {
 
     MessageType parsed = parseMessageType(message);
     MessageType expected = buildMessage()
-        .requiredGroup()
+        .requiredGroup().as(LIST)
         .repeated(BINARY).as(UTF8).named("string")
         .named("aList")
         .named("Message");

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/e5cd652a/parquet-column/src/test/java/org/apache/parquet/schema/TestMessageType.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/test/java/org/apache/parquet/schema/TestMessageType.java b/parquet-column/src/test/java/org/apache/parquet/schema/TestMessageType.java
index 438fae9..4add174 100644
--- a/parquet-column/src/test/java/org/apache/parquet/schema/TestMessageType.java
+++ b/parquet-column/src/test/java/org/apache/parquet/schema/TestMessageType.java
@@ -19,6 +19,7 @@
 package org.apache.parquet.schema;
 
 import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY;
+import static org.apache.parquet.schema.OriginalType.LIST;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
@@ -144,7 +145,7 @@ public class TestMessageType {
       t9.union(t10);
       fail("moving from BINARY (UTF8) to BINARY");
     } catch (IncompatibleSchemaModificationException e) {
-      assertEquals("can not merge type optional binary a into optional binary a (UTF8)",
e.getMessage());
+      assertEquals("cannot merge original type null into UTF8", e.getMessage());
     }
 
     MessageType t11 = Types.buildMessage()
@@ -162,6 +163,32 @@ public class TestMessageType {
   }
 
   @Test
+  public void testMergeSchemaWithOriginalType() throws Exception {
+    MessageType t5 = new MessageType("root1",
+        new GroupType(REQUIRED, "g1", LIST,
+            new PrimitiveType(OPTIONAL, BINARY, "a")),
+        new GroupType(REQUIRED, "g2",
+            new PrimitiveType(OPTIONAL, BINARY, "b")));
+    MessageType t6 = new MessageType("root1",
+        new GroupType(REQUIRED, "g1", LIST,
+            new PrimitiveType(OPTIONAL, BINARY, "a")),
+        new GroupType(REQUIRED, "g2", LIST,
+            new GroupType(REQUIRED, "g3",
+                new PrimitiveType(OPTIONAL, BINARY, "c")),
+            new PrimitiveType(OPTIONAL, BINARY, "b")));
+
+    assertEquals(
+        new MessageType("root1",
+            new GroupType(REQUIRED, "g1", LIST,
+                new PrimitiveType(OPTIONAL, BINARY, "a")),
+            new GroupType(REQUIRED, "g2", LIST,
+                new PrimitiveType(OPTIONAL, BINARY, "b"),
+                new GroupType(REQUIRED, "g3",
+                    new PrimitiveType(OPTIONAL, BINARY, "c")))),
+        t5.union(t6));
+  }
+
+  @Test
   public void testIDs() throws Exception {
     MessageType schema = new MessageType("test",
         new PrimitiveType(REQUIRED, BINARY, "foo").withId(4),

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/e5cd652a/parquet-pig/src/test/java/org/apache/parquet/pig/TestPigSchemaConverter.java
----------------------------------------------------------------------
diff --git a/parquet-pig/src/test/java/org/apache/parquet/pig/TestPigSchemaConverter.java
b/parquet-pig/src/test/java/org/apache/parquet/pig/TestPigSchemaConverter.java
index 7641232..1fe0ccd 100644
--- a/parquet-pig/src/test/java/org/apache/parquet/pig/TestPigSchemaConverter.java
+++ b/parquet-pig/src/test/java/org/apache/parquet/pig/TestPigSchemaConverter.java
@@ -114,11 +114,11 @@ public class TestPigSchemaConverter {
     testConversion(
         "a:chararray, b:{t:(c:chararray, d:chararray)}",
         "message pig_schema {\n" +
-        "  optional binary a;\n" +
-        "  optional group b {\n" +
+        "  optional binary a (UTF8);\n" +
+        "  optional group b (LIST) {\n" +
         "    repeated group t {\n" +
-        "      optional binary c;\n" +
-        "      optional binary d;\n" +
+        "      optional binary c (UTF8);\n" +
+        "      optional binary d (UTF8);\n" +
         "    }\n" +
         "  }\n" +
         "}\n");
@@ -129,11 +129,11 @@ public class TestPigSchemaConverter {
     testConversion(
         "a:chararray, b:{(c:chararray, d:chararray)}",
         "message pig_schema {\n" +
-        "  optional binary a;\n" +
-        "  optional group b {\n" +
+        "  optional binary a (UTF8);\n" +
+        "  optional group b (LIST) {\n" +
         "    repeated group bag {\n" + // the inner field in the bag is called "bag"
-        "      optional binary c;\n" +
-        "      optional binary d;\n" +
+        "      optional binary c (UTF8);\n" +
+        "      optional binary d (UTF8);\n" +
         "    }\n" +
         "  }\n" +
         "}\n");
@@ -144,13 +144,13 @@ public class TestPigSchemaConverter {
     testConversion(
         "a:chararray, b:[(c:chararray, d:chararray)]",
         "message pig_schema {\n" +
-        "  optional binary a;\n" +
-        "  optional group b {\n" +
-        "    repeated group map {\n" +
-        "      required binary key;\n" +
+        "  optional binary a (UTF8);\n" +
+        "  optional group b (MAP) {\n" +
+        "    repeated group map (MAP_KEY_VALUE) {\n" +
+        "      required binary key (UTF8);\n" +
         "      optional group value {\n" +
-        "        optional binary c;\n" +
-        "        optional binary d;\n" +
+        "        optional binary c (UTF8);\n" +
+        "        optional binary d (UTF8);\n" +
         "      }\n" +
         "    }\n" +
         "  }\n" +
@@ -161,9 +161,9 @@ public class TestPigSchemaConverter {
   public void testMap2() throws Exception {
     testConversion("a:map[int]",
         "message pig_schema {\n" +
-        "  optional group a {\n" +
-        "    repeated group map {\n" +
-        "      required binary key;\n" +
+        "  optional group a (MAP) {\n" +
+        "    repeated group map (MAP_KEY_VALUE) {\n" +
+        "      required binary key (UTF8);\n" +
         "      optional int32 value;" +
         "    }\n" +
         "  }\n" +
@@ -174,12 +174,12 @@ public class TestPigSchemaConverter {
   public void testMap3() throws Exception {
     testConversion("a:map[map[int]]",
         "message pig_schema {\n" +
-        "  optional group a {\n" +
-        "    repeated group map {\n" +
-        "      required binary key;\n" +
-        "      optional group value {\n" +
-        "        repeated group map {\n" +
-        "          required binary key;\n" +
+        "  optional group a (MAP) {\n" +
+        "    repeated group map (MAP_KEY_VALUE) {\n" +
+        "      required binary key (UTF8);\n" +
+        "      optional group value (MAP) {\n" +
+        "        repeated group map (MAP_KEY_VALUE) {\n" +
+        "          required binary key (UTF8);\n" +
         "          optional int32 value;\n" +
         "        }\n" +
         "      }\n" +
@@ -192,10 +192,10 @@ public class TestPigSchemaConverter {
   public void testMap4() throws Exception {
     testConversion("a:map[bag{(a:int)}]",
         "message pig_schema {\n" +
-        "  optional group a {\n" +
-        "    repeated group map {\n" +
-        "      required binary key;\n" +
-        "      optional group value {\n" +
+        "  optional group a (MAP) {\n" +
+        "    repeated group map (MAP_KEY_VALUE) {\n" +
+        "      required binary key (UTF8);\n" +
+        "      optional group value (LIST) {\n" +
         "        repeated group bag {\n" +
         "          optional int32 a;\n" +
         "        }\n" +


Mime
View raw message