From a4d4994d724b86c2f8e9bdcbbd2424f7d2723c67 Mon Sep 17 00:00:00 2001 From: Arya Rizky Date: Sun, 31 May 2026 21:13:56 +0700 Subject: [PATCH] parquet: Add tests for disabling statistics on multiple columns Add two new test cases to verify per-column statistics configuration works correctly when stats are disabled on more than one column: - testColumnStatisticsDisabledMultipleColumns: verifies that when stats are enabled on one column and disabled on two others, both disabled columns correctly omit statistics in the Parquet file. - testColumnStatisticsDisabledAllColumns: verifies that when stats are disabled on every column, all columns correctly omit statistics. These tests address the edge case reported in issue #15347, where disabling statistics across multiple columns could result in some columns still writing statistics. --- .../apache/iceberg/parquet/TestParquet.java | 91 +++++++++++++++++++ 1 file changed, 91 insertions(+) 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 5f1e0c83cc0f..2363ba0fd26b 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java @@ -273,6 +273,97 @@ public void testColumnStatisticsEnabled() throws Exception { } } + @Test + public void testColumnStatisticsDisabledMultipleColumns() throws Exception { + Schema schema = + new Schema( + optional(1, "int_field", IntegerType.get()), + optional(2, "string_field", Types.StringType.get()), + optional(3, "double_field", Types.DoubleType.get())); + + File file = createTempFile(temp); + + List records = Lists.newArrayListWithCapacity(5); + org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); + for (int i = 1; i <= 5; i++) { + GenericData.Record record = new GenericData.Record(avroSchema); + record.put("int_field", i); + record.put("string_field", "test"); + record.put("double_field", i * 1.5); + records.add(record); + } + + write( + file, + schema, + ImmutableMap.builder() + .put(PARQUET_COLUMN_STATS_ENABLED_PREFIX + "int_field", "true") + .put(PARQUET_COLUMN_STATS_ENABLED_PREFIX + "string_field", "false") + .put(PARQUET_COLUMN_STATS_ENABLED_PREFIX + "double_field", "false") + .buildOrThrow(), + ParquetAvroWriter::buildWriter, + records.toArray(new GenericData.Record[] {})); + + InputFile inputFile = Files.localInput(file); + + try (ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(inputFile))) { + for (BlockMetaData block : reader.getFooter().getBlocks()) { + for (ColumnChunkMetaData column : block.getColumns()) { + boolean emptyStats = column.getStatistics().isEmpty(); + if (column.getPath().toDotString().equals("int_field")) { + assertThat(emptyStats).as("int_field has statistics").isEqualTo(false); + } else if (column.getPath().toDotString().equals("string_field")) { + assertThat(emptyStats).as("string_field has statistics disabled").isEqualTo(true); + } else if (column.getPath().toDotString().equals("double_field")) { + assertThat(emptyStats).as("double_field has statistics disabled").isEqualTo(true); + } + } + } + } + } + + @Test + public void testColumnStatisticsDisabledAllColumns() throws Exception { + Schema schema = + new Schema( + optional(1, "int_field", IntegerType.get()), + optional(2, "string_field", Types.StringType.get())); + + File file = createTempFile(temp); + + List records = Lists.newArrayListWithCapacity(5); + org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); + for (int i = 1; i <= 5; i++) { + GenericData.Record record = new GenericData.Record(avroSchema); + record.put("int_field", i); + record.put("string_field", "test"); + records.add(record); + } + + write( + file, + schema, + ImmutableMap.builder() + .put(PARQUET_COLUMN_STATS_ENABLED_PREFIX + "int_field", "false") + .put(PARQUET_COLUMN_STATS_ENABLED_PREFIX + "string_field", "false") + .buildOrThrow(), + ParquetAvroWriter::buildWriter, + records.toArray(new GenericData.Record[] {})); + + InputFile inputFile = Files.localInput(file); + + try (ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(inputFile))) { + for (BlockMetaData block : reader.getFooter().getBlocks()) { + for (ColumnChunkMetaData column : block.getColumns()) { + boolean emptyStats = column.getStatistics().isEmpty(); + assertThat(emptyStats) + .as(column.getPath().toDotString() + " has statistics disabled") + .isEqualTo(true); + } + } + } + } + @Test public void testFooterMetricsWithNameMappingForFileWithoutIds() throws IOException { Schema schemaWithIds =