You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by bl...@apache.org on 2022/05/02 23:49:07 UTC
[iceberg] branch master updated: Flink: Support malformed Parquet lists added by addFiles API (#4555)
This is an automated email from the ASF dual-hosted git repository.
blue pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iceberg.git
The following commit(s) were added to refs/heads/master by this push:
new c38f7b539 Flink: Support malformed Parquet lists added by addFiles API (#4555)
c38f7b539 is described below
commit c38f7b5398fe88aaedeca76d3932138aa83128fc
Author: Ashish Singh <as...@pinterest.com>
AuthorDate: Mon May 2 16:49:02 2022 -0700
Flink: Support malformed Parquet lists added by addFiles API (#4555)
---
.../flink/data/ParquetWithFlinkSchemaVisitor.java | 2 --
.../apache/iceberg/parquet/ParquetTypeVisitor.java | 2 --
.../iceberg/parquet/TypeWithSchemaVisitor.java | 2 --
.../iceberg/parquet/TestParquetSchemaUtil.java | 25 ++++++++++++++++++++++
4 files changed, 25 insertions(+), 6 deletions(-)
diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java
index 541986f93..87501891a 100644
--- a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java
+++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java
@@ -53,8 +53,6 @@ public class ParquetWithFlinkSchemaVisitor<T> {
if (annotation != null) {
switch (annotation) {
case LIST:
- Preconditions.checkArgument(!group.isRepetition(Type.Repetition.REPEATED),
- "Invalid list: top-level group is repeated: %s", group);
Preconditions.checkArgument(group.getFieldCount() == 1,
"Invalid list: does not contain single repeated field: %s", group);
diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java
index b7668f0a8..894835600 100644
--- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java
+++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java
@@ -61,8 +61,6 @@ public class ParquetTypeVisitor<T> {
}
private static <T> T visitList(GroupType list, ParquetTypeVisitor<T> visitor) {
- Preconditions.checkArgument(!list.isRepetition(Type.Repetition.REPEATED),
- "Invalid list: top-level group is repeated: %s", list);
Preconditions.checkArgument(list.getFieldCount() == 1,
"Invalid list: does not contain single repeated field: %s", list);
diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java b/parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java
index 4bc8301f3..0081bc333 100644
--- a/parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java
+++ b/parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java
@@ -58,8 +58,6 @@ public class TypeWithSchemaVisitor<T> {
if (annotation != null) {
switch (annotation) {
case LIST:
- Preconditions.checkArgument(!group.isRepetition(Type.Repetition.REPEATED),
- "Invalid list: top-level group is repeated: %s", group);
Preconditions.checkArgument(group.getFieldCount() == 1,
"Invalid list: does not contain single repeated field: %s", group);
diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java
index 170a167c0..6922111b7 100644
--- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java
+++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java
@@ -303,6 +303,31 @@ public class TestParquetSchemaUtil {
Assert.assertEquals("Schema must match", expectedSchema.asStruct(), actualSchema.asStruct());
}
+ @Test
+ public void testLegacyTwoLevelListGenByParquetThrift1() {
+ String messageType =
+ "message root {" +
+ " optional group my_list (LIST) {" +
+ " repeated group my_list_tuple (LIST) {" +
+ " repeated int32 my_list_tuple_tuple;" +
+ " }" +
+ " }" +
+ "}";
+
+ MessageType parquetScehma = MessageTypeParser.parseMessageType(messageType);
+ Schema expectedSchema = new Schema(
+ optional(1, "my_list", Types.ListType.ofRequired(
+ 1001,
+ Types.ListType.ofRequired(
+ 1000, Types.IntegerType.get()
+ )
+ ))
+ );
+
+ Schema actualSchema = ParquetSchemaUtil.convert(parquetScehma);
+ Assert.assertEquals("Schema must match", expectedSchema.asStruct(), actualSchema.asStruct());
+ }
+
private Type primitive(Integer id, String name, PrimitiveTypeName typeName, Repetition repetition) {
PrimitiveBuilder<PrimitiveType> builder = org.apache.parquet.schema.Types.primitive(typeName, repetition);
if (id != null) {