nsivabalan commented on a change in pull request #1469: URL: https://github.com/apache/incubator-hudi/pull/1469#discussion_r412955401
########## File path: hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java ########## @@ -430,6 +430,14 @@ public boolean getBloomIndexUpdatePartitionPath() { return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_UPDATE_PARTITION_PATH)); } + public int getBloomIndexV2Parallelism() { + return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_V2_PARALLELISM_PROP)); + } + + public long getBloomIndexV2BufferSize() { Review comment: nit: MaxBufferSize() when you change the name as per previous comment ########## File path: hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexV2.java ########## @@ -0,0 +1,321 @@ +/* + * 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.index.bloom; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.utils.LazyIterableIterator; +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.DefaultSizeEstimator; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ExternalSpillableMap; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.io.HoodieBloomRangeInfoHandle; +import org.apache.hudi.io.HoodieKeyLookupHandle; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Simplified re-implementation of {@link HoodieBloomIndex} that does not rely on caching, or + * incurs the overhead of auto-tuning parallelism. + */ +public class HoodieBloomIndexV2<T extends HoodieRecordPayload> extends HoodieIndex<T> { + + private static final Logger LOG = LogManager.getLogger(HoodieBloomIndexV2.class); + + public HoodieBloomIndexV2(HoodieWriteConfig config) { + super(config); + } + + @Override + public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, + JavaSparkContext jsc, + HoodieTable<T> hoodieTable) { + return recordRDD + .sortBy((record) -> String.format("%s-%s", record.getPartitionPath(), record.getRecordKey()), + true, config.getBloomIndexV2Parallelism()) + .mapPartitions((itr) -> new LazyRangeBloomChecker(itr, hoodieTable)).flatMap(List::iterator) + .sortBy(Pair::getRight, true, config.getBloomIndexV2Parallelism()) + .mapPartitions((itr) -> new LazyKeyChecker(itr, hoodieTable)) + .filter(Option::isPresent) + .map(Option::get); + } + + @Override + public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys, + JavaSparkContext jsc, HoodieTable<T> hoodieTable) { + throw new UnsupportedOperationException("Not yet implemented"); + } + + @Override + public boolean rollbackCommit(String commitTime) { + // Nope, don't need to do anything. + return true; + } + + /** + * This is not global, since we depend on the partitionPath to do the lookup. + */ + @Override + public boolean isGlobal() { + return false; + } + + /** + * No indexes into log files yet. + */ + @Override + public boolean canIndexLogFiles() { + return false; + } + + /** + * Bloom filters are stored, into the same data files. + */ + @Override + public boolean isImplicitWithStorage() { + return true; + } + + @Override + public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc, + HoodieTable<T> hoodieTable) { + return writeStatusRDD; + } + + /** + * Given an iterator of hoodie records, returns a pair of candidate HoodieRecord, FileID pairs. Review comment: can we have more descriptive docs to convey what exactly the class is doing. Something like "Given an iterator of hoodie records, returns a pair of candidate HoodieRecord, FileID pairs, by filtering for ranges and bloom for all records with all fileIds" sort of. ########## File path: hudi-common/src/main/java/org/apache/hudi/common/util/HoodieTimer.java ########## @@ -69,4 +76,13 @@ public long endTimer() { } return timeInfoDeque.pop().stop(); } + + public long deltaTime() { Review comment: yeah, looks like HoodieTimer already gives you the delta. So, don't really understand why we need the changes. T0: HoodieTimer timer = new HoodieTimer(); T1: timer.startTimer(); T4: timer.endTimer(); // T4 - T1 T4: timer.startTimer(); T6: timer.endTimer(); // T6 - T4 T8: timer.startTimer(); T9: timer.startTimer(); T12: timer.endTimer(); // T12 - T9 T15: timer.endTimer(); // T15 - t8 ########## File path: hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexV2.java ########## @@ -0,0 +1,321 @@ +/* + * 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.index.bloom; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.utils.LazyIterableIterator; +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.DefaultSizeEstimator; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ExternalSpillableMap; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.io.HoodieBloomRangeInfoHandle; +import org.apache.hudi.io.HoodieKeyLookupHandle; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Simplified re-implementation of {@link HoodieBloomIndex} that does not rely on caching, or + * incurs the overhead of auto-tuning parallelism. + */ +public class HoodieBloomIndexV2<T extends HoodieRecordPayload> extends HoodieIndex<T> { + + private static final Logger LOG = LogManager.getLogger(HoodieBloomIndexV2.class); + + public HoodieBloomIndexV2(HoodieWriteConfig config) { + super(config); + } + + @Override + public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, + JavaSparkContext jsc, + HoodieTable<T> hoodieTable) { + return recordRDD + .sortBy((record) -> String.format("%s-%s", record.getPartitionPath(), record.getRecordKey()), + true, config.getBloomIndexV2Parallelism()) + .mapPartitions((itr) -> new LazyRangeBloomChecker(itr, hoodieTable)).flatMap(List::iterator) + .sortBy(Pair::getRight, true, config.getBloomIndexV2Parallelism()) + .mapPartitions((itr) -> new LazyKeyChecker(itr, hoodieTable)) + .filter(Option::isPresent) + .map(Option::get); + } + + @Override + public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys, + JavaSparkContext jsc, HoodieTable<T> hoodieTable) { + throw new UnsupportedOperationException("Not yet implemented"); + } + + @Override + public boolean rollbackCommit(String commitTime) { + // Nope, don't need to do anything. + return true; + } + + /** + * This is not global, since we depend on the partitionPath to do the lookup. + */ + @Override + public boolean isGlobal() { + return false; + } + + /** + * No indexes into log files yet. + */ + @Override + public boolean canIndexLogFiles() { + return false; + } + + /** + * Bloom filters are stored, into the same data files. + */ + @Override + public boolean isImplicitWithStorage() { + return true; + } + + @Override + public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc, + HoodieTable<T> hoodieTable) { + return writeStatusRDD; + } + + /** + * Given an iterator of hoodie records, returns a pair of candidate HoodieRecord, FileID pairs. + */ + class LazyRangeBloomChecker extends + LazyIterableIterator<HoodieRecord<T>, List<Pair<HoodieRecord<T>, String>>> { + + private HoodieTable<T> table; + private String currentPartitionPath; + private Set<String> fileIDs; + private IndexFileFilter indexFileFilter; + private ExternalSpillableMap<String, BloomFilter> fileIDToBloomFilter; + private HoodieTimer hoodieTimer; + private long totalTimeMs; + private long totalCount; + private long totalMetadataReadTimeMs; + private long totalRangeCheckTimeMs; + private long totalBloomCheckTimeMs; + private long totalMatches; + + public LazyRangeBloomChecker(Iterator<HoodieRecord<T>> in, final HoodieTable<T> table) { + super(in); + this.table = table; + } + + @Override + protected List<Pair<HoodieRecord<T>, String>> computeNext() { + + List<Pair<HoodieRecord<T>, String>> candidates = new ArrayList<>(); + if (inputItr.hasNext()) { + + HoodieRecord<T> record = inputItr.next(); + try { + initIfNeeded(record.getPartitionPath()); + + totalMetadataReadTimeMs += hoodieTimer.deltaTime(); + } catch (IOException e) { + throw new HoodieIOException("Error reading index metadata for " + record.getPartitionPath(), e); + } + + // <Partition path, file name> + Set<Pair<String, String>> matchingFiles = indexFileFilter + .getMatchingFilesAndPartition(record.getPartitionPath(), record.getRecordKey()); + + totalRangeCheckTimeMs += hoodieTimer.deltaTime(); + + matchingFiles.forEach(partitionFileIdPair -> { + BloomFilter filter = fileIDToBloomFilter.get(partitionFileIdPair.getRight()); + if (filter.mightContain(record.getRecordKey())) { + totalMatches++; + candidates.add(Pair.of(record, partitionFileIdPair.getRight())); + } + }); + totalBloomCheckTimeMs += hoodieTimer.deltaTime(); + + if (candidates.isEmpty()) { + candidates.add(Pair.of(record, "")); + } + } + + totalCount++; + return candidates; + } + + @Override + protected void start() { + totalTimeMs = 0; + totalMatches = 0; + totalCount = 0; + hoodieTimer = new HoodieTimer().startTimer(); + } + + @Override + protected void end() { + totalTimeMs = hoodieTimer.endTimer(); + String rangeCheckInfo = "LazyRangeBloomChecker: " + + "totalCount: " + totalCount + ", " + + "totalMatches: " + totalMatches + "ms, " + + "totalTimeMs: " + totalTimeMs + "ms, " + + "totalMetadataReadTimeMs: " + totalMetadataReadTimeMs + "ms, " + + "totalRangeCheckTimeMs: " + totalRangeCheckTimeMs + "ms, " + + "totalBloomCheckTimeMs: " + totalBloomCheckTimeMs + "ms"; + LOG.info(rangeCheckInfo); + + cleanup(); + } + + private void cleanup() { + if (this.fileIDs != null) { + this.fileIDs.clear(); + } + if (this.fileIDToBloomFilter != null) { + this.fileIDToBloomFilter.clear(); + } + } + + private void initIfNeeded(String partitionPath) throws IOException { + if (!Objects.equals(currentPartitionPath, partitionPath)) { + this.currentPartitionPath = partitionPath; + cleanup(); + populateFileIDs(); + populateRangeAndBloomFilters(); + } + } + + private void populateFileIDs() { + Option<HoodieInstant> latestCommitTime = table.getMetaClient().getCommitsTimeline().filterCompletedInstants().lastInstant(); + this.fileIDs = latestCommitTime.map(commitTime -> + table.getBaseFileOnlyView() + .getLatestBaseFilesBeforeOrOn(currentPartitionPath, commitTime.getTimestamp()) + .map(HoodieBaseFile::getFileId) + .collect(Collectors.toSet()) + ).orElse(Collections.emptySet()); + } + + private void populateRangeAndBloomFilters() throws IOException { + this.fileIDToBloomFilter = new ExternalSpillableMap<>(config.getBloomIndexV2BufferSize(), + config.getSpillableMapBasePath(), new DefaultSizeEstimator<>(), new DefaultSizeEstimator<>()); + List<BloomIndexFileInfo> fileInfos = fileIDs.stream().map(fileID -> { + HoodieBloomRangeInfoHandle<T> indexMetadataHandle = new HoodieBloomRangeInfoHandle<T>( + config, table, Pair.of(currentPartitionPath, fileID)); + this.fileIDToBloomFilter.put(fileID, indexMetadataHandle.getBloomFilter()); + return indexMetadataHandle.getRangeInfo(); + }).collect(Collectors.toList()); + this.indexFileFilter = new IntervalTreeBasedIndexFileFilter(Collections.singletonMap(currentPartitionPath, fileInfos)); + } + + } + + /** + * Double check HoodieRecord by key. Review comment: can we have more descriptive docs. ########## File path: hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexV2.java ########## @@ -0,0 +1,321 @@ +/* + * 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.index.bloom; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.utils.LazyIterableIterator; +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.DefaultSizeEstimator; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ExternalSpillableMap; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.io.HoodieBloomRangeInfoHandle; +import org.apache.hudi.io.HoodieKeyLookupHandle; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Simplified re-implementation of {@link HoodieBloomIndex} that does not rely on caching, or + * incurs the overhead of auto-tuning parallelism. + */ +public class HoodieBloomIndexV2<T extends HoodieRecordPayload> extends HoodieIndex<T> { + + private static final Logger LOG = LogManager.getLogger(HoodieBloomIndexV2.class); + + public HoodieBloomIndexV2(HoodieWriteConfig config) { + super(config); + } + + @Override + public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, + JavaSparkContext jsc, + HoodieTable<T> hoodieTable) { + return recordRDD + .sortBy((record) -> String.format("%s-%s", record.getPartitionPath(), record.getRecordKey()), + true, config.getBloomIndexV2Parallelism()) + .mapPartitions((itr) -> new LazyRangeBloomChecker(itr, hoodieTable)).flatMap(List::iterator) + .sortBy(Pair::getRight, true, config.getBloomIndexV2Parallelism()) + .mapPartitions((itr) -> new LazyKeyChecker(itr, hoodieTable)) + .filter(Option::isPresent) + .map(Option::get); + } + + @Override + public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys, + JavaSparkContext jsc, HoodieTable<T> hoodieTable) { + throw new UnsupportedOperationException("Not yet implemented"); + } + + @Override + public boolean rollbackCommit(String commitTime) { + // Nope, don't need to do anything. + return true; + } + + /** + * This is not global, since we depend on the partitionPath to do the lookup. + */ + @Override + public boolean isGlobal() { + return false; + } + + /** + * No indexes into log files yet. + */ + @Override + public boolean canIndexLogFiles() { + return false; + } + + /** + * Bloom filters are stored, into the same data files. + */ + @Override + public boolean isImplicitWithStorage() { + return true; + } + + @Override + public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc, + HoodieTable<T> hoodieTable) { + return writeStatusRDD; + } + + /** + * Given an iterator of hoodie records, returns a pair of candidate HoodieRecord, FileID pairs. + */ + class LazyRangeBloomChecker extends + LazyIterableIterator<HoodieRecord<T>, List<Pair<HoodieRecord<T>, String>>> { + + private HoodieTable<T> table; + private String currentPartitionPath; + private Set<String> fileIDs; + private IndexFileFilter indexFileFilter; + private ExternalSpillableMap<String, BloomFilter> fileIDToBloomFilter; + private HoodieTimer hoodieTimer; + private long totalTimeMs; + private long totalCount; + private long totalMetadataReadTimeMs; + private long totalRangeCheckTimeMs; + private long totalBloomCheckTimeMs; + private long totalMatches; + + public LazyRangeBloomChecker(Iterator<HoodieRecord<T>> in, final HoodieTable<T> table) { + super(in); + this.table = table; + } + + @Override + protected List<Pair<HoodieRecord<T>, String>> computeNext() { + + List<Pair<HoodieRecord<T>, String>> candidates = new ArrayList<>(); + if (inputItr.hasNext()) { + + HoodieRecord<T> record = inputItr.next(); + try { + initIfNeeded(record.getPartitionPath()); + + totalMetadataReadTimeMs += hoodieTimer.deltaTime(); + } catch (IOException e) { + throw new HoodieIOException("Error reading index metadata for " + record.getPartitionPath(), e); + } + + // <Partition path, file name> + Set<Pair<String, String>> matchingFiles = indexFileFilter + .getMatchingFilesAndPartition(record.getPartitionPath(), record.getRecordKey()); + + totalRangeCheckTimeMs += hoodieTimer.deltaTime(); + + matchingFiles.forEach(partitionFileIdPair -> { + BloomFilter filter = fileIDToBloomFilter.get(partitionFileIdPair.getRight()); + if (filter.mightContain(record.getRecordKey())) { + totalMatches++; + candidates.add(Pair.of(record, partitionFileIdPair.getRight())); + } + }); + totalBloomCheckTimeMs += hoodieTimer.deltaTime(); + + if (candidates.isEmpty()) { + candidates.add(Pair.of(record, "")); + } + } + + totalCount++; + return candidates; + } + + @Override + protected void start() { + totalTimeMs = 0; + totalMatches = 0; + totalCount = 0; + hoodieTimer = new HoodieTimer().startTimer(); + } + + @Override + protected void end() { + totalTimeMs = hoodieTimer.endTimer(); + String rangeCheckInfo = "LazyRangeBloomChecker: " Review comment: yep, please remove an debugging statements and any other metrics that was added for perf tuning. ########## File path: hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndexV2.java ########## @@ -0,0 +1,235 @@ +/* + * 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.index.bloom; + +import org.apache.avro.Schema; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.HoodieClientTestHarness; +import org.apache.hudi.common.HoodieClientTestUtils; +import org.apache.hudi.common.TestRawTripPayload; +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.bloom.BloomFilterFactory; +import org.apache.hudi.common.bloom.BloomFilterTypeCode; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.FileIOUtils; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.spark.api.java.JavaRDD; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.UUID; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +@RunWith(Parameterized.class) +public class TestHoodieBloomIndexV2 extends HoodieClientTestHarness { Review comment: once you have fixed fetchRecordLocation, do write tests for the same. ########## File path: hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexV2.java ########## @@ -0,0 +1,321 @@ +/* + * 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.index.bloom; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.utils.LazyIterableIterator; +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.DefaultSizeEstimator; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ExternalSpillableMap; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.io.HoodieBloomRangeInfoHandle; +import org.apache.hudi.io.HoodieKeyLookupHandle; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Simplified re-implementation of {@link HoodieBloomIndex} that does not rely on caching, or + * incurs the overhead of auto-tuning parallelism. + */ +public class HoodieBloomIndexV2<T extends HoodieRecordPayload> extends HoodieIndex<T> { + + private static final Logger LOG = LogManager.getLogger(HoodieBloomIndexV2.class); + + public HoodieBloomIndexV2(HoodieWriteConfig config) { + super(config); + } + + @Override + public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, + JavaSparkContext jsc, + HoodieTable<T> hoodieTable) { + return recordRDD + .sortBy((record) -> String.format("%s-%s", record.getPartitionPath(), record.getRecordKey()), + true, config.getBloomIndexV2Parallelism()) + .mapPartitions((itr) -> new LazyRangeBloomChecker(itr, hoodieTable)).flatMap(List::iterator) + .sortBy(Pair::getRight, true, config.getBloomIndexV2Parallelism()) + .mapPartitions((itr) -> new LazyKeyChecker(itr, hoodieTable)) + .filter(Option::isPresent) + .map(Option::get); + } + + @Override + public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys, + JavaSparkContext jsc, HoodieTable<T> hoodieTable) { + throw new UnsupportedOperationException("Not yet implemented"); + } + + @Override + public boolean rollbackCommit(String commitTime) { + // Nope, don't need to do anything. + return true; + } + + /** + * This is not global, since we depend on the partitionPath to do the lookup. + */ + @Override + public boolean isGlobal() { + return false; + } + + /** + * No indexes into log files yet. + */ + @Override + public boolean canIndexLogFiles() { + return false; + } + + /** + * Bloom filters are stored, into the same data files. + */ + @Override + public boolean isImplicitWithStorage() { + return true; + } + + @Override + public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc, + HoodieTable<T> hoodieTable) { + return writeStatusRDD; + } + + /** + * Given an iterator of hoodie records, returns a pair of candidate HoodieRecord, FileID pairs. + */ + class LazyRangeBloomChecker extends Review comment: nit. take your call. Can we name this LazyRangeAndBloomFilter/ LazyRangeAndBloomChecker. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org