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 2022/02/02 10:01:30 UTC

[GitHub] [hudi] codope commented on a change in pull request #4352: [HUDI-1295] Metadata Index - Bloom filter and Column stats index to speed up index lookups

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
##########
@@ -145,22 +152,86 @@
       ValidationUtils.checkArgument(!this.metadataWriteConfig.isMetadataTableEnabled(),
           "File listing cannot be used for Metadata Table");
 
-      initRegistry();
       this.dataMetaClient =
           HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(dataWriteConfig.getBasePath()).build();
+      enablePartitions();
+      initRegistry();
       initialize(engineContext, actionMetadata, inflightInstantTimestamp);
       initTableMetadata();
     } else {
       enabled = false;
-      this.metrics = Option.empty();
     }
   }
 
   public HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig,
-      HoodieEngineContext engineContext) {
+                                         HoodieEngineContext engineContext) {
     this(hadoopConf, writeConfig, engineContext, Option.empty(), Option.empty());
   }
 
+  /**
+   * Enable metadata table partitions based on config.
+   */
+  private void enablePartitions() {
+    final HoodieMetadataConfig metadataConfig = dataWriteConfig.getMetadataConfig();
+    boolean isBootstrapCompleted = false;

Review comment:
       nit: remove redundant initializer

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
##########
@@ -145,22 +152,86 @@
       ValidationUtils.checkArgument(!this.metadataWriteConfig.isMetadataTableEnabled(),
           "File listing cannot be used for Metadata Table");
 
-      initRegistry();
       this.dataMetaClient =
           HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(dataWriteConfig.getBasePath()).build();
+      enablePartitions();
+      initRegistry();
       initialize(engineContext, actionMetadata, inflightInstantTimestamp);
       initTableMetadata();
     } else {
       enabled = false;
-      this.metrics = Option.empty();
     }
   }
 
   public HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig,
-      HoodieEngineContext engineContext) {
+                                         HoodieEngineContext engineContext) {
     this(hadoopConf, writeConfig, engineContext, Option.empty(), Option.empty());
   }
 
