Skip to content

Commit a3c538f

Browse files
authored
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
1 parent f154daf commit a3c538f

File tree

2 files changed

+51
-2
lines changed

2 files changed

+51
-2
lines changed

parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -79,10 +79,10 @@ public static Metrics footerMetrics(
7979
MetricsConfig metricsConfig,
8080
NameMapping nameMapping) {
8181
Preconditions.checkNotNull(fieldMetrics, "fieldMetrics should not be null");
82-
MessageType messageType = metadata.getFileMetaData().getSchema();
8382
MessageType parquetTypeWithIds = getParquetTypeWithIds(metadata, nameMapping);
8483
Schema fileSchema = ParquetSchemaUtil.convertAndPrune(parquetTypeWithIds);
85-
return ParquetMetrics.metrics(fileSchema, messageType, metricsConfig, metadata, fieldMetrics);
84+
return ParquetMetrics.metrics(
85+
fileSchema, parquetTypeWithIds, metricsConfig, metadata, fieldMetrics);
8686
}
8787

8888
private static MessageType getParquetTypeWithIds(

parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java

Lines changed: 49 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@
2828
import static org.apache.iceberg.parquet.ParquetWritingTestUtils.write;
2929
import static org.apache.iceberg.relocated.com.google.common.collect.Iterables.getOnlyElement;
3030
import static org.apache.iceberg.types.Types.NestedField.optional;
31+
import static org.apache.iceberg.types.Types.NestedField.required;
3132
import static org.assertj.core.api.Assertions.assertThat;
3233

3334
import java.io.File;
@@ -37,6 +38,7 @@
3738
import java.util.Collections;
3839
import java.util.List;
3940
import java.util.function.Function;
41+
import java.util.stream.Stream;
4042
import org.apache.avro.generic.GenericData;
4143
import org.apache.avro.generic.GenericRecord;
4244
import org.apache.avro.generic.GenericRecordBuilder;
@@ -46,6 +48,8 @@
4648
import org.apache.iceberg.Schema;
4749
import org.apache.iceberg.avro.AvroSchemaUtil;
4850
import org.apache.iceberg.io.InputFile;
51+
import org.apache.iceberg.mapping.MappingUtil;
52+
import org.apache.iceberg.mapping.NameMapping;
4953
import org.apache.iceberg.relocated.com.google.common.base.Strings;
5054
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
5155
import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
@@ -265,6 +269,51 @@ public void testColumnStatisticsEnabled() throws Exception {
265269
}
266270
}
267271

272+
@Test
273+
public void testFooterMetricsWithNameMappingForFileWithoutIds() throws IOException {
274+
Schema schemaWithIds =
275+
new Schema(
276+
required(1, "id", Types.LongType.get()), optional(2, "data", Types.StringType.get()));
277+
278+
NameMapping nameMapping = MappingUtil.create(schemaWithIds);
279+
280+
File file = createTempFile(temp);
281+
282+
// Write a Parquet file WITHOUT field IDs using plain Avro schema
283+
org.apache.avro.Schema avroSchemaWithoutIds =
284+
org.apache.avro.SchemaBuilder.record("test")
285+
.fields()
286+
.requiredLong("id")
287+
.optionalString("data")
288+
.endRecord();
289+
290+
try (ParquetWriter<GenericData.Record> writer =
291+
AvroParquetWriter.<GenericData.Record>builder(ParquetIO.file(Files.localOutput(file)))
292+
.withDataModel(GenericData.get())
293+
.withSchema(avroSchemaWithoutIds)
294+
.build()) {
295+
296+
GenericData.Record record = new GenericData.Record(avroSchemaWithoutIds);
297+
record.put("id", 1L);
298+
record.put("data", "a");
299+
writer.write(record);
300+
}
301+
302+
InputFile inputFile = Files.localInput(file);
303+
304+
try (ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(inputFile))) {
305+
MessageType parquetSchema = reader.getFooter().getFileMetaData().getSchema();
306+
assertThat(ParquetSchemaUtil.hasIds(parquetSchema)).isFalse();
307+
308+
Metrics metrics =
309+
ParquetUtil.footerMetrics(
310+
reader.getFooter(), Stream.empty(), MetricsConfig.getDefault(), nameMapping);
311+
312+
// The key assertion: column sizes should be keyed by field IDs from NameMapping
313+
assertThat(metrics.columnSizes()).containsOnlyKeys(1, 2);
314+
}
315+
}
316+
268317
private Pair<File, Long> generateFile(
269318
Function<MessageType, ParquetValueWriter<?>> createWriterFunc,
270319
int desiredRecordCount,

0 commit comments

Comments
 (0)