ebyhr commented on code in PR #14617:
URL: https://github.com/apache/iceberg/pull/14617#discussion_r2540778041


##########
parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java:
##########
@@ -264,6 +268,51 @@ public void testColumnStatisticsEnabled() throws Exception 
{
     }
   }
 
+  @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();
+
+    ParquetWriter<GenericData.Record> writer =
+        AvroParquetWriter.<GenericData.Record>builder(new 
org.apache.hadoop.fs.Path(file.toURI()))
+            .withDataModel(GenericData.get())
+            .withSchema(avroSchemaWithoutIds)
+            .build();
+
+    GenericData.Record record = new GenericData.Record(avroSchemaWithoutIds);
+    record.put("id", 1L);
+    record.put("data", "a");
+    writer.write(record);
+    writer.close();
+
+    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()).containsKeys(1, 2);

Review Comment:
   This assertion passes even when the `metrics.columnSizes()` contains 
additional keys. Could you use `containsOnlyKeys` instead? 



##########
parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java:
##########
@@ -264,6 +268,51 @@ public void testColumnStatisticsEnabled() throws Exception 
{
     }
   }
 
+  @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();
+
+    ParquetWriter<GenericData.Record> writer =
+        AvroParquetWriter.<GenericData.Record>builder(new 
org.apache.hadoop.fs.Path(file.toURI()))

Review Comment:
   `builder(Path file)` is deprecated. Can we `builder(OutputFile file)` 
instead? You can refer to #14620



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