rdblue commented on a change in pull request #3774:
URL: https://github.com/apache/iceberg/pull/3774#discussion_r791316278



##########
File path: 
parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java
##########
@@ -164,4 +165,62 @@ public Boolean primitive(PrimitiveType primitive) {
     }
   }
 
+  public static Type getListElementType(GroupType array) {

Review comment:
       Iceberg doesn't use `get` in method names. There is probably a more 
specific verb you should use here instead, like `determine` or `find`.

##########
File path: 
parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java
##########
@@ -164,4 +165,62 @@ public Boolean primitive(PrimitiveType primitive) {
     }
   }
 
+  public static Type getListElementType(GroupType array) {
+    Type repeated = array.getFields().get(0);
+    boolean isOldListElementType = 
ParquetSchemaUtil.isOldListElementType(array);

Review comment:
       Nit: this uses the current class as a prefix for a static method call.

##########
File path: 
parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java
##########
@@ -67,11 +67,17 @@ public Type list(GroupType list, Type elementType) {
     Preconditions.checkArgument(elementType != null,
         "List type must have element field");
 
+    boolean isOldListElementType = 
ParquetSchemaUtil.isOldListElementType(list);
     MappedField field = nameMapping.find(currentPath());
-    Type listType = Types.buildGroup(list.getRepetition())
-        .as(LogicalTypeAnnotation.listType())
-        .repeatedGroup().addFields(elementType).named(list.getFieldName(0))
-        .named(list.getName());
+
+    Types.GroupBuilder<GroupType> listBuilder = 
Types.buildGroup(list.getRepetition())
+        .as(LogicalTypeAnnotation.listType());
+    if (isOldListElementType) {

Review comment:
       I think this can check whether the repetition level of the element is 
`REPEATED` to determine whether it is a 2-level list. I think that would be 
better than allowing the `isOldListElementType` check in multiple places.

##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java
##########
@@ -108,15 +108,20 @@ public Type struct(GroupType struct, List<Type> fields) {
 
   @Override
   public Type list(GroupType list, Type element) {
-    GroupType repeated = list.getType(0).asGroupType();
-    Type originalElement = repeated.getType(0);
+    Type repeated = list.getType(0);
+    boolean isOldListElementType = 
ParquetSchemaUtil.isOldListElementType(list);
+    Type originalElement = ParquetSchemaUtil.getListElementType(list);
     Integer elementId = getId(originalElement);
 
     if (elementId != null && selectedIds.contains(elementId)) {
       return list;
     } else if (element != null) {
       if (!Objects.equal(element, originalElement)) {
-        return list.withNewFields(repeated.withNewFields(element));
+        if (isOldListElementType) {

Review comment:
       I think this should be a check for whether the element is `REPEATED` 
instead.

##########
File path: 
parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java
##########
@@ -63,29 +63,14 @@
             Preconditions.checkArgument(group.getFieldCount() == 1,
                 "Invalid list: does not contain single repeated field: %s", 
group);
 
-            GroupType repeatedElement = group.getFields().get(0).asGroupType();
+            Type repeatedElement = group.getFields().get(0);
             
Preconditions.checkArgument(repeatedElement.isRepetition(Type.Repetition.REPEATED),
                 "Invalid list: inner group is not repeated");
-            Preconditions.checkArgument(repeatedElement.getFieldCount() <= 1,
-                "Invalid list: repeated group is not a single field: %s", 
group);
 
-            Types.ListType list = null;
-            Types.NestedField element = null;
-            if (iType != null) {
-              list = iType.asListType();
-              element = list.fields().get(0);
-            }
-
-            visitor.fieldNames.push(repeatedElement.getName());
-            try {
-              T elementResult = null;
-              if (repeatedElement.getFieldCount() > 0) {
-                elementResult = visitField(element, 
repeatedElement.getType(0), visitor);
-              }
-
-              return visitor.list(list, group, elementResult);
-            } finally {
-              visitor.fieldNames.pop();
+            if (ParquetSchemaUtil.isOldListElementType(group)) {

Review comment:
       Check for REPEATED and pass the element?

##########
File path: 
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java
##########
@@ -610,7 +623,71 @@ public void testStructOfThreeLevelLists() throws Exception 
{
     structOfThreeLevelLists(false);
   }
 
-  public void threeLevelList(boolean useLegacyMode) throws Exception {
+  @Test
+  public void testTwoLevelList() throws IOException {
+    spark.conf().set("spark.sql.parquet.writeLegacyFormat", true);
+
+    String tableName = sourceName("testTwoLevelList");
+    File location = temp.newFolder();
+
+    StructType sparkSchema =
+        new StructType(
+            new StructField[]{
+                new StructField(
+                        "col1", new ArrayType(
+                            new StructType(
+                                new StructField[]{
+                                    new StructField(
+                                        "col2",
+                                        DataTypes.IntegerType,
+                                        false,
+                                        Metadata.empty())
+                                }), false), true, Metadata.empty())});
+    String expectedParquetSchema =
+        "message spark_schema {\n" +
+            "  optional group col1 (LIST) {\n" +
+            "    repeated group array {\n" +
+            "      required int32 col2;\n" +
+            "    }\n" +
+            "  }\n" +
+            "}\n";
+
+
+    // generate parquet file with required schema
+    List<String> testData = Collections.singletonList("{\"col1\": [{\"col2\": 
1}]}");
+    spark.read().schema(sparkSchema).json(
+            
JavaSparkContext.fromSparkContext(spark.sparkContext()).parallelize(testData))
+        
.coalesce(1).write().format("parquet").mode(SaveMode.Append).save(location.getPath());
+
+    File parquetFile = 
Arrays.stream(Objects.requireNonNull(location.listFiles(new FilenameFilter() {
+      @Override
+      public boolean accept(File dir, String name) {
+        return name.endsWith("parquet");
+      }
+    }))).findAny().get();
+
+    // verify generated parquet file has expected schema
+    ParquetFileReader pqReader = 
ParquetFileReader.open(HadoopInputFile.fromPath(new Path(parquetFile.getPath()),
+        new Configuration()));
+    MessageType schema = pqReader.getFooter().getFileMetaData().getSchema();
+    Assert.assertEquals(expectedParquetSchema, schema.toString());

Review comment:
       You're converting them to String and then checking, which makes this 
brittle. Can you compare the schemas without converting to String?

##########
File path: 
parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java
##########
@@ -149,6 +134,50 @@
     }
   }
 
+  private static <T> T visitTwoLevelList(
+      org.apache.iceberg.types.Type iType,
+      TypeWithSchemaVisitor<T> visitor,
+      GroupType group,
+      Type repeatedElement) {

Review comment:
       Iceberg doesn't use one line per argument. Can you wrap these at 120 
chars?

##########
File path: 
parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java
##########
@@ -149,6 +134,50 @@
     }
   }
 
+  private static <T> T visitTwoLevelList(
+      org.apache.iceberg.types.Type iType,
+      TypeWithSchemaVisitor<T> visitor,
+      GroupType group,
+      Type repeatedElement) {
+    Types.ListType list = null;
+    Types.NestedField element = null;
+    if (iType != null) {
+      list = iType.asListType();

Review comment:
       This can be pushed up to the method call. `iType.asListType()` passed in 
here makes the signature of this method simpler, `Types.ListType`. Same with 
the other method.

##########
File path: 
parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java
##########
@@ -66,17 +66,16 @@
     Preconditions.checkArgument(list.getFieldCount() == 1,
         "Invalid list: does not contain single repeated field: %s", list);
 
-    GroupType repeatedElement = list.getFields().get(0).asGroupType();
+    Type repeatedElement = list.getFields().get(0);
     
Preconditions.checkArgument(repeatedElement.isRepetition(Type.Repetition.REPEATED),
         "Invalid list: inner group is not repeated");
-    Preconditions.checkArgument(repeatedElement.getFieldCount() <= 1,
-        "Invalid list: repeated group is not a single field: %s", list);
+
+    Type elementField = ParquetSchemaUtil.getListElementType(list);
 
     visitor.beforeRepeatedElement(repeatedElement);
     try {
       T elementResult = null;
-      if (repeatedElement.getFieldCount() > 0) {
-        Type elementField = repeatedElement.getType(0);
+      if (repeatedElement.isPrimitive() || 
repeatedElement.asGroupType().getFieldCount() > 0) {

Review comment:
       The changes to this class are inconsistent with the changes to the 
`TypeWithSchemaVisitor`. Here, the repeated element is always visited 
(`beforeRepeatedElement` call above) and may be processed again as the element. 
The other avoids pushing the name on the stack. If `beforeRepeatedElement` were 
used to track names, I think it would get a duplicate name for the repeated 
group.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to