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/09/15 08:57:08 UTC

[GitHub] [hudi] YuweiXiao opened a new pull request, #6680: [WIP][HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

YuweiXiao opened a new pull request, #6680:
URL: https://github.com/apache/hudi/pull/6680

   
   ### Change Logs
   
   Lazy fetching partition path & file slice for HoodieFileIndex
   
   ### Impact
   
   No API change, and will improve performance for spark query with partition filter.
   
   **Risk level: none | low | medium | high**
   
   Low.
   
   ### Contributor's checklist
   
   - [x] Read through [contributor's guide](https://hudi.apache.org/contribute/how-to-contribute)
   - [x] Change Logs and Impact were stated clearly
   - [ ] Adequate tests were added if applicable
   - [ ] CI passed
   


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


[GitHub] [hudi] psendyk commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
psendyk commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1023364717


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -179,17 +175,94 @@ public void close() throws Exception {
     resetTableMetadata(null);
   }
 
+  protected String[] getPartitionColumns() {
+    return partitionColumns;
+  }
+
+  protected List<Path> getQueryPaths() {
+    return queryPaths;
+  }
+
+  /**
+   * Returns all partition paths matching the ones explicitly provided by the query (if any)
+   */
   protected List<PartitionPath> getAllQueryPartitionPaths() {
-    List<String> queryRelativePartitionPaths = queryPaths.stream()
-        .map(path -> FSUtils.getRelativePartitionPath(basePath, path))
-        .collect(Collectors.toList());
+    if (cachedAllPartitionPaths == null) {
+      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))
-        .collect(Collectors.toList());
+      this.cachedAllPartitionPaths = listPartitionPaths(queryRelativePartitionPaths);
+    }
+
+    return cachedAllPartitionPaths;
+  }
+
+  /**
+   * Returns all listed file-slices w/in the partition paths returned by {@link #getAllQueryPartitionPaths()}
+   */
+  protected Map<PartitionPath, List<FileSlice>> getAllInputFileSlices() {
+    if (!areAllFileSlicesCached()) {
+      // Fetching file slices for partitions that have not been cached yet
+      List<PartitionPath> missingPartitions = getAllQueryPartitionPaths().stream()
+          .filter(p -> !cachedAllInputFileSlices.containsKey(p))
+          .collect(Collectors.toList());
+
+      // NOTE: Individual partitions are always cached in full, therefore if partition is cached
+      //       it will hold all the file-slices residing w/in the partition
+      cachedAllInputFileSlices.putAll(loadFileSlicesForPartitions(missingPartitions));
+    }
+
+    return cachedAllInputFileSlices;
+  }
+
+  /**
+   * Get input file slice for the given partition. Will use cache directly if it is computed before.
+   */
+  protected List<FileSlice> getInputFileSlices(PartitionPath partition) {
+    return cachedAllInputFileSlices.computeIfAbsent(partition,
+        p -> loadFileSlicesForPartitions(Collections.singletonList(p)).get(p));
+  }
+
+  private Map<PartitionPath, List<FileSlice>> loadFileSlicesForPartitions(List<PartitionPath> partitions) {
+    Map<PartitionPath, FileStatus[]> partitionFiles = partitions.stream()
+        .collect(Collectors.toMap(p -> p, this::loadPartitionPathFiles));
+    HoodieTimeline activeTimeline = getActiveTimeline();
+    Option<HoodieInstant> latestInstant = activeTimeline.lastInstant();
+
+    FileStatus[] allFiles = partitionFiles.values().stream().flatMap(Arrays::stream).toArray(FileStatus[]::new);
+    HoodieTableFileSystemView fileSystemView = new HoodieTableFileSystemView(metaClient, activeTimeline, allFiles);

Review Comment:
   And to add to this, I'm also seeing the same behavior in eager loading mode, except it's much worse as it's listing all partitions in the table. So if I'm correct, then this PR introduces a regression to all MT-based file listing. When I use the previous implementation (0.12.0), the `files` partition is only computed once.



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1000222754


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala:
##########
@@ -187,12 +189,16 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
    * @param predicates     The filter condition.
    * @return The pruned partition paths.
    */
-  protected def prunePartition(partitionPaths: Seq[PartitionPath], predicates: Seq[Expression]): Seq[PartitionPath] = {
+  protected def prunePartition(predicates: Seq[Expression]): Seq[PartitionPath] = {
     val partitionColumnNames = partitionSchema.fields.map(_.name).toSet
     val partitionPruningPredicates = predicates.filter {
       _.references.map(_.name).toSet.subsetOf(partitionColumnNames)
     }
     if (partitionPruningPredicates.nonEmpty) {
+      val equalPredicate = partitionPruningPredicates.filter(_.isInstanceOf[EqualTo])
+      val names = equalPredicate.map(_.asInstanceOf[EqualTo]).map(_.left.asInstanceOf[AttributeReference].name).toArray

Review Comment:
   Ah never mind about this one. I was thinking to remove the config, and handle the lazy listing internally (i.e., turn on or not). 



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1294408411

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * df0a3523720f59827e74c67354d465966e17e176 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630) 
   * f12d60f65b7e33cf69783b80eac0463e62f6f170 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1000209438


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -310,35 +256,159 @@ private void doRefresh() {
             )
         );
 
-    cachedFileSize = cachedAllInputFileSlices.values().stream()
+    this.cachedFileSize += ret.values().stream()
         .flatMap(Collection::stream)
         .mapToLong(BaseHoodieTableFileIndex::fileSliceSize)
         .sum();
 
-    // If the partition value contains InternalRow.empty, we query it as a non-partitioned table.
-    queryAsNonePartitionedTable = partitionFiles.keySet().stream().anyMatch(p -> p.values.length == 0);
+    return ret;
+  }
 
-    long duration = System.currentTimeMillis() - startTime;
+  /**
+   * 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 (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
 
-    LOG.info(String.format("Refresh table %s, spent: %d ms", metaClient.getTableConfig().getTableName(), duration));
+    Pair<String, Boolean> relativeQueryPartitionPathPair = composeRelativePartitionPaths(partitionNames, values);

Review Comment:
   Yeah, sure. The construction is based on the partition order from schema rather than the input order.



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1288311641

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a09114e1c326791e33e910b2f660aaa6882dcfc9 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980877328


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala:
##########
@@ -138,6 +138,14 @@ object DataSourceReadOptions {
         " read from the data file (in Hudi partition columns are persisted by default)." +
         " This config is a fallback allowing to preserve existing behavior, and should not be used otherwise.")
 
+  val REFRESH_PARTITION_AND_FILES_IN_INITIALIZATION: ConfigProperty[Boolean] =

Review Comment:
   Do you mean setting the config default to false? Indeed, I intend to have this option as a fallback.



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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980383463


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala:
##########
@@ -187,12 +189,16 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
    * @param predicates     The filter condition.
    * @return The pruned partition paths.
    */
-  protected def prunePartition(partitionPaths: Seq[PartitionPath], predicates: Seq[Expression]): Seq[PartitionPath] = {
+  protected def prunePartition(predicates: Seq[Expression]): Seq[PartitionPath] = {
     val partitionColumnNames = partitionSchema.fields.map(_.name).toSet
     val partitionPruningPredicates = predicates.filter {
       _.references.map(_.name).toSet.subsetOf(partitionColumnNames)
     }
     if (partitionPruningPredicates.nonEmpty) {
+      val equalPredicate = partitionPruningPredicates.filter(_.isInstanceOf[EqualTo])
+      val names = equalPredicate.map(_.asInstanceOf[EqualTo]).map(_.left.asInstanceOf[AttributeReference].name).toArray

Review Comment:
   Yeah, we need to make it a proper restriction on the semantic:
   
    - When filters are passed in we check whether we could do lazy-merging on them (checking if its strictly `EqualTo`, `In` predicates) 
    - If not, we fallback to eager loading



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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980386522


##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java:
##########
@@ -143,6 +143,13 @@ protected Option<HoodieRecord<HoodieMetadataPayload>> getRecordByKey(String key,
     return recordsByKeys.size() == 0 ? Option.empty() : recordsByKeys.get(0).getValue();
   }
 
+  @Override
+  public List<String> getPartitionPathsWithPrefix(String prefix) throws IOException {
+    return getAllPartitionPaths().stream()

Review Comment:
   We should instead fetch for all the prefixes at once, instead of doing it in a for loop



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r979367148


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(queryRelativePartitionPaths);

Review Comment:
   Fixed.



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (p) -> {
+      FileStatus[] files = loadPartitionPathFiles(p);

Review Comment:
   Fixed.



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1305047029

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fe18f26f8ab63a28c98b23c6940b6135b6155d91 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804) 
   * 13b8d331e0dcf5ac1f958b91770e2024bce04c43 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819) 
   * c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1250220263

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1000187071


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -139,7 +142,20 @@ public BaseHoodieTableFileIndex(HoodieEngineContext engineContext,
     this.engineContext = engineContext;
     this.fileStatusCache = fileStatusCache;
 
-    doRefresh();
+    /**
+     * The `shouldListLazily` variable controls how we initialize the TableFileIndex:
+     *  - non-lazy/eager listing (shouldListLazily=true):  all partitions and file slices will be loaded eagerly during initialization.

Review Comment:
   Yes! Thanks!



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r979367133


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -221,60 +329,46 @@ protected HoodieTimeline getActiveTimeline() {
   }
 
   /**
-   * Load all partition paths and it's files under the query table path.
+   * Load 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 =
-          getAllFilesInPartitionsUnchecked(fullPartitionPathsMapToFetch.keySet())
-              .entrySet()
-              .stream()
-              .collect(Collectors.toMap(e -> fullPartitionPathsMapToFetch.get(e.getKey()), e -> e.getValue()));
-
+  private FileStatus[] loadPartitionPathFiles(PartitionPath partition) {
+    // Try fetch from the FileStatusCache first
+    Option<FileStatus[]> files = fileStatusCache.get(partition.fullPartitionPath(basePath));
+    if (files.isPresent()) {
+      return files.get();
     }
 
-    // Update the fileStatusCache
-    fetchedPartitionToFiles.forEach((partitionPath, filesInPartition) -> {
-      fileStatusCache.put(partitionPath.fullPartitionPath(basePath), filesInPartition);
-    });
+    try {
+      Path path = partition.fullPartitionPath(basePath);
+      FileStatus[] fetchedFiles = tableMetadata.getAllFilesInPartition(path);
 
-    return CollectionUtils.combine(cachedPartitionToFiles, fetchedPartitionToFiles);
+      // Update the fileStatusCache
+      fileStatusCache.put(partition.fullPartitionPath(basePath), fetchedFiles);
+      return fetchedFiles;
+    } catch (IOException e) {
+      throw new HoodieIOException("Failed to list partition path (" + partition + ") for a table", e);
+    }
   }
 
   private void doRefresh() {
+    doRefresh(false);
+  }
+  
+  private void doRefresh(boolean initMetadataOnly) {

Review Comment:
   Done.



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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980385727


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNamesToIdx.containsKey(columnNames)) {
+        int k = partitionNamesToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");

Review Comment:
   We have to handle general use-case, we can't just support the primary one. Please take a look at my comment above regarding making this a restriction on lazy loading semantic



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1257143633

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d8495f35bf97ed4ec27a3841dad66307d862d65e Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475) 
   * 21b5c2377dd577e4ff8257f87fc0cda3f089c7d3 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980877328


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala:
##########
@@ -138,6 +138,14 @@ object DataSourceReadOptions {
         " read from the data file (in Hudi partition columns are persisted by default)." +
         " This config is a fallback allowing to preserve existing behavior, and should not be used otherwise.")
 
+  val REFRESH_PARTITION_AND_FILES_IN_INITIALIZATION: ConfigProperty[Boolean] =

Review Comment:
   Do you mean setting the config to false? Indeed, I intend to have this option as a fallback.



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1259163769

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 21b5c2377dd577e4ff8257f87fc0cda3f089c7d3 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708) 
   * efc19bfcfb86bf582d4bd2584462083b8178c1c6 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r990601721


##########
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) {

Review Comment:
   Yeah, good point. 1) generalize to batch get  2) load only remaining partitions



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1286546211

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a65056adaa4e9fabda4205c1b5c7be2e48bdd67f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382) 
   * 903ac79702d02fe968d271b08a217f8401d63700 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1272295291

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5f6d4f624c5f20cf3c4c38384e17c7bb13e56991 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1272505910

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5f6d4f624c5f20cf3c4c38384e17c7bb13e56991 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065) 
   * fd5036ba6e5768ec3f846a41d766e776b8106551 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1294404537

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a09114e1c326791e33e910b2f660aaa6882dcfc9 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493) 
   * df0a3523720f59827e74c67354d465966e17e176 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630) 
   * f12d60f65b7e33cf69783b80eac0463e62f6f170 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1007526943


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/PartitionPathFormatterBase.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.keygen;
+
+import org.apache.spark.unsafe.types.UTF8String;
+
+import java.util.List;
+import java.util.function.Supplier;
+
+import static org.apache.hudi.common.util.ValidationUtils.checkState;
+import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_PARTITION_PATH_SEPARATOR;
+
+/**
+ * Partition path formatter allows to efficiently combine partition paths into
+ * generic Java {@link String} (as well as engine-specific containers like
+ * {@code UTF8String} for ex), while being flexible in terms of
+ *
+ * <ul>
+ *   <li>Allowing to configure how to handle empty values</li>
+ *   <li>Allowing to encode individual values</li>
+ *   <li>Supporting Hive-style partitioning ({@code column=value})</li>
+ * </ul>
+ *
+ * @param <S> string type
+ */
+public abstract class PartitionPathFormatterBase<S> {

Review Comment:
   These partition-path formatter were extracted from `BuilinKeyGenerator` to be reused subsequently for partition-path composition in `SparkHoodieFileIndex`



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1305147415

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1305009832

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fe18f26f8ab63a28c98b23c6940b6135b6155d91 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804) 
   * 13b8d331e0dcf5ac1f958b91770e2024bce04c43 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1305215450

   @alexeykudinkin Hey Alex, the test is added and ut is fixed. Please take another look. By the way, I see the lazy listing mode is eager by default, is it intended?


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1257166416

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 21b5c2377dd577e4ff8257f87fc0cda3f089c7d3 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1000236939


##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java:
##########
@@ -132,6 +132,11 @@ static HoodieBackedTableMetadata createHoodieBackedTableMetadata(HoodieEngineCon
    */
   FileStatus[] getAllFilesInPartition(Path partitionPath) throws IOException;
 
+  /**
+   * Fetch list of all partitions path that match the given prefix

Review Comment:
   Improved, also added an example to illustrate the idea.



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1311101313

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     }, {
       "hash" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937",
       "triggerID" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12938",
       "triggerID" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 522204901637ca82f8232c63d9b26adce3f484b8 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937) 
   * 5341fff4dfb30afde48c370a7c6cb6e31b389539 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12938) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] xushiyan commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
xushiyan commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1022548772


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala:
##########
@@ -237,70 +246,64 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
     }
   }
 
-  private def listMatchingPartitionPathsInternal(partitionColumnNames: Seq[String],
-                                                 partitionColumnPredicates: Seq[Expression]): Seq[PartitionPath] = {
-    // NOTE: Here we try to to achieve efficiency in avoiding necessity to recursively list deep folder structures of
-    //       partitioned tables w/ multiple partition columns, by carefully analyzing provided partition predicates:
-    //
-    //       In cases when partition-predicates have
-    //         - The form of equality predicates w/ static literals (for ex, like `date = '2022-01-01'`)
-    //         - Fully specified proper prefix of the partition schema (ie fully binding first N columns
-    //           of the partition schema adhering to hereby described rules)
-    //
-    // We will try to exploit this specific structure, and try to reduce the scope of a
-    // necessary file-listings of partitions of the table to just the sub-folder under relative prefix
-    // of the partition-path derived from the partition-column predicates. For ex, consider following
-    // scenario:
-    //
-    // Table's partition schema (in-order):
-    //
-    //    country_code: string (for ex, 'us')
-    //    date: string (for ex, '2022-01-01')
-    //
-    // Table's folder structure:
-    //    us/
-    //     |- 2022-01-01/
-    //     |- 2022-01-02/
-    //     ...
-    //
-    // In case we have incoming query specifies following predicates:
-    //
-    //    `... WHERE country_code = 'us' AND date = '2022-01-01'`
-    //
-    // We can deduce full partition-path w/o doing a single listing: `us/2022-01-01`
-    if (areAllPartitionPathsCached || !shouldUsePartitionPathPrefixAnalysis(configProperties)) {
-      logDebug("All partition paths have already been cached, use it directly")
+  // NOTE: Here we try to to achieve efficiency in avoiding necessity to recursively list deep folder structures of
+  //       partitioned tables w/ multiple partition columns, by carefully analyzing provided partition predicates:
+  //
+  //       In cases when partition-predicates have
+  //         - The form of equality predicates w/ static literals (for ex, like `date = '2022-01-01'`)
+  //         - Fully specified proper prefix of the partition schema (ie fully binding first N columns
+  //           of the partition schema adhering to hereby described rules)
+  //
+  // We will try to exploit this specific structure, and try to reduce the scope of a
+  // necessary file-listings of partitions of the table to just the sub-folder under relative prefix
+  // of the partition-path derived from the partition-column predicates. For ex, consider following
+  // scenario:
+  //
+  // Table's partition schema (in-order):
+  //
+  //    country_code: string (for ex, 'us')
+  //    date: string (for ex, '2022-01-01')
+  //
+  // Table's folder structure:
+  //    us/
+  //     |- 2022-01-01/
+  //     |- 2022-01-02/
+  //     ...
+  //
+  // In case we have incoming query specifies following predicates:
+  //
+  //    `... WHERE country_code = 'us' AND date = '2022-01-01'`
+  //
+  // We can deduce full partition-path w/o doing a single listing: `us/2022-01-01`

Review Comment:
   correct



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r979366164


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();

Review Comment:
   Yes, I added comments to this.



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r979365431


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala:
##########
@@ -187,12 +189,16 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
    * @param predicates     The filter condition.
    * @return The pruned partition paths.
    */
-  protected def prunePartition(partitionPaths: Seq[PartitionPath], predicates: Seq[Expression]): Seq[PartitionPath] = {
+  protected def prunePartition(predicates: Seq[Expression]): Seq[PartitionPath] = {
     val partitionColumnNames = partitionSchema.fields.map(_.name).toSet
     val partitionPruningPredicates = predicates.filter {
       _.references.map(_.name).toSet.subsetOf(partitionColumnNames)
     }
     if (partitionPruningPredicates.nonEmpty) {
+      val equalPredicate = partitionPruningPredicates.filter(_.isInstanceOf[EqualTo])
+      val names = equalPredicate.map(_.asInstanceOf[EqualTo]).map(_.left.asInstanceOf[AttributeReference].name).toArray

Review Comment:
   Yes, but here the lazy file fetching is only possible for equal predicate (maybe also `in` predicate). The code is kind of hard-coded. Any better way to achieve this would be appreciated! Not very familiar with spark scala coding.



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1257142821

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d8495f35bf97ed4ec27a3841dad66307d862d65e Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475) 
   * 21b5c2377dd577e4ff8257f87fc0cda3f089c7d3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980881190


##########
hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java:
##########
@@ -74,8 +75,13 @@ public List<String> getAllPartitionPaths() throws IOException {
       return FSUtils.getAllPartitionFoldersThreeLevelsDown(fs, datasetBasePath);
     }
 
+    return getPartitionPathsWithPrefix("");
+  }
+
+  @Override
+  public List<String> getPartitionPathsWithPrefix(String prefix) throws IOException {
     List<Path> pathsToList = new CopyOnWriteArrayList<>();
-    pathsToList.add(basePath);
+    pathsToList.add(StringUtils.isNullOrEmpty(prefix) ? new Path(datasetBasePath) : new Path(datasetBasePath, prefix));

Review Comment:
   Actually, it could be a prefix of the name of the folder, e.g., prefix=`year=1992`, and the actual structure is `year=1992/month=xx/day=xx`.
   
   The reason I added a check here (i.e., Empty check) is that the Path constructor cannot handle empty "" input for its second parameter.



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1272532564

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fd5036ba6e5768ec3f846a41d766e776b8106551 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1285189590

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fd5036ba6e5768ec3f846a41d766e776b8106551 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087) 
   * a65056adaa4e9fabda4205c1b5c7be2e48bdd67f Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1294360119

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a09114e1c326791e33e910b2f660aaa6882dcfc9 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493) 
   * df0a3523720f59827e74c67354d465966e17e176 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r984065742


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNamesToIdx.containsKey(columnNames)) {
+        int k = partitionNamesToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");
+      } else {
+        break;
+      }
+    }
+    queryPartitionPath.deleteCharAt(queryPartitionPath.length() - 1);
+    // Return directly if all partition values are specified.
+    if (idx == this.partitionColumns.length) {

Review Comment:
   De-duplication we should perform in the method analyzing predicates (which is a caller of this method).
   
   It's not about a time complexity, it's more about the _code complexity_ -- making it invariant of the idx simplifies this code



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1311152024

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     }, {
       "hash" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937",
       "triggerID" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12938",
       "triggerID" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12942",
       "triggerID" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5341fff4dfb30afde48c370a7c6cb6e31b389539 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12938) 
   * 8bafc71d82607b40cb99505d145bbfddb8c81ae3 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12942) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1015834881


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java:
##########
@@ -269,56 +255,40 @@ private <S> S combineCompositeRecordKeyInternal(
     }
   }
 
-  private <S> S combinePartitionPathInternal(Supplier<StringBuilder<S>> builderFactory,
-                                             Function<Object, S> converter,
-                                             Function<S, S> encoder,
-                                             Function<S, S> emptyHandler,
-                                             Object... partitionPathParts) {
-    checkState(partitionPathParts.length == partitionPathFields.size());
-    // Avoid creating [[StringBuilder]] in case there's just one partition-path part,
-    // and Hive-style of partitioning is not required
-    if (!hiveStylePartitioning && partitionPathParts.length == 1) {
-      return emptyHandler.apply(converter.apply(partitionPathParts[0]));
-    }
-
-    StringBuilder<S> sb = builderFactory.get();
-    for (int i = 0; i < partitionPathParts.length; ++i) {
-      S partitionPathPartStr = encoder.apply(emptyHandler.apply(converter.apply(partitionPathParts[i])));
-
-      if (hiveStylePartitioning) {
-        sb.appendJava(partitionPathFields.get(i))
-            .appendJava("=")
-            .append(partitionPathPartStr);
-      } else {
-        sb.append(partitionPathPartStr);
-      }
-
-      if (i < partitionPathParts.length - 1) {
-        sb.appendJava(DEFAULT_PARTITION_PATH_SEPARATOR);
+  private void tryInitRowConverter(StructType structType) {

Review Comment:
   Check [this comment](https://github.com/apache/hudi/pull/6680/files#r1007526943) for more details



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1312193054

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     }, {
       "hash" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937",
       "triggerID" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12938",
       "triggerID" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12942",
       "triggerID" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f7e288c8e5e69a3a406c8e22761d752a068b7f3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12964",
       "triggerID" : "1f7e288c8e5e69a3a406c8e22761d752a068b7f3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8bafc71d82607b40cb99505d145bbfddb8c81ae3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12942) 
   * 1f7e288c8e5e69a3a406c8e22761d752a068b7f3 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12964) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] xushiyan commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
xushiyan commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1022550825


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/UTF8StringPartitionPathFormatter.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.keygen;
+
+import org.apache.hudi.common.util.PartitionPathEncodeUtils;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import java.util.function.Supplier;
+
+import static org.apache.hudi.keygen.BuiltinKeyGenerator.toUTF8String;

Review Comment:
   Not feeling too strong about this. Optional: update javadoc for PartitionPathFormatterBase to clarify its usage - it is to be used exclusively with KeyGen. 



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1314396354

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     }, {
       "hash" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937",
       "triggerID" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12938",
       "triggerID" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12942",
       "triggerID" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f7e288c8e5e69a3a406c8e22761d752a068b7f3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12964",
       "triggerID" : "1f7e288c8e5e69a3a406c8e22761d752a068b7f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e6cd657efc1ab3ac8e472b32b025d87be8671b5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2e6cd657efc1ab3ac8e472b32b025d87be8671b5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1f7e288c8e5e69a3a406c8e22761d752a068b7f3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12964) 
   * 2e6cd657efc1ab3ac8e472b32b025d87be8671b5 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1022034859


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/UTF8StringPartitionPathFormatter.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.keygen;
+
+import org.apache.hudi.common.util.PartitionPathEncodeUtils;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import java.util.function.Supplier;
+
+import static org.apache.hudi.keygen.BuiltinKeyGenerator.toUTF8String;

Review Comment:
   Good call. There was a reason why it didn't do it in the first place, but i can't recollect now. 
   Let me try again



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1023408717


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -179,17 +175,94 @@ public void close() throws Exception {
     resetTableMetadata(null);
   }
 
+  protected String[] getPartitionColumns() {
+    return partitionColumns;
+  }
+
+  protected List<Path> getQueryPaths() {
+    return queryPaths;
+  }
+
+  /**
+   * Returns all partition paths matching the ones explicitly provided by the query (if any)
+   */
   protected List<PartitionPath> getAllQueryPartitionPaths() {
-    List<String> queryRelativePartitionPaths = queryPaths.stream()
-        .map(path -> FSUtils.getRelativePartitionPath(basePath, path))
-        .collect(Collectors.toList());
+    if (cachedAllPartitionPaths == null) {
+      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))
-        .collect(Collectors.toList());
+      this.cachedAllPartitionPaths = listPartitionPaths(queryRelativePartitionPaths);
+    }
+
+    return cachedAllPartitionPaths;
+  }
+
+  /**
+   * Returns all listed file-slices w/in the partition paths returned by {@link #getAllQueryPartitionPaths()}
+   */
+  protected Map<PartitionPath, List<FileSlice>> getAllInputFileSlices() {
+    if (!areAllFileSlicesCached()) {
+      // Fetching file slices for partitions that have not been cached yet
+      List<PartitionPath> missingPartitions = getAllQueryPartitionPaths().stream()
+          .filter(p -> !cachedAllInputFileSlices.containsKey(p))
+          .collect(Collectors.toList());
+
+      // NOTE: Individual partitions are always cached in full, therefore if partition is cached
+      //       it will hold all the file-slices residing w/in the partition
+      cachedAllInputFileSlices.putAll(loadFileSlicesForPartitions(missingPartitions));
+    }
+
+    return cachedAllInputFileSlices;
+  }
+
+  /**
+   * Get input file slice for the given partition. Will use cache directly if it is computed before.
+   */
+  protected List<FileSlice> getInputFileSlices(PartitionPath partition) {
+    return cachedAllInputFileSlices.computeIfAbsent(partition,
+        p -> loadFileSlicesForPartitions(Collections.singletonList(p)).get(p));
+  }
+
+  private Map<PartitionPath, List<FileSlice>> loadFileSlicesForPartitions(List<PartitionPath> partitions) {
+    Map<PartitionPath, FileStatus[]> partitionFiles = partitions.stream()
+        .collect(Collectors.toMap(p -> p, this::loadPartitionPathFiles));
+    HoodieTimeline activeTimeline = getActiveTimeline();
+    Option<HoodieInstant> latestInstant = activeTimeline.lastInstant();
+
+    FileStatus[] allFiles = partitionFiles.values().stream().flatMap(Arrays::stream).toArray(FileStatus[]::new);
+    HoodieTableFileSystemView fileSystemView = new HoodieTableFileSystemView(metaClient, activeTimeline, allFiles);

Review Comment:
   Yeah, I notice the `loadPartitionPathFile` too. I'll check it with Alex and make sure it works for all cases. 
   
   Definitely good to hear your opinion and test feedback, Thanks!



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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1003677144


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -347,6 +417,17 @@ private void validate(HoodieTimeline activeTimeline, Option<String> queryInstant
     }
   }
 
+  protected boolean isAllInputFileSlicesCached() {
+    if (cachedAllPartitionPaths == null) {
+      return false;
+    }
+    return cachedAllPartitionPaths.stream().allMatch(p -> cachedAllInputFileSlices.containsKey(p));
+  }
+
+  protected boolean isPartitionedTable() {
+    return !queryAsNonePartitionedTable && (partitionColumns.length > 0 || HoodieTableMetadata.isMetadataTable(basePath.toString()));

Review Comment:
   I see. Makes sense, let's keep it then for now and tackle this later. Thanks for checking though!



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1304575624

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f12d60f65b7e33cf69783b80eac0463e62f6f170 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631) 
   * fe18f26f8ab63a28c98b23c6940b6135b6155d91 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1305050588

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 13b8d331e0dcf5ac1f958b91770e2024bce04c43 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819) 
   * c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r979365479


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)

Review Comment:
   Fixed, thanks!



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980861704


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +189,125 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
+    loadAllQueryPartitionPaths();
+    return this.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 = getQueryPartitionPaths(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::loadInputFileSlicesOfPartition);
+  }
+
+  private List<FileSlice> loadInputFileSlicesOfPartition(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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNameToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNameToIdx.containsKey(columnNames)) {
+        int k = partitionNameToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");
+      } else {
+        break;
+      }
+    }
+    queryPartitionPath.deleteCharAt(queryPartitionPath.length() - 1);
+    // Return directly if all partition values are specified.
+    if (idx == this.partitionColumns.length) {
+      return Collections.singletonList(new PartitionPath(queryPartitionPath.toString(), parsePartitionColumnValues(partitionColumns, queryPartitionPath.toString())));
+    }
+    // The predicate forms a prefix of partition path, do listing to the path only.
+    return getQueryPartitionPaths(Collections.singletonList(queryPartitionPath.toString()));
+  }
+
+  private List<PartitionPath> getQueryPartitionPaths(List<String> queryRelativePartitionPaths) {

Review Comment:
   Sure, fixed.



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1259500344

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * efc19bfcfb86bf582d4bd2584462083b8178c1c6 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1000193074


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -347,6 +417,17 @@ private void validate(HoodieTimeline activeTimeline, Option<String> queryInstant
     }
   }
 
