[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:
committed by
vinoth chandar
parent
7498ca71cb
commit
14881e99e0
@@ -21,6 +21,9 @@ package org.apache.hudi.common;
|
||||
import org.apache.hudi.HoodieReadClient;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
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.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
@@ -147,7 +150,7 @@ public class HoodieClientTestUtils {
|
||||
}
|
||||
|
||||
public static HashMap<String, String> getLatestFileIDsToFullPath(String basePath, HoodieTimeline commitTimeline,
|
||||
List<HoodieInstant> commitsToReturn) throws IOException {
|
||||
List<HoodieInstant> commitsToReturn) throws IOException {
|
||||
HashMap<String, String> fileIdToFullPath = new HashMap<>();
|
||||
for (HoodieInstant commit : commitsToReturn) {
|
||||
HoodieCommitMetadata metadata =
|
||||
@@ -158,7 +161,7 @@ public class HoodieClientTestUtils {
|
||||
}
|
||||
|
||||
public static Dataset<Row> readCommit(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline,
|
||||
String commitTime) {
|
||||
String commitTime) {
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||
if (!commitTimeline.containsInstant(commitInstant)) {
|
||||
new HoodieException("No commit exists at " + commitTime);
|
||||
@@ -178,7 +181,7 @@ public class HoodieClientTestUtils {
|
||||
* Obtain all new data written into the Hoodie dataset since the given timestamp.
|
||||
*/
|
||||
public static Dataset<Row> readSince(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline,
|
||||
String lastCommitTime) {
|
||||
String lastCommitTime) {
|
||||
List<HoodieInstant> commitsToReturn =
|
||||
commitTimeline.findInstantsAfter(lastCommitTime, Integer.MAX_VALUE).getInstants().collect(Collectors.toList());
|
||||
try {
|
||||
@@ -195,7 +198,7 @@ public class HoodieClientTestUtils {
|
||||
* Reads the paths under the a hoodie dataset out as a DataFrame.
|
||||
*/
|
||||
public static Dataset<Row> read(JavaSparkContext jsc, String basePath, SQLContext sqlContext, FileSystem fs,
|
||||
String... paths) {
|
||||
String... paths) {
|
||||
List<String> filteredPaths = new ArrayList<>();
|
||||
try {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true);
|
||||
@@ -214,10 +217,11 @@ public class HoodieClientTestUtils {
|
||||
}
|
||||
|
||||
public static String writeParquetFile(String basePath, String partitionPath, String filename,
|
||||
List<HoodieRecord> records, Schema schema, BloomFilter filter, boolean createCommitTime) throws IOException {
|
||||
List<HoodieRecord> records, Schema schema, BloomFilter filter, boolean createCommitTime) throws IOException {
|
||||
|
||||
if (filter == null) {
|
||||
filter = new BloomFilter(10000, 0.0000001);
|
||||
filter = BloomFilterFactory
|
||||
.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name());
|
||||
}
|
||||
HoodieAvroWriteSupport writeSupport =
|
||||
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
@@ -245,7 +249,7 @@ public class HoodieClientTestUtils {
|
||||
}
|
||||
|
||||
public static String writeParquetFile(String basePath, String partitionPath, List<HoodieRecord> records,
|
||||
Schema schema, BloomFilter filter, boolean createCommitTime) throws IOException, InterruptedException {
|
||||
Schema schema, BloomFilter filter, boolean createCommitTime) throws IOException, InterruptedException {
|
||||
Thread.sleep(1000);
|
||||
String commitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
|
||||
@@ -19,9 +19,11 @@
|
||||
package org.apache.hudi.index.bloom;
|
||||
|
||||
import org.apache.hudi.HoodieClientTestHarness;
|
||||
import org.apache.hudi.common.BloomFilter;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.TestRawTripPayload;
|
||||
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.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
@@ -248,7 +250,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
|
||||
// We write record1, record2 to a parquet file, but the bloom filter contains (record1,
|
||||
// record2, record3).
|
||||
BloomFilter filter = new BloomFilter(10000, 0.0000001);
|
||||
BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name());
|
||||
filter.add(record3.getRecordKey());
|
||||
String filename = HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Arrays.asList(record1, record2),
|
||||
schema, filter, true);
|
||||
@@ -451,7 +453,8 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
HoodieRecord record2 =
|
||||
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||
|
||||
BloomFilter filter = new BloomFilter(10000, 0.0000001);
|
||||
BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1,
|
||||
BloomFilterTypeCode.SIMPLE.name());
|
||||
filter.add(record2.getRecordKey());
|
||||
String filename =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Arrays.asList(record1), schema, filter, true);
|
||||
|
||||
@@ -20,11 +20,11 @@ package org.apache.hudi.table;
|
||||
|
||||
import org.apache.hudi.HoodieClientTestHarness;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.BloomFilter;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.TestRawTripPayload;
|
||||
import org.apache.hudi.common.TestRawTripPayload.MetadataMergeWriteStatus;
|
||||
import org.apache.hudi.common.bloom.filter.BloomFilter;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
|
||||
Reference in New Issue
Block a user