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



##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java
##########
@@ -108,15 +108,16 @@ 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(repeated, list.getName());
+    Type originalElement = isOldListElementType ? repeated : 
repeated.asGroupType().getType(0);
     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));
+        return 
list.withNewFields(repeated.asGroupType().withNewFields(element));

Review comment:
       I think this is wrong. If this is a 2-level list, then `repeated` is the 
element type and could be primitive. So this needs to rebuild the list 
correctly if it is a 2-level list.




-- 
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