+  protected boolean isAllInputFileSlicesCached() {
+    if (cachedAllPartitionPaths == null) {
+      return false;
+    }
+    return cachedAllPartitionPaths.stream().allMatch(p -> cachedAllInputFileSlices.containsKey(p));

Review Comment:
   Sure, added.



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1000200644


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -310,35 +256,159 @@ private void doRefresh() {
             )
         );
 
-    cachedFileSize = cachedAllInputFileSlices.values().stream()
+    this.cachedFileSize += ret.values().stream()
         .flatMap(Collection::stream)
         .mapToLong(BaseHoodieTableFileIndex::fileSliceSize)
         .sum();
 
-    // If the partition value contains InternalRow.empty, we query it as a non-partitioned table.
-    queryAsNonePartitionedTable = partitionFiles.keySet().stream().anyMatch(p -> p.values.length == 0);
+    return ret;
+  }
 
-    long duration = System.currentTimeMillis() - startTime;
+  /**
+   * 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 (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
 
-    LOG.info(String.format("Refresh table %s, spent: %d ms", metaClient.getTableConfig().getTableName(), duration));
+    Pair<String, Boolean> relativeQueryPartitionPathPair = composeRelativePartitionPaths(partitionNames, values);
+    // If the composed partition path is complete, we return it directly, to save extra DFS listing operations.
+    if (relativeQueryPartitionPathPair.getRight()) {
+      return Collections.singletonList(new PartitionPath(relativeQueryPartitionPathPair.getLeft(),
+          parsePartitionColumnValues(partitionColumns, relativeQueryPartitionPathPair.getLeft())));
+    }
+    // The input partition values (from query predicate) forms a prefix of partition path, do listing to the path only.
+    return listPartitionPaths(Collections.singletonList(relativeQueryPartitionPathPair.getLeft()));
   }
 
-  private Map<String, FileStatus[]> getAllFilesInPartitionsUnchecked(Collection<String> fullPartitionPathsMapToFetch) {
-    try {
-      return tableMetadata.getAllFilesInPartitions(new ArrayList<>(fullPartitionPathsMapToFetch));
-    } catch (IOException e) {
-      throw new HoodieIOException("Failed to list partition paths for a table", e);
+  /**
+   * Construct relative partition (i.e., partition prefix) from the given partition values

Review Comment:
   Fixed.



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -310,35 +256,159 @@ private void doRefresh() {
             )
         );
 
-    cachedFileSize = cachedAllInputFileSlices.values().stream()
+    this.cachedFileSize += ret.values().stream()
         .flatMap(Collection::stream)
         .mapToLong(BaseHoodieTableFileIndex::fileSliceSize)
         .sum();
 
-    // If the partition value contains InternalRow.empty, we query it as a non-partitioned table.
-    queryAsNonePartitionedTable = partitionFiles.keySet().stream().anyMatch(p -> p.values.length == 0);
+    return ret;
+  }
 
-    long duration = System.currentTimeMillis() - startTime;
+  /**
+   * 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) {

Review Comment:
   Yeah, sure!



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1000223772


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNamesToIdx.containsKey(columnNames)) {
+        int k = partitionNamesToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");
+      } else {
+        break;
+      }
+    }
+    queryPartitionPath.deleteCharAt(queryPartitionPath.length() - 1);
+    // Return directly if all partition values are specified.
+    if (idx == this.partitionColumns.length) {
+      return Collections.singletonList(new PartitionPath(queryPartitionPath.toString(), parsePartitionColumnValues(partitionColumns, queryPartitionPath.toString())));
+    }
+    return getQueryPartitionPaths(Collections.singletonList(queryPartitionPath.toString()));
+  }
+
+  private List<PartitionPath> getQueryPartitionPaths(List<String> queryRelativePartitionPaths) {
+    List<String> matchedPartitionPaths = queryRelativePartitionPaths.stream()
+        .flatMap(prefix -> {
+          try {
+            // Handle wildcard specially. This will have FileIndex to query the table as non-partitioned-table
+            if (prefix.contains("*")) {

Review Comment:
   I encountered one when running the UT. Let me remove it and check again.



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r979365943


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNamesToIdx.containsKey(columnNames)) {
+        int k = partitionNamesToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");

Review Comment:
   Yes, you are right. The current logic only handles predicates that could form the prefix of the partitions. If we want to handle the more general cases, we need also to modify the listing implementation inside the metadata table.
   
   In our use case, all queries will give predicate regarding the 1st partition field  first, then the 2nd and so on. And I believe it should cover most user cases. I could create a JIRA to track this if you think necessary.



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


[GitHub] [hudi] psendyk commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
psendyk commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1023403989


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -179,17 +175,94 @@ public void close() throws Exception {
     resetTableMetadata(null);
   }
 
+  protected String[] getPartitionColumns() {
+    return partitionColumns;
+  }
+
+  protected List<Path> getQueryPaths() {
+    return queryPaths;
+  }
+
+  /**
+   * Returns all partition paths matching the ones explicitly provided by the query (if any)
+   */
   protected List<PartitionPath> getAllQueryPartitionPaths() {
-    List<String> queryRelativePartitionPaths = queryPaths.stream()
-        .map(path -> FSUtils.getRelativePartitionPath(basePath, path))
-        .collect(Collectors.toList());
+    if (cachedAllPartitionPaths == null) {
+      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))
-        .collect(Collectors.toList());
+      this.cachedAllPartitionPaths = listPartitionPaths(queryRelativePartitionPaths);
+    }
+
+    return cachedAllPartitionPaths;
+  }
+
+  /**
+   * Returns all listed file-slices w/in the partition paths returned by {@link #getAllQueryPartitionPaths()}
+   */
+  protected Map<PartitionPath, List<FileSlice>> getAllInputFileSlices() {
+    if (!areAllFileSlicesCached()) {
+      // Fetching file slices for partitions that have not been cached yet
+      List<PartitionPath> missingPartitions = getAllQueryPartitionPaths().stream()
+          .filter(p -> !cachedAllInputFileSlices.containsKey(p))
+          .collect(Collectors.toList());
+
+      // NOTE: Individual partitions are always cached in full, therefore if partition is cached
+      //       it will hold all the file-slices residing w/in the partition
+      cachedAllInputFileSlices.putAll(loadFileSlicesForPartitions(missingPartitions));
+    }
+
+    return cachedAllInputFileSlices;
+  }
+
+  /**
+   * Get input file slice for the given partition. Will use cache directly if it is computed before.
+   */
+  protected List<FileSlice> getInputFileSlices(PartitionPath partition) {
+    return cachedAllInputFileSlices.computeIfAbsent(partition,
+        p -> loadFileSlicesForPartitions(Collections.singletonList(p)).get(p));
+  }
+
+  private Map<PartitionPath, List<FileSlice>> loadFileSlicesForPartitions(List<PartitionPath> partitions) {
+    Map<PartitionPath, FileStatus[]> partitionFiles = partitions.stream()
+        .collect(Collectors.toMap(p -> p, this::loadPartitionPathFiles));
+    HoodieTimeline activeTimeline = getActiveTimeline();
+    Option<HoodieInstant> latestInstant = activeTimeline.lastInstant();
+
+    FileStatus[] allFiles = partitionFiles.values().stream().flatMap(Arrays::stream).toArray(FileStatus[]::new);
+    HoodieTableFileSystemView fileSystemView = new HoodieTableFileSystemView(metaClient, activeTimeline, allFiles);

Review Comment:
   Yeah, that makes sense @YuweiXiao, I see how the caching works here. I corrected my initial statement above, i.e. I believe this behavior is actually triggered by calling `loadPartitionPathFiles` on L229 for each partition individually, so before the `HoodieTableFileSystemView` is initialized with the cached files on L234. If you go down the call chain starting from L229, you'll find that in `BaseTableMetadata.fetchAllFilesInPartition`, the call to `getRecordByKey` on L329, results in a call to `ensurePartitionLoadedCorrectly` with the `files` partition. However, along the way in `HoodieTableMetadataUtil.getPartitionFileSlices`, the filesystem view of the MT is re-initialized without any cached files. I believe this forces the `files` partitions to be recomputed each time. Each time the filesystem view is initialized, `partitionToFileGroupsMap` is initialized. I see that `partitionToFileGroupsMap` is being populated in `HoodieTableFileSystemView` from the provided files if you use
  the constructor on L177 but not in the one on L97 which is what `HoodieTableMetadataUtil.getPartitionFileSlices` uses. Not trying to derail this PR and please feel free to ignore if you don't the see the `files` partition being recomputed in your env.



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


[GitHub] [hudi] psendyk commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
psendyk commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1023448955


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -179,17 +175,94 @@ public void close() throws Exception {
     resetTableMetadata(null);
   }
 
+  protected String[] getPartitionColumns() {
+    return partitionColumns;
+  }
+
+  protected List<Path> getQueryPaths() {
+    return queryPaths;
+  }
+
+  /**
+   * Returns all partition paths matching the ones explicitly provided by the query (if any)
+   */
   protected List<PartitionPath> getAllQueryPartitionPaths() {
-    List<String> queryRelativePartitionPaths = queryPaths.stream()
-        .map(path -> FSUtils.getRelativePartitionPath(basePath, path))
-        .collect(Collectors.toList());
+    if (cachedAllPartitionPaths == null) {
+      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))
-        .collect(Collectors.toList());
+      this.cachedAllPartitionPaths = listPartitionPaths(queryRelativePartitionPaths);
+    }
+
+    return cachedAllPartitionPaths;
+  }
+
+  /**
+   * Returns all listed file-slices w/in the partition paths returned by {@link #getAllQueryPartitionPaths()}
+   */
+  protected Map<PartitionPath, List<FileSlice>> getAllInputFileSlices() {
+    if (!areAllFileSlicesCached()) {
+      // Fetching file slices for partitions that have not been cached yet
+      List<PartitionPath> missingPartitions = getAllQueryPartitionPaths().stream()
+          .filter(p -> !cachedAllInputFileSlices.containsKey(p))
+          .collect(Collectors.toList());
+
+      // NOTE: Individual partitions are always cached in full, therefore if partition is cached
+      //       it will hold all the file-slices residing w/in the partition
+      cachedAllInputFileSlices.putAll(loadFileSlicesForPartitions(missingPartitions));
+    }
+
+    return cachedAllInputFileSlices;
+  }
+
+  /**
+   * Get input file slice for the given partition. Will use cache directly if it is computed before.
+   */
+  protected List<FileSlice> getInputFileSlices(PartitionPath partition) {
+    return cachedAllInputFileSlices.computeIfAbsent(partition,
+        p -> loadFileSlicesForPartitions(Collections.singletonList(p)).get(p));
+  }
+
+  private Map<PartitionPath, List<FileSlice>> loadFileSlicesForPartitions(List<PartitionPath> partitions) {
+    Map<PartitionPath, FileStatus[]> partitionFiles = partitions.stream()
+        .collect(Collectors.toMap(p -> p, this::loadPartitionPathFiles));
+    HoodieTimeline activeTimeline = getActiveTimeline();
+    Option<HoodieInstant> latestInstant = activeTimeline.lastInstant();
+
+    FileStatus[] allFiles = partitionFiles.values().stream().flatMap(Arrays::stream).toArray(FileStatus[]::new);
+    HoodieTableFileSystemView fileSystemView = new HoodieTableFileSystemView(metaClient, activeTimeline, allFiles);

Review Comment:
   Gotcha, that makes sense. So from the perspective of the file listing, the issue is the number of partial lookups, right? Out of curiosity, would it be possible to do the lookups on a less granular level? For instance, if a table is partitioned by 2 columns but the query only provides one equality predicate with a partitioning column, the algorithm in this PR will first fetch all leaf partitions that match the predicate and then list the files in all leaf partitions one by one. Instead, could you do a single partial lookup just on the column provided in the predicate? I'm not too familiar with the internals of the files index so this is just for my own curiosity.
   
   Thank you both for the explanation and for implementing this. I'm still excited for this patch to land, even with MT disabled. For context, I have a table with a few PBs of data partitioned by 2 columns for a total of ~1.5M partitions. On 0.12.0 with MT enabled, the file listing takes ~15min regardless of the query. Most of the queries have an equality predicate on one of the columns, and when I tested this patch yesterday the listing with MT disabled only takes ~30s. 



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1317802568

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     }, {
       "hash" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937",
       "triggerID" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12938",
       "triggerID" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12942",
       "triggerID" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f7e288c8e5e69a3a406c8e22761d752a068b7f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12964",
       "triggerID" : "1f7e288c8e5e69a3a406c8e22761d752a068b7f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e6cd657efc1ab3ac8e472b32b025d87be8671b5",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=13012",
       "triggerID" : "2e6cd657efc1ab3ac8e472b32b025d87be8671b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1455e0679d0264cf960f1d064fefe5183e1cc2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4d1455e0679d0264cf960f1d064fefe5183e1cc2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2e6cd657efc1ab3ac8e472b32b025d87be8671b5 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=13012) 
   * 4d1455e0679d0264cf960f1d064fefe5183e1cc2 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] alexeykudinkin merged pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin merged PR #6680:
