SinghAsDev commented on a change in pull request #3774:
URL: https://github.com/apache/iceberg/pull/3774#discussion_r791088471
##########
File path:
parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java
##########
@@ -96,11 +96,13 @@ public Type struct(GroupType struct, List<Type> fieldTypes)
{
@Override
public Type list(GroupType array, Type elementType) {
- GroupType repeated = array.getType(0).asGroupType();
- org.apache.parquet.schema.Type element = repeated.getType(0);
+ org.apache.parquet.schema.Type repeated = array.getType(0);
+ org.apache.parquet.schema.Type repeatedElement = array.getFields().get(0);
+ boolean isElementType =
ParquetSchemaUtil.isListElementType(repeatedElement, array.getName());
+ org.apache.parquet.schema.Type element = isElementType ? repeated :
repeated.asGroupType().getType(0);
Preconditions.checkArgument(
- !element.isRepetition(Repetition.REPEATED),
+ isElementType || !element.isRepetition(Repetition.REPEATED),
"Elements cannot have repetition REPEATED: %s", element);
Review comment:
Yea, forgot to drop this in earlier update. Dropping it.
##########
File path:
parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java
##########
@@ -164,4 +164,48 @@ public Boolean primitive(PrimitiveType primitive) {
}
}
+ // Parquet LIST backwards-compatibility rules.
+ //
https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#backward-compatibility-rules
+ public static boolean isOldListElementType(Type repeatedType, String
parentName) {
Review comment:
This is also used outside of iceberg-parquet, like iceberg-spark. Good
point on making method to accept list directly, updating.
##########
File path:
parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java
##########
@@ -96,11 +96,13 @@ public Type struct(GroupType struct, List<Type> fieldTypes)
{
@Override
public Type list(GroupType array, Type elementType) {
- GroupType repeated = array.getType(0).asGroupType();
- org.apache.parquet.schema.Type element = repeated.getType(0);
+ org.apache.parquet.schema.Type repeated = array.getType(0);
+ org.apache.parquet.schema.Type repeatedElement = array.getFields().get(0);
Review comment:
Likely a copy paste error, we only need one of them. Updating.
##########
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:
Thanks for catching this.
--
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]