+  /**
+   * Enable metadata table partitions based on config.
+   */
+  private void enablePartitions() {
+    final HoodieMetadataConfig metadataConfig = dataWriteConfig.getMetadataConfig();
+    boolean isBootstrapCompleted = false;
+    Option<HoodieTableMetaClient> metaClient = Option.empty();
+    try {
+      isBootstrapCompleted = dataMetaClient.getFs().exists(new Path(metadataWriteConfig.getBasePath(), HoodieTableMetaClient.METAFOLDER_NAME));
+      if (isBootstrapCompleted) {
+        metaClient = Option.of(HoodieTableMetaClient.builder().setConf(hadoopConf.get())
+            .setBasePath(metadataWriteConfig.getBasePath()).build());
+      }
+    } catch (IOException e) {
+      throw new HoodieException("Failed to enable metadata partitions!", e);
+    }
+
+    Option<HoodieTableFileSystemView> fsView = Option.ofNullable(
+        metaClient.isPresent() ? HoodieTableMetadataUtil.getFileSystemView(metaClient.get()) : null);
+    enablePartition(MetadataPartitionType.FILES, metadataConfig, metaClient, fsView, isBootstrapCompleted);
+    if (metadataConfig.isBloomFilterIndexEnabled()) {
+      enablePartition(MetadataPartitionType.BLOOM_FILTERS, metadataConfig, metaClient, fsView, isBootstrapCompleted);
+    }
+    if (metadataConfig.isColumnStatsIndexEnabled()) {
+      enablePartition(MetadataPartitionType.COLUMN_STATS, metadataConfig, metaClient, fsView, isBootstrapCompleted);
+    }
+  }
+
+  /**
+   * Enable metadata table partition.
+   *
+   * @param partitionType        - Metadata table partition type
+   * @param metadataConfig       - Table config
+   * @param metaClient           - Meta client for the metadata table
+   * @param fsView               - Metadata table filesystem view to use
+   * @param isBootstrapCompleted - Is metadata table bootstrap completed
+   */
+  private void enablePartition(final MetadataPartitionType partitionType, final HoodieMetadataConfig metadataConfig,
+                               final Option<HoodieTableMetaClient> metaClient, Option<HoodieTableFileSystemView> fsView, boolean isBootstrapCompleted) {
+    final int fileGroupCount = getPartitionFileGroupCount(partitionType, metaClient, fsView, metadataConfig, isBootstrapCompleted);
+    partitionType.setFileGroupCount(fileGroupCount);
+    this.enabledPartitionTypes.add(partitionType);
+  }
+
+  private static int getPartitionFileGroupCount(final MetadataPartitionType partitionType,

Review comment:
       if possible, let's keep all static methods in a util e.g. we can move this static method to `HoodieTableMetadataUtil`.

##########
File path: hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java
##########
@@ -22,27 +22,60 @@
 import java.util.List;
 
 public enum MetadataPartitionType {
-  FILES("files", "files-");
+  FILES(HoodieTableMetadataUtil.PARTITION_NAME_FILES, "files-"),
+  COLUMN_STATS(HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS, "col-stats-"),
+  BLOOM_FILTERS(HoodieTableMetadataUtil.PARTITION_NAME_BLOOM_FILTERS, "bloom-filters-");
 
-  // refers to partition path in metadata table.
+  // Partition path in metadata table.
   private final String partitionPath;
-  // refers to fileId prefix used for all file groups in this partition.
+  // FileId prefix used for all file groups in this partition.
   private final String fileIdPrefix;
+  // Total file groups
+  private int fileGroupCount = 1;
 
-  MetadataPartitionType(String partitionPath, String fileIdPrefix) {
+  MetadataPartitionType(final String partitionPath, final String fileIdPrefix) {
     this.partitionPath = partitionPath;
     this.fileIdPrefix = fileIdPrefix;
   }
 
-  public String partitionPath() {
+  public String getPartitionPath() {
     return partitionPath;
   }
 
   public String getFileIdPrefix() {
     return fileIdPrefix;
   }
 
-  public static List<String> all() {
-    return Arrays.asList(MetadataPartitionType.FILES.partitionPath());
+  void setFileGroupCount(final int fileGroupCount) {
+    this.fileGroupCount = fileGroupCount;
+  }
+
+  public int getFileGroupCount() {
+    return this.fileGroupCount;
+  }
+
+  public static List<String> allPaths() {
+    return Arrays.asList(
+        FILES.getPartitionPath(),
+        COLUMN_STATS.getPartitionPath(),
+        BLOOM_FILTERS.getPartitionPath()
+    );
+  }
+
+  public static List<MetadataPartitionType> allTypes() {

Review comment:
       remove this method as it is not being used anywhere?

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieMetadataBloomIndexCheckFunction.java
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.client.utils.LazyIterableIterator;
+import org.apache.hudi.common.bloom.BloomFilterTypeCode;
+import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+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.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.HoodieIndexUtils;
+import org.apache.hudi.io.HoodieKeyLookupResult;
+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.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Spark Function2 implementation for checking bloom filters for the
+ * requested keys from the metadata table index. The bloom filter
+ * checking for keys and the actual file verification for the
+ * candidate keys is done in an iterative fashion. In each iteration,
+ * bloom filters are requested for a batch of partition files and the
+ * keys are checked against them.
+ */
+public class HoodieMetadataBloomIndexCheckFunction implements
+    Function2<Integer, Iterator<Tuple2<String, HoodieKey>>, Iterator<List<HoodieKeyLookupResult>>> {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieMetadataBloomIndexCheckFunction.class);
+
+  // Assuming each file bloom filter takes up 512K, sizing the max file count
+  // per batch so that the total fetched bloom filters would not cross 128 MB.
+  private static final long BLOOM_FILTER_CHECK_MAX_FILE_COUNT_PER_BATCH = 256;
+  private final HoodieTable hoodieTable;
+  private final HoodieWriteConfig config;

Review comment:
       `config` is not being used anywhere




-- 
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-unsubscribe@hudi.apache.org

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