URL: https://github.com/apache/hudi/pull/6680


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r979365591


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNamesToIdx.containsKey(columnNames)) {
+        int k = partitionNamesToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");
+      } else {
+        break;
+      }
+    }
+    queryPartitionPath.deleteCharAt(queryPartitionPath.length() - 1);
+    // Return directly if all partition values are specified.
+    if (idx == this.partitionColumns.length) {

Review Comment:
   Do you mean before the queryPartitionPath construction? But the idx is valid only after the iteration.



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980656812


##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java:
##########
@@ -143,6 +143,13 @@ protected Option<HoodieRecord<HoodieMetadataPayload>> getRecordByKey(String key,
     return recordsByKeys.size() == 0 ? Option.empty() : recordsByKeys.get(0).getValue();
   }
 
+  @Override
+  public List<String> getPartitionPathsWithPrefix(String prefix) throws IOException {
+    return getAllPartitionPaths().stream()

Review Comment:
   Do you mean API like `getPartitionPathsWithPrefix(List<String> prefixes)`



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980668999


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNamesToIdx.containsKey(columnNames)) {
+        int k = partitionNamesToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");
+      } else {
+        break;
+      }
+    }
+    queryPartitionPath.deleteCharAt(queryPartitionPath.length() - 1);
+    // Return directly if all partition values are specified.
+    if (idx == this.partitionColumns.length) {

Review Comment:
   Yes, `partitionNames.length` may serve the purpose. But we also need to handle some corner cases, e.g., user give two conditions on the same partition.
   
   Regarding the time complexity, moving condition up cannot bring any benefits. I would say we could keep the current implementation. WDYT?



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980669969


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +189,125 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
+    loadAllQueryPartitionPaths();
+    return this.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 = getQueryPartitionPaths(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::loadInputFileSlicesOfPartition);
+  }
+
+  private List<FileSlice> loadInputFileSlicesOfPartition(PartitionPath p) {

Review Comment:
   Fixed.



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980869861


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala:
##########
@@ -187,12 +189,16 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
    * @param predicates     The filter condition.
    * @return The pruned partition paths.
    */
-  protected def prunePartition(partitionPaths: Seq[PartitionPath], predicates: Seq[Expression]): Seq[PartitionPath] = {
+  protected def prunePartition(predicates: Seq[Expression]): Seq[PartitionPath] = {

Review Comment:
   Yeah, improved.



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980886811


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +189,125 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
+    loadAllQueryPartitionPaths();
+    return this.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 = getQueryPartitionPaths(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::loadInputFileSlicesOfPartition);
+  }
+
+  private List<FileSlice> loadInputFileSlicesOfPartition(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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());

Review Comment:
   Like constructing a key-gen and have info from it instead?



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1000237144


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala:
##########
@@ -165,7 +164,7 @@ case class HoodieFileIndex(spark: SparkSession,
 
       logInfo(s"Total base files: $totalFileSize; " +
         s"candidate files after data skipping : $candidateFileSize; " +
-        s"skipping percent ${if (allFiles.nonEmpty && totalFileSize > 0) (totalFileSize - candidateFileSize) / totalFileSize.toDouble else 0}")
+        s"skipping percent ${if (!isAllInputFileSlicesCached) "is disable" else if (allFiles.nonEmpty && totalFileSize > 0) (totalFileSize - candidateFileSize) / totalFileSize.toDouble else 0}")

Review Comment:
   Fixed.



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1000224278


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala:
##########
@@ -138,6 +138,14 @@ object DataSourceReadOptions {
         " read from the data file (in Hudi partition columns are persisted by default)." +
         " This config is a fallback allowing to preserve existing behavior, and should not be used otherwise.")
 
+  val REFRESH_PARTITION_AND_FILES_IN_INITIALIZATION: ConfigProperty[Boolean] =

Review Comment:
   OK, will turn it on by default.



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


[GitHub] [hudi] YuweiXiao commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1288289575

   @hudi-bot run azure


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r990606063


##########
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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNameToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNameToIdx.containsKey(columnNames)) {
+        int k = partitionNameToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");
+      } else {
+        break;
+      }
+    }
+    queryPartitionPath.deleteCharAt(queryPartitionPath.length() - 1);
+    // Return directly if all partition values are specified.
+    if (idx == this.partitionColumns.length) {
+      return Collections.singletonList(new PartitionPath(queryPartitionPath.toString(), parsePartitionColumnValues(partitionColumns, queryPartitionPath.toString())));
+    }
+    // The predicate forms a prefix of partition path, do listing to the path only.
+    return listQueryPartitionPaths(Collections.singletonList(queryPartitionPath.toString()));
+  }
+
+  private List<PartitionPath> listQueryPartitionPaths(List<String> queryRelativePartitionPaths) {

Review Comment:
   I updated the name 
   
   The method will reference member variables and another abstract member function, i.e., `parsePartitionColumnValues`. To make it static, we may need to change the return signature to List<String> and leave `String` to `PartitionPath` conversion to the caller. What is your idea, Alex?



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1272269333

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * efc19bfcfb86bf582d4bd2584462083b8178c1c6 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783) 
   * 5f6d4f624c5f20cf3c4c38384e17c7bb13e56991 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1311053705

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     }, {
       "hash" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 59cdd09e3190c3646e1e3ea6ca3f076526ec0473 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912) 
   * 522204901637ca82f8232c63d9b26adce3f484b8 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1024669448


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -179,17 +175,94 @@ public void close() throws Exception {
     resetTableMetadata(null);
   }
 
+  protected String[] getPartitionColumns() {
+    return partitionColumns;
+  }
+
+  protected List<Path> getQueryPaths() {
+    return queryPaths;
+  }
+
+  /**
+   * Returns all partition paths matching the ones explicitly provided by the query (if any)
+   */
   protected List<PartitionPath> getAllQueryPartitionPaths() {
-    List<String> queryRelativePartitionPaths = queryPaths.stream()
-        .map(path -> FSUtils.getRelativePartitionPath(basePath, path))
-        .collect(Collectors.toList());
+    if (cachedAllPartitionPaths == null) {
+      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))
-        .collect(Collectors.toList());
+      this.cachedAllPartitionPaths = listPartitionPaths(queryRelativePartitionPaths);
+    }
+
+    return cachedAllPartitionPaths;
+  }
+
+  /**
+   * Returns all listed file-slices w/in the partition paths returned by {@link #getAllQueryPartitionPaths()}
+   */
+  protected Map<PartitionPath, List<FileSlice>> getAllInputFileSlices() {
+    if (!areAllFileSlicesCached()) {
+      // Fetching file slices for partitions that have not been cached yet
+      List<PartitionPath> missingPartitions = getAllQueryPartitionPaths().stream()
+          .filter(p -> !cachedAllInputFileSlices.containsKey(p))
+          .collect(Collectors.toList());
+
+      // NOTE: Individual partitions are always cached in full, therefore if partition is cached
+      //       it will hold all the file-slices residing w/in the partition
+      cachedAllInputFileSlices.putAll(loadFileSlicesForPartitions(missingPartitions));
+    }
+
+    return cachedAllInputFileSlices;
+  }
+
+  /**
+   * Get input file slice for the given partition. Will use cache directly if it is computed before.
+   */
+  protected List<FileSlice> getInputFileSlices(PartitionPath partition) {
+    return cachedAllInputFileSlices.computeIfAbsent(partition,
+        p -> loadFileSlicesForPartitions(Collections.singletonList(p)).get(p));
+  }
+
+  private Map<PartitionPath, List<FileSlice>> loadFileSlicesForPartitions(List<PartitionPath> partitions) {
+    Map<PartitionPath, FileStatus[]> partitionFiles = partitions.stream()
+        .collect(Collectors.toMap(p -> p, this::loadPartitionPathFiles));
+    HoodieTimeline activeTimeline = getActiveTimeline();
+    Option<HoodieInstant> latestInstant = activeTimeline.lastInstant();
+
+    FileStatus[] allFiles = partitionFiles.values().stream().flatMap(Arrays::stream).toArray(FileStatus[]::new);
+    HoodieTableFileSystemView fileSystemView = new HoodieTableFileSystemView(metaClient, activeTimeline, allFiles);

Review Comment:
   Yeah, `partial lookup` should be the next step. The current algorithm is more like a `prefix lookup`. 
   
   Just to make sure I understand you correctly, `partial lookup` means exploiting the knowledge of the predicates during the partition listing. Taking a table with two partition columns (e.g., year & month) as an example. If the predicate is on the month column only, e.g., `month=03`, the current `prefix lookup` still have to do a full listing. While the `partial lookup` could do a single listing on base path, and then padding the result with `month=03`.



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1311094657

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     }, {
       "hash" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937",
       "triggerID" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 59cdd09e3190c3646e1e3ea6ca3f076526ec0473 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912) 
   * 522204901637ca82f8232c63d9b26adce3f484b8 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1022045154


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/UTF8StringPartitionPathFormatter.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.keygen;
+
+import org.apache.hudi.common.util.PartitionPathEncodeUtils;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import java.util.function.Supplier;
+
+import static org.apache.hudi.keygen.BuiltinKeyGenerator.toUTF8String;

