YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r990607541


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -179,15 +197,125 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (cachedAllPartitionPaths != null) {
+      return cachedAllPartitionPaths;
+    }
+
+    loadAllQueryPartitionPaths();
+    return cachedAllPartitionPaths;
+  }
+
+  private void loadAllQueryPartitionPaths() {
     List<String> queryRelativePartitionPaths = queryPaths.stream()
         .map(path -> FSUtils.getRelativePartitionPath(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 = getAllPartitionPathsUnchecked()
-        .stream()
-        .filter(path -> 
queryRelativePartitionPaths.stream().anyMatch(path::startsWith))
+    this.cachedAllPartitionPaths = 
listQueryPartitionPaths(queryRelativePartitionPaths);
+
+    // If the partition value contains InternalRow.empty, we query it as a 
non-partitioned table.
+    this.queryAsNonePartitionedTable = 
this.cachedAllPartitionPaths.stream().anyMatch(p -> p.values.length == 0);
+  }
+
+  protected Map<PartitionPath, List<FileSlice>> getAllInputFileSlices() {
+    if (!isAllInputFileSlicesCached) {
+      doRefresh();
+    }
+    return cachedAllInputFileSlices;
+  }
+
+  /**
+   * Get input file slice for the given partition. Will use cache directly if 
it is computed before.
+   */
+  protected List<FileSlice> getCachedInputFileSlices(PartitionPath partition) {
+    return cachedAllInputFileSlices.computeIfAbsent(partition, 
this::loadFileSlicesForPartition);
+  }
+
+  private List<FileSlice> loadFileSlicesForPartition(PartitionPath p) {
+    FileStatus[] files = loadPartitionPathFiles(p);
+    HoodieTimeline activeTimeline = getActiveTimeline();
+    Option<HoodieInstant> latestInstant = activeTimeline.lastInstant();
+
+    HoodieTableFileSystemView fileSystemView = new 
HoodieTableFileSystemView(metaClient, activeTimeline, files);
+
+    Option<String> queryInstant = specifiedQueryInstant.or(() -> 
latestInstant.map(HoodieInstant::getTimestamp));
+
+    validate(activeTimeline, queryInstant);
+
+    List<FileSlice> ret;
+    if (tableType.equals(HoodieTableType.MERGE_ON_READ) && 
queryType.equals(HoodieTableQueryType.SNAPSHOT)) {
+      ret = queryInstant.map(instant ->
+              fileSystemView.getLatestMergedFileSlicesBeforeOrOn(p.path, 
queryInstant.get())
+                  .collect(Collectors.toList())
+          )
+          .orElse(Collections.emptyList());
+    } else {
+      ret = queryInstant.map(instant ->
+              fileSystemView.getLatestFileSlicesBeforeOrOn(p.path, instant, 
true)
+          )
+          .orElse(fileSystemView.getLatestFileSlices(p.path))
+          .collect(Collectors.toList());
+    }
+
+    cachedFileSize += 
ret.stream().mapToLong(BaseHoodieTableFileIndex::fileSliceSize).sum();
+    return ret;
+  }
+
+  /**
+   * Get partition path with the given partition value
+   * @param partitionNames partition names
+   * @param values partition values
+   * @return partitions that match the given partition values
+   */
+  protected List<PartitionPath> getPartitionPaths(String[] partitionNames, 
String[] values) {
+    if (partitionNames.length == 0 || partitionNames.length != values.length) {

Review Comment:
   Yeah, I cleaned up the code accordingly. I added `isPartial` to replace the 
role of `idx`. Could u take another look?



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