nsivabalan commented on a change in pull request #1469: URL: https://github.com/apache/incubator-hudi/pull/1469#discussion_r426250288
########## File path: hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndexV2.java ########## @@ -0,0 +1,223 @@ +/* + * 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.utils.LazyIterableIterator; +import org.apache.hudi.common.bloom.BloomFilter; +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.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableMetaClient; +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.io.HoodieBloomRangeInfoHandle; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import scala.Tuple2; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +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 HoodieGlobalBloomIndexV2<T extends HoodieRecordPayload> extends HoodieBloomIndexV2<T> { + + private static final Logger LOG = LogManager.getLogger(HoodieGlobalBloomIndexV2.class); + + public HoodieGlobalBloomIndexV2(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 LazyRangeAndBloomChecker(itr, hoodieTable)).flatMap(List::iterator) + .sortBy(Pair::getRight, true, config.getBloomIndexV2Parallelism()) + .mapPartitions((itr) -> new LazyKeyChecker(itr, hoodieTable)) + .filter(Option::isPresent) + .map(Option::get); + } + + /** + * This is not global, since we depend on the partitionPath to do the lookup. + */ + @Override + public boolean isGlobal() { + return true; + } + + /** + * 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. + */ + class LazyRangeAndBloomChecker extends + LazyIterableIterator<HoodieRecord<T>, List<Pair<HoodieRecord<T>, String>>> { + + private HoodieTable<T> table; + private List<Pair<String, String>> partitionPathFileIDList; + 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 LazyRangeAndBloomChecker(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()) { + return candidates; + } + + HoodieRecord<T> record = inputItr.next(); + + // <Partition path, file name> + hoodieTimer.startTimer(); + Set<Pair<String, String>> matchingFiles = indexFileFilter Review comment: bottom line, we need to consider all partitions, not just the ones where incoming records are tagged with. ---------------------------------------------------------------- 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: [email protected]
