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/08 23:08:53 UTC

[GitHub] [hudi] nsivabalan commented on a change in pull request #4669: [HUDI-3239] Convert `BaseHoodieTableFileIndex` to Java

nsivabalan commented on a change in pull request #4669:
URL: https://github.com/apache/hudi/pull/4669#discussion_r802123222



##########
File path: hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java
##########
@@ -0,0 +1,351 @@
+/*
+ * 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;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.BaseFile;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodieTableQueryType;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
+import org.apache.hudi.common.util.CollectionUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Common (engine-agnostic) File Index implementation enabling individual query engines to
+ * list Hudi Table contents based on the
+ *
+ * <ul>
+ *   <li>Table type (MOR, COW)</li>
+ *   <li>Query type (snapshot, read_optimized, incremental)</li>
+ *   <li>Query instant/range</li>
+ * </ul>
+ */
+public abstract class BaseHoodieTableFileIndex {
+
+  private static final Logger LOG = LogManager.getLogger(BaseHoodieTableFileIndex.class);
+
+  private final String[] partitionColumns;
+
+  private final FileSystemViewStorageConfig fileSystemStorageConfig;
+  private final HoodieMetadataConfig metadataConfig;
+
+  private final HoodieTableQueryType queryType;
+  private final Option<String> specifiedQueryInstant;
+  protected final List<Path> queryPaths;
+
+  private final boolean shouldIncludePendingCommits;
+
+  private final HoodieTableType tableType;
+  protected final String basePath;
+
+  private final HoodieTableMetaClient metaClient;
+  private final HoodieEngineContext engineContext;
+
+  private final transient FileStatusCache fileStatusCache;
+
+  protected transient volatile long cachedFileSize = 0L;
+  protected transient volatile Map<PartitionPath, List<FileSlice>> cachedAllInputFileSlices;
+
+  protected volatile boolean queryAsNonePartitionedTable = false;
+
+  private transient volatile HoodieTableFileSystemView fileSystemView = null;
+
+  /**
+   * @param engineContext Hudi engine-specific context
+   * @param metaClient Hudi table's meta-client
+   * @param configProperties unifying configuration (in the form of generic properties)
+   * @param queryType target query type
+   * @param queryPaths target DFS paths being queried
+   * @param specifiedQueryInstant instant as of which table is being queried
+   * @param shouldIncludePendingCommits flags whether file-index should exclude any pending operations
+   * @param fileStatusCache transient cache of fetched [[FileStatus]]es
+   */
+  public BaseHoodieTableFileIndex(HoodieEngineContext engineContext,
+                                  HoodieTableMetaClient metaClient,
+                                  TypedProperties configProperties,
+                                  HoodieTableQueryType queryType,
+                                  List<Path> queryPaths,
+                                  Option<String> specifiedQueryInstant,
+                                  boolean shouldIncludePendingCommits,
+                                  FileStatusCache fileStatusCache) {
+    this.partitionColumns = metaClient.getTableConfig().getPartitionFields()
+        .orElse(new String[0]);
+
+    this.fileSystemStorageConfig = FileSystemViewStorageConfig.newBuilder()
+        .fromProperties(configProperties)
+        .build();
+    this.metadataConfig = HoodieMetadataConfig.newBuilder()
+        .fromProperties(configProperties)
+        .build();
+
+    this.queryType = queryType;
+    this.queryPaths = queryPaths;
+    this.specifiedQueryInstant = specifiedQueryInstant;
+    this.shouldIncludePendingCommits = shouldIncludePendingCommits;
+
+    this.tableType = metaClient.getTableType();
+    this.basePath = metaClient.getBasePath();
+
+    this.metaClient = metaClient;
+    this.engineContext = engineContext;
+    this.fileStatusCache = fileStatusCache;
+
+    doRefresh();
+  }
+
+  protected abstract Object[] parsePartitionColumnValues(String[] partitionColumns, String partitionPath);
+
+  /**
+   * Returns latest completed instant as seen by this instance of the file-index
+   */
+  public Option<HoodieInstant> getLatestCompletedInstant() {
+    return getActiveTimeline().filterCompletedInstants().lastInstant();
+  }
+
+  /**
+   * Fetch list of latest base files and log files per partition.
+   *
+   * @return mapping from string partition paths to its base/log files
+   */
+  public Map<String, List<FileSlice>> listFileSlices() {
+    return cachedAllInputFileSlices.entrySet()
+        .stream()
+        .collect(Collectors.toMap(e -> e.getKey().path, Map.Entry::getValue));
+  }
+
+  protected List<PartitionPath> getAllQueryPartitionPaths() {
+    List<String> queryRelativePartitionPaths = queryPaths.stream()
+        .map(path -> FSUtils.getRelativePartitionPath(new Path(basePath), path))
+        .collect(Collectors.toList());
+
+    // Load all the partition path from the basePath, and filter by the query partition path.
+    // TODO load files from the queryRelativePartitionPaths directly.
+    List<String> matchedPartitionPaths = FSUtils.getAllPartitionPaths(engineContext, metadataConfig, basePath)
+        .stream()
+        .filter(path -> queryRelativePartitionPaths.stream().anyMatch(path::startsWith))
+        .collect(Collectors.toList());
+
+    // Convert partition's path into partition descriptor
+    return matchedPartitionPaths.stream()
+        .map(partitionPath -> {
+          Object[] partitionColumnValues = parsePartitionColumnValues(partitionColumns, partitionPath);
+          return new PartitionPath(partitionPath, partitionColumnValues);
+        })
+        .collect(Collectors.toList());
+  }
+
+  protected void refresh() {
+    fileStatusCache.invalidate();
+    doRefresh();
+  }
+
+  protected HoodieTimeline getActiveTimeline() {
+    // NOTE: We have to use commits and compactions timeline, to make sure that we're properly
+    //       handling the following case: when records are inserted into the new log-file w/in the file-group
+    //       that is under the pending compaction process, new log-file will bear the compaction's instant (on the
+    //       timeline) in its name, as opposed to the base-file's commit instant. To make sure we're not filtering
+    //       such log-file we have to _always_ include pending compaction instants into consideration
+    // TODO(HUDI-3302) re-evaluate whether we should not filter any commits in here
+    HoodieTimeline timeline = metaClient.getCommitsAndCompactionTimeline();
+    if (shouldIncludePendingCommits) {
+      return timeline;
+    } else {
+      return timeline.filterCompletedAndCompactionInstants();
+    }
+  }
+
+  /**
+   * Load all partition paths and it's files under the query table path.
+   */
+  private Map<PartitionPath, FileStatus[]> loadPartitionPathFiles() {
+    // List files in all partition paths
+    List<PartitionPath> pathToFetch = new ArrayList<>();
+    Map<PartitionPath, FileStatus[]> cachedPartitionToFiles = new HashMap<>();
+
+    // Fetch from the FileStatusCache
+    List<PartitionPath> partitionPaths = getAllQueryPartitionPaths();
+    partitionPaths.forEach(partitionPath -> {
+      Option<FileStatus[]> filesInPartition = fileStatusCache.get(partitionPath.fullPartitionPath(basePath));
+      if (filesInPartition.isPresent()) {
+        cachedPartitionToFiles.put(partitionPath, filesInPartition.get());
+      } else {
+        pathToFetch.add(partitionPath);
+      }
+    });
+
+    Map<PartitionPath, FileStatus[]> fetchedPartitionToFiles;
+
+    if (pathToFetch.isEmpty()) {
+      fetchedPartitionToFiles = Collections.emptyMap();
+    } else {
+      Map<String, PartitionPath> fullPartitionPathsMapToFetch = pathToFetch.stream()
+          .collect(Collectors.toMap(
+              partitionPath -> partitionPath.fullPartitionPath(basePath).toString(),
+              Function.identity())
+          );
+
+      fetchedPartitionToFiles =
+          FSUtils.getFilesInPartitions(
+                  engineContext,
+                  metadataConfig,
+                  basePath,
+                  fullPartitionPathsMapToFetch.keySet().toArray(new String[0]),

Review comment:
       in master I see here we are using fullPartitionPathsMapToFetch.values.toArray() ? 

##########
File path: hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java
##########
@@ -0,0 +1,351 @@
+/*
+ * 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;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.BaseFile;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodieTableQueryType;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
+import org.apache.hudi.common.util.CollectionUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Common (engine-agnostic) File Index implementation enabling individual query engines to
+ * list Hudi Table contents based on the
+ *
+ * <ul>
+ *   <li>Table type (MOR, COW)</li>
+ *   <li>Query type (snapshot, read_optimized, incremental)</li>
+ *   <li>Query instant/range</li>
+ * </ul>
+ */
+public abstract class BaseHoodieTableFileIndex {
+
+  private static final Logger LOG = LogManager.getLogger(BaseHoodieTableFileIndex.class);
+
+  private final String[] partitionColumns;
+
+  private final FileSystemViewStorageConfig fileSystemStorageConfig;
+  private final HoodieMetadataConfig metadataConfig;
+
+  private final HoodieTableQueryType queryType;
+  private final Option<String> specifiedQueryInstant;
+  protected final List<Path> queryPaths;
+
+  private final boolean shouldIncludePendingCommits;
+
+  private final HoodieTableType tableType;
+  protected final String basePath;
+
+  private final HoodieTableMetaClient metaClient;
+  private final HoodieEngineContext engineContext;
+
+  private final transient FileStatusCache fileStatusCache;
+
+  protected transient volatile long cachedFileSize = 0L;
+  protected transient volatile Map<PartitionPath, List<FileSlice>> cachedAllInputFileSlices;
+
+  protected volatile boolean queryAsNonePartitionedTable = false;
+
+  private transient volatile HoodieTableFileSystemView fileSystemView = null;
+
+  /**
+   * @param engineContext Hudi engine-specific context
+   * @param metaClient Hudi table's meta-client
+   * @param configProperties unifying configuration (in the form of generic properties)
+   * @param queryType target query type
+   * @param queryPaths target DFS paths being queried
+   * @param specifiedQueryInstant instant as of which table is being queried
+   * @param shouldIncludePendingCommits flags whether file-index should exclude any pending operations
+   * @param fileStatusCache transient cache of fetched [[FileStatus]]es
+   */
+  public BaseHoodieTableFileIndex(HoodieEngineContext engineContext,
+                                  HoodieTableMetaClient metaClient,
+                                  TypedProperties configProperties,
+                                  HoodieTableQueryType queryType,
+                                  List<Path> queryPaths,
+                                  Option<String> specifiedQueryInstant,
+                                  boolean shouldIncludePendingCommits,
+                                  FileStatusCache fileStatusCache) {
+    this.partitionColumns = metaClient.getTableConfig().getPartitionFields()
+        .orElse(new String[0]);
+
+    this.fileSystemStorageConfig = FileSystemViewStorageConfig.newBuilder()
+        .fromProperties(configProperties)
+        .build();
+    this.metadataConfig = HoodieMetadataConfig.newBuilder()
+        .fromProperties(configProperties)
+        .build();
+
+    this.queryType = queryType;
+    this.queryPaths = queryPaths;
+    this.specifiedQueryInstant = specifiedQueryInstant;
+    this.shouldIncludePendingCommits = shouldIncludePendingCommits;
+
+    this.tableType = metaClient.getTableType();
+    this.basePath = metaClient.getBasePath();
+
+    this.metaClient = metaClient;
+    this.engineContext = engineContext;
+    this.fileStatusCache = fileStatusCache;
+
+    doRefresh();
+  }
+
+  protected abstract Object[] parsePartitionColumnValues(String[] partitionColumns, String partitionPath);
+
+  /**
+   * Returns latest completed instant as seen by this instance of the file-index
+   */
+  public Option<HoodieInstant> getLatestCompletedInstant() {
+    return getActiveTimeline().filterCompletedInstants().lastInstant();
+  }
+
+  /**
+   * Fetch list of latest base files and log files per partition.
+   *
+   * @return mapping from string partition paths to its base/log files
+   */
+  public Map<String, List<FileSlice>> listFileSlices() {
+    return cachedAllInputFileSlices.entrySet()
+        .stream()
+        .collect(Collectors.toMap(e -> e.getKey().path, Map.Entry::getValue));
+  }
+
+  protected List<PartitionPath> getAllQueryPartitionPaths() {
+    List<String> queryRelativePartitionPaths = queryPaths.stream()
+        .map(path -> FSUtils.getRelativePartitionPath(new Path(basePath), path))
+        .collect(Collectors.toList());
+
+    // Load all the partition path from the basePath, and filter by the query partition path.
+    // TODO load files from the queryRelativePartitionPaths directly.
+    List<String> matchedPartitionPaths = FSUtils.getAllPartitionPaths(engineContext, metadataConfig, basePath)
+        .stream()
+        .filter(path -> queryRelativePartitionPaths.stream().anyMatch(path::startsWith))
+        .collect(Collectors.toList());
+
+    // Convert partition's path into partition descriptor
+    return matchedPartitionPaths.stream()
+        .map(partitionPath -> {
+          Object[] partitionColumnValues = parsePartitionColumnValues(partitionColumns, partitionPath);
+          return new PartitionPath(partitionPath, partitionColumnValues);
+        })
+        .collect(Collectors.toList());
+  }
+
+  protected void refresh() {
+    fileStatusCache.invalidate();
+    doRefresh();
+  }
+
+  protected HoodieTimeline getActiveTimeline() {
+    // NOTE: We have to use commits and compactions timeline, to make sure that we're properly
+    //       handling the following case: when records are inserted into the new log-file w/in the file-group
+    //       that is under the pending compaction process, new log-file will bear the compaction's instant (on the
+    //       timeline) in its name, as opposed to the base-file's commit instant. To make sure we're not filtering
+    //       such log-file we have to _always_ include pending compaction instants into consideration
+    // TODO(HUDI-3302) re-evaluate whether we should not filter any commits in here
+    HoodieTimeline timeline = metaClient.getCommitsAndCompactionTimeline();
+    if (shouldIncludePendingCommits) {
+      return timeline;
+    } else {
+      return timeline.filterCompletedAndCompactionInstants();
+    }
+  }
+
+  /**
+   * Load all partition paths and it's files under the query table path.
+   */
+  private Map<PartitionPath, FileStatus[]> loadPartitionPathFiles() {
+    // List files in all partition paths
+    List<PartitionPath> pathToFetch = new ArrayList<>();
+    Map<PartitionPath, FileStatus[]> cachedPartitionToFiles = new HashMap<>();
+
+    // Fetch from the FileStatusCache
+    List<PartitionPath> partitionPaths = getAllQueryPartitionPaths();
+    partitionPaths.forEach(partitionPath -> {
+      Option<FileStatus[]> filesInPartition = fileStatusCache.get(partitionPath.fullPartitionPath(basePath));
+      if (filesInPartition.isPresent()) {
+        cachedPartitionToFiles.put(partitionPath, filesInPartition.get());
+      } else {
+        pathToFetch.add(partitionPath);
+      }
+    });
+
+    Map<PartitionPath, FileStatus[]> fetchedPartitionToFiles;
+
+    if (pathToFetch.isEmpty()) {
+      fetchedPartitionToFiles = Collections.emptyMap();
+    } else {
+      Map<String, PartitionPath> fullPartitionPathsMapToFetch = pathToFetch.stream()
+          .collect(Collectors.toMap(
+              partitionPath -> partitionPath.fullPartitionPath(basePath).toString(),
+              Function.identity())
+          );
+
+      fetchedPartitionToFiles =
+          FSUtils.getFilesInPartitions(
+                  engineContext,
+                  metadataConfig,
+                  basePath,
+                  fullPartitionPathsMapToFetch.keySet().toArray(new String[0]),
+                  fileSystemStorageConfig.getSpillableDir())
+              .entrySet()
+              .stream()
+              .collect(Collectors.toMap(e -> fullPartitionPathsMapToFetch.get(e.getKey()), e -> e.getValue()));
+
+    }
+
+    // Update the fileStatusCache
+    fetchedPartitionToFiles.forEach((partitionPath, filesInPartition) -> {
+      fileStatusCache.put(partitionPath.fullPartitionPath(basePath), filesInPartition);
+    });
+
+    return CollectionUtils.combine(cachedPartitionToFiles, fetchedPartitionToFiles);
+  }
+
+  private void doRefresh() {
+    long startTime = System.currentTimeMillis();
+
+    Map<PartitionPath, FileStatus[]> partitionFiles = loadPartitionPathFiles();
+    FileStatus[] allFiles = partitionFiles.values().stream().flatMap(Arrays::stream).toArray(FileStatus[]::new);
+
+    metaClient.reloadActiveTimeline();
+
+    HoodieTimeline activeTimeline = getActiveTimeline();
+    Option<HoodieInstant> latestInstant = activeTimeline.lastInstant();
+
+    // TODO we can optimize the flow by:
+    //  - First fetch list of files from instants of interest
+    //  - Load FileStatus's
+    fileSystemView = new HoodieTableFileSystemView(metaClient, activeTimeline, allFiles);
+
+    Option<String> queryInstant =
+        specifiedQueryInstant.or(() -> latestInstant.map(HoodieInstant::getTimestamp));

Review comment:
       what happens if latestInstant is empty ? 

##########
File path: hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java
##########
@@ -0,0 +1,351 @@
+/*
+ * 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;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.BaseFile;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodieTableQueryType;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
+import org.apache.hudi.common.util.CollectionUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Common (engine-agnostic) File Index implementation enabling individual query engines to
+ * list Hudi Table contents based on the
+ *
+ * <ul>
+ *   <li>Table type (MOR, COW)</li>
+ *   <li>Query type (snapshot, read_optimized, incremental)</li>
+ *   <li>Query instant/range</li>
+ * </ul>
+ */
+public abstract class BaseHoodieTableFileIndex {
+
+  private static final Logger LOG = LogManager.getLogger(BaseHoodieTableFileIndex.class);
+
+  private final String[] partitionColumns;
+
+  private final FileSystemViewStorageConfig fileSystemStorageConfig;
+  private final HoodieMetadataConfig metadataConfig;
+
+  private final HoodieTableQueryType queryType;
+  private final Option<String> specifiedQueryInstant;
+  protected final List<Path> queryPaths;
+
+  private final boolean shouldIncludePendingCommits;
+
+  private final HoodieTableType tableType;
+  protected final String basePath;
+
+  private final HoodieTableMetaClient metaClient;
+  private final HoodieEngineContext engineContext;
+
+  private final transient FileStatusCache fileStatusCache;
+
+  protected transient volatile long cachedFileSize = 0L;
+  protected transient volatile Map<PartitionPath, List<FileSlice>> cachedAllInputFileSlices;
+
+  protected volatile boolean queryAsNonePartitionedTable = false;
+
+  private transient volatile HoodieTableFileSystemView fileSystemView = null;
+
+  /**
+   * @param engineContext Hudi engine-specific context
+   * @param metaClient Hudi table's meta-client
+   * @param configProperties unifying configuration (in the form of generic properties)
+   * @param queryType target query type
+   * @param queryPaths target DFS paths being queried
+   * @param specifiedQueryInstant instant as of which table is being queried
+   * @param shouldIncludePendingCommits flags whether file-index should exclude any pending operations
+   * @param fileStatusCache transient cache of fetched [[FileStatus]]es
+   */
+  public BaseHoodieTableFileIndex(HoodieEngineContext engineContext,
+                                  HoodieTableMetaClient metaClient,
+                                  TypedProperties configProperties,
+                                  HoodieTableQueryType queryType,
+                                  List<Path> queryPaths,
+                                  Option<String> specifiedQueryInstant,
+                                  boolean shouldIncludePendingCommits,
+                                  FileStatusCache fileStatusCache) {
+    this.partitionColumns = metaClient.getTableConfig().getPartitionFields()
+        .orElse(new String[0]);
+
+    this.fileSystemStorageConfig = FileSystemViewStorageConfig.newBuilder()
+        .fromProperties(configProperties)
+        .build();
+    this.metadataConfig = HoodieMetadataConfig.newBuilder()
+        .fromProperties(configProperties)
+        .build();
+
+    this.queryType = queryType;
+    this.queryPaths = queryPaths;
+    this.specifiedQueryInstant = specifiedQueryInstant;
+    this.shouldIncludePendingCommits = shouldIncludePendingCommits;
+
+    this.tableType = metaClient.getTableType();
+    this.basePath = metaClient.getBasePath();
+
+    this.metaClient = metaClient;
+    this.engineContext = engineContext;
+    this.fileStatusCache = fileStatusCache;
+
+    doRefresh();
+  }
+
+  protected abstract Object[] parsePartitionColumnValues(String[] partitionColumns, String partitionPath);
+
+  /**
+   * Returns latest completed instant as seen by this instance of the file-index
+   */
+  public Option<HoodieInstant> getLatestCompletedInstant() {
+    return getActiveTimeline().filterCompletedInstants().lastInstant();
+  }
+
+  /**
+   * Fetch list of latest base files and log files per partition.
+   *
+   * @return mapping from string partition paths to its base/log files
+   */
+  public Map<String, List<FileSlice>> listFileSlices() {
+    return cachedAllInputFileSlices.entrySet()
+        .stream()
+        .collect(Collectors.toMap(e -> e.getKey().path, Map.Entry::getValue));
+  }
+
+  protected List<PartitionPath> getAllQueryPartitionPaths() {
+    List<String> queryRelativePartitionPaths = queryPaths.stream()
+        .map(path -> FSUtils.getRelativePartitionPath(new Path(basePath), path))
+        .collect(Collectors.toList());
+
+    // Load all the partition path from the basePath, and filter by the query partition path.
+    // TODO load files from the queryRelativePartitionPaths directly.
+    List<String> matchedPartitionPaths = FSUtils.getAllPartitionPaths(engineContext, metadataConfig, basePath)
+        .stream()
+        .filter(path -> queryRelativePartitionPaths.stream().anyMatch(path::startsWith))
+        .collect(Collectors.toList());
+
+    // Convert partition's path into partition descriptor
+    return matchedPartitionPaths.stream()
+        .map(partitionPath -> {
+          Object[] partitionColumnValues = parsePartitionColumnValues(partitionColumns, partitionPath);
+          return new PartitionPath(partitionPath, partitionColumnValues);
+        })
+        .collect(Collectors.toList());
+  }
+
+  protected void refresh() {
+    fileStatusCache.invalidate();
+    doRefresh();
+  }
+
+  protected HoodieTimeline getActiveTimeline() {
+    // NOTE: We have to use commits and compactions timeline, to make sure that we're properly
+    //       handling the following case: when records are inserted into the new log-file w/in the file-group
+    //       that is under the pending compaction process, new log-file will bear the compaction's instant (on the
+    //       timeline) in its name, as opposed to the base-file's commit instant. To make sure we're not filtering
+    //       such log-file we have to _always_ include pending compaction instants into consideration
+    // TODO(HUDI-3302) re-evaluate whether we should not filter any commits in here
+    HoodieTimeline timeline = metaClient.getCommitsAndCompactionTimeline();
+    if (shouldIncludePendingCommits) {
+      return timeline;
+    } else {
+      return timeline.filterCompletedAndCompactionInstants();
+    }
+  }
+
+  /**
+   * Load all partition paths and it's files under the query table path.
+   */
+  private Map<PartitionPath, FileStatus[]> loadPartitionPathFiles() {
+    // List files in all partition paths
+    List<PartitionPath> pathToFetch = new ArrayList<>();
+    Map<PartitionPath, FileStatus[]> cachedPartitionToFiles = new HashMap<>();
+
+    // Fetch from the FileStatusCache
+    List<PartitionPath> partitionPaths = getAllQueryPartitionPaths();
+    partitionPaths.forEach(partitionPath -> {
+      Option<FileStatus[]> filesInPartition = fileStatusCache.get(partitionPath.fullPartitionPath(basePath));
+      if (filesInPartition.isPresent()) {
+        cachedPartitionToFiles.put(partitionPath, filesInPartition.get());
+      } else {
+        pathToFetch.add(partitionPath);
+      }
+    });
+
+    Map<PartitionPath, FileStatus[]> fetchedPartitionToFiles;

Review comment:
       we can instantiate to empty map here only. and later follow it as 
   ```
   if (!pathToFetch.isEmpty()) {) {
    // update 
   } 
   ```
   
   or was it intentionally written this way ?

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/HoodieTableQueryType.java
##########
@@ -30,7 +30,7 @@
  * </ol>
  */
 public enum HoodieTableQueryType {
-  QUERY_TYPE_SNAPSHOT,
-  QUERY_TYPE_INCREMENTAL,
-  QUERY_TYPE_READ_OPTIMIZED
+  SNAPSHOT,

Review comment:
       I see deduceQueryType in SparkHoodieTableFileIndex. so if we are converting the old vars to new ones, we should be good. Can you do some sanity testing w/ quick start guide that old variables work w/ this patch too. 
   

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/HoodieTableQueryType.java
##########
@@ -30,7 +30,7 @@
  * </ol>
  */
 public enum HoodieTableQueryType {
-  QUERY_TYPE_SNAPSHOT,
-  QUERY_TYPE_INCREMENTAL,
-  QUERY_TYPE_READ_OPTIMIZED
+  SNAPSHOT,

Review comment:
       I am not sure if we can do this. Users might have to fix all of their existing pipelines w/ these new variables. atleast for read optimized query and incremental query pipelines. 

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/HoodieTableQueryType.java
##########
@@ -30,7 +30,7 @@
  * </ol>
  */
 public enum HoodieTableQueryType {
-  QUERY_TYPE_SNAPSHOT,
-  QUERY_TYPE_INCREMENTAL,
-  QUERY_TYPE_READ_OPTIMIZED
+  SNAPSHOT,

Review comment:
       and does the variable conversion happen in all code paths. for eg, there is a config where in you can disable fileIndex altogether. So, presto, trino, hive connector, beeline, spark datasource from all possible paths, ensure old to new is taken care of. 
   

##########
File path: hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java
##########
@@ -0,0 +1,351 @@
+/*
+ * 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;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.BaseFile;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodieTableQueryType;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
+import org.apache.hudi.common.util.CollectionUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Common (engine-agnostic) File Index implementation enabling individual query engines to
+ * list Hudi Table contents based on the
+ *
+ * <ul>
+ *   <li>Table type (MOR, COW)</li>
+ *   <li>Query type (snapshot, read_optimized, incremental)</li>
+ *   <li>Query instant/range</li>
+ * </ul>
+ */
+public abstract class BaseHoodieTableFileIndex {
+
+  private static final Logger LOG = LogManager.getLogger(BaseHoodieTableFileIndex.class);
+
+  private final String[] partitionColumns;
+
+  private final FileSystemViewStorageConfig fileSystemStorageConfig;
+  private final HoodieMetadataConfig metadataConfig;
+
+  private final HoodieTableQueryType queryType;
+  private final Option<String> specifiedQueryInstant;
+  protected final List<Path> queryPaths;
+
+  private final boolean shouldIncludePendingCommits;
+
+  private final HoodieTableType tableType;
+  protected final String basePath;
+
+  private final HoodieTableMetaClient metaClient;
+  private final HoodieEngineContext engineContext;
+
+  private final transient FileStatusCache fileStatusCache;
+
+  protected transient volatile long cachedFileSize = 0L;
+  protected transient volatile Map<PartitionPath, List<FileSlice>> cachedAllInputFileSlices;
+
+  protected volatile boolean queryAsNonePartitionedTable = false;
+
+  private transient volatile HoodieTableFileSystemView fileSystemView = null;
+
+  /**
+   * @param engineContext Hudi engine-specific context
+   * @param metaClient Hudi table's meta-client
+   * @param configProperties unifying configuration (in the form of generic properties)
+   * @param queryType target query type
+   * @param queryPaths target DFS paths being queried
+   * @param specifiedQueryInstant instant as of which table is being queried
+   * @param shouldIncludePendingCommits flags whether file-index should exclude any pending operations
+   * @param fileStatusCache transient cache of fetched [[FileStatus]]es
+   */
+  public BaseHoodieTableFileIndex(HoodieEngineContext engineContext,
+                                  HoodieTableMetaClient metaClient,
+                                  TypedProperties configProperties,
+                                  HoodieTableQueryType queryType,
+                                  List<Path> queryPaths,
+                                  Option<String> specifiedQueryInstant,
+                                  boolean shouldIncludePendingCommits,
+                                  FileStatusCache fileStatusCache) {
+    this.partitionColumns = metaClient.getTableConfig().getPartitionFields()
+        .orElse(new String[0]);
+
+    this.fileSystemStorageConfig = FileSystemViewStorageConfig.newBuilder()
+        .fromProperties(configProperties)
+        .build();
+    this.metadataConfig = HoodieMetadataConfig.newBuilder()
+        .fromProperties(configProperties)
+        .build();
+
+    this.queryType = queryType;
+    this.queryPaths = queryPaths;
+    this.specifiedQueryInstant = specifiedQueryInstant;
+    this.shouldIncludePendingCommits = shouldIncludePendingCommits;
+
+    this.tableType = metaClient.getTableType();
+    this.basePath = metaClient.getBasePath();
+
+    this.metaClient = metaClient;
+    this.engineContext = engineContext;
+    this.fileStatusCache = fileStatusCache;
+
+    doRefresh();
+  }
+
+  protected abstract Object[] parsePartitionColumnValues(String[] partitionColumns, String partitionPath);
+
+  /**
+   * Returns latest completed instant as seen by this instance of the file-index
+   */
+  public Option<HoodieInstant> getLatestCompletedInstant() {
+    return getActiveTimeline().filterCompletedInstants().lastInstant();
+  }
+
+  /**
+   * Fetch list of latest base files and log files per partition.
+   *
+   * @return mapping from string partition paths to its base/log files
+   */
+  public Map<String, List<FileSlice>> listFileSlices() {
+    return cachedAllInputFileSlices.entrySet()
+        .stream()
+        .collect(Collectors.toMap(e -> e.getKey().path, Map.Entry::getValue));
+  }
+
+  protected List<PartitionPath> getAllQueryPartitionPaths() {
+    List<String> queryRelativePartitionPaths = queryPaths.stream()
+        .map(path -> FSUtils.getRelativePartitionPath(new Path(basePath), path))
+        .collect(Collectors.toList());
+
+    // Load all the partition path from the basePath, and filter by the query partition path.
+    // TODO load files from the queryRelativePartitionPaths directly.
+    List<String> matchedPartitionPaths = FSUtils.getAllPartitionPaths(engineContext, metadataConfig, basePath)
+        .stream()
+        .filter(path -> queryRelativePartitionPaths.stream().anyMatch(path::startsWith))
+        .collect(Collectors.toList());
+
+    // Convert partition's path into partition descriptor
+    return matchedPartitionPaths.stream()
+        .map(partitionPath -> {
+          Object[] partitionColumnValues = parsePartitionColumnValues(partitionColumns, partitionPath);
+          return new PartitionPath(partitionPath, partitionColumnValues);
+        })
+        .collect(Collectors.toList());
+  }
+
+  protected void refresh() {
+    fileStatusCache.invalidate();
+    doRefresh();
+  }
+
+  protected HoodieTimeline getActiveTimeline() {
+    // NOTE: We have to use commits and compactions timeline, to make sure that we're properly
+    //       handling the following case: when records are inserted into the new log-file w/in the file-group
+    //       that is under the pending compaction process, new log-file will bear the compaction's instant (on the
+    //       timeline) in its name, as opposed to the base-file's commit instant. To make sure we're not filtering
+    //       such log-file we have to _always_ include pending compaction instants into consideration
+    // TODO(HUDI-3302) re-evaluate whether we should not filter any commits in here
+    HoodieTimeline timeline = metaClient.getCommitsAndCompactionTimeline();
+    if (shouldIncludePendingCommits) {
+      return timeline;
+    } else {
+      return timeline.filterCompletedAndCompactionInstants();
+    }
+  }
+
+  /**
+   * Load all partition paths and it's files under the query table path.
+   */
+  private Map<PartitionPath, FileStatus[]> loadPartitionPathFiles() {
+    // List files in all partition paths
+    List<PartitionPath> pathToFetch = new ArrayList<>();
+    Map<PartitionPath, FileStatus[]> cachedPartitionToFiles = new HashMap<>();
+
+    // Fetch from the FileStatusCache
+    List<PartitionPath> partitionPaths = getAllQueryPartitionPaths();
+    partitionPaths.forEach(partitionPath -> {
+      Option<FileStatus[]> filesInPartition = fileStatusCache.get(partitionPath.fullPartitionPath(basePath));
+      if (filesInPartition.isPresent()) {
+        cachedPartitionToFiles.put(partitionPath, filesInPartition.get());
+      } else {
+        pathToFetch.add(partitionPath);
+      }
+    });
+
+    Map<PartitionPath, FileStatus[]> fetchedPartitionToFiles;
+
+    if (pathToFetch.isEmpty()) {
+      fetchedPartitionToFiles = Collections.emptyMap();
+    } else {
+      Map<String, PartitionPath> fullPartitionPathsMapToFetch = pathToFetch.stream()
+          .collect(Collectors.toMap(
+              partitionPath -> partitionPath.fullPartitionPath(basePath).toString(),
+              Function.identity())
+          );
+
+      fetchedPartitionToFiles =
+          FSUtils.getFilesInPartitions(
+                  engineContext,
+                  metadataConfig,
+                  basePath,
+                  fullPartitionPathsMapToFetch.keySet().toArray(new String[0]),

Review comment:
       oh, we are reverting the key and value in fullPartitionPathsMapToFetch. got 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.

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

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