This is an automated email from the ASF dual-hosted git repository.

singhpk234 pushed a commit to branch 1.10.x
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/1.10.x by this push:
     new 75b7f60f10 Fix NameMapping loss in ParquetUtil.footerMetrics (#14617)
75b7f60f10 is described below

commit 75b7f60f103b86b295fe4f108c8487896e439dd3
Author: Tamas Mate <[email protected]>
AuthorDate: Tue Nov 25 01:11:04 2025 +0100

    Fix NameMapping loss in ParquetUtil.footerMetrics (#14617)
    
    * Fix NameMapping loss in ParquetUtil.footerMetrics
    
    When footerMetrics was called with a NameMapping, it correctly applied
    the mapping to get field IDs but then passed the wrong MessageType to
    ParquetMetrics.metrics, causing the IDs to be lost. This resulted in
    empty metrics for Parquet files without embedded field IDs.
    
    Fixed by passing parquetTypeWithIds instead of the original messageType.
    
    * Addressed review comments
    
    (cherry picked from commit a3c538f647a113cf61dbfd7855d9c71da8c722ba)
---
 .../org/apache/iceberg/parquet/ParquetUtil.java    |  4 +-
 .../org/apache/iceberg/parquet/TestParquet.java    | 49 ++++++++++++++++++++++
 2 files changed, 51 insertions(+), 2 deletions(-)

diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java 
b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java
index f83a7ea77a..9f5e1caa24 100644
--- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java
+++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java
@@ -79,10 +79,10 @@ public class ParquetUtil {
       MetricsConfig metricsConfig,
       NameMapping nameMapping) {
     Preconditions.checkNotNull(fieldMetrics, "fieldMetrics should not be 
null");
-    MessageType messageType = metadata.getFileMetaData().getSchema();
     MessageType parquetTypeWithIds = getParquetTypeWithIds(metadata, 
nameMapping);
     Schema fileSchema = ParquetSchemaUtil.convertAndPrune(parquetTypeWithIds);
-    return ParquetMetrics.metrics(fileSchema, messageType, metricsConfig, 
metadata, fieldMetrics);
+    return ParquetMetrics.metrics(
+        fileSchema, parquetTypeWithIds, metricsConfig, metadata, fieldMetrics);
   }
 
   private static MessageType getParquetTypeWithIds(
diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java 
b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java
index c33229ee8f..59a4efefbb 100644
--- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java
+++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java
@@ -28,6 +28,7 @@ import static 
org.apache.iceberg.parquet.ParquetWritingTestUtils.createTempFile;
 import static org.apache.iceberg.parquet.ParquetWritingTestUtils.write;
 import static 
org.apache.iceberg.relocated.com.google.common.collect.Iterables.getOnlyElement;
 import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
@@ -37,6 +38,7 @@ import java.nio.file.Path;
 import java.util.Collections;
 import java.util.List;
 import java.util.function.Function;
+import java.util.stream.Stream;
 import org.apache.avro.generic.GenericData;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.generic.GenericRecordBuilder;
@@ -46,6 +48,8 @@ import org.apache.iceberg.MetricsConfig;
 import org.apache.iceberg.Schema;
 import org.apache.iceberg.avro.AvroSchemaUtil;
 import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.MappingUtil;
+import org.apache.iceberg.mapping.NameMapping;
 import org.apache.iceberg.relocated.com.google.common.base.Strings;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
 import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
@@ -264,6 +268,51 @@ public class TestParquet {
     }
   }
 
+  @Test
+  public void testFooterMetricsWithNameMappingForFileWithoutIds() throws 
IOException {
+    Schema schemaWithIds =
+        new Schema(
+            required(1, "id", Types.LongType.get()), optional(2, "data", 
Types.StringType.get()));
+
+    NameMapping nameMapping = MappingUtil.create(schemaWithIds);
+
+    File file = createTempFile(temp);
+
+    // Write a Parquet file WITHOUT field IDs using plain Avro schema
+    org.apache.avro.Schema avroSchemaWithoutIds =
+        org.apache.avro.SchemaBuilder.record("test")
+            .fields()
+            .requiredLong("id")
+            .optionalString("data")
+            .endRecord();
+
+    try (ParquetWriter<GenericData.Record> writer =
+        
AvroParquetWriter.<GenericData.Record>builder(ParquetIO.file(Files.localOutput(file)))
+            .withDataModel(GenericData.get())
+            .withSchema(avroSchemaWithoutIds)
+            .build()) {
+
+      GenericData.Record record = new GenericData.Record(avroSchemaWithoutIds);
+      record.put("id", 1L);
+      record.put("data", "a");
+      writer.write(record);
+    }
+
+    InputFile inputFile = Files.localInput(file);
+
+    try (ParquetFileReader reader = 
ParquetFileReader.open(ParquetIO.file(inputFile))) {
+      MessageType parquetSchema = 
reader.getFooter().getFileMetaData().getSchema();
+      assertThat(ParquetSchemaUtil.hasIds(parquetSchema)).isFalse();
+
+      Metrics metrics =
+          ParquetUtil.footerMetrics(
+              reader.getFooter(), Stream.empty(), MetricsConfig.getDefault(), 
nameMapping);
+
+      // The key assertion: column sizes should be keyed by field IDs from 
NameMapping
+      assertThat(metrics.columnSizes()).containsOnlyKeys(1, 2);
+    }
+  }
+
   private Pair<File, Long> generateFile(
       Function<MessageType, ParquetValueWriter<?>> createWriterFunc,
       int desiredRecordCount,

Reply via email to