|
28 | 28 | import static org.apache.iceberg.parquet.ParquetWritingTestUtils.write; |
29 | 29 | import static org.apache.iceberg.relocated.com.google.common.collect.Iterables.getOnlyElement; |
30 | 30 | import static org.apache.iceberg.types.Types.NestedField.optional; |
| 31 | +import static org.apache.iceberg.types.Types.NestedField.required; |
31 | 32 | import static org.assertj.core.api.Assertions.assertThat; |
32 | 33 |
|
33 | 34 | import java.io.File; |
|
37 | 38 | import java.util.Collections; |
38 | 39 | import java.util.List; |
39 | 40 | import java.util.function.Function; |
| 41 | +import java.util.stream.Stream; |
40 | 42 | import org.apache.avro.generic.GenericData; |
41 | 43 | import org.apache.avro.generic.GenericRecord; |
42 | 44 | import org.apache.avro.generic.GenericRecordBuilder; |
|
46 | 48 | import org.apache.iceberg.Schema; |
47 | 49 | import org.apache.iceberg.avro.AvroSchemaUtil; |
48 | 50 | import org.apache.iceberg.io.InputFile; |
| 51 | +import org.apache.iceberg.mapping.MappingUtil; |
| 52 | +import org.apache.iceberg.mapping.NameMapping; |
49 | 53 | import org.apache.iceberg.relocated.com.google.common.base.Strings; |
50 | 54 | import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; |
51 | 55 | import org.apache.iceberg.relocated.com.google.common.collect.Iterables; |
@@ -265,6 +269,51 @@ public void testColumnStatisticsEnabled() throws Exception { |
265 | 269 | } |
266 | 270 | } |
267 | 271 |
|
| 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 | + |
268 | 317 | private Pair<File, Long> generateFile( |
269 | 318 | Function<MessageType, ParquetValueWriter<?>> createWriterFunc, |
270 | 319 | int desiredRecordCount, |
|
0 commit comments