-
Notifications
You must be signed in to change notification settings - Fork 3k
Core: Support writing parquet bloom filter #2642
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| 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()); | ||
| } | ||
| } | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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 { | ||
|
|
||
|
|
@@ -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))); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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[] {})); | ||
| } | ||
| } | ||
There was a problem hiding this comment.
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 ?
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
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
There was a problem hiding this comment.
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 ?
There was a problem hiding this comment.
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.