You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2020/04/11 19:55:33 UTC

[GitHub] [incubator-hudi] lamber-ken commented on a change in pull request #1469: [HUDI-686] Implement BloomIndexV2 that does not depend on memory caching

lamber-ken commented on a change in pull request #1469: [HUDI-686] Implement BloomIndexV2 that does not depend on memory caching
URL: https://github.com/apache/incubator-hudi/pull/1469#discussion_r407104334
 
 

 ##########
 File path: hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexV2.java
 ##########
 @@ -0,0 +1,312 @@
+/*
+ * 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.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.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);
+  }
+
+  /**
+   * 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 long totalTimeNs;
+    private long totalCount;
+    private long totalMetadataReadTimeNs;
+    private long totalRangeCheckTimeNs;
+    private long totalBloomCheckTimeNs;
+    private long totalMatches;
+
+    public LazyRangeBloomChecker(Iterator<HoodieRecord<T>> in, final HoodieTable<T> table) {
+      super(in);
+      this.table = table;
+    }
+
+    private void cleanup() {
+      if (this.fileIDToBloomFilter != null) {
+        this.fileIDToBloomFilter.clear();
+      }
+    }
+
+    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.EMPTY_SET);
+    }
+
+    private void populateRangeAndBloomFilters() throws IOException {
+      this.fileIDToBloomFilter = new ExternalSpillableMap<>(1000000000L,
+          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));
+    }
+
+    private void initIfNeeded(String partitionPath) throws IOException {
+      if (!Objects.equals(partitionPath, currentPartitionPath)) {
+        cleanup();
+        this.currentPartitionPath = partitionPath;
+        populateFileIDs();
+        populateRangeAndBloomFilters();
+      }
+    }
+
+    @Override
+    protected void start() {
+      totalCount = 0;
+      totalTimeNs = 0;
+    }
+
+    @Override
+    protected List<Pair<HoodieRecord<T>, String>> computeNext() {
+
+      final long startNs = System.nanoTime();
+      List<Pair<HoodieRecord<T>, String>> candidates = new ArrayList<>();
+      if (inputItr.hasNext()) {
+        long timerStart = System.nanoTime();
+        HoodieRecord<T> record = inputItr.next();
+        try {
+          initIfNeeded(record.getPartitionPath());
+        } catch (IOException e) {
+          throw new HoodieIOException(
+              "Error reading index metadata for " + record.getPartitionPath(), e);
+        }
+
+        totalMetadataReadTimeNs += System.nanoTime() - timerStart;
+        timerStart = System.nanoTime();
+
+        Set<Pair<String, String>> matchingFiles = indexFileFilter
+            .getMatchingFilesAndPartition(record.getPartitionPath(), record.getRecordKey());
+        totalRangeCheckTimeNs += System.nanoTime() - timerStart;
+        timerStart = System.nanoTime();
+
+        matchingFiles.forEach(partitionFileIdPair -> {
+          BloomFilter filter = fileIDToBloomFilter.get(partitionFileIdPair.getRight());
+          if (filter.mightContain(record.getRecordKey())) {
+            candidates.add(Pair.of(record, partitionFileIdPair.getRight()));
+          }
+        });
+        totalBloomCheckTimeNs += System.nanoTime() - timerStart;
+
+        if (candidates.size() == 0) {
+          candidates.add(Pair.of(record, ""));
 
 Review comment:
   If it is a new record, there is no file group associated with it. We can't ignore it.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services