Skip to content
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
91 changes: 91 additions & 0 deletions parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java
Original file line number Diff line number Diff line change
Expand Up @@ -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<GenericData.Record> 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.<String, String>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<GenericData.Record> 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.<String, String>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 =
Expand Down