manojpec commented on a change in pull request #4352:
URL: https://github.com/apache/hudi/pull/4352#discussion_r780117989



##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java
##########
@@ -111,13 +124,14 @@ public HoodieBloomIndex(HoodieWriteConfig config, 
BaseHoodieBloomIndexHelper blo
   private HoodiePairData<HoodieKey, HoodieRecordLocation> lookupIndex(
       HoodiePairData<String, String> partitionRecordKeyPairs, final 
HoodieEngineContext context,
       final HoodieTable hoodieTable) {
-    // Obtain records per partition, in the incoming records
+    // Step 1: Obtain records per partition, in the incoming records
     Map<String, Long> recordsPerPartition = 
partitionRecordKeyPairs.countByKey();
     List<String> affectedPartitionPathList = new 
ArrayList<>(recordsPerPartition.keySet());
 
     // Step 2: Load all involved files as <Partition, filename> pairs
-    List<Pair<String, BloomIndexFileInfo>> fileInfoList =
-        loadInvolvedFiles(affectedPartitionPathList, context, hoodieTable);
+    List<Pair<String, BloomIndexFileInfo>> fileInfoList = 
(config.getMetadataConfig().isMetaIndexColumnStatsEnabled()

Review comment:
       Please take a look at the latest revision. Now, the bloom index and col 
stat index keys includes the full file name instead of just the file id part. 
Full file name are needed so that base file revisions are captured in the index 
properly.

##########
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomMetaIndexBatchCheckFunction.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.hadoop.fs.Path;
+import org.apache.hudi.common.bloom.BloomFilterTypeCode;
+import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.common.util.hash.FileIndexID;
+import org.apache.hudi.common.util.hash.PartitionIndexID;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import 
org.apache.hudi.io.HoodieKeyMetaIndexBatchLookupHandle.MetaBloomIndexGroupedKeyLookupResult;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileReaderFactory;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.function.Function2;
+import scala.Tuple2;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Function performing actual checking of RDD partition containing (fileId, 
hoodieKeys) against the actual files.
+ */
+public class HoodieBloomMetaIndexBatchCheckFunction implements
+    Function2<Integer, Iterator<Tuple2<String, HoodieKey>>, 
Iterator<List<MetaBloomIndexGroupedKeyLookupResult>>> {
+
+  private static final Logger LOG = 
LogManager.getLogger(HoodieBloomMetaIndexBatchCheckFunction.class);
+  private final HoodieTable hoodieTable;
+  private final HoodieWriteConfig config;
+
+  public HoodieBloomMetaIndexBatchCheckFunction(HoodieTable hoodieTable, 
HoodieWriteConfig config) {
+    this.hoodieTable = hoodieTable;
+    this.config = config;
+  }
+
+  @Override
+  public Iterator<List<MetaBloomIndexGroupedKeyLookupResult>> call(Integer 
integer, Iterator<Tuple2<String, HoodieKey>> tuple2Iterator) throws Exception {
+    List<List<MetaBloomIndexGroupedKeyLookupResult>> resultList = new 
ArrayList<>();
+    Map<Pair<String, String>, List<HoodieKey>> fileToKeysMap = new HashMap<>();
+
+    while (tuple2Iterator.hasNext()) {
+      Tuple2<String, HoodieKey> entry = tuple2Iterator.next();
+      fileToKeysMap.computeIfAbsent(Pair.of(entry._2.getPartitionPath(), 
entry._1), k -> new ArrayList<>()).add(entry._2);
+    }
+
+    List<Pair<PartitionIndexID, FileIndexID>> partitionIDFileIDList =
+        fileToKeysMap.keySet().stream().map(partitionFileIdPair -> {
+          return Pair.of(new PartitionIndexID(partitionFileIdPair.getLeft()),
+              new FileIndexID(partitionFileIdPair.getRight()));
+        }).collect(Collectors.toList());
+
+    Map<String, ByteBuffer> fileIDToBloomFilterByteBufferMap =

Review comment:
       yes, getBloomFilters() internally has a treeSet to order the keys to be 
lookedup. Since I don't want to force on the callers the order of the keys, i 
kept the sorting at the next level. Btw, HoodieBloomMetaIndexBatchCheckFunction 
already operates at the minimal files as it is already sorted by file id by the 
caller. 

##########
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomMetaIndexBatchCheckFunction.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.hadoop.fs.Path;
+import org.apache.hudi.common.bloom.BloomFilterTypeCode;
+import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.common.util.hash.FileIndexID;
+import org.apache.hudi.common.util.hash.PartitionIndexID;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import 
org.apache.hudi.io.HoodieKeyMetaIndexBatchLookupHandle.MetaBloomIndexGroupedKeyLookupResult;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileReaderFactory;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.function.Function2;
+import scala.Tuple2;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Function performing actual checking of RDD partition containing (fileId, 
hoodieKeys) against the actual files.
+ */
+public class HoodieBloomMetaIndexBatchCheckFunction implements
+    Function2<Integer, Iterator<Tuple2<String, HoodieKey>>, 
Iterator<List<MetaBloomIndexGroupedKeyLookupResult>>> {
+
+  private static final Logger LOG = 
LogManager.getLogger(HoodieBloomMetaIndexBatchCheckFunction.class);
+  private final HoodieTable hoodieTable;
+  private final HoodieWriteConfig config;
+
+  public HoodieBloomMetaIndexBatchCheckFunction(HoodieTable hoodieTable, 
HoodieWriteConfig config) {
+    this.hoodieTable = hoodieTable;
+    this.config = config;
+  }
+
+  @Override
+  public Iterator<List<MetaBloomIndexGroupedKeyLookupResult>> call(Integer 
integer, Iterator<Tuple2<String, HoodieKey>> tuple2Iterator) throws Exception {

Review comment:
       This is fixed in the latest revision. The input tuple has FileID, and 
there is a translation to file name happening in this class. 

##########
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomMetaIndexBatchCheckFunction.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.hadoop.fs.Path;
+import org.apache.hudi.common.bloom.BloomFilterTypeCode;
+import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.common.util.hash.FileIndexID;
+import org.apache.hudi.common.util.hash.PartitionIndexID;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import 
org.apache.hudi.io.HoodieKeyMetaIndexBatchLookupHandle.MetaBloomIndexGroupedKeyLookupResult;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileReaderFactory;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.function.Function2;
+import scala.Tuple2;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Function performing actual checking of RDD partition containing (fileId, 
hoodieKeys) against the actual files.
+ */
+public class HoodieBloomMetaIndexBatchCheckFunction implements
+    Function2<Integer, Iterator<Tuple2<String, HoodieKey>>, 
Iterator<List<MetaBloomIndexGroupedKeyLookupResult>>> {
+
+  private static final Logger LOG = 
LogManager.getLogger(HoodieBloomMetaIndexBatchCheckFunction.class);
+  private final HoodieTable hoodieTable;
+  private final HoodieWriteConfig config;
+
+  public HoodieBloomMetaIndexBatchCheckFunction(HoodieTable hoodieTable, 
HoodieWriteConfig config) {
+    this.hoodieTable = hoodieTable;
+    this.config = config;
+  }
+
+  @Override
+  public Iterator<List<MetaBloomIndexGroupedKeyLookupResult>> call(Integer 
integer, Iterator<Tuple2<String, HoodieKey>> tuple2Iterator) throws Exception {
+    List<List<MetaBloomIndexGroupedKeyLookupResult>> resultList = new 
ArrayList<>();
+    Map<Pair<String, String>, List<HoodieKey>> fileToKeysMap = new HashMap<>();
+
+    while (tuple2Iterator.hasNext()) {
+      Tuple2<String, HoodieKey> entry = tuple2Iterator.next();
+      fileToKeysMap.computeIfAbsent(Pair.of(entry._2.getPartitionPath(), 
entry._1), k -> new ArrayList<>()).add(entry._2);
+    }
+
+    List<Pair<PartitionIndexID, FileIndexID>> partitionIDFileIDList =
+        fileToKeysMap.keySet().stream().map(partitionFileIdPair -> {
+          return Pair.of(new PartitionIndexID(partitionFileIdPair.getLeft()),
+              new FileIndexID(partitionFileIdPair.getRight()));
+        }).collect(Collectors.toList());
+
+    Map<String, ByteBuffer> fileIDToBloomFilterByteBufferMap =
+        hoodieTable.getMetadataTable().getBloomFilters(partitionIDFileIDList);
+
+    fileToKeysMap.forEach((partitionPathFileIdPair, hoodieKeyList) -> {
+      final String partitionPath = partitionPathFileIdPair.getLeft();
+      final String fileId = partitionPathFileIdPair.getRight();
+      ValidationUtils.checkState(!fileId.isEmpty());
+
+      final String partitionIDHash = new 
PartitionIndexID(partitionPath).asBase64EncodedString();
+      final String fileIDHash = new 
FileIndexID(fileId).asBase64EncodedString();
+      final String bloomKey = partitionIDHash.concat(fileIDHash);
+      if (!fileIDToBloomFilterByteBufferMap.containsKey(bloomKey)) {
+        throw new HoodieIndexException("Failed to get the bloom filter for " + 
partitionPathFileIdPair);
+      }
+      final ByteBuffer fileBloomFilterByteBuffer = 
fileIDToBloomFilterByteBufferMap.get(bloomKey);
+
+      HoodieDynamicBoundedBloomFilter fileBloomFilter =

Review comment:
       Right, this is a TODO item. 

##########
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomMetaIndexBatchCheckFunction.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.hadoop.fs.Path;
+import org.apache.hudi.common.bloom.BloomFilterTypeCode;
+import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.common.util.hash.FileIndexID;
+import org.apache.hudi.common.util.hash.PartitionIndexID;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import 
org.apache.hudi.io.HoodieKeyMetaIndexBatchLookupHandle.MetaBloomIndexGroupedKeyLookupResult;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileReaderFactory;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.function.Function2;
+import scala.Tuple2;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Function performing actual checking of RDD partition containing (fileId, 
hoodieKeys) against the actual files.
+ */
+public class HoodieBloomMetaIndexBatchCheckFunction implements
+    Function2<Integer, Iterator<Tuple2<String, HoodieKey>>, 
Iterator<List<MetaBloomIndexGroupedKeyLookupResult>>> {
+
+  private static final Logger LOG = 
LogManager.getLogger(HoodieBloomMetaIndexBatchCheckFunction.class);
+  private final HoodieTable hoodieTable;
+  private final HoodieWriteConfig config;
+
+  public HoodieBloomMetaIndexBatchCheckFunction(HoodieTable hoodieTable, 
HoodieWriteConfig config) {
+    this.hoodieTable = hoodieTable;
+    this.config = config;
+  }
+
+  @Override
+  public Iterator<List<MetaBloomIndexGroupedKeyLookupResult>> call(Integer 
integer, Iterator<Tuple2<String, HoodieKey>> tuple2Iterator) throws Exception {
+    List<List<MetaBloomIndexGroupedKeyLookupResult>> resultList = new 
ArrayList<>();
+    Map<Pair<String, String>, List<HoodieKey>> fileToKeysMap = new HashMap<>();
+
+    while (tuple2Iterator.hasNext()) {
+      Tuple2<String, HoodieKey> entry = tuple2Iterator.next();
+      fileToKeysMap.computeIfAbsent(Pair.of(entry._2.getPartitionPath(), 
entry._1), k -> new ArrayList<>()).add(entry._2);
+    }
+
+    List<Pair<PartitionIndexID, FileIndexID>> partitionIDFileIDList =
+        fileToKeysMap.keySet().stream().map(partitionFileIdPair -> {
+          return Pair.of(new PartitionIndexID(partitionFileIdPair.getLeft()),
+              new FileIndexID(partitionFileIdPair.getRight()));
+        }).collect(Collectors.toList());
+
+    Map<String, ByteBuffer> fileIDToBloomFilterByteBufferMap =
+        hoodieTable.getMetadataTable().getBloomFilters(partitionIDFileIDList);
+
+    fileToKeysMap.forEach((partitionPathFileIdPair, hoodieKeyList) -> {
+      final String partitionPath = partitionPathFileIdPair.getLeft();
+      final String fileId = partitionPathFileIdPair.getRight();
+      ValidationUtils.checkState(!fileId.isEmpty());
+
+      final String partitionIDHash = new 
PartitionIndexID(partitionPath).asBase64EncodedString();
+      final String fileIDHash = new 
FileIndexID(fileId).asBase64EncodedString();

Review comment:
       This is fixed.

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
##########
@@ -735,14 +768,19 @@ protected void bootstrapCommit(List<DirectoryInfo> 
partitionInfoList, String cre
     List<String> partitions = partitionInfoList.stream().map(p ->
         p.getRelativePath().isEmpty() ? NON_PARTITIONED_NAME : 
p.getRelativePath()).collect(Collectors.toList());
     final int totalFiles = partitionInfoList.stream().mapToInt(p -> 
p.getTotalFiles()).sum();
+    final Map<MetadataPartitionType, HoodieData<HoodieRecord>> 
partitionToRecordsMap = new HashMap<>();
 
     // Record which saves the list of all partitions
     HoodieRecord allPartitionRecord = 
HoodieMetadataPayload.createPartitionListRecord(partitions);
     if (partitions.isEmpty()) {
-      // in case of boostrapping of a fresh table, there won't be any 
partitions, but we need to make a boostrap commit
-      
commit(engineContext.parallelize(Collections.singletonList(allPartitionRecord), 
1), MetadataPartitionType.FILES.partitionPath(), createInstantTime, false);
+      // in case of bootstrapping of a fresh table, there won't be any 
partitions, but we need to make a boostrap commit
+      final HoodieData<HoodieRecord> allPartitionRecordsRDD = 
engineContext.parallelize(
+          Collections.singletonList(allPartitionRecord), 1);
+      partitionToRecordsMap.put(MetadataPartitionType.FILES, 
allPartitionRecordsRDD);
+      commit(createInstantTime, partitionToRecordsMap, false);

Review comment:
       Bootstrapping is not part of this PR. There is a separate ticket for it.

##########
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomMetaIndexColStatFunction.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.avro.model.HoodieColumnStats;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import scala.Tuple2;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+/**
+ * Function performing actual checking of RDD partition containing (fileId, 
hoodieKeys) against the actual files.
+ */
+public class HoodieBloomMetaIndexColStatFunction
+    implements FlatMapFunction<Iterator<Tuple2<Tuple2<String, String>, 
HoodieKey>>, Tuple2<Tuple2<String, String>,
+    HoodieKey>> {
+
+  private static final Logger LOG = 
LogManager.getLogger(HoodieBloomMetaIndexColStatFunction.class);
+  private final HoodieTable hoodieTable;
+  private final HoodieWriteConfig config;
+
+  public HoodieBloomMetaIndexColStatFunction(HoodieTable hoodieTable, 
HoodieWriteConfig config) {

Review comment:
       This is removed. 




-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to