parquet-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From tians...@apache.org
Subject parquet-mr git commit: PARQUET-278 : enforce non empty group on MessageType level
Date Fri, 15 May 2015 20:08:35 GMT
Repository: parquet-mr
Updated Branches:
  refs/heads/master 1dbcdf2e3 -> 60edcf9df


PARQUET-278 : enforce non empty group on MessageType level

As columnar format, parquet currently does not support empty struct/group without leaves.
We should throw when constructing an empty GroupType to give a clear message.

Author: Tianshuo Deng <tdeng@twitter.com>

Closes #195 from tsdeng/message_type_enforce_non_empty_group and squashes the following commits:

a286c58 [Tianshuo Deng] revert change to merge_parquet_pr
a09f6ba [Tianshuo Deng] fix test
ac63567 [Tianshuo Deng] fix tests
aa2633c [Tianshuo Deng] enforce non empty group on MessageType level


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

Branch: refs/heads/master
Commit: 60edcf9df1bbe271b1414b04e914641937395d8a
Parents: 1dbcdf2
Author: Tianshuo Deng <tdeng@twitter.com>
Authored: Fri May 15 13:07:14 2015 -0700
Committer: Tianshuo Deng <tdeng@twitter.com>
Committed: Fri May 15 13:07:14 2015 -0700

----------------------------------------------------------------------
 .../org/apache/parquet/schema/GroupType.java    |  3 ++
 ...IncompatibleSchemaModificationException.java |  6 ++--
 .../parquet/schema/InvalidSchemaException.java  | 31 ++++++++++++++++++++
 .../apache/parquet/schema/TestMessageType.java  | 14 +++++++++
 .../parquet/pig/TestPigSchemaConverter.java     |  7 ++++-
 .../parquet/thrift/ThriftSchemaConverter.java   |  2 +-
 6 files changed, 58 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/60edcf9d/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 027fbc0..bf98bfd 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
@@ -88,6 +88,9 @@ public class GroupType extends Type {
    */
   GroupType(Repetition repetition, String name, OriginalType originalType, List<Type>
fields, ID id) {
     super(name, repetition, originalType, id);
+    if (fields.isEmpty()) {
+      throw new InvalidSchemaException("A group type can not be empty. Parquet does not support
empty group without leaves. Empty group: " + name);
+    }
     this.fields = fields;
     this.indexByName = new HashMap<String, Integer>();
     for (int i = 0; i < fields.size(); i++) {

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/60edcf9d/parquet-column/src/main/java/org/apache/parquet/schema/IncompatibleSchemaModificationException.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/schema/IncompatibleSchemaModificationException.java
b/parquet-column/src/main/java/org/apache/parquet/schema/IncompatibleSchemaModificationException.java
index cde5e75..ea64d95 100644
--- a/parquet-column/src/main/java/org/apache/parquet/schema/IncompatibleSchemaModificationException.java
+++ b/parquet-column/src/main/java/org/apache/parquet/schema/IncompatibleSchemaModificationException.java
@@ -1,4 +1,4 @@
-/* 
+/*
  * 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
@@ -6,9 +6,9 @@
  * 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

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/60edcf9d/parquet-column/src/main/java/org/apache/parquet/schema/InvalidSchemaException.java
----------------------------------------------------------------------
diff --git a/parquet-column/src/main/java/org/apache/parquet/schema/InvalidSchemaException.java
b/parquet-column/src/main/java/org/apache/parquet/schema/InvalidSchemaException.java
new file mode 100644
index 0000000..32caaa1
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/schema/InvalidSchemaException.java
@@ -0,0 +1,31 @@
+/*
+ * 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.parquet.schema;
+
+import org.apache.parquet.ParquetRuntimeException;
+
+public class InvalidSchemaException extends ParquetRuntimeException {
+  public InvalidSchemaException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public InvalidSchemaException(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/60edcf9d/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 5bc3122..b730ec8 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
@@ -145,4 +145,18 @@ public class TestMessageType {
     assertEquals(schema, schema2);
     assertEquals(schema.toString(), schema2.toString());
   }
+
+  @Test
+  public void testEmptyGroup() {
+    try {
+      MessageType t5 = new MessageType("root1",
+          new GroupType(REQUIRED, "g1"),
+          new GroupType(REQUIRED, "g2",
+              new PrimitiveType(OPTIONAL, BINARY, "b")));
+      fail("should throw InvalidSchemaException when GroupType contains no child");
+    } catch (InvalidSchemaException e) {
+      assertEquals("A group type can not be empty. Parquet does not support empty group without
leaves. Empty group: g1", e.getMessage());
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/60edcf9d/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 6e75ede..8b61b4b 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
@@ -18,6 +18,8 @@
  */
 package org.apache.parquet.pig;
 
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.*;
+import static org.apache.parquet.schema.Type.Repetition.OPTIONAL;
 import static org.junit.Assert.assertEquals;
 import static org.apache.parquet.pig.PigSchemaConverter.pigSchemaToString;
 import static org.apache.parquet.pig.TupleReadSupport.getPigSchemaFromMultipleFiles;
@@ -28,6 +30,9 @@ import java.util.LinkedHashSet;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
+import org.apache.parquet.schema.Type;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.impl.util.Utils;
 import org.junit.Test;
@@ -221,7 +226,7 @@ public class TestPigSchemaConverter {
     map.put("pig.schema", new LinkedHashSet<String>(Arrays.asList(
         "a:int, b:int, c:int, d:int, e:int, f:int",
         "aa:int, aaa:int, b:int, c:int, ee:int")));
-    Schema result = getPigSchemaFromMultipleFiles(new MessageType("empty"), map);
+    Schema result = getPigSchemaFromMultipleFiles(new MessageType("file_schema", new PrimitiveType(OPTIONAL,
INT32,"a")), map);
     assertEquals("a: int,b: int,c: int,d: int,e: int,f: int,aa: int,aaa: int,ee: int", pigSchemaToString(result));
   }
 

http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/60edcf9d/parquet-thrift/src/main/java/org/apache/parquet/thrift/ThriftSchemaConverter.java
----------------------------------------------------------------------
diff --git a/parquet-thrift/src/main/java/org/apache/parquet/thrift/ThriftSchemaConverter.java
b/parquet-thrift/src/main/java/org/apache/parquet/thrift/ThriftSchemaConverter.java
index f9f23eb..112913e 100644
--- a/parquet-thrift/src/main/java/org/apache/parquet/thrift/ThriftSchemaConverter.java
+++ b/parquet-thrift/src/main/java/org/apache/parquet/thrift/ThriftSchemaConverter.java
@@ -59,8 +59,8 @@ public class ThriftSchemaConverter {
   public MessageType convert(StructType thriftClass) {
     ThriftSchemaConvertVisitor visitor = new ThriftSchemaConvertVisitor(fieldProjectionFilter);
     thriftClass.accept(visitor);
-    MessageType convertedMessageType = visitor.getConvertedMessageType();
     fieldProjectionFilter.assertNoUnmatchedPatterns();
+    MessageType convertedMessageType = visitor.getConvertedMessageType();
     return convertedMessageType;
   }
 


Mime
View raw message