xushiyan commented on code in PR #7143:
URL: https://github.com/apache/hudi/pull/7143#discussion_r1192695414


##########
hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java:
##########
@@ -41,6 +41,13 @@ public class HoodieCommonConfig extends HoodieConfig {
       .noDefaultValue()
       .withDocumentation("The query instant for time travel. Without specified 
this option, we query the latest snapshot.");
 
+  public static final ConfigProperty<Integer> TABLE_LOADING_PARALLELISM = 
ConfigProperty
+      .key("hoodie.datasource.read.loading.parallelism")
+      .defaultValue(-1)

Review Comment:
   we don't normally use -1 to disable a feature. usually we have a xxxEnabled. 
and we should mark this advanced() config. and can we auto infer it somehow?



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -229,17 +238,93 @@ protected List<FileSlice> 
getInputFileSlices(PartitionPath partition) {
   }
 
   private Map<PartitionPath, List<FileSlice>> 
loadFileSlicesForPartitions(List<PartitionPath> partitions) {
-    FileStatus[] allFiles = listPartitionPathFiles(partitions);
+    Pair<Map<Path, FileStatus[]>, Map<String, FileStatus[]>> 
partitionFilesPair = listPartitionPathFiles(partitions);
     HoodieTimeline activeTimeline = getActiveTimeline();
     Option<HoodieInstant> latestInstant = activeTimeline.lastInstant();
 
-    HoodieTableFileSystemView fileSystemView =
-        new HoodieTableFileSystemView(metaClient, activeTimeline, allFiles);
-
     Option<String> queryInstant = specifiedQueryInstant.or(() -> 
latestInstant.map(HoodieInstant::getTimestamp));
 
     validate(activeTimeline, queryInstant);
 
+    int parallelism = 
Integer.parseInt(String.valueOf(configProperties.getOrDefault(HoodieCommonConfig.TABLE_LOADING_PARALLELISM.key(),
+        HoodieCommonConfig.TABLE_LOADING_PARALLELISM.defaultValue())));
+
+    Map<PartitionPath, List<FileSlice>> cachedAllInputFileSlices;
+    long buildCacheFileSlicesLocalStart = System.currentTimeMillis();
+    if (parallelism > 0 && partitions.size() > 0) {
+
+      // convert Map<Path, FileStatus[]> to Map<String, FileStatus[]>
+      Map<String, FileStatus[]> left = 
partitionFilesPair.getLeft().entrySet().stream().map(entry -> {
+        String partitionPath = entry.getKey().toString();
+        FileStatus[] statuses = entry.getValue();
+        return Pair.of(partitionPath, statuses);
+      }).collect(Collectors.toMap(Pair::getKey, Pair::getValue));
+
+      Map<String, FileStatus[]> partitionFiles = combine(left, 
partitionFilesPair.getRight());
+
+      cachedAllInputFileSlices = 
buildCacheFileSlicesLocalParallel(parallelism, partitions, partitionFiles, 
activeTimeline, queryInstant);
+    } else {
+      FileStatus[] allFiles = 
combine(flatMap(partitionFilesPair.getLeft().values()), 
flatMap(partitionFilesPair.getRight().values()));
+      HoodieTableFileSystemView fileSystemView =
+          new HoodieTableFileSystemView(metaClient, activeTimeline, allFiles);
+
+      cachedAllInputFileSlices =  getCandidateFileSlices(partitions, 
queryInstant, fileSystemView);
+    }
+
+    long buildCacheFileSlicesLocalEnd = System.currentTimeMillis();
+    LOG.info(String.format("Build cache file slices, spent: %d ms", 
buildCacheFileSlicesLocalEnd - buildCacheFileSlicesLocalStart));
+
+    return cachedAllInputFileSlices;
+  }
+
+  private Map<PartitionPath, List<FileSlice>> 
buildCacheFileSlicesLocalParallel(int parallelism, List<PartitionPath> 
partitions, Map<String, FileStatus[]> partitionFiles,
+                                                                               
 HoodieTimeline activeTimeline, Option<String> queryInstant) {
+    HashMap<PartitionPath, List<FileSlice>> res = new HashMap<>();
+    parallelism = Math.max(1, Math.min(parallelism, partitionFiles.size()));
+    int totalPartitions = partitionFiles.size();
+    int cursor = 0;
+    int step = totalPartitions / parallelism;
+
+    ExecutorService pool =  Executors.newFixedThreadPool((parallelism + 1));
+    ArrayList<CompletableFuture<Map<PartitionPath, List<FileSlice>>>> 
futureList = new ArrayList<>(parallelism + 1);
+
+    while (cursor + step <= totalPartitions) {

Review Comment:
   +1 for parallelStream to simplify the logic



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndexParallelLoad.scala:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.hudi.common.config.HoodieCommonConfig
+
+/**
+ * running all the UT TestHoodieFileIndex test in local parallelism mode
+ */
+class TestHoodieFileIndexParallelLoad extends TestHoodieFileIndex {

Review Comment:
   pls enhance the tests



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