Review Comment:
   Alright, i recollected now:
   
   - PartitionPathFormatter is actually not completely decoupled from KG: partition-paths formatting make sense and only occurs w/in the scope of key-generation (record-key, partition-path). Outside of it there's no use-case for it. In other words, we're not planning to use it completely detached from the KG.
   - What you're saying makes total sense nonetheless, but then i weighed in whether it's worth to add a new Utility class for this 1 method and ultimately decided that it's not.
   - `SparkKeyGenUtils` has actually a slightly different scope -- it hosts utils to work w/ KGs rather than internal helpers for KGs
   
   Let me know if you feel strongly about it, i can extract to a separate utility class.
   



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1304576431

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f12d60f65b7e33cf69783b80eac0463e62f6f170 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631) 
   * fe18f26f8ab63a28c98b23c6940b6135b6155d91 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980670227


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +189,125 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
+    loadAllQueryPartitionPaths();
+    return this.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 = getQueryPartitionPaths(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::loadInputFileSlicesOfPartition);
+  }
+
+  private List<FileSlice> loadInputFileSlicesOfPartition(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));

Review Comment:
   Sure~



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980863729


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +189,125 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
+    loadAllQueryPartitionPaths();
+    return this.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 = getQueryPartitionPaths(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::loadInputFileSlicesOfPartition);
+  }
+
+  private List<FileSlice> loadInputFileSlicesOfPartition(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));

Review Comment:
   Just came to me mind, java does not support lazy field natively. Do you mean a separate get function with lazy initialization?



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1259170237

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 21b5c2377dd577e4ff8257f87fc0cda3f089c7d3 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708) 
   * efc19bfcfb86bf582d4bd2584462083b8178c1c6 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980851604


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  protected Map<PartitionPath, List<FileSlice>> getAllInputFileSlices() {
+    if (!isAllInputFileSlicesCached) {

Review Comment:
   Not sure if `shouldRefreshLazily` is appropriate. E.g., in `shouldRefreshLazily=true` case, calling getAllInputFileSlices multiple times will trigger multiple `refresh`, though the interface seems only used in `non-partitioned-table`.



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1000193983


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -298,7 +244,7 @@ private void doRefresh() {
     // This logic is realized by `AbstractTableFileSystemView::getLatestMergedFileSlicesBeforeOrOn`
     // API.  Note that for COW table, the merging logic of two slices does not happen as there
     // is no compaction, thus there is no performance impact.
-    cachedAllInputFileSlices = partitionFiles.keySet().stream()
+    Map<PartitionPath, List<FileSlice>> ret = partitionFiles.keySet().stream()

Review Comment:
   Fixed.



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1286196069

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a65056adaa4e9fabda4205c1b5c7be2e48bdd67f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r984047423


##########
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);

Review Comment:
   We don't need this field anymore we can use `isPartitionedTable` method



##########
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:
   Let's actually extract composing of the relative partition path (from values) into a standalone method. Then we can get eliminate this one and then just do:
   
   ```
   val relativePaths = composeRelativePartitionPaths(partitionCols, partitionVals)
   listQueryPartitionPaths(relativePaths)
   ```



##########
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();

Review Comment:
   Let's do following:
   
    - Make `loadAllQueryPartitionPaths` a) static, b) accept list of paths to list, c) return listed partition paths (this will be necessary in other place as well)
    - Reshape our conditional here like following:
   
   ```
   if (cached == null) {
     this.cached = load(...)
   }
   
   return cached;
   ```



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -138,7 +143,20 @@ public BaseHoodieTableFileIndex(HoodieEngineContext engineContext,
     this.engineContext = engineContext;
     this.fileStatusCache = fileStatusCache;
 
-    doRefresh();
+    /**
+     * The `shouldRefresh` variable controls how we initialize the TableFileIndex:

Review Comment:
   Let's actually 
    - Call it `shouldListLazily`
    - Rename `isAllInputFileSlicesCached` accordingly



##########
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) {

Review Comment:
   This will refresh the whole cache. We don't want that, instead we want to load the partition paths we haven't already listed. 



##########
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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNameToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNameToIdx.containsKey(columnNames)) {
+        int k = partitionNameToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");
+      } else {
+        break;
+      }
+    }
+    queryPartitionPath.deleteCharAt(queryPartitionPath.length() - 1);
+    // Return directly if all partition values are specified.
+    if (idx == this.partitionColumns.length) {
+      return Collections.singletonList(new PartitionPath(queryPartitionPath.toString(), parsePartitionColumnValues(partitionColumns, queryPartitionPath.toString())));
+    }
+    // The predicate forms a prefix of partition path, do listing to the path only.
+    return listQueryPartitionPaths(Collections.singletonList(queryPartitionPath.toString()));
+  }
+
+  private List<PartitionPath> listQueryPartitionPaths(List<String> queryRelativePartitionPaths) {

Review Comment:
   Let's rename this to be more generic `listPartitionPath` (it's listing any paths not only query ones)



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNamesToIdx.containsKey(columnNames)) {
+        int k = partitionNamesToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");
+      } else {
+        break;
+      }
+    }
+    queryPartitionPath.deleteCharAt(queryPartitionPath.length() - 1);
+    // Return directly if all partition values are specified.
+    if (idx == this.partitionColumns.length) {

Review Comment:
   De-duplication we should be perform in the method analyzing predicates (which is a caller of this method).
   
   It's not about a time complexity, it's more about the _code complexity_ -- making it invariant of the idx simplifies this code



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +189,125 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
+    loadAllQueryPartitionPaths();
+    return this.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 = getQueryPartitionPaths(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::loadInputFileSlicesOfPartition);
+  }
+
+  private List<FileSlice> loadInputFileSlicesOfPartition(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));

Review Comment:
   There's already utility called `Lazy`
   
   P.S. I actually realized we can't use Lazy here b/c this object needs to be serializable (and Lazy could not be unfortunately)



##########
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();

Review Comment:
   Actually, reading on my comment here seems like we can actually inline `loadAllQueryPartitionPaths` it altogether after we remove `queryAsNonePartitionedTable`



##########
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) {

Review Comment:
   We already have that method actually (`getCachedInputFileSlices`) we just need to generalize it to be able to batch-list partitions



##########
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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNameToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNameToIdx.containsKey(columnNames)) {
+        int k = partitionNameToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");
+      } else {
+        break;
+      }
+    }
+    queryPartitionPath.deleteCharAt(queryPartitionPath.length() - 1);
+    // Return directly if all partition values are specified.
+    if (idx == this.partitionColumns.length) {
+      return Collections.singletonList(new PartitionPath(queryPartitionPath.toString(), parsePartitionColumnValues(partitionColumns, queryPartitionPath.toString())));
+    }
+    // The predicate forms a prefix of partition path, do listing to the path only.
+    return listQueryPartitionPaths(Collections.singletonList(queryPartitionPath.toString()));
+  }
+
+  private List<PartitionPath> listQueryPartitionPaths(List<String> queryRelativePartitionPaths) {

Review Comment:
   We can also make it static



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -323,30 +424,13 @@ private void doRefresh() {
         .mapToLong(BaseHoodieTableFileIndex::fileSliceSize)
         .sum();
 
-    // If the partition value contains InternalRow.empty, we query it as a non-partitioned table.
-    queryAsNonePartitionedTable = partitionFiles.keySet().stream().anyMatch(p -> p.values.length == 0);

Review Comment:
   Correct



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1250219495

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1250231508

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473) 
   * d8495f35bf97ed4ec27a3841dad66307d862d65e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r990601573


##########
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();

Review Comment:
   Yes, you are right. I will have it inlined.



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r990605731


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -138,7 +143,20 @@ public BaseHoodieTableFileIndex(HoodieEngineContext engineContext,
     this.engineContext = engineContext;
     this.fileStatusCache = fileStatusCache;
 
-    doRefresh();
+    /**
+     * The `shouldRefresh` variable controls how we initialize the TableFileIndex:

Review Comment:
   I removed `isAllInputFileSlicesCached ` and have following logic to check is all file slices cached:
   
   ```
   if (cachedAllPartitionPaths == null) {
         return false;
       }
   return cachedAllPartitionPaths.stream().allMatch(p -> cachedAllInputFileSlices.containsKey(p));
   ```
   
   Basically, we check if all partitions are loaded. Then we check if all partitions is contained in the `cachedAllInputFileSlices`. It should be cleaner instead of maintaining a separate flag variable.



##########
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);

Review Comment:
   Fixed.



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1312298377

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     }, {
       "hash" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937",
       "triggerID" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12938",
       "triggerID" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12942",
       "triggerID" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f7e288c8e5e69a3a406c8e22761d752a068b7f3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12964",
       "triggerID" : "1f7e288c8e5e69a3a406c8e22761d752a068b7f3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1f7e288c8e5e69a3a406c8e22761d752a068b7f3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12964) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1023389281


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -179,17 +175,94 @@ public void close() throws Exception {
     resetTableMetadata(null);
   }
 
+  protected String[] getPartitionColumns() {
+    return partitionColumns;
+  }
+
+  protected List<Path> getQueryPaths() {
+    return queryPaths;
+  }
+
+  /**
+   * Returns all partition paths matching the ones explicitly provided by the query (if any)
+   */
   protected List<PartitionPath> getAllQueryPartitionPaths() {
-    List<String> queryRelativePartitionPaths = queryPaths.stream()
-        .map(path -> FSUtils.getRelativePartitionPath(basePath, path))
-        .collect(Collectors.toList());
+    if (cachedAllPartitionPaths == null) {
+      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))
-        .collect(Collectors.toList());
+      this.cachedAllPartitionPaths = listPartitionPaths(queryRelativePartitionPaths);
+    }
+
+    return cachedAllPartitionPaths;
+  }
+
+  /**
+   * Returns all listed file-slices w/in the partition paths returned by {@link #getAllQueryPartitionPaths()}
+   */
+  protected Map<PartitionPath, List<FileSlice>> getAllInputFileSlices() {
+    if (!areAllFileSlicesCached()) {
+      // Fetching file slices for partitions that have not been cached yet
+      List<PartitionPath> missingPartitions = getAllQueryPartitionPaths().stream()
+          .filter(p -> !cachedAllInputFileSlices.containsKey(p))
+          .collect(Collectors.toList());
+
+      // NOTE: Individual partitions are always cached in full, therefore if partition is cached
+      //       it will hold all the file-slices residing w/in the partition
+      cachedAllInputFileSlices.putAll(loadFileSlicesForPartitions(missingPartitions));
+    }
+
+    return cachedAllInputFileSlices;
+  }
+
+  /**
+   * Get input file slice for the given partition. Will use cache directly if it is computed before.
+   */
+  protected List<FileSlice> getInputFileSlices(PartitionPath partition) {
+    return cachedAllInputFileSlices.computeIfAbsent(partition,
+        p -> loadFileSlicesForPartitions(Collections.singletonList(p)).get(p));
+  }
+
+  private Map<PartitionPath, List<FileSlice>> loadFileSlicesForPartitions(List<PartitionPath> partitions) {
+    Map<PartitionPath, FileStatus[]> partitionFiles = partitions.stream()
+        .collect(Collectors.toMap(p -> p, this::loadPartitionPathFiles));
+    HoodieTimeline activeTimeline = getActiveTimeline();
+    Option<HoodieInstant> latestInstant = activeTimeline.lastInstant();
+
+    FileStatus[] allFiles = partitionFiles.values().stream().flatMap(Arrays::stream).toArray(FileStatus[]::new);
+    HoodieTableFileSystemView fileSystemView = new HoodieTableFileSystemView(metaClient, activeTimeline, allFiles);

Review Comment:
   In `ensurePartitionLoadedCorrectly`, we will check `isPartitionAvailableInStore` for the input partition (L303 in `AbstractTableFileSystemView`), which will hold for every input partition because we construct the `HoodieTableFileSystemView` with cached file status (i.e. L234 in `BaseHoodieTableFileIndex`).
   
   I checked the 0.12.0 version, the `getLatestMergedFileSlicesBeforeOrOn` is also called for each data partition (i.e., in the refresh() method). As Alex said, the file index 'manually' list all file status and the `FileSystemView` is only responsible for projections. 
   
   So if you see regression (i.e., spend much more time in this patch compared to 0.12.0), I guess it is not caused by the `FileSystemView` you mentioned. 



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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1023416704


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -179,17 +175,94 @@ public void close() throws Exception {
     resetTableMetadata(null);
   }
 
+  protected String[] getPartitionColumns() {
+    return partitionColumns;
+  }
+
+  protected List<Path> getQueryPaths() {
+    return queryPaths;
+  }
+
+  /**
+   * Returns all partition paths matching the ones explicitly provided by the query (if any)
+   */
   protected List<PartitionPath> getAllQueryPartitionPaths() {
-    List<String> queryRelativePartitionPaths = queryPaths.stream()
-        .map(path -> FSUtils.getRelativePartitionPath(basePath, path))
-        .collect(Collectors.toList());
+    if (cachedAllPartitionPaths == null) {
+      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))
-        .collect(Collectors.toList());
+      this.cachedAllPartitionPaths = listPartitionPaths(queryRelativePartitionPaths);
+    }
+
+    return cachedAllPartitionPaths;
+  }
+
+  /**
+   * Returns all listed file-slices w/in the partition paths returned by {@link #getAllQueryPartitionPaths()}
+   */
+  protected Map<PartitionPath, List<FileSlice>> getAllInputFileSlices() {
+    if (!areAllFileSlicesCached()) {
+      // Fetching file slices for partitions that have not been cached yet
+      List<PartitionPath> missingPartitions = getAllQueryPartitionPaths().stream()
+          .filter(p -> !cachedAllInputFileSlices.containsKey(p))
+          .collect(Collectors.toList());
+
+      // NOTE: Individual partitions are always cached in full, therefore if partition is cached
+      //       it will hold all the file-slices residing w/in the partition
+      cachedAllInputFileSlices.putAll(loadFileSlicesForPartitions(missingPartitions));
+    }
+
+    return cachedAllInputFileSlices;
+  }
+
+  /**
+   * Get input file slice for the given partition. Will use cache directly if it is computed before.
+   */
+  protected List<FileSlice> getInputFileSlices(PartitionPath partition) {
+    return cachedAllInputFileSlices.computeIfAbsent(partition,
+        p -> loadFileSlicesForPartitions(Collections.singletonList(p)).get(p));
+  }
+
+  private Map<PartitionPath, List<FileSlice>> loadFileSlicesForPartitions(List<PartitionPath> partitions) {
+    Map<PartitionPath, FileStatus[]> partitionFiles = partitions.stream()
+        .collect(Collectors.toMap(p -> p, this::loadPartitionPathFiles));
+    HoodieTimeline activeTimeline = getActiveTimeline();
+    Option<HoodieInstant> latestInstant = activeTimeline.lastInstant();
+
+    FileStatus[] allFiles = partitionFiles.values().stream().flatMap(Arrays::stream).toArray(FileStatus[]::new);
+    HoodieTableFileSystemView fileSystemView = new HoodieTableFileSystemView(metaClient, activeTimeline, allFiles);

Review Comment:
   Oh, i see your point now @psendyk!
   
   Yes, that's an issue w/ how MT caching is working -- currently, whenever you do partial lookup (as compared to parsing whole files partition) it will clear the cache before proceeding essentially re-parsing files partition every time.
   
   I have a [PR](https://github.com/apache/hudi/pull/6782) out to address this, but need to put in a few more cycles into it to address the issues carefully.
   
   



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1250254510

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d8495f35bf97ed4ec27a3841dad66307d862d65e Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980885367


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -323,30 +424,13 @@ private void doRefresh() {
         .mapToLong(BaseHoodieTableFileIndex::fileSliceSize)
         .sum();
 
-    // If the partition value contains InternalRow.empty, we query it as a non-partitioned table.
-    queryAsNonePartitionedTable = partitionFiles.keySet().stream().anyMatch(p -> p.values.length == 0);

Review Comment:
   Do you mean remove the variable at all, and have `isPartitionedTable ` to play its functionality?



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980864178


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala:
##########
@@ -165,7 +164,7 @@ case class HoodieFileIndex(spark: SparkSession,
 
       logInfo(s"Total base files: $totalFileSize; " +
         s"candidate files after data skipping : $candidateFileSize; " +
-        s"skipping percent ${if (allFiles.nonEmpty && totalFileSize > 0) (totalFileSize - candidateFileSize) / totalFileSize.toDouble else 0}")
+        s"skipping percent ${if (isAllInputFileSlicesCached && allFiles.nonEmpty && totalFileSize > 0) (totalFileSize - candidateFileSize) / totalFileSize.toDouble else 0}")

Review Comment:
   Yeah, fixed.



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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r979062168


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -221,60 +329,46 @@ protected HoodieTimeline getActiveTimeline() {
   }
 
   /**
-   * Load all partition paths and it's files under the query table path.
+   * Load 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 =
-          getAllFilesInPartitionsUnchecked(fullPartitionPathsMapToFetch.keySet())
-              .entrySet()
-              .stream()
-              .collect(Collectors.toMap(e -> fullPartitionPathsMapToFetch.get(e.getKey()), e -> e.getValue()));
-
+  private FileStatus[] loadPartitionPathFiles(PartitionPath partition) {
+    // Try fetch from the FileStatusCache first
+    Option<FileStatus[]> files = fileStatusCache.get(partition.fullPartitionPath(basePath));
+    if (files.isPresent()) {
+      return files.get();
     }
 
-    // Update the fileStatusCache
-    fetchedPartitionToFiles.forEach((partitionPath, filesInPartition) -> {
-      fileStatusCache.put(partitionPath.fullPartitionPath(basePath), filesInPartition);
-    });
+    try {
+      Path path = partition.fullPartitionPath(basePath);
+      FileStatus[] fetchedFiles = tableMetadata.getAllFilesInPartition(path);
 
-    return CollectionUtils.combine(cachedPartitionToFiles, fetchedPartitionToFiles);
+      // Update the fileStatusCache
+      fileStatusCache.put(partition.fullPartitionPath(basePath), fetchedFiles);
+      return fetchedFiles;
+    } catch (IOException e) {
+      throw new HoodieIOException("Failed to list partition path (" + partition + ") for a table", e);
+    }
   }
 
   private void doRefresh() {
+    doRefresh(false);
+  }
+  
+  private void doRefresh(boolean initMetadataOnly) {

Review Comment:
   Instead of boolean, let's split this method into 2:
   
    - One init-ing only MT
    - One also doing file-listing



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(queryRelativePartitionPaths);

Review Comment:
   Let's extract any state modifications to a separate method w/ explicit contract to do so: for ex, previously all the loading has been done by `loadX` methods invoked from `doRefresh`
   
   We should not be modifying the state w/in getters in this case



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  protected Map<PartitionPath, List<FileSlice>> getAllInputFileSlices() {
+    if (!isAllInputFileSlicesCached) {

Review Comment:
   Why do we need separate boolean for that?



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (p) -> {
+      FileStatus[] files = loadPartitionPathFiles(p);

Review Comment:
   Let's extract actual listing to a standalone method



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala:
##########
@@ -187,12 +189,16 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
    * @param predicates     The filter condition.
    * @return The pruned partition paths.
    */
-  protected def prunePartition(partitionPaths: Seq[PartitionPath], predicates: Seq[Expression]): Seq[PartitionPath] = {
+  protected def prunePartition(predicates: Seq[Expression]): Seq[PartitionPath] = {
     val partitionColumnNames = partitionSchema.fields.map(_.name).toSet
     val partitionPruningPredicates = predicates.filter {
       _.references.map(_.name).toSet.subsetOf(partitionColumnNames)
     }
     if (partitionPruningPredicates.nonEmpty) {
+      val equalPredicate = partitionPruningPredicates.filter(_.isInstanceOf[EqualTo])
+      val names = equalPredicate.map(_.asInstanceOf[EqualTo]).map(_.left.asInstanceOf[AttributeReference].name).toArray

Review Comment:
   We can not assume what those predicates might be



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNamesToIdx.containsKey(columnNames)) {
+        int k = partitionNamesToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");
+      } else {
+        break;
+      }
+    }
+    queryPartitionPath.deleteCharAt(queryPartitionPath.length() - 1);
+    // Return directly if all partition values are specified.
+    if (idx == this.partitionColumns.length) {

Review Comment:
   We can move this conditional up before the iteration



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNamesToIdx.containsKey(columnNames)) {
+        int k = partitionNamesToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");
+      } else {
+        break;
+      }
+    }
+    queryPartitionPath.deleteCharAt(queryPartitionPath.length() - 1);
+    // Return directly if all partition values are specified.
+    if (idx == this.partitionColumns.length) {
+      return Collections.singletonList(new PartitionPath(queryPartitionPath.toString(), parsePartitionColumnValues(partitionColumns, queryPartitionPath.toString())));
+    }
+    return getQueryPartitionPaths(Collections.singletonList(queryPartitionPath.toString()));
+  }
+
+  private List<PartitionPath> getQueryPartitionPaths(List<String> queryRelativePartitionPaths) {
+    List<String> matchedPartitionPaths = queryRelativePartitionPaths.stream()
+        .flatMap(prefix -> {
+          try {
+            // Handle wildcard specially. This will have FileIndex to query the table as non-partitioned-table
+            if (prefix.contains("*")) {

Review Comment:
   How could there be a wildcard?



##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java:
##########
@@ -143,6 +143,13 @@ protected Option<HoodieRecord<HoodieMetadataPayload>> getRecordByKey(String key,
     return recordsByKeys.size() == 0 ? Option.empty() : recordsByKeys.get(0).getValue();
   }
 
+  @Override
+  public List<String> getPartitionPathsWithPrefix(String prefix) throws IOException {
+    return getAllPartitionPaths().stream()

Review Comment:
   This is going to be incredibly wasteful in case of MT due to HUDI-4911



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();

Review Comment:
   Let's make it clear that this could be a path's prefix



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)

Review Comment:
   Should be singular (`partitionNameToIdx`)



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNamesToIdx.containsKey(columnNames)) {
+        int k = partitionNamesToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");

Review Comment:
   If i understood your intention correctly, then there would be a following problems: in the partition path ordering of the column is fixed, therefore if query specifies partition column predicates which only referenced columns that do not constitute the prefix of the actual partition path, then this would not work. For ex, let's say there are 2 partition columns A and B, and the path looks like `A=foo/B=bar`, but then the query only specifies predicate containing only B



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980669333


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +189,125 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {

Review Comment:
   Ah ok, will fix!



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1317807709

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     }, {
       "hash" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937",
       "triggerID" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12938",
       "triggerID" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12942",
       "triggerID" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f7e288c8e5e69a3a406c8e22761d752a068b7f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12964",
       "triggerID" : "1f7e288c8e5e69a3a406c8e22761d752a068b7f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e6cd657efc1ab3ac8e472b32b025d87be8671b5",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=13012",
       "triggerID" : "2e6cd657efc1ab3ac8e472b32b025d87be8671b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1455e0679d0264cf960f1d064fefe5183e1cc2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=13065",
       "triggerID" : "4d1455e0679d0264cf960f1d064fefe5183e1cc2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2e6cd657efc1ab3ac8e472b32b025d87be8671b5 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=13012) 
   * 4d1455e0679d0264cf960f1d064fefe5183e1cc2 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=13065) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1285179822

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fd5036ba6e5768ec3f846a41d766e776b8106551 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087) 
   * a65056adaa4e9fabda4205c1b5c7be2e48bdd67f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1286861133

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 903ac79702d02fe968d271b08a217f8401d63700 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1288306886

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * a09114e1c326791e33e910b2f660aaa6882dcfc9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1309472827

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822) 
   * 59cdd09e3190c3646e1e3ea6ca3f076526ec0473 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1272506886

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5f6d4f624c5f20cf3c4c38384e17c7bb13e56991 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065) 
   * fd5036ba6e5768ec3f846a41d766e776b8106551 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] xushiyan commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
xushiyan commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1021670354


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/UTF8StringPartitionPathFormatter.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.keygen;
+
+import org.apache.hudi.common.util.PartitionPathEncodeUtils;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import java.util.function.Supplier;
+
+import static org.apache.hudi.keygen.BuiltinKeyGenerator.toUTF8String;

Review Comment:
   import this here is a bit against the hierarchy where BuiltinKeyGenerator uses PartitionPathFormatter. how about move  `toUTF8String` to `SparkKeyGenUtils`?



##########
hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java:
##########
@@ -74,8 +76,23 @@ public List<String> getAllPartitionPaths() throws IOException {
       return FSUtils.getAllPartitionFoldersThreeLevelsDown(fs, datasetBasePath);
     }
 
+    return getPartitionPathsWithPrefixes(Collections.singletonList(""));
+  }
+
+  @Override
+  public List<String> getPartitionPathsWithPrefixes(List<String> prefixes) {
+    return prefixes.stream().flatMap(prefix -> {
+      try {
+        return getPartitionPathsWithPrefix(prefix).stream();
+      } catch (IOException e) {
+        throw new HoodieIOException("Error fetching partition paths with prefix: " + prefix, e);
+      }
+    }).collect(Collectors.toList());
+  }
+
+  public List<String> getPartitionPathsWithPrefix(String prefix) throws IOException {

Review Comment:
   can be private as it's internal specifics of this class



##########
hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetFileFormat.scala:
##########
@@ -22,6 +22,7 @@ import org.apache.hadoop.fs.Path
 import org.apache.hadoop.mapreduce.lib.input.FileSplit
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
 import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.hudi.DataSourceReadOptions

Review Comment:
   unused import



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1022214084


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala:
##########
@@ -237,70 +246,64 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
     }
   }
 
-  private def listMatchingPartitionPathsInternal(partitionColumnNames: Seq[String],
-                                                 partitionColumnPredicates: Seq[Expression]): Seq[PartitionPath] = {
-    // NOTE: Here we try to to achieve efficiency in avoiding necessity to recursively list deep folder structures of
-    //       partitioned tables w/ multiple partition columns, by carefully analyzing provided partition predicates:
-    //
-    //       In cases when partition-predicates have
-    //         - The form of equality predicates w/ static literals (for ex, like `date = '2022-01-01'`)
-    //         - Fully specified proper prefix of the partition schema (ie fully binding first N columns
-    //           of the partition schema adhering to hereby described rules)
-    //
-    // We will try to exploit this specific structure, and try to reduce the scope of a
-    // necessary file-listings of partitions of the table to just the sub-folder under relative prefix
-    // of the partition-path derived from the partition-column predicates. For ex, consider following
-    // scenario:
-    //
-    // Table's partition schema (in-order):
-    //
-    //    country_code: string (for ex, 'us')
-    //    date: string (for ex, '2022-01-01')
-    //
-    // Table's folder structure:
-    //    us/
-    //     |- 2022-01-01/
-    //     |- 2022-01-02/
-    //     ...
-    //
-    // In case we have incoming query specifies following predicates:
-    //
-    //    `... WHERE country_code = 'us' AND date = '2022-01-01'`
-    //
-    // We can deduce full partition-path w/o doing a single listing: `us/2022-01-01`
-    if (areAllPartitionPathsCached || !shouldUsePartitionPathPrefixAnalysis(configProperties)) {
-      logDebug("All partition paths have already been cached, use it directly")
+  // NOTE: Here we try to to achieve efficiency in avoiding necessity to recursively list deep folder structures of
+  //       partitioned tables w/ multiple partition columns, by carefully analyzing provided partition predicates:
+  //
+  //       In cases when partition-predicates have
+  //         - The form of equality predicates w/ static literals (for ex, like `date = '2022-01-01'`)
+  //         - Fully specified proper prefix of the partition schema (ie fully binding first N columns
+  //           of the partition schema adhering to hereby described rules)
+  //
+  // We will try to exploit this specific structure, and try to reduce the scope of a
+  // necessary file-listings of partitions of the table to just the sub-folder under relative prefix
+  // of the partition-path derived from the partition-column predicates. For ex, consider following
+  // scenario:
+  //
+  // Table's partition schema (in-order):
+  //
+  //    country_code: string (for ex, 'us')
+  //    date: string (for ex, '2022-01-01')
+  //
+  // Table's folder structure:
+  //    us/
+  //     |- 2022-01-01/
+  //     |- 2022-01-02/
+  //     ...
+  //
+  // In case we have incoming query specifies following predicates:
+  //
+  //    `... WHERE country_code = 'us' AND date = '2022-01-01'`
+  //
+  // We can deduce full partition-path w/o doing a single listing: `us/2022-01-01`

Review Comment:
   looks like a typo. _w/o_ means _without_? 



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


[GitHub] [hudi] psendyk commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
psendyk commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1023090340


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -179,17 +175,94 @@ public void close() throws Exception {
     resetTableMetadata(null);
   }
 
+  protected String[] getPartitionColumns() {
+    return partitionColumns;
+  }
+
+  protected List<Path> getQueryPaths() {
+    return queryPaths;
+  }
+
+  /**
+   * Returns all partition paths matching the ones explicitly provided by the query (if any)
+   */
   protected List<PartitionPath> getAllQueryPartitionPaths() {
-    List<String> queryRelativePartitionPaths = queryPaths.stream()
-        .map(path -> FSUtils.getRelativePartitionPath(basePath, path))
-        .collect(Collectors.toList());
+    if (cachedAllPartitionPaths == null) {
+      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))
-        .collect(Collectors.toList());
+      this.cachedAllPartitionPaths = listPartitionPaths(queryRelativePartitionPaths);
+    }
+
+    return cachedAllPartitionPaths;
+  }
+
+  /**
+   * Returns all listed file-slices w/in the partition paths returned by {@link #getAllQueryPartitionPaths()}
+   */
+  protected Map<PartitionPath, List<FileSlice>> getAllInputFileSlices() {
+    if (!areAllFileSlicesCached()) {
+      // Fetching file slices for partitions that have not been cached yet
+      List<PartitionPath> missingPartitions = getAllQueryPartitionPaths().stream()
+          .filter(p -> !cachedAllInputFileSlices.containsKey(p))
+          .collect(Collectors.toList());
+
+      // NOTE: Individual partitions are always cached in full, therefore if partition is cached
+      //       it will hold all the file-slices residing w/in the partition
+      cachedAllInputFileSlices.putAll(loadFileSlicesForPartitions(missingPartitions));
+    }
+
+    return cachedAllInputFileSlices;
+  }
+
+  /**
+   * Get input file slice for the given partition. Will use cache directly if it is computed before.
+   */
+  protected List<FileSlice> getInputFileSlices(PartitionPath partition) {
+    return cachedAllInputFileSlices.computeIfAbsent(partition,
+        p -> loadFileSlicesForPartitions(Collections.singletonList(p)).get(p));
+  }
+
+  private Map<PartitionPath, List<FileSlice>> loadFileSlicesForPartitions(List<PartitionPath> partitions) {
+    Map<PartitionPath, FileStatus[]> partitionFiles = partitions.stream()
+        .collect(Collectors.toMap(p -> p, this::loadPartitionPathFiles));
+    HoodieTimeline activeTimeline = getActiveTimeline();
+    Option<HoodieInstant> latestInstant = activeTimeline.lastInstant();
+
+    FileStatus[] allFiles = partitionFiles.values().stream().flatMap(Arrays::stream).toArray(FileStatus[]::new);
+    HoodieTableFileSystemView fileSystemView = new HoodieTableFileSystemView(metaClient, activeTimeline, allFiles);

Review Comment:
   @YuweiXiao Please correct me if I'm wrong but I think that since `loadFileSlicesForPartitions` is called separately for each data partition after pruning, when using the MT, the FILES partition needs to be re-constructed from scratch when listing files in each data partition, one by one. This can become pretty expensive when listing multiple partitions as the merging of the base and delta files in the FILES partition is expensive. Would it be possible to either (a) list all pruned partitions in a single call, similarly to how the eager implementation (`doRefresh()`) does it, or (b) only initialize the `HoodieTableFileSystemView` once for listing all partitions in a single query? Anyways, this is a massive improvement to the query listing performance so thank you for implementing it, just thought I'd raise this as when I tested this branch in my env I actually got better listing perf with MT disabled. I can provide the driver logs if helpful where you can see the FILES partition be
 ing loaded when listing each data partition.



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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1023185853


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -179,17 +175,94 @@ public void close() throws Exception {
     resetTableMetadata(null);
   }
 
+  protected String[] getPartitionColumns() {
+    return partitionColumns;
+  }
+
+  protected List<Path> getQueryPaths() {
+    return queryPaths;
+  }
+
+  /**
+   * Returns all partition paths matching the ones explicitly provided by the query (if any)
+   */
   protected List<PartitionPath> getAllQueryPartitionPaths() {
-    List<String> queryRelativePartitionPaths = queryPaths.stream()
-        .map(path -> FSUtils.getRelativePartitionPath(basePath, path))
-        .collect(Collectors.toList());
+    if (cachedAllPartitionPaths == null) {
+      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))
-        .collect(Collectors.toList());
+      this.cachedAllPartitionPaths = listPartitionPaths(queryRelativePartitionPaths);
+    }
+
+    return cachedAllPartitionPaths;
+  }
+
+  /**
+   * Returns all listed file-slices w/in the partition paths returned by {@link #getAllQueryPartitionPaths()}
+   */
+  protected Map<PartitionPath, List<FileSlice>> getAllInputFileSlices() {
+    if (!areAllFileSlicesCached()) {
+      // Fetching file slices for partitions that have not been cached yet
+      List<PartitionPath> missingPartitions = getAllQueryPartitionPaths().stream()
+          .filter(p -> !cachedAllInputFileSlices.containsKey(p))
+          .collect(Collectors.toList());
+
+      // NOTE: Individual partitions are always cached in full, therefore if partition is cached
+      //       it will hold all the file-slices residing w/in the partition
+      cachedAllInputFileSlices.putAll(loadFileSlicesForPartitions(missingPartitions));
+    }
+
+    return cachedAllInputFileSlices;
+  }
+
+  /**
+   * Get input file slice for the given partition. Will use cache directly if it is computed before.
+   */
+  protected List<FileSlice> getInputFileSlices(PartitionPath partition) {
+    return cachedAllInputFileSlices.computeIfAbsent(partition,
+        p -> loadFileSlicesForPartitions(Collections.singletonList(p)).get(p));
+  }
+
+  private Map<PartitionPath, List<FileSlice>> loadFileSlicesForPartitions(List<PartitionPath> partitions) {
+    Map<PartitionPath, FileStatus[]> partitionFiles = partitions.stream()
+        .collect(Collectors.toMap(p -> p, this::loadPartitionPathFiles));
+    HoodieTimeline activeTimeline = getActiveTimeline();
+    Option<HoodieInstant> latestInstant = activeTimeline.lastInstant();
+
+    FileStatus[] allFiles = partitionFiles.values().stream().flatMap(Arrays::stream).toArray(FileStatus[]::new);
+    HoodieTableFileSystemView fileSystemView = new HoodieTableFileSystemView(metaClient, activeTimeline, allFiles);

Review Comment:
   Watchful eye, @psendyk! 
   It's actually already done exactly the way you're describing -- `tableMetadata` is an object that holds the state of the MT, and is the one that FileIndex is caching internally. Hence no matter which way you access it, it won't be re-parsing MT.
   
   `HoodieTableFileSystemView` on the other end is _already being_ provided w/ the files previously listed and as such is only responsible for projecting (read, filtering) which files should be read for particular query (based on instant snapshot, etc).



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


[GitHub] [hudi] psendyk commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
psendyk commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1023350011


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -179,17 +175,94 @@ public void close() throws Exception {
     resetTableMetadata(null);
   }
 
+  protected String[] getPartitionColumns() {
+    return partitionColumns;
+  }
+
+  protected List<Path> getQueryPaths() {
+    return queryPaths;
+  }
+
+  /**
+   * Returns all partition paths matching the ones explicitly provided by the query (if any)
+   */
   protected List<PartitionPath> getAllQueryPartitionPaths() {
-    List<String> queryRelativePartitionPaths = queryPaths.stream()
-        .map(path -> FSUtils.getRelativePartitionPath(basePath, path))
-        .collect(Collectors.toList());
+    if (cachedAllPartitionPaths == null) {
+      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))
-        .collect(Collectors.toList());
+      this.cachedAllPartitionPaths = listPartitionPaths(queryRelativePartitionPaths);
+    }
+
+    return cachedAllPartitionPaths;
+  }
+
+  /**
+   * Returns all listed file-slices w/in the partition paths returned by {@link #getAllQueryPartitionPaths()}
+   */
+  protected Map<PartitionPath, List<FileSlice>> getAllInputFileSlices() {
+    if (!areAllFileSlicesCached()) {
+      // Fetching file slices for partitions that have not been cached yet
+      List<PartitionPath> missingPartitions = getAllQueryPartitionPaths().stream()
+          .filter(p -> !cachedAllInputFileSlices.containsKey(p))
+          .collect(Collectors.toList());
+
+      // NOTE: Individual partitions are always cached in full, therefore if partition is cached
+      //       it will hold all the file-slices residing w/in the partition
+      cachedAllInputFileSlices.putAll(loadFileSlicesForPartitions(missingPartitions));
+    }
+
+    return cachedAllInputFileSlices;
+  }
+
+  /**
+   * Get input file slice for the given partition. Will use cache directly if it is computed before.
+   */
+  protected List<FileSlice> getInputFileSlices(PartitionPath partition) {
+    return cachedAllInputFileSlices.computeIfAbsent(partition,
+        p -> loadFileSlicesForPartitions(Collections.singletonList(p)).get(p));
+  }
+
+  private Map<PartitionPath, List<FileSlice>> loadFileSlicesForPartitions(List<PartitionPath> partitions) {
+    Map<PartitionPath, FileStatus[]> partitionFiles = partitions.stream()
+        .collect(Collectors.toMap(p -> p, this::loadPartitionPathFiles));
+    HoodieTimeline activeTimeline = getActiveTimeline();
+    Option<HoodieInstant> latestInstant = activeTimeline.lastInstant();
+
+    FileStatus[] allFiles = partitionFiles.values().stream().flatMap(Arrays::stream).toArray(FileStatus[]::new);
+    HoodieTableFileSystemView fileSystemView = new HoodieTableFileSystemView(metaClient, activeTimeline, allFiles);

Review Comment:
   Yes, you're right the `tableMetadata` is being reused but I'm not seeing how the caching of the file index works, would you mind pointing me to it? I spent some time in a debugger and it seems like this behavior is triggered when computing `allFiles` on L228-229, not initializing `HoodieTableFileSystemView` as I originally thought. This is just my poor attempt at debugging this with a very limited understanding of the codebase so apologies if I'm making some incorrect assumptions. I think it might be happening because `HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices` is being called with the `files` partition for each data partition listing operation, which then re-initializes the `HoodieTableFileSystemView` without any cached files. Then when `ensurePartitionLoadedCorrectly` is called with the `files` partition, `addedPartitions` is empty and the partition needs to be recomputed. There's also a chance something is wrong with my env but I'd suggest at least trying to re
 produce it in your env; if you can confirm the `files` partition isn't being recomputed for every data partition listing, that's great! Let me know if it works for you as expected and I'll move on to debugging my own env. In case you find it helpful, here's the stack trace from my debugger that I get for every data partition listing operation, where at the end (`ensurePartitionLoadedCorrectly`), the `files` partition is computed each time:
   ```
   AbstractTableFileSystemView.ensurePartitionLoadedCorrectly
   AbstractTableFileSystemView.getLatestMergedFileSlicesBeforeOrOn
   HoodieTableMetadataUtil.getPartitionFileSlices
   HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices
   HoodieBackedTableMetadata.getPartitionFileSliceToKeysMapping
   HoodieBackedTableMetadata.getRecordsByKeys
   HoodieBackedTableMetadata.getRecordByKey
   BaseTableMetadata.fetchAllFilesInPartition
   BaseTableMetadata.getAllFilesInPartition
   BaseHoodieTableFileIndex.loadPartitionPathFiles
   BaseHoodieTableFileIndex.loadFileSlicesForPartitions
   ```



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r979366643


##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java:
##########
@@ -143,6 +143,13 @@ protected Option<HoodieRecord<HoodieMetadataPayload>> getRecordByKey(String key,
     return recordsByKeys.size() == 0 ? Option.empty() : recordsByKeys.get(0).getValue();
   }
 
+  @Override
+  public List<String> getPartitionPathsWithPrefix(String prefix) throws IOException {
+    return getAllPartitionPaths().stream()

Review Comment:
   Yeah, I only put a naive implementation for MDT here. Do you think the lazy fetching is necessary for MDT case? Or is there API to load partitions with given prefix only when using MDT?



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r979364969


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  protected Map<PartitionPath, List<FileSlice>> getAllInputFileSlices() {
+    if (!isAllInputFileSlicesCached) {

Review Comment:
   Because there are two cases:
   1. all input files are loaded, e.g., the old doRefresh logic.
   2. input files are loaded on demand when using the lazy initialization. In which case, we are uncertain if all file slices are loaded.



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1000194169


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -310,35 +256,159 @@ private void doRefresh() {
             )
         );
 
-    cachedFileSize = cachedAllInputFileSlices.values().stream()
+    this.cachedFileSize += ret.values().stream()
         .flatMap(Collection::stream)
         .mapToLong(BaseHoodieTableFileIndex::fileSliceSize)
         .sum();
 
-    // If the partition value contains InternalRow.empty, we query it as a non-partitioned table.
-    queryAsNonePartitionedTable = partitionFiles.keySet().stream().anyMatch(p -> p.values.length == 0);
+    return ret;
+  }
 
-    long duration = System.currentTimeMillis() - startTime;
+  /**
+   * 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 (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");

Review Comment:
   Sure.



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1000207659


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -310,35 +256,159 @@ private void doRefresh() {
             )
         );
 
-    cachedFileSize = cachedAllInputFileSlices.values().stream()
+    this.cachedFileSize += ret.values().stream()
         .flatMap(Collection::stream)
         .mapToLong(BaseHoodieTableFileIndex::fileSliceSize)
         .sum();
 
-    // If the partition value contains InternalRow.empty, we query it as a non-partitioned table.
-    queryAsNonePartitionedTable = partitionFiles.keySet().stream().anyMatch(p -> p.values.length == 0);
+    return ret;
+  }
 
-    long duration = System.currentTimeMillis() - startTime;
+  /**
+   * 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 (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
 
-    LOG.info(String.format("Refresh table %s, spent: %d ms", metaClient.getTableConfig().getTableName(), duration));
+    Pair<String, Boolean> relativeQueryPartitionPathPair = composeRelativePartitionPaths(partitionNames, values);
+    // If the composed partition path is complete, we return it directly, to save extra DFS listing operations.
+    if (relativeQueryPartitionPathPair.getRight()) {
+      return Collections.singletonList(new PartitionPath(relativeQueryPartitionPathPair.getLeft(),
+          parsePartitionColumnValues(partitionColumns, relativeQueryPartitionPathPair.getLeft())));
+    }
+    // The input partition values (from query predicate) forms a prefix of partition path, do listing to the path only.
+    return listPartitionPaths(Collections.singletonList(relativeQueryPartitionPathPair.getLeft()));
   }
 
-  private Map<String, FileStatus[]> getAllFilesInPartitionsUnchecked(Collection<String> fullPartitionPathsMapToFetch) {
-    try {
-      return tableMetadata.getAllFilesInPartitions(new ArrayList<>(fullPartitionPathsMapToFetch));
-    } catch (IOException e) {
-      throw new HoodieIOException("Failed to list partition paths for a table", e);
+  /**
+   * Construct relative partition (i.e., partition prefix) from the given partition values
+   * @return relative partition path and a flag to indicate if the path is complete (i.e., not a prefix)
+   */
+  private Pair<String, Boolean> composeRelativePartitionPaths(String[] partitionNames, String[] values) {

Review Comment:
   So how about putting the `composeRelativePartitionPaths ` in `BuiltinKeyGenerator`, and reuse `combinePartitionPathInternal` to construct the prefix of the partition path?



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1286540938

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a65056adaa4e9fabda4205c1b5c7be2e48bdd67f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382) 
   * 903ac79702d02fe968d271b08a217f8401d63700 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] alexeykudinkin commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1306108771

   @YuweiXiao yeah, i had to switch it back to eager by default realizing its limitations (wouldn't work w/ encoded paths, etc). 
   
   I'm actually planning to make one more change to separate out a) actual lazy listing from b) prefix-based partition pruning, and then we can enable lazy-listing as default (which will greatly benefit all use-cases and not just jumbo tables)


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1305007619

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fe18f26f8ab63a28c98b23c6940b6135b6155d91 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804) 
   * 13b8d331e0dcf5ac1f958b91770e2024bce04c43 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1304609751

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fe18f26f8ab63a28c98b23c6940b6135b6155d91 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1314404805

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     }, {
       "hash" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937",
       "triggerID" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12938",
       "triggerID" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12942",
       "triggerID" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f7e288c8e5e69a3a406c8e22761d752a068b7f3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12964",
       "triggerID" : "1f7e288c8e5e69a3a406c8e22761d752a068b7f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e6cd657efc1ab3ac8e472b32b025d87be8671b5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=13012",
       "triggerID" : "2e6cd657efc1ab3ac8e472b32b025d87be8671b5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1f7e288c8e5e69a3a406c8e22761d752a068b7f3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12964) 
   * 2e6cd657efc1ab3ac8e472b32b025d87be8671b5 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=13012) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1311275586

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     }, {
       "hash" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937",
       "triggerID" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12938",
       "triggerID" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12942",
       "triggerID" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8bafc71d82607b40cb99505d145bbfddb8c81ae3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12942) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980871724


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala:
##########
@@ -209,7 +215,8 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
         s" after partition prune size is: ${prunedPartitionPaths.size}")
       prunedPartitionPaths
     } else {
-      partitionPaths
+      logInfo(s"No partition predicate provided, total partition size is: ${getAllQueryPartitionPaths.asScala.size}")

Review Comment:
   Yeah, I added some description on FileIndex constructor.



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
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-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1272270347

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * efc19bfcfb86bf582d4bd2584462083b8178c1c6 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783) 
   * 5f6d4f624c5f20cf3c4c38384e17c7bb13e56991 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1250230751

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473) 
   * d8495f35bf97ed4ec27a3841dad66307d862d65e UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1309616517

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 59cdd09e3190c3646e1e3ea6ca3f076526ec0473 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1000217067


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -347,6 +417,17 @@ private void validate(HoodieTimeline activeTimeline, Option<String> queryInstant
     }
   }
 
