1
0

[HUDI-106] Adding support for DynamicBloomFilter (#976)

- Introduced configs for bloom filter type
- Implemented dynamic bloom filter with configurable max number of keys
- BloomFilterFactory abstractions; Defaults to current simple bloom filter
This commit is contained in:
Sivabalan Narayanan
2019-12-17 19:06:24 -08:00
committed by vinoth chandar
parent 7498ca71cb
commit 14881e99e0
23 changed files with 1069 additions and 83 deletions

View File

@@ -1,49 +0,0 @@
/*
* 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.hudi.common;
import org.junit.Test;
import java.io.IOException;
/**
* Tests bloom filter {@link BloomFilter}.
*/
public class TestBloomFilter {
@Test
public void testAddKey() {
BloomFilter filter = new BloomFilter(100, 0.0000001);
filter.add("key1");
assert (filter.mightContain("key1"));
}
@Test
public void testSerialize() throws IOException, ClassNotFoundException {
BloomFilter filter = new BloomFilter(1000, 0.0000001);
filter.add("key1");
filter.add("key2");
String filterStr = filter.serializeToString();
// Rebuild
BloomFilter newFilter = new BloomFilter(filterStr);
assert (newFilter.mightContain("key1"));
assert (newFilter.mightContain("key2"));
}
}

View File

@@ -0,0 +1,110 @@
/*
* 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.hudi.common.bloom.filter;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.UUID;
/**
* Unit tests {@link SimpleBloomFilter} and {@link HoodieDynamicBoundedBloomFilter}.
*/
@RunWith(Parameterized.class)
public class TestBloomFilter {
private final String versionToTest;
// name attribute is optional, provide an unique name for test
// multiple parameters, uses Collection<Object[]>
@Parameters()
public static Collection<Object[]> data() {
return Arrays.asList(new Object[][] {
{BloomFilterTypeCode.SIMPLE.name()},
{BloomFilterTypeCode.DYNAMIC_V0.name()}
});
}
public TestBloomFilter(String versionToTest) {
this.versionToTest = versionToTest;
}
@Test
public void testAddKey() {
List<String> inputs = new ArrayList<>();
int[] sizes = {100, 1000, 10000};
for (int size : sizes) {
inputs = new ArrayList<>();
BloomFilter filter = getBloomFilter(versionToTest, size, 0.000001, size * 10);
for (int i = 0; i < size; i++) {
String key = UUID.randomUUID().toString();
inputs.add(key);
filter.add(key);
}
for (java.lang.String key : inputs) {
Assert.assertTrue("Filter should have returned true for " + key, filter.mightContain(key));
}
for (int i = 0; i < 100; i++) {
String randomKey = UUID.randomUUID().toString();
if (inputs.contains(randomKey)) {
Assert.assertTrue("Filter should have returned true for " + randomKey, filter.mightContain(randomKey));
}
}
}
}
@Test
public void testSerialize() throws IOException, ClassNotFoundException {
List<String> inputs = new ArrayList<>();
int[] sizes = {100, 1000, 10000};
for (int size : sizes) {
inputs = new ArrayList<>();
BloomFilter filter = getBloomFilter(versionToTest, size, 0.000001, size * 10);
for (int i = 0; i < size; i++) {
String key = UUID.randomUUID().toString();
inputs.add(key);
filter.add(key);
}
String serString = filter.serializeToString();
BloomFilter recreatedBloomFilter = BloomFilterFactory
.fromString(serString, versionToTest);
for (String key : inputs) {
Assert.assertTrue("Filter should have returned true for " + key, recreatedBloomFilter.mightContain(key));
}
}
}
BloomFilter getBloomFilter(String typeCode, int numEntries, double errorRate, int maxEntries) {
if (typeCode.equalsIgnoreCase(BloomFilterTypeCode.SIMPLE.name())) {
return BloomFilterFactory.createBloomFilter(numEntries, errorRate, -1, typeCode);
} else {
return BloomFilterFactory.createBloomFilter(numEntries, errorRate, maxEntries, typeCode);
}
}
}

View File

@@ -0,0 +1,60 @@
/*
* 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.hudi.common.bloom.filter;
import org.apache.hadoop.util.hash.Hash;
import org.junit.Assert;
import org.junit.Test;
import java.util.UUID;
/**
* Unit tests {@link InternalDynamicBloomFilter} for size bounding.
*/
public class TestInternalDynamicBloomFilter {
@Test
public void testBoundedSize() {
int[] batchSizes = {1000, 10000, 10000, 100000, 100000, 10000};
int indexForMaxGrowth = 3;
int maxSize = batchSizes[0] * 100;
BloomFilter filter = new HoodieDynamicBoundedBloomFilter(batchSizes[0], 0.000001, Hash.MURMUR_HASH, maxSize);
int index = 0;
int lastKnownBloomSize = 0;
while (index < batchSizes.length) {
for (int i = 0; i < batchSizes[index]; i++) {
String key = UUID.randomUUID().toString();
filter.add(key);
}
String serString = filter.serializeToString();
if (index != 0) {
int curLength = serString.length();
if (index > indexForMaxGrowth) {
Assert.assertEquals("Length should not increase after hitting max entries", curLength, lastKnownBloomSize);
} else {
Assert.assertTrue("Length should increase until max entries are reached", curLength > lastKnownBloomSize);
}
}
lastKnownBloomSize = serString.length();
index++;
}
}
}

View File

@@ -19,22 +19,30 @@
package org.apache.hudi.common.util;
import org.apache.hudi.avro.HoodieAvroWriteSupport;
import org.apache.hudi.common.BloomFilter;
import org.apache.hudi.common.HoodieCommonTestHarness;
import org.apache.hudi.common.bloom.filter.BloomFilter;
import org.apache.hudi.common.bloom.filter.BloomFilterFactory;
import org.apache.hudi.common.bloom.filter.BloomFilterTypeCode;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.avro.AvroSchemaConverter;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
@@ -47,8 +55,23 @@ import static org.junit.Assert.assertTrue;
/**
* Tests parquet utils.
*/
@RunWith(Parameterized.class)
public class TestParquetUtils extends HoodieCommonTestHarness {
String bloomFilterTypeToTest;
@Parameters()
public static Collection<Object[]> data() {
return Arrays.asList(new Object[][] {
{BloomFilterTypeCode.SIMPLE.name()},
{BloomFilterTypeCode.DYNAMIC_V0.name()}
});
}
public TestParquetUtils(String bloomFilterTypeToTest) {
this.bloomFilterTypeToTest = bloomFilterTypeToTest;
}
@Before
public void setup() {
initPath();
@@ -78,6 +101,16 @@ public class TestParquetUtils extends HoodieCommonTestHarness {
}
}
private Configuration getConfiguration() {
if (bloomFilterTypeToTest.equalsIgnoreCase(BloomFilterTypeCode.SIMPLE.name())) {
return HoodieTestUtils.getDefaultHadoopConf();
} else {
org.apache.hadoop.conf.Configuration conf = HoodieTestUtils.getDefaultHadoopConf();
// conf.set();
return conf;
}
}
@Test
public void testFilterParquetRowKeys() throws Exception {
List<String> rowKeys = new ArrayList<>();
@@ -107,7 +140,8 @@ public class TestParquetUtils extends HoodieCommonTestHarness {
private void writeParquetFile(String filePath, List<String> rowKeys) throws Exception {
// Write out a parquet file
Schema schema = HoodieAvroUtils.getRecordKeySchema();
BloomFilter filter = new BloomFilter(1000, 0.0001);
BloomFilter filter = BloomFilterFactory
.createBloomFilter(1000, 0.0001, 10000, bloomFilterTypeToTest);
HoodieAvroWriteSupport writeSupport =
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
ParquetWriter writer = new ParquetWriter(new Path(filePath), writeSupport, CompressionCodecName.GZIP,