diff --git a/xtable-hudi-support/xtable-hudi-support-extensions/src/main/java/org/apache/xtable/hudi/extensions/ApplyNameMappingForParquet.java b/xtable-hudi-support/xtable-hudi-support-extensions/src/main/java/org/apache/xtable/hudi/extensions/ApplyNameMappingForParquet.java
new file mode 100644
index 000000000..1dadd2a78
--- /dev/null
+++ b/xtable-hudi-support/xtable-hudi-support-extensions/src/main/java/org/apache/xtable/hudi/extensions/ApplyNameMappingForParquet.java
@@ -0,0 +1,316 @@
+/*
+ * 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.xtable.hudi.extensions;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.apache.parquet.schema.Types;
+
+import org.apache.iceberg.mapping.MappedField;
+import org.apache.iceberg.mapping.NameMapping;
+
+/**
+ * A Parquet schema visitor that applies a NameMapping to add field IDs to a Parquet schema.
+ *
+ * Why this class exists
+ *
+ * This is a local implementation adapted from Iceberg's {@code
+ * org.apache.iceberg.parquet.ApplyNameMapping} to work around a dependency conflict introduced in
+ * Iceberg 1.10+.
+ *
+ *
The issue: Iceberg 1.10 added Parquet variant type support, which requires parquet 1.16.0's
+ * {@code LogicalTypeAnnotation.variantType()} method. However, other dependencies in the XTable
+ * ecosystem (Hudi 0.14.0, Paimon 1.3.1) are not yet compatible with parquet 1.16:
+ *
+ *
+ * - Hudi 0.14.0 - compiled against parquet 1.10.1, fails with {@code
+ * GeospatialStatistics.writePage()} errors on parquet 1.16
+ *
- Paimon 1.3.1 - bundles shaded parquet with older fastutil, causing {@code LongList.of()}
+ * conflicts
+ *
+ *
+ * The original {@code iceberg-parquet} module's {@code ParquetSchemaUtil.applyNameMapping()}
+ * internally uses {@code ParquetTypeVisitor.visit()}, which unconditionally checks for variant
+ * types even when not processing variants - triggering the parquet 1.16 requirement.
+ *
+ *
This implementation avoids extending Iceberg's {@code ParquetTypeVisitor} and instead
+ * implements the traversal logic directly, skipping variant type handling entirely since it's not
+ * needed for the field ID mapping use case.
+ *
+ *
Deprecation Plan
+ *
+ * This class should be removed in favor of using {@code
+ * org.apache.iceberg.parquet.ParquetSchemaUtil.applyNameMapping()} directly once the following
+ * conditions are met:
+ *
+ *
+ * - Hudi upgrades to a version compatible with parquet 1.16+ (likely Hudi 0.16+)
+ *
- Paimon upgrades to a version compatible with parquet 1.16+
+ *
- XTable can safely set {@code parquet.version} to 1.16.0 or later globally
+ *
+ *
+ * To migrate back to iceberg-parquet:
+ *
+ *
+ * - Add {@code iceberg-parquet} dependency to xtable-hudi-support-extensions/pom.xml
+ *
- Replace usage of {@code ApplyNameMappingForParquet.applyNameMapping()} with {@code
+ * ParquetSchemaUtil.applyNameMapping()}
+ *
- Delete this class
+ *
- Update parquet.version to 1.16.0+ in the root pom.xml
+ *
+ *
+ * @see Iceberg PR #14588 - Add variant type
+ * support to ParquetTypeVisitor
+ */
+class ApplyNameMappingForParquet {
+
+ private static final String LIST_ELEMENT_NAME = "element";
+ private static final String MAP_KEY_NAME = "key";
+ private static final String MAP_VALUE_NAME = "value";
+
+ private ApplyNameMappingForParquet() {}
+
+ /**
+ * Applies a NameMapping to a Parquet MessageType, adding field IDs based on the mapping.
+ *
+ * @param fileSchema the Parquet schema to apply the mapping to
+ * @param nameMapping the NameMapping containing field ID assignments
+ * @return a new MessageType with field IDs applied
+ */
+ public static MessageType applyNameMapping(MessageType fileSchema, NameMapping nameMapping) {
+ Visitor visitor = new Visitor(nameMapping);
+ return (MessageType) visit(fileSchema, visitor);
+ }
+
+ private static Type visit(Type type, Visitor visitor) {
+ if (type instanceof MessageType) {
+ MessageType message = (MessageType) type;
+ List fields = new ArrayList<>();
+ for (Type field : message.getFields()) {
+ visitor.beforeField(field);
+ try {
+ fields.add(visit(field, visitor));
+ } finally {
+ visitor.afterField(field);
+ }
+ }
+ return visitor.message(message, fields);
+ } else if (type.isPrimitive()) {
+ return visitor.primitive(type.asPrimitiveType());
+ } else {
+ GroupType group = type.asGroupType();
+ LogicalTypeAnnotation annotation = group.getLogicalTypeAnnotation();
+
+ if (LogicalTypeAnnotation.listType().equals(annotation)) {
+ return visitList(group, visitor);
+ } else if (LogicalTypeAnnotation.mapType().equals(annotation)) {
+ return visitMap(group, visitor);
+ }
+
+ // Regular struct
+ List fields = new ArrayList<>();
+ for (Type field : group.getFields()) {
+ visitor.beforeField(field);
+ try {
+ fields.add(visit(field, visitor));
+ } finally {
+ visitor.afterField(field);
+ }
+ }
+ return visitor.struct(group, fields);
+ }
+ }
+
+ private static Type visitList(GroupType list, Visitor visitor) {
+ if (list.getFieldCount() != 1) {
+ throw new IllegalArgumentException("Invalid list: " + list);
+ }
+
+ Type repeatedElement = list.getType(0);
+ Type elementResult;
+
+ if (isElementType(list, repeatedElement)) {
+ visitor.beforeElementField(repeatedElement);
+ try {
+ elementResult = visit(repeatedElement, visitor);
+ } finally {
+ visitor.afterField(repeatedElement);
+ }
+ } else {
+ GroupType repeated = repeatedElement.asGroupType();
+ Type element = repeated.getType(0);
+ visitor.beforeElementField(element);
+ try {
+ elementResult = visit(element, visitor);
+ } finally {
+ visitor.afterField(element);
+ }
+ }
+
+ return visitor.list(list, elementResult);
+ }
+
+ private static boolean isElementType(GroupType list, Type repeatedElement) {
+ // Check for 2-level list encoding
+ return repeatedElement.isPrimitive()
+ || repeatedElement.asGroupType().getFieldCount() > 1
+ || repeatedElement.getName().equals("array")
+ || repeatedElement.getName().equals(list.getName() + "_tuple");
+ }
+
+ private static Type visitMap(GroupType map, Visitor visitor) {
+ if (map.getFieldCount() != 1) {
+ throw new IllegalArgumentException("Invalid map: " + map);
+ }
+
+ GroupType keyValue = map.getType(0).asGroupType();
+ if (keyValue.getFieldCount() != 2) {
+ throw new IllegalArgumentException("Invalid map key-value: " + keyValue);
+ }
+
+ Type key = keyValue.getType(0);
+ Type value = keyValue.getType(1);
+
+ visitor.beforeKeyField(key);
+ Type keyResult;
+ try {
+ keyResult = visit(key, visitor);
+ } finally {
+ visitor.afterField(key);
+ }
+
+ visitor.beforeValueField(value);
+ Type valueResult;
+ try {
+ valueResult = visit(value, visitor);
+ } finally {
+ visitor.afterField(value);
+ }
+
+ return visitor.map(map, keyResult, valueResult);
+ }
+
+ private static class Visitor {
+ private final NameMapping nameMapping;
+ private final Deque fieldNames = new ArrayDeque<>();
+
+ Visitor(NameMapping nameMapping) {
+ this.nameMapping = nameMapping;
+ }
+
+ Type message(MessageType message, List fields) {
+ Types.MessageTypeBuilder builder = Types.buildMessage();
+ fields.stream().filter(Objects::nonNull).forEach(builder::addField);
+ return builder.named(message.getName());
+ }
+
+ Type struct(GroupType struct, List types) {
+ MappedField field = nameMapping.find(currentPath());
+ List actualTypes = types.stream().filter(Objects::nonNull).collect(Collectors.toList());
+ Type structType = struct.withNewFields(actualTypes);
+ return field == null ? structType : structType.withId(field.id());
+ }
+
+ Type list(GroupType list, Type elementType) {
+ if (elementType == null) {
+ throw new IllegalArgumentException("List type must have element field");
+ }
+
+ Type listElement = determineListElementType(list);
+ MappedField field = nameMapping.find(currentPath());
+
+ Types.GroupBuilder listBuilder =
+ Types.buildGroup(list.getRepetition()).as(LogicalTypeAnnotation.listType());
+ if (listElement.isRepetition(Type.Repetition.REPEATED)) {
+ listBuilder.addFields(elementType);
+ } else {
+ listBuilder.repeatedGroup().addFields(elementType).named(list.getFieldName(0));
+ }
+ Type listType = listBuilder.named(list.getName());
+
+ return field == null ? listType : listType.withId(field.id());
+ }
+
+ Type map(GroupType map, Type keyType, Type valueType) {
+ if (keyType == null || valueType == null) {
+ throw new IllegalArgumentException("Map type must have both key field and value field");
+ }
+
+ MappedField field = nameMapping.find(currentPath());
+ Type mapType =
+ Types.buildGroup(map.getRepetition())
+ .as(LogicalTypeAnnotation.mapType())
+ .repeatedGroup()
+ .addFields(keyType, valueType)
+ .named(map.getFieldName(0))
+ .named(map.getName());
+
+ return field == null ? mapType : mapType.withId(field.id());
+ }
+
+ Type primitive(PrimitiveType primitive) {
+ MappedField field = nameMapping.find(currentPath());
+ return field == null ? primitive : primitive.withId(field.id());
+ }
+
+ void beforeField(Type type) {
+ fieldNames.push(type.getName());
+ }
+
+ void afterField(Type type) {
+ fieldNames.pop();
+ }
+
+ void beforeElementField(Type element) {
+ fieldNames.push(LIST_ELEMENT_NAME);
+ }
+
+ void beforeKeyField(Type key) {
+ fieldNames.push(MAP_KEY_NAME);
+ }
+
+ void beforeValueField(Type value) {
+ fieldNames.push(MAP_VALUE_NAME);
+ }
+
+ private String[] currentPath() {
+ List path = new ArrayList<>(fieldNames);
+ java.util.Collections.reverse(path);
+ return path.toArray(new String[0]);
+ }
+
+ private static Type determineListElementType(GroupType list) {
+ Type repeated = list.getType(0);
+ if (isElementType(list, repeated)) {
+ return repeated;
+ }
+ return repeated.asGroupType().getType(0);
+ }
+ }
+}
diff --git a/xtable-hudi-support/xtable-hudi-support-extensions/src/main/java/org/apache/xtable/hudi/extensions/HoodieAvroWriteSupportWithFieldIds.java b/xtable-hudi-support/xtable-hudi-support-extensions/src/main/java/org/apache/xtable/hudi/extensions/HoodieAvroWriteSupportWithFieldIds.java
index d3a1e788e..ed34010df 100644
--- a/xtable-hudi-support/xtable-hudi-support-extensions/src/main/java/org/apache/xtable/hudi/extensions/HoodieAvroWriteSupportWithFieldIds.java
+++ b/xtable-hudi-support/xtable-hudi-support-extensions/src/main/java/org/apache/xtable/hudi/extensions/HoodieAvroWriteSupportWithFieldIds.java
@@ -33,7 +33,6 @@
import org.apache.iceberg.mapping.MappedField;
import org.apache.iceberg.mapping.MappedFields;
import org.apache.iceberg.mapping.NameMapping;
-import org.apache.iceberg.parquet.ParquetSchemaUtil;
import org.apache.xtable.hudi.idtracking.IdTracker;
import org.apache.xtable.hudi.idtracking.models.IdMapping;
@@ -88,7 +87,7 @@ private static MessageType addFieldIdsToParquetSchema(
idMappings.stream()
.map(HoodieAvroWriteSupportWithFieldIds::toMappedField)
.collect(Collectors.toList()));
- return ParquetSchemaUtil.applyNameMapping(messageType, nameMapping);
+ return ApplyNameMappingForParquet.applyNameMapping(messageType, nameMapping);
})
.orElse(messageType);
}
diff --git a/xtable-hudi-support/xtable-hudi-support-extensions/src/main/resources/META-INF/LICENSE-bundled b/xtable-hudi-support/xtable-hudi-support-extensions/src/main/resources/META-INF/LICENSE-bundled
index 892ec28ab..6c11e7e27 100644
--- a/xtable-hudi-support/xtable-hudi-support-extensions/src/main/resources/META-INF/LICENSE-bundled
+++ b/xtable-hudi-support/xtable-hudi-support-extensions/src/main/resources/META-INF/LICENSE-bundled
@@ -233,9 +233,9 @@ org.apache.hudi:hudi-client-common:0.14.0
org.apache.hudi:hudi-common:0.14.0
org.apache.hudi:hudi-timeline-service:0.14.0
org.apache.hudi:hudi-sync-common:0.14.0
-org.apache.iceberg:iceberg-api:1.9.2
-org.apache.iceberg:iceberg-core:1.9.2
-org.apache.iceberg:iceberg-parquet:1.9.2
+org.apache.iceberg:iceberg-api:1.10.1
+org.apache.iceberg:iceberg-core:1.10.1
+org.apache.iceberg:iceberg-parquet:1.10.1
org.apache.logging.log4j:log4j-api:2.22.0
org.apache.logging.log4j:log4j-1.2-api:2.22.0
org.roaringbitmap:RoaringBitmap:0.9.47