+  protected boolean isAllInputFileSlicesCached() {
+    if (cachedAllPartitionPaths == null) {
+      return false;
+    }
+    return cachedAllPartitionPaths.stream().allMatch(p -> cachedAllInputFileSlices.containsKey(p));
+  }
+
+  protected boolean isPartitionedTable() {
+    return !queryAsNonePartitionedTable && (partitionColumns.length > 0 || HoodieTableMetadata.isMetadataTable(basePath.toString()));

Review Comment:
   Yeah, I was trying to remove it but some tests will fail. The thing is if we fail to parse partition values from the partition path (e.g., non-encoded partition value containing `/`), the original implementation will query the table as non-partitioned-table to obtain correct results.
   
   Check out `SparkHoodieTableFileIndex::parsePartitionColumnValues`. And in the failure case we will flag the query:
   
   ```
   this.queryAsNonePartitionedTable = getAllQueryPartitionPaths().stream().anyMatch(p -> p.values.length == 0);
   ```
   
   Can not come up a good way to get rid of the variable completely :(



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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r998681902


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -310,35 +256,159 @@ private void doRefresh() {
             )
         );
 
-    cachedFileSize = cachedAllInputFileSlices.values().stream()
+    this.cachedFileSize += ret.values().stream()
         .flatMap(Collection::stream)
         .mapToLong(BaseHoodieTableFileIndex::fileSliceSize)
         .sum();
 
-    // If the partition value contains InternalRow.empty, we query it as a non-partitioned table.
-    queryAsNonePartitionedTable = partitionFiles.keySet().stream().anyMatch(p -> p.values.length == 0);
+    return ret;
+  }
 
-    long duration = System.currentTimeMillis() - startTime;
+  /**
+   * 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 (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");

Review Comment:
   Let's downgrade this to debug



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -347,6 +417,17 @@ private void validate(HoodieTimeline activeTimeline, Option<String> queryInstant
     }
   }
 
+  protected boolean isAllInputFileSlicesCached() {
+    if (cachedAllPartitionPaths == null) {
+      return false;
+    }
+    return cachedAllPartitionPaths.stream().allMatch(p -> cachedAllInputFileSlices.containsKey(p));

Review Comment:
   Let's add a comment to the `cachedAllPartitionPaths` elaborating that it's always fetched fully, and never incrementally (otherwise this check would be incorrect)



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala:
##########
@@ -187,12 +189,16 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
    * @param predicates     The filter condition.
    * @return The pruned partition paths.
    */
