Skip to content
Closed
Show file tree
Hide file tree
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
12 changes: 12 additions & 0 deletions core/src/main/java/org/apache/iceberg/TableProperties.java
Original file line number Diff line number Diff line change
Expand Up @@ -111,6 +111,18 @@ private TableProperties() {
public static final String DELETE_AVRO_COMPRESSION = "write.delete.avro.compression-codec";
public static final String AVRO_COMPRESSION_DEFAULT = "gzip";

public static final String DEFAULT_PARQUET_BLOOM_FILTER_ENABLED = "write.parquet.bloom-filter-enabled.default";
public static final boolean DEFAULT_PARQUET_BLOOM_FILTER_ENABLED_DEFAULT = false;
public static final String PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX = "write.parquet.bloom-filter-enabled.column.";
public static final boolean PARQUET_BLOOM_FILTER_COLUMN_ENABLED_DEFAULT = false;

public static final String PARQUET_BLOOM_FILTER_COLUMN_EXPECTED_NDV_PREFIX =
"write.parquet.bloom-filter-expected-ndv.column.";
public static final long PARQUET_BLOOM_FILTER_COLUMN_EXPECTED_NDV_DEFAULT = -1L;

public static final String PARQUET_BLOOM_FILTER_MAX_BYTES = "write.parquet.bloom-filter-max-bytes";
public static final int PARQUET_BLOOM_FILTER_MAX_BYTES_DEFAULT = 1024 * 1024;

public static final String SPLIT_SIZE = "read.split.target-size";
public static final long SPLIT_SIZE_DEFAULT = 134217728; // 128 MB

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.apache.iceberg.parquet;

import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.function.BiConsumer;
import java.util.function.Function;

/**
* Copied from parquet-hadoop
*/
class ColumnConfigParser {

private static class ConfigHelper<T> {
private final String prefix;
private final Function<String, T> function;
private final BiConsumer<String, T> consumer;

ConfigHelper(String prefix, Function<String, T> function, BiConsumer<String, T> consumer) {
this.prefix = prefix;
this.function = function;
this.consumer = consumer;
}

public void processKey(String key) {
if (key.startsWith(prefix)) {
String columnPath = key.substring(prefix.length());
T value = function.apply(key);
consumer.accept(columnPath, value);
}
}
}

private final List<ConfigHelper<?>> helpers = new ArrayList<>();

public <T> ColumnConfigParser withColumnConfig(String prefix, Function<String, T> function,
BiConsumer<String, T> consumer) {
helpers.add(new ConfigHelper<>(prefix, function, consumer));
return this;
}

public void parseConfig(Map<String, String> conf) {
for (Map.Entry<String, String> entry : conf.entrySet()) {
for (ConfigHelper<?> helper : helpers) {
// We retrieve the value from function instead of parsing from the string here to get the correct type
helper.processKey(entry.getKey());
}
}
}
}
51 changes: 47 additions & 4 deletions parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java
Original file line number Diff line number Diff line change
Expand Up @@ -78,11 +78,19 @@
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.parquet.schema.MessageType;

import static org.apache.iceberg.TableProperties.DEFAULT_PARQUET_BLOOM_FILTER_ENABLED;
import static org.apache.iceberg.TableProperties.DEFAULT_PARQUET_BLOOM_FILTER_ENABLED_DEFAULT;
import static org.apache.iceberg.TableProperties.DELETE_PARQUET_COMPRESSION;
import static org.apache.iceberg.TableProperties.DELETE_PARQUET_COMPRESSION_LEVEL;
import static org.apache.iceberg.TableProperties.DELETE_PARQUET_DICT_SIZE_BYTES;
import static org.apache.iceberg.TableProperties.DELETE_PARQUET_PAGE_SIZE_BYTES;
import static org.apache.iceberg.TableProperties.DELETE_PARQUET_ROW_GROUP_SIZE_BYTES;
import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_ENABLED_DEFAULT;
import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX;
import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_EXPECTED_NDV_DEFAULT;
import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_EXPECTED_NDV_PREFIX;
import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_MAX_BYTES;
import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_MAX_BYTES_DEFAULT;
import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION;
import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_DEFAULT;
import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL;
Expand Down Expand Up @@ -225,6 +233,10 @@ public <D> FileAppender<D> build() throws IOException {
int dictionaryPageSize = context.dictionaryPageSize();
String compressionLevel = context.compressionLevel();
CompressionCodecName codec = context.codec();
boolean bloomFilterEnabled = PropertyUtil.propertyAsBoolean(config, DEFAULT_PARQUET_BLOOM_FILTER_ENABLED,
DEFAULT_PARQUET_BLOOM_FILTER_ENABLED_DEFAULT);
int bloomFilterMaxBytes = PropertyUtil.propertyAsInt(config, PARQUET_BLOOM_FILTER_MAX_BYTES,
PARQUET_BLOOM_FILTER_MAX_BYTES_DEFAULT);

if (compressionLevel != null) {
switch (codec) {
Expand Down Expand Up @@ -259,17 +271,31 @@ public <D> FileAppender<D> build() throws IOException {
conf.set(entry.getKey(), entry.getValue());
}

ParquetProperties parquetProperties = ParquetProperties.builder()
ParquetProperties.Builder propsBuilder = ParquetProperties.builder()
.withWriterVersion(writerVersion)
.withPageSize(pageSize)
.withDictionaryPageSize(dictionaryPageSize)
.build();
.withMaxBloomFilterBytes(bloomFilterMaxBytes)
.withBloomFilterEnabled(bloomFilterEnabled);

new ColumnConfigParser()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

looks a bit weird, is this intended, new instance without reference to it ?

Copy link
Contributor Author

@jshmchenxi jshmchenxi Feb 9, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This ColumnConfigParser instance is created to parse column configs and apply to propsBuilder

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I know it's probably not a big deal, but I mean could new instances be avoided ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The parser is designed to be used in this way, just like we need creating builder instances to build objects.

.withColumnConfig(
PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX,
key -> PropertyUtil.propertyAsBoolean(config, key, PARQUET_BLOOM_FILTER_COLUMN_ENABLED_DEFAULT),
propsBuilder::withBloomFilterEnabled)
.withColumnConfig(
PARQUET_BLOOM_FILTER_COLUMN_EXPECTED_NDV_PREFIX,
key -> PropertyUtil.propertyAsLong(config, key, PARQUET_BLOOM_FILTER_COLUMN_EXPECTED_NDV_DEFAULT),
propsBuilder::withBloomFilterNDV)
.parseConfig(config);

ParquetProperties parquetProperties = propsBuilder.build();

return new org.apache.iceberg.parquet.ParquetWriter<>(
conf, file, schema, rowGroupSize, metadata, createWriterFunc, codec,
parquetProperties, metricsConfig, writeMode);
} else {
return new ParquetWriteAdapter<>(new ParquetWriteBuilder<D>(ParquetIO.file(file))
ParquetWriteBuilder<D> parquetWriteBuilder = new ParquetWriteBuilder<D>(ParquetIO.file(file))
.withWriterVersion(writerVersion)
.setType(type)
.setConfig(config)
Expand All @@ -280,7 +306,22 @@ public <D> FileAppender<D> build() throws IOException {
.withRowGroupSize(rowGroupSize)
.withPageSize(pageSize)
.withDictionaryPageSize(dictionaryPageSize)
.build(),
// TODO: add .withMaxBloomFilterBytes(bloomFilterMaxBytes) once ParquetWriter.Builder supports it
.withBloomFilterEnabled(bloomFilterEnabled);

new ColumnConfigParser()
.withColumnConfig(
PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX,
key -> PropertyUtil.propertyAsBoolean(config, key, PARQUET_BLOOM_FILTER_COLUMN_ENABLED_DEFAULT),
parquetWriteBuilder::withBloomFilterEnabled)
.withColumnConfig(
PARQUET_BLOOM_FILTER_COLUMN_EXPECTED_NDV_PREFIX,
key -> PropertyUtil.propertyAsLong(config, key, PARQUET_BLOOM_FILTER_COLUMN_EXPECTED_NDV_DEFAULT),
parquetWriteBuilder::withBloomFilterNDV)
.parseConfig(config);

return new ParquetWriteAdapter<>(
parquetWriteBuilder.build(),
metricsConfig);
}
}
Expand Down Expand Up @@ -846,12 +887,14 @@ public <D> CloseableIterable<D> build() {
Schema fileSchema = ParquetSchemaUtil.convert(type);
builder.useStatsFilter()
.useDictionaryFilter()
.useBloomFilter()
.useRecordFilter(filterRecords)
.withFilter(ParquetFilters.convert(fileSchema, filter, caseSensitive));
} else {
// turn off filtering
builder.useStatsFilter(false)
.useDictionaryFilter(false)
.useBloomFilter(false)
.useRecordFilter(false);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -337,6 +337,10 @@ public static boolean hasNonDictionaryPages(ColumnChunkMetaData meta) {
}
}

public static boolean hasNonBloomFilterPages(ColumnChunkMetaData meta) {
return meta.getBloomFilterOffset() < 0;
}

public static Dictionary readDictionary(ColumnDescriptor desc, PageReader pageSource) {
DictionaryPage dictionaryPage = pageSource.readDictionaryPage();
if (dictionaryPage != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import org.apache.parquet.column.ColumnWriteStore;
import org.apache.parquet.column.ParquetProperties;
import org.apache.parquet.column.page.PageWriteStore;
import org.apache.parquet.column.values.bloomfilter.BloomFilterWriteStore;
import org.apache.parquet.hadoop.CodecFactory;
import org.apache.parquet.hadoop.ParquetFileWriter;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
Expand Down Expand Up @@ -199,7 +200,7 @@ private void startRowGroup() {
compressor, parquetSchema, props.getAllocator(), this.columnIndexTruncateLength);

this.flushPageStoreToWriter = flushToWriter.bind(pageStore);
this.writeStore = props.newColumnWriteStore(parquetSchema, pageStore);
this.writeStore = props.newColumnWriteStore(parquetSchema, pageStore, (BloomFilterWriteStore) pageStore);

model.setColumnStore(writeStore);
}
Expand Down
90 changes: 90 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 @@ -26,23 +26,37 @@
import java.util.List;
import java.util.function.Function;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecordBuilder;
import org.apache.iceberg.Schema;
import org.apache.iceberg.avro.AvroSchemaUtil;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.types.Types.DoubleType;
import org.apache.iceberg.types.Types.FloatType;
import org.apache.iceberg.types.Types.IntegerType;
import org.apache.iceberg.types.Types.LongType;
import org.apache.iceberg.types.Types.StringType;
import org.apache.iceberg.util.Pair;
import org.apache.parquet.column.values.bloomfilter.BloomFilter;
import org.apache.parquet.hadoop.BloomFilterReader;
import org.apache.parquet.hadoop.ParquetFileReader;
import org.apache.parquet.hadoop.metadata.BlockMetaData;
import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
import org.apache.parquet.io.api.Binary;
import org.apache.parquet.schema.MessageType;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;

import static org.apache.iceberg.Files.localInput;
import static org.apache.iceberg.TableProperties.DEFAULT_PARQUET_BLOOM_FILTER_ENABLED;
import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX;
import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES;
import static org.apache.iceberg.parquet.ParquetWritingTestUtils.createTempFile;
import static org.apache.iceberg.parquet.ParquetWritingTestUtils.write;
import static org.apache.iceberg.parquet.ParquetWritingTestUtils.writeRecords;
import static org.apache.iceberg.types.Types.NestedField.optional;
import static org.apache.iceberg.types.Types.NestedField.required;

public class TestParquet {

Expand Down Expand Up @@ -127,4 +141,80 @@ private Pair<File, Long> generateFileWithTwoRowGroups(Function<MessageType, Parq
records.toArray(new GenericData.Record[]{}));
return Pair.of(file, size);
}

@Test
public void testBloomFilterWriteRead() throws IOException {
File parquetFile = generateFileWithBloomFilter();

try (ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(localInput(parquetFile)))) {
BlockMetaData rowGroup = reader.getRowGroups().get(0);
BloomFilterReader bloomFilterDataReader = reader.getBloomFilterDataReader(rowGroup);

ColumnChunkMetaData intColumn = rowGroup.getColumns().get(0);
BloomFilter intBloomFilter = bloomFilterDataReader.readBloomFilter(intColumn);
Assert.assertTrue(intBloomFilter.findHash(intBloomFilter.hash(30)));
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we should add in a few negative tests as well since all of these check for inclusion and none for exclusion. I'm not super worried about this since I assume this code path is already well tested in Parquet.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The hash values might conflict so I didn't add exclusion tests.

Assert.assertTrue(intBloomFilter.findHash(intBloomFilter.hash(40)));
Assert.assertTrue(intBloomFilter.findHash(intBloomFilter.hash(79)));

ColumnChunkMetaData longColumn = rowGroup.getColumns().get(1);
BloomFilter longBloomFilter = bloomFilterDataReader.readBloomFilter(longColumn);
Assert.assertTrue(longBloomFilter.findHash(longBloomFilter.hash(130L)));
Assert.assertTrue(longBloomFilter.findHash(longBloomFilter.hash(140L)));
Assert.assertTrue(longBloomFilter.findHash(longBloomFilter.hash(179L)));

ColumnChunkMetaData floatColumn = rowGroup.getColumns().get(2);
BloomFilter floatBloomFilter = bloomFilterDataReader.readBloomFilter(floatColumn);
Assert.assertTrue(floatBloomFilter.findHash(floatBloomFilter.hash(1030F)));
Assert.assertTrue(floatBloomFilter.findHash(floatBloomFilter.hash(1040F)));
Assert.assertTrue(floatBloomFilter.findHash(floatBloomFilter.hash(1079F)));

ColumnChunkMetaData doubleColumn = rowGroup.getColumns().get(3);
BloomFilter doubleBloomFilter = bloomFilterDataReader.readBloomFilter(doubleColumn);
Assert.assertTrue(doubleBloomFilter.findHash(doubleBloomFilter.hash(10030D)));
Assert.assertTrue(doubleBloomFilter.findHash(doubleBloomFilter.hash(10040D)));
Assert.assertTrue(doubleBloomFilter.findHash(doubleBloomFilter.hash(10079D)));

ColumnChunkMetaData binaryColumn = rowGroup.getColumns().get(4);
BloomFilter binaryBloomFilter = bloomFilterDataReader.readBloomFilter(binaryColumn);
Assert.assertTrue(binaryBloomFilter.findHash(binaryBloomFilter.hash(Binary.fromString("BINARY_30"))));
Assert.assertTrue(binaryBloomFilter.findHash(binaryBloomFilter.hash(Binary.fromString("BINARY_40"))));
Assert.assertTrue(binaryBloomFilter.findHash(binaryBloomFilter.hash(Binary.fromString("BINARY_79"))));

ColumnChunkMetaData noBloomColumn = rowGroup.getColumns().get(5);
Assert.assertFalse(ParquetUtil.hasNonBloomFilterPages(intColumn));
Assert.assertFalse(ParquetUtil.hasNonBloomFilterPages(longColumn));
Assert.assertFalse(ParquetUtil.hasNonBloomFilterPages(floatColumn));
Assert.assertFalse(ParquetUtil.hasNonBloomFilterPages(doubleColumn));
Assert.assertFalse(ParquetUtil.hasNonBloomFilterPages(binaryColumn));
Assert.assertTrue(ParquetUtil.hasNonBloomFilterPages(noBloomColumn));
}
}

private File generateFileWithBloomFilter() throws IOException {
Schema schema = new Schema(
required(0, "id", IntegerType.get()),
required(1, "id_long", LongType.get()),
required(2, "id_float", FloatType.get()),
required(3, "id_double", DoubleType.get()),
required(4, "id_binary", StringType.get()),
required(5, "id_no_bloom", IntegerType.get())
);
int recordCount = 50;
List<GenericData.Record> records = new ArrayList<>(recordCount);
GenericRecordBuilder builder = new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "bloom_filter_test"));
for (int i = 0; i < recordCount; i++) {
builder.set("id", 30 + i); // min=30, max=79
builder.set("id_long", 100L + 30 + i); // min=130L, max=179L
builder.set("id_float", 1000F + 30 + i); // min=1030F, max=10079F
builder.set("id_double", 10000D + 30 + i); // min=10030D, max=1079D
builder.set("id_binary", "BINARY_" + (30 + i)); // min=BINARY_30, max=BINARY_79
builder.set("id_no_bloom", 30 + i); // min=30, max=79
records.add(builder.build());
}

ImmutableMap<String, String> config = ImmutableMap.of(
DEFAULT_PARQUET_BLOOM_FILTER_ENABLED, "true",
PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_no_bloom", "false");
return writeRecords(temp, schema, config, records.toArray(new GenericData.Record[] {}));
}
}
4 changes: 4 additions & 0 deletions site/docs/configuration.md
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,10 @@ Iceberg tables support table properties to configure table behavior, like the de
| write.parquet.dict-size-bytes | 2097152 (2 MB) | Parquet dictionary page size |
| write.parquet.compression-codec | gzip | Parquet compression codec |
| write.parquet.compression-level | null | Parquet compression level |
| write.parquet.bloom-filter-enabled.default | false | Whether to enable writing bloom filter for all columns |
| write.parquet.bloom-filter-enabled.column.col1 | (not set) | Whether to enable writing bloom filter for column 'col1' to allow per-column configuration; This property overrides `bloom-filter-enabled.default` for the specified column; For example, setting both `write.parquet.bloom-filter-enabled.default=true` and `write.parquet.bloom-filter-enabled.column.some_col=false` will enable bloom filter for all columns except `some_col` |
| write.parquet.bloom-filter-expected-ndv.column.col1 | (not set) | The expected number of distinct values in a column, it is used to compute the optimal size of the bloom filter; Note that if this property is not set, the bloom filter will use the maximum size set in `bloom-filter-max-bytes`; This property overrides `bloom-filter-enabled`, automatically enabling bloom filters for any columns specified; For example, setting both `write.parquet.bloom-filter-enabled.column.some_col=false` and `write.parquet.bloom-filter-expected-ndv.column.some_col=200` will enable bloom filter for `some_col` with expected number of distinct values equals to 200 |
| write.parquet.bloom-filter-max-bytes | 1048576 (1 MB) | The maximum number of bytes for a bloom filter bitset |
| write.avro.compression-codec | gzip | Avro compression codec |
| write.location-provider.impl | null | Optional custom implemention for LocationProvider |
| write.metadata.compression-codec | none | Metadata compression codec; none or gzip |
Expand Down