-  protected def prunePartition(partitionPaths: Seq[PartitionPath], predicates: Seq[Expression]): Seq[PartitionPath] = {
+  protected def prunePartition(predicates: Seq[Expression]): Seq[PartitionPath] = {
     val partitionColumnNames = partitionSchema.fields.map(_.name).toSet
     val partitionPruningPredicates = predicates.filter {
       _.references.map(_.name).toSet.subsetOf(partitionColumnNames)
     }
     if (partitionPruningPredicates.nonEmpty) {
+      val equalPredicate = partitionPruningPredicates.filter(_.isInstanceOf[EqualTo])
+      val names = equalPredicate.map(_.asInstanceOf[EqualTo]).map(_.left.asInstanceOf[AttributeReference].name).toArray

Review Comment:
   > By the way, currently I put an option to control the loading behavior, REFRESH_PARTITION_AND_FILES_IN_INITIALIZATION. With the above semantic, maybe we can remote it and let it be handled automatically. WDYT?
   
   Sorry, not sure i fully follow this one: what do you want to remove?



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -298,7 +244,7 @@ private void doRefresh() {
     // This logic is realized by `AbstractTableFileSystemView::getLatestMergedFileSlicesBeforeOrOn`
     // API.  Note that for COW table, the merging logic of two slices does not happen as there
     // is no compaction, thus there is no performance impact.
-    cachedAllInputFileSlices = partitionFiles.keySet().stream()
+    Map<PartitionPath, List<FileSlice>> ret = partitionFiles.keySet().stream()

Review Comment:
   nit: `listedPartitions`



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -310,35 +256,159 @@ private void doRefresh() {
             )
         );
 
-    cachedFileSize = cachedAllInputFileSlices.values().stream()
+    this.cachedFileSize += ret.values().stream()
         .flatMap(Collection::stream)
         .mapToLong(BaseHoodieTableFileIndex::fileSliceSize)
         .sum();
 
-    // If the partition value contains InternalRow.empty, we query it as a non-partitioned table.
-    queryAsNonePartitionedTable = partitionFiles.keySet().stream().anyMatch(p -> p.values.length == 0);
+    return ret;
+  }
 
-    long duration = System.currentTimeMillis() - startTime;
+  /**
+   * 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) {

Review Comment:
   Let's make it accept `List<Pair<String, String>> partitionColumnValuePairs`



##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java:
##########
@@ -132,6 +132,11 @@ static HoodieBackedTableMetadata createHoodieBackedTableMetadata(HoodieEngineCon
    */
   FileStatus[] getAllFilesInPartition(Path partitionPath) throws IOException;
 
+  /**
+   * Fetch list of all partitions path that match the given prefix

Review Comment:
   Let's make it very clear that the prefix here is referring to "partition-path being the prefix of the complete relative partition path"



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -310,35 +256,159 @@ private void doRefresh() {
             )
         );
 
-    cachedFileSize = cachedAllInputFileSlices.values().stream()
+    this.cachedFileSize += ret.values().stream()
         .flatMap(Collection::stream)
         .mapToLong(BaseHoodieTableFileIndex::fileSliceSize)
         .sum();
 
-    // If the partition value contains InternalRow.empty, we query it as a non-partitioned table.
-    queryAsNonePartitionedTable = partitionFiles.keySet().stream().anyMatch(p -> p.values.length == 0);
+    return ret;
+  }
 
-    long duration = System.currentTimeMillis() - startTime;
+  /**
+   * 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 (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
 
-    LOG.info(String.format("Refresh table %s, spent: %d ms", metaClient.getTableConfig().getTableName(), duration));
+    Pair<String, Boolean> relativeQueryPartitionPathPair = composeRelativePartitionPaths(partitionNames, values);
+    // If the composed partition path is complete, we return it directly, to save extra DFS listing operations.
+    if (relativeQueryPartitionPathPair.getRight()) {
+      return Collections.singletonList(new PartitionPath(relativeQueryPartitionPathPair.getLeft(),
+          parsePartitionColumnValues(partitionColumns, relativeQueryPartitionPathPair.getLeft())));
+    }
+    // The input partition values (from query predicate) forms a prefix of partition path, do listing to the path only.
+    return listPartitionPaths(Collections.singletonList(relativeQueryPartitionPathPair.getLeft()));
   }
 
-  private Map<String, FileStatus[]> getAllFilesInPartitionsUnchecked(Collection<String> fullPartitionPathsMapToFetch) {
-    try {
-      return tableMetadata.getAllFilesInPartitions(new ArrayList<>(fullPartitionPathsMapToFetch));
-    } catch (IOException e) {
-      throw new HoodieIOException("Failed to list partition paths for a table", e);
+  /**
+   * Construct relative partition (i.e., partition prefix) from the given partition values

Review Comment:
   nit: "relative partition path"



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -310,35 +256,159 @@ private void doRefresh() {
             )
         );
 
-    cachedFileSize = cachedAllInputFileSlices.values().stream()
+    this.cachedFileSize += ret.values().stream()
         .flatMap(Collection::stream)
         .mapToLong(BaseHoodieTableFileIndex::fileSliceSize)
         .sum();
 
-    // If the partition value contains InternalRow.empty, we query it as a non-partitioned table.
-    queryAsNonePartitionedTable = partitionFiles.keySet().stream().anyMatch(p -> p.values.length == 0);
+    return ret;
+  }
 
-    long duration = System.currentTimeMillis() - startTime;
+  /**
+   * 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 (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
 
-    LOG.info(String.format("Refresh table %s, spent: %d ms", metaClient.getTableConfig().getTableName(), duration));
+    Pair<String, Boolean> relativeQueryPartitionPathPair = composeRelativePartitionPaths(partitionNames, values);

Review Comment:
   We actually don't need to return boolean flag, we can detect whether this would be a full-path here as well (comparing # of values against # of partition columns)



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -139,7 +142,20 @@ public BaseHoodieTableFileIndex(HoodieEngineContext engineContext,
     this.engineContext = engineContext;
     this.fileStatusCache = fileStatusCache;
 
-    doRefresh();
+    /**
+     * The `shouldListLazily` variable controls how we initialize the TableFileIndex:
+     *  - non-lazy/eager listing (shouldListLazily=true):  all partitions and file slices will be loaded eagerly during initialization.

Review Comment:
   Typo: should be `shouldListLazy=false`



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -310,35 +256,159 @@ private void doRefresh() {
             )
         );
 
-    cachedFileSize = cachedAllInputFileSlices.values().stream()
+    this.cachedFileSize += ret.values().stream()
         .flatMap(Collection::stream)
         .mapToLong(BaseHoodieTableFileIndex::fileSliceSize)
         .sum();
 
-    // If the partition value contains InternalRow.empty, we query it as a non-partitioned table.
-    queryAsNonePartitionedTable = partitionFiles.keySet().stream().anyMatch(p -> p.values.length == 0);
+    return ret;
+  }
 
-    long duration = System.currentTimeMillis() - startTime;
+  /**
+   * 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 (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
 
-    LOG.info(String.format("Refresh table %s, spent: %d ms", metaClient.getTableConfig().getTableName(), duration));
+    Pair<String, Boolean> relativeQueryPartitionPathPair = composeRelativePartitionPaths(partitionNames, values);
+    // If the composed partition path is complete, we return it directly, to save extra DFS listing operations.
+    if (relativeQueryPartitionPathPair.getRight()) {
+      return Collections.singletonList(new PartitionPath(relativeQueryPartitionPathPair.getLeft(),
+          parsePartitionColumnValues(partitionColumns, relativeQueryPartitionPathPair.getLeft())));
+    }
+    // The input partition values (from query predicate) forms a prefix of partition path, do listing to the path only.
+    return listPartitionPaths(Collections.singletonList(relativeQueryPartitionPathPair.getLeft()));
   }
 
-  private Map<String, FileStatus[]> getAllFilesInPartitionsUnchecked(Collection<String> fullPartitionPathsMapToFetch) {
-    try {
-      return tableMetadata.getAllFilesInPartitions(new ArrayList<>(fullPartitionPathsMapToFetch));
-    } catch (IOException e) {
-      throw new HoodieIOException("Failed to list partition paths for a table", e);
+  /**
+   * Construct relative partition (i.e., partition prefix) from the given partition values
+   * @return relative partition path and a flag to indicate if the path is complete (i.e., not a prefix)
+   */
+  private Pair<String, Boolean> composeRelativePartitionPaths(String[] partitionNames, String[] values) {

Review Comment:
   This is somewhat non-trivial logic and we need to make sure we're not duplicating it: we already all of this handled w/in `BuiltinKeyGenerator.combinePartitionPathInternal`, let's generalize it and apply it here
   



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -347,6 +417,17 @@ private void validate(HoodieTimeline activeTimeline, Option<String> queryInstant
     }
   }
 
+  protected boolean isAllInputFileSlicesCached() {
+    if (cachedAllPartitionPaths == null) {
+      return false;
+    }
+    return cachedAllPartitionPaths.stream().allMatch(p -> cachedAllInputFileSlices.containsKey(p));
+  }
+
+  protected boolean isPartitionedTable() {
+    return !queryAsNonePartitionedTable && (partitionColumns.length > 0 || HoodieTableMetadata.isMetadataTable(basePath.toString()));

Review Comment:
   We actually should remove `queryAsNonePartitionedTable` completely



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala:
##########
@@ -165,7 +164,7 @@ case class HoodieFileIndex(spark: SparkSession,
 
       logInfo(s"Total base files: $totalFileSize; " +
         s"candidate files after data skipping : $candidateFileSize; " +
-        s"skipping percent ${if (allFiles.nonEmpty && totalFileSize > 0) (totalFileSize - candidateFileSize) / totalFileSize.toDouble else 0}")
+        s"skipping percent ${if (!isAllInputFileSlicesCached) "is disable" else if (allFiles.nonEmpty && totalFileSize > 0) (totalFileSize - candidateFileSize) / totalFileSize.toDouble else 0}")

Review Comment:
   nit: "is disabled"



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNamesToIdx.containsKey(columnNames)) {
+        int k = partitionNamesToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");
+      } else {
+        break;
+      }
+    }
+    queryPartitionPath.deleteCharAt(queryPartitionPath.length() - 1);
+    // Return directly if all partition values are specified.
+    if (idx == this.partitionColumns.length) {
+      return Collections.singletonList(new PartitionPath(queryPartitionPath.toString(), parsePartitionColumnValues(partitionColumns, queryPartitionPath.toString())));
+    }
+    return getQueryPartitionPaths(Collections.singletonList(queryPartitionPath.toString()));
+  }
+
+  private List<PartitionPath> getQueryPartitionPaths(List<String> queryRelativePartitionPaths) {
+    List<String> matchedPartitionPaths = queryRelativePartitionPaths.stream()
+        .flatMap(prefix -> {
+          try {
+            // Handle wildcard specially. This will have FileIndex to query the table as non-partitioned-table
+            if (prefix.contains("*")) {

Review Comment:
   I don't think we should be handling it here though: paths globbing is handled w/in the `DefaultSource` and it shouldn't be reaching the FileIndex



##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java:
##########
@@ -143,6 +143,13 @@ protected Option<HoodieRecord<HoodieMetadataPayload>> getRecordByKey(String key,
     return recordsByKeys.size() == 0 ? Option.empty() : recordsByKeys.get(0).getValue();
   }
 
+  @Override
+  public List<String> getPartitionPathsWithPrefix(String prefix) throws IOException {
+    return getAllPartitionPaths().stream()

Review Comment:
   Correct



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -310,35 +256,159 @@ private void doRefresh() {
             )
         );
 
-    cachedFileSize = cachedAllInputFileSlices.values().stream()
+    this.cachedFileSize += ret.values().stream()
         .flatMap(Collection::stream)
         .mapToLong(BaseHoodieTableFileIndex::fileSliceSize)
         .sum();
 
-    // If the partition value contains InternalRow.empty, we query it as a non-partitioned table.
-    queryAsNonePartitionedTable = partitionFiles.keySet().stream().anyMatch(p -> p.values.length == 0);
+    return ret;
+  }
 
-    long duration = System.currentTimeMillis() - startTime;
+  /**
+   * 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 (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
 
-    LOG.info(String.format("Refresh table %s, spent: %d ms", metaClient.getTableConfig().getTableName(), duration));
+    Pair<String, Boolean> relativeQueryPartitionPathPair = composeRelativePartitionPaths(partitionNames, values);

Review Comment:
   We also need to make sure that the partition columns are ordered in the same way they are w/in the partition schema (otherwise paths won't match)



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala:
##########
@@ -138,6 +138,14 @@ object DataSourceReadOptions {
         " read from the data file (in Hudi partition columns are persisted by default)." +
         " This config is a fallback allowing to preserve existing behavior, and should not be used otherwise.")
 
+  val REFRESH_PARTITION_AND_FILES_IN_INITIALIZATION: ConfigProperty[Boolean] =

Review Comment:
   @YuweiXiao the other way around:
    - We should list lazily by default
    - We should add config to override this behavior and fallback to eager listing (if necessary)



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980860355


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -139,7 +144,11 @@ public BaseHoodieTableFileIndex(HoodieEngineContext engineContext,
     this.engineContext = engineContext;
     this.fileStatusCache = fileStatusCache;
 
-    doRefresh();
+    if (shouldRefresh) {

Review Comment:
   Added.



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r979366415


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNamesToIdx.containsKey(columnNames)) {
+        int k = partitionNamesToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");
+      } else {
+        break;
+      }
+    }
+    queryPartitionPath.deleteCharAt(queryPartitionPath.length() - 1);
+    // Return directly if all partition values are specified.
+    if (idx == this.partitionColumns.length) {
+      return Collections.singletonList(new PartitionPath(queryPartitionPath.toString(), parsePartitionColumnValues(partitionColumns, queryPartitionPath.toString())));
+    }
+    return getQueryPartitionPaths(Collections.singletonList(queryPartitionPath.toString()));
+  }
+
+  private List<PartitionPath> getQueryPartitionPaths(List<String> queryRelativePartitionPaths) {
+    List<String> matchedPartitionPaths = queryRelativePartitionPaths.stream()
+        .flatMap(prefix -> {
+          try {
+            // Handle wildcard specially. This will have FileIndex to query the table as non-partitioned-table
+            if (prefix.contains("*")) {

Review Comment:
   Yeah, in non-sql read path, we may have spark.read.format("hudi").load("path-to-table/*/*/*"). The current handle follows the origin logic, which will fallback to non-partition-table querying.



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r979365943


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNamesToIdx.containsKey(columnNames)) {
+        int k = partitionNamesToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");

Review Comment:
   Yes, you are write. The current logic only handles predicates that could form the prefix of the partition. If we want to handle the more general case, we need also to modify the listing implementation inside the metadata table.
   
   In our case, all queries will give first partition field predicate first, then the second and so on. And I believe it should cover most user cases. I could create a JIRA to track this if you think necessary.



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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980388571


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -139,7 +144,11 @@ public BaseHoodieTableFileIndex(HoodieEngineContext engineContext,
     this.engineContext = engineContext;
     this.fileStatusCache = fileStatusCache;
 
-    doRefresh();
+    if (shouldRefresh) {

Review Comment:
   Let's add a comment explaining the difference b/w these branches (elaborating on lazy-semantic)



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +189,125 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {

Review Comment:
   nit: We can avoid prefixing `this.` to fields when accessing; this is usually reserved to assign fields w/in ctors when there's local var w/ the same name is present in the scope



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +189,125 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
+    loadAllQueryPartitionPaths();
+    return this.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 = getQueryPartitionPaths(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::loadInputFileSlicesOfPartition);
+  }
+
+  private List<FileSlice> loadInputFileSlicesOfPartition(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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNameToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNameToIdx.containsKey(columnNames)) {
+        int k = partitionNameToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");
+      } else {
+        break;
+      }
+    }
+    queryPartitionPath.deleteCharAt(queryPartitionPath.length() - 1);
+    // Return directly if all partition values are specified.
+    if (idx == this.partitionColumns.length) {
+      return Collections.singletonList(new PartitionPath(queryPartitionPath.toString(), parsePartitionColumnValues(partitionColumns, queryPartitionPath.toString())));
+    }
+    // The predicate forms a prefix of partition path, do listing to the path only.
+    return getQueryPartitionPaths(Collections.singletonList(queryPartitionPath.toString()));
+  }
+
+  private List<PartitionPath> getQueryPartitionPaths(List<String> queryRelativePartitionPaths) {

Review Comment:
   nit: Let's rename this to `listQueryPartitionPath` to make it apparent that this will entail MT/FS listing operation



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +189,125 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
+    loadAllQueryPartitionPaths();
+    return this.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 = getQueryPartitionPaths(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::loadInputFileSlicesOfPartition);
+  }
+
+  private List<FileSlice> loadInputFileSlicesOfPartition(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));

Review Comment:
   Let's create a `Lazy` field for this one (this is not supposed to change for given FileIndex instance)



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala:
##########
@@ -165,7 +164,7 @@ case class HoodieFileIndex(spark: SparkSession,
 
       logInfo(s"Total base files: $totalFileSize; " +
         s"candidate files after data skipping : $candidateFileSize; " +
-        s"skipping percent ${if (allFiles.nonEmpty && totalFileSize > 0) (totalFileSize - candidateFileSize) / totalFileSize.toDouble else 0}")
+        s"skipping percent ${if (isAllInputFileSlicesCached && allFiles.nonEmpty && totalFileSize > 0) (totalFileSize - candidateFileSize) / totalFileSize.toDouble else 0}")

Review Comment:
   If we're loading lazily let's instead output `"is disabled"`



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +189,125 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
+    loadAllQueryPartitionPaths();
+    return this.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 = getQueryPartitionPaths(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::loadInputFileSlicesOfPartition);
+  }
+
+  private List<FileSlice> loadInputFileSlicesOfPartition(PartitionPath p) {

Review Comment:
   nit: `loadFileSlicesForPartition`



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +189,125 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
+    loadAllQueryPartitionPaths();
+    return this.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 = getQueryPartitionPaths(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::loadInputFileSlicesOfPartition);
+  }
+
+  private List<FileSlice> loadInputFileSlicesOfPartition(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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());

Review Comment:
   Let's add a tracking task to consolidate this logic w/in key-gens



##########
hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java:
##########
@@ -74,8 +75,13 @@ public List<String> getAllPartitionPaths() throws IOException {
       return FSUtils.getAllPartitionFoldersThreeLevelsDown(fs, datasetBasePath);
     }
 
+    return getPartitionPathsWithPrefix("");
+  }
+
+  @Override
+  public List<String> getPartitionPathsWithPrefix(String prefix) throws IOException {
     List<Path> pathsToList = new CopyOnWriteArrayList<>();
-    pathsToList.add(basePath);
+    pathsToList.add(StringUtils.isNullOrEmpty(prefix) ? new Path(datasetBasePath) : new Path(datasetBasePath, prefix));

Review Comment:
   This implementation currently assumes that either a) prefix is "" or b) prefix is an actual relative path (instead of, say, prefix of the name of the folder) which could be misleading
   
   Instead, i'd suggest we do the following:
   
    - Let's create a separate method `getPartitionPathsWithPrefix(Option<String> prefix)`
    - Not allow empty prefixes to be passed in
    - Make sure that implementation can handle actual prefixes (instead of currently digesting only the full-paths)



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  protected Map<PartitionPath, List<FileSlice>> getAllInputFileSlices() {
+    if (!isAllInputFileSlicesCached) {

Review Comment:
   Let's also add a comment explaining why we're doing full refresh here in that case



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNamesToIdx.containsKey(columnNames)) {
+        int k = partitionNamesToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");
+      } else {
+        break;
+      }
+    }
+    queryPartitionPath.deleteCharAt(queryPartitionPath.length() - 1);
+    // Return directly if all partition values are specified.
+    if (idx == this.partitionColumns.length) {

Review Comment:
   We don't need to use `idx` for it (we can use `partitionNames.length`)



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala:
##########
@@ -187,12 +189,16 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
    * @param predicates     The filter condition.
    * @return The pruned partition paths.
    */
-  protected def prunePartition(partitionPaths: Seq[PartitionPath], predicates: Seq[Expression]): Seq[PartitionPath] = {
+  protected def prunePartition(predicates: Seq[Expression]): Seq[PartitionPath] = {

Review Comment:
   Let's rename this to `listMatchingPartitionPaths` (current name is misaligned after signature change)



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala:
##########
@@ -209,7 +215,8 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
         s" after partition prune size is: ${prunedPartitionPaths.size}")
       prunedPartitionPaths
     } else {
-      partitionPaths
+      logInfo(s"No partition predicate provided, total partition size is: ${getAllQueryPartitionPaths.asScala.size}")

Review Comment:
   Let's make sure we add comments elaborating on the lazy and non-lazy listing flows



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -323,30 +424,13 @@ private void doRefresh() {
         .mapToLong(BaseHoodieTableFileIndex::fileSliceSize)
         .sum();
 
-    // If the partition value contains InternalRow.empty, we query it as a non-partitioned table.
-    queryAsNonePartitionedTable = partitionFiles.keySet().stream().anyMatch(p -> p.values.length == 0);

Review Comment:
   Let's actually replace this flag w/ `isPartitionedTable` predicate (sorry, it's my bad that i missed it cleaning it up last time)



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala:
##########
@@ -138,6 +138,14 @@ object DataSourceReadOptions {
         " read from the data file (in Hudi partition columns are persisted by default)." +
         " This config is a fallback allowing to preserve existing behavior, and should not be used otherwise.")
 
+  val REFRESH_PARTITION_AND_FILES_IN_INITIALIZATION: ConfigProperty[Boolean] =

Review Comment:
   I don't think we need a config for that, instead:
   
   - Lazy loading should be default behavior
   - We should add a config that can override default behavior and always load eagerly (fallback)



##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  protected Map<PartitionPath, List<FileSlice>> getAllInputFileSlices() {
+    if (!isAllInputFileSlicesCached) {

Review Comment:
   Understood. Let's rename this flag to actually associate it w/ lazy-loading semantic: `shouldRefreshLazily`



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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r979076389


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -180,15 +185,118 @@ public void close() throws Exception {
   }
 
   protected List<PartitionPath> getAllQueryPartitionPaths() {
+    if (this.cachedAllPartitionPaths != null) {
+      return this.cachedAllPartitionPaths;
+    }
+
     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 = getQueryPartitionPaths(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);
+    return this.cachedAllPartitionPaths;
+  }
+
+  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, (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) {
+      LOG.info("The input partition names or value is empty, fallback to return all partition paths");
+      return getAllQueryPartitionPaths();
+    }
+
+    if (cachedAllPartitionPaths != null) {
+      LOG.info("All partition paths have already loaded, use it directly");
+      return cachedAllPartitionPaths;
+    }
+
+    boolean hiveStylePartitioning = Boolean.parseBoolean(metaClient.getTableConfig().getHiveStylePartitioningEnable());
+    boolean urlEncodePartitioning = Boolean.parseBoolean(this.metaClient.getTableConfig().getUrlEncodePartitioning());
+    Map<String, Integer> partitionNamesToIdx = IntStream.range(0, partitionNames.length)
+        .mapToObj(i -> Pair.of(i, partitionNames[i]))
+        .collect(Collectors.toMap(Pair::getValue, Pair::getKey));
+    StringBuilder queryPartitionPath = new StringBuilder();
+    int idx = 0;
+    for (; idx < partitionNames.length; ++idx) {
+      String columnNames = this.partitionColumns[idx];
+      if (partitionNamesToIdx.containsKey(columnNames)) {
+        int k = partitionNamesToIdx.get(columnNames);
+        String value =  urlEncodePartitioning ? PartitionPathEncodeUtils.escapePathName(values[k]) : values[k];
+        queryPartitionPath.append(hiveStylePartitioning ? columnNames + "=" : "").append(value).append("/");

Review Comment:
   If i understood your intention correctly, then there would be a following problem: in the partition path ordering of the columns is fixed, therefore if query specifies partition column predicates which only referenced columns that do not constitute the prefix of the actual partition path, then this would not work. For ex, let's say there are 2 partition columns A and B, and the path looks like `A=foo/B=bar`, but then the query only specifies predicate containing only B



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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r980665830


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala:
##########
@@ -187,12 +189,16 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
    * @param predicates     The filter condition.
    * @return The pruned partition paths.
    */
-  protected def prunePartition(partitionPaths: Seq[PartitionPath], predicates: Seq[Expression]): Seq[PartitionPath] = {
+  protected def prunePartition(predicates: Seq[Expression]): Seq[PartitionPath] = {
     val partitionColumnNames = partitionSchema.fields.map(_.name).toSet
     val partitionPruningPredicates = predicates.filter {
       _.references.map(_.name).toSet.subsetOf(partitionColumnNames)
     }
     if (partitionPruningPredicates.nonEmpty) {
+      val equalPredicate = partitionPruningPredicates.filter(_.isInstanceOf[EqualTo])
+      val names = equalPredicate.map(_.asInstanceOf[EqualTo]).map(_.left.asInstanceOf[AttributeReference].name).toArray

Review Comment:
   There is also another case, where we eager loading all partition paths and fetch file group lazily. E.g., User specifies a complex predicates on partitions or only gives a single predicate for the second partition field. 
   
   So maybe restrict to the following two cases:
   - We have predicate on partition columns, we do lazy loading of file groups. And partition loading based on prefix is another optimization to avoid full partition listing.
   - If not, we fallback to eager loading of partitions and file groups.
   
   By the way, currently I put an option to control the loading behavior, `REFRESH_PARTITION_AND_FILES_IN_INITIALIZATION`. With the above semantic, maybe we can remote it and let it be handled automatically. WDYT?



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1294355717

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a09114e1c326791e33e910b2f660aaa6882dcfc9 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493) 
   * df0a3523720f59827e74c67354d465966e17e176 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1294503468

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f12d60f65b7e33cf69783b80eac0463e62f6f170 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1317997586

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     }, {
       "hash" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937",
       "triggerID" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12938",
       "triggerID" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12942",
       "triggerID" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f7e288c8e5e69a3a406c8e22761d752a068b7f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12964",
       "triggerID" : "1f7e288c8e5e69a3a406c8e22761d752a068b7f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e6cd657efc1ab3ac8e472b32b025d87be8671b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=13012",
       "triggerID" : "2e6cd657efc1ab3ac8e472b32b025d87be8671b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1455e0679d0264cf960f1d064fefe5183e1cc2",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=13065",
       "triggerID" : "4d1455e0679d0264cf960f1d064fefe5183e1cc2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4d1455e0679d0264cf960f1d064fefe5183e1cc2 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=13065) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
YuweiXiao commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1022214226


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala:
##########
@@ -237,70 +246,64 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
     }
   }
 
-  private def listMatchingPartitionPathsInternal(partitionColumnNames: Seq[String],
-                                                 partitionColumnPredicates: Seq[Expression]): Seq[PartitionPath] = {
-    // NOTE: Here we try to to achieve efficiency in avoiding necessity to recursively list deep folder structures of
-    //       partitioned tables w/ multiple partition columns, by carefully analyzing provided partition predicates:
-    //
-    //       In cases when partition-predicates have
-    //         - The form of equality predicates w/ static literals (for ex, like `date = '2022-01-01'`)
-    //         - Fully specified proper prefix of the partition schema (ie fully binding first N columns
-    //           of the partition schema adhering to hereby described rules)
-    //
-    // We will try to exploit this specific structure, and try to reduce the scope of a
-    // necessary file-listings of partitions of the table to just the sub-folder under relative prefix
-    // of the partition-path derived from the partition-column predicates. For ex, consider following
-    // scenario:
-    //
-    // Table's partition schema (in-order):
-    //
-    //    country_code: string (for ex, 'us')
-    //    date: string (for ex, '2022-01-01')
-    //
-    // Table's folder structure:
-    //    us/
-    //     |- 2022-01-01/
-    //     |- 2022-01-02/
-    //     ...
-    //
-    // In case we have incoming query specifies following predicates:
-    //
-    //    `... WHERE country_code = 'us' AND date = '2022-01-01'`
-    //
-    // We can deduce full partition-path w/o doing a single listing: `us/2022-01-01`
-    if (areAllPartitionPathsCached || !shouldUsePartitionPathPrefixAnalysis(configProperties)) {
-      logDebug("All partition paths have already been cached, use it directly")
+  // NOTE: Here we try to to achieve efficiency in avoiding necessity to recursively list deep folder structures of
+  //       partitioned tables w/ multiple partition columns, by carefully analyzing provided partition predicates:
+  //
+  //       In cases when partition-predicates have
+  //         - The form of equality predicates w/ static literals (for ex, like `date = '2022-01-01'`)
+  //         - Fully specified proper prefix of the partition schema (ie fully binding first N columns
+  //           of the partition schema adhering to hereby described rules)
+  //
+  // We will try to exploit this specific structure, and try to reduce the scope of a
+  // necessary file-listings of partitions of the table to just the sub-folder under relative prefix
+  // of the partition-path derived from the partition-column predicates. For ex, consider following
+  // scenario:
+  //
+  // Table's partition schema (in-order):
+  //
+  //    country_code: string (for ex, 'us')
+  //    date: string (for ex, '2022-01-01')
+  //
+  // Table's folder structure:
+  //    us/
+  //     |- 2022-01-01/
+  //     |- 2022-01-02/
+  //     ...
+  //
+  // In case we have incoming query specifies following predicates:
+  //
+  //    `... WHERE country_code = 'us' AND date = '2022-01-01'`
+  //
+  // We can deduce full partition-path w/o doing a single listing: `us/2022-01-01`

Review Comment:
   looks like a typo. w/o means without?



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


[GitHub] [hudi] alexeykudinkin commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1314386314

   CI is green:
   <img width="1634" alt="Screenshot 2022-11-14 at 1 09 01 PM" src="https://user-images.githubusercontent.com/428277/201766442-14a24ee8-10ab-401c-9664-86c72f7d5a35.png">
   
   https://dev.azure.com/apache-hudi-ci-org/apache-hudi-ci/_build/results?buildId=12964&view=results


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1314565361

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     }, {
       "hash" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937",
       "triggerID" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12938",
       "triggerID" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12942",
       "triggerID" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f7e288c8e5e69a3a406c8e22761d752a068b7f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12964",
       "triggerID" : "1f7e288c8e5e69a3a406c8e22761d752a068b7f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e6cd657efc1ab3ac8e472b32b025d87be8671b5",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=13012",
       "triggerID" : "2e6cd657efc1ab3ac8e472b32b025d87be8671b5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2e6cd657efc1ab3ac8e472b32b025d87be8671b5 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=13012) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6680:
URL: https://github.com/apache/hudi/pull/6680#discussion_r1023416704


##########
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##########
@@ -179,17 +175,94 @@ public void close() throws Exception {
     resetTableMetadata(null);
   }
 
+  protected String[] getPartitionColumns() {
+    return partitionColumns;
+  }
+
+  protected List<Path> getQueryPaths() {
+    return queryPaths;
+  }
+
+  /**
+   * Returns all partition paths matching the ones explicitly provided by the query (if any)
+   */
   protected List<PartitionPath> getAllQueryPartitionPaths() {
-    List<String> queryRelativePartitionPaths = queryPaths.stream()
-        .map(path -> FSUtils.getRelativePartitionPath(basePath, path))
-        .collect(Collectors.toList());
+    if (cachedAllPartitionPaths == null) {
+      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))
-        .collect(Collectors.toList());
+      this.cachedAllPartitionPaths = listPartitionPaths(queryRelativePartitionPaths);
+    }
+
+    return cachedAllPartitionPaths;
+  }
+
+  /**
+   * Returns all listed file-slices w/in the partition paths returned by {@link #getAllQueryPartitionPaths()}
+   */
+  protected Map<PartitionPath, List<FileSlice>> getAllInputFileSlices() {
+    if (!areAllFileSlicesCached()) {
+      // Fetching file slices for partitions that have not been cached yet
+      List<PartitionPath> missingPartitions = getAllQueryPartitionPaths().stream()
+          .filter(p -> !cachedAllInputFileSlices.containsKey(p))
+          .collect(Collectors.toList());
+
+      // NOTE: Individual partitions are always cached in full, therefore if partition is cached
+      //       it will hold all the file-slices residing w/in the partition
+      cachedAllInputFileSlices.putAll(loadFileSlicesForPartitions(missingPartitions));
+    }
+
+    return cachedAllInputFileSlices;
+  }
+
+  /**
+   * Get input file slice for the given partition. Will use cache directly if it is computed before.
+   */
+  protected List<FileSlice> getInputFileSlices(PartitionPath partition) {
+    return cachedAllInputFileSlices.computeIfAbsent(partition,
+        p -> loadFileSlicesForPartitions(Collections.singletonList(p)).get(p));
+  }
+
+  private Map<PartitionPath, List<FileSlice>> loadFileSlicesForPartitions(List<PartitionPath> partitions) {
+    Map<PartitionPath, FileStatus[]> partitionFiles = partitions.stream()
+        .collect(Collectors.toMap(p -> p, this::loadPartitionPathFiles));
+    HoodieTimeline activeTimeline = getActiveTimeline();
+    Option<HoodieInstant> latestInstant = activeTimeline.lastInstant();
+
+    FileStatus[] allFiles = partitionFiles.values().stream().flatMap(Arrays::stream).toArray(FileStatus[]::new);
+    HoodieTableFileSystemView fileSystemView = new HoodieTableFileSystemView(metaClient, activeTimeline, allFiles);

Review Comment:
   Oh, i see your point now @psendyk!
   
   Yes, that's an issue w/ how MT caching is working -- currently, whenever you do partial lookup (as compared to parsing whole files partition) it will clear the cache before proceeding, essentially re-parsing files partition every time.
   
   I have a [PR](https://github.com/apache/hudi/pull/6782) out to address this, but need to put in a few more cycles into it to address the issues carefully.
   
   



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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1250241469

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473) 
   * d8495f35bf97ed4ec27a3841dad66307d862d65e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1309466349

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822) 
   * 59cdd09e3190c3646e1e3ea6ca3f076526ec0473 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1311148593

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     }, {
       "hash" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937",
       "triggerID" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12938",
       "triggerID" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 522204901637ca82f8232c63d9b26adce3f484b8 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937) 
   * 5341fff4dfb30afde48c370a7c6cb6e31b389539 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12938) 
   * 8bafc71d82607b40cb99505d145bbfddb8c81ae3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1311098147

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     }, {
       "hash" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937",
       "triggerID" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 59cdd09e3190c3646e1e3ea6ca3f076526ec0473 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912) 
   * 522204901637ca82f8232c63d9b26adce3f484b8 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937) 
   * 5341fff4dfb30afde48c370a7c6cb6e31b389539 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #6680: [HUDI-4812] Lazy fetching partition path & file slice for HoodieFileIndex

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6680:
URL: https://github.com/apache/hudi/pull/6680#issuecomment-1312189257

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11473",
       "triggerID" : "d39358bf6cd185715f9cd4d7cdbb3a10cb6b8d0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11475",
       "triggerID" : "d8495f35bf97ed4ec27a3841dad66307d862d65e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11708",
       "triggerID" : "21b5c2377dd577e4ff8257f87fc0cda3f089c7d3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11783",
       "triggerID" : "efc19bfcfb86bf582d4bd2584462083b8178c1c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12065",
       "triggerID" : "5f6d4f624c5f20cf3c4c38384e17c7bb13e56991",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12087",
       "triggerID" : "fd5036ba6e5768ec3f846a41d766e776b8106551",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12382",
       "triggerID" : "a65056adaa4e9fabda4205c1b5c7be2e48bdd67f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "903ac79702d02fe968d271b08a217f8401d63700",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12413",
       "triggerID" : "903ac79702d02fe968d271b08a217f8401d63700",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12493",
       "triggerID" : "a09114e1c326791e33e910b2f660aaa6882dcfc9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1288289575",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "df0a3523720f59827e74c67354d465966e17e176",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12630",
       "triggerID" : "df0a3523720f59827e74c67354d465966e17e176",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12631",
       "triggerID" : "f12d60f65b7e33cf69783b80eac0463e62f6f170",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12804",
       "triggerID" : "fe18f26f8ab63a28c98b23c6940b6135b6155d91",
       "triggerType" : "PUSH"
     }, {
       "hash" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12819",
       "triggerID" : "13b8d331e0dcf5ac1f958b91770e2024bce04c43",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12822",
       "triggerID" : "c3aba0dc3e2f7c2c6240d3aa5bc279cf8f359153",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12912",
       "triggerID" : "59cdd09e3190c3646e1e3ea6ca3f076526ec0473",
       "triggerType" : "PUSH"
     }, {
       "hash" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12937",
       "triggerID" : "522204901637ca82f8232c63d9b26adce3f484b8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12938",
       "triggerID" : "5341fff4dfb30afde48c370a7c6cb6e31b389539",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12942",
       "triggerID" : "8bafc71d82607b40cb99505d145bbfddb8c81ae3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1f7e288c8e5e69a3a406c8e22761d752a068b7f3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1f7e288c8e5e69a3a406c8e22761d752a068b7f3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8bafc71d82607b40cb99505d145bbfddb8c81ae3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12942) 
   * 1f7e288c8e5e69a3a406c8e22761d752a068b7f3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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