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/06/01 04:18:44 UTC

[GitHub] [hudi] boneanxs opened a new pull request, #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   ## *Tips*
   - *Thank you very much for contributing to Apache Hudi.*
   - *Please review https://hudi.apache.org/contribute/how-to-contribute before opening a pull request.*
   
   ## What is the purpose of the pull request
   If a hudi table is all logs files without any base files, following query will return 0
   
   ```scala
   spark.read.format("hudi")
   .load(basePath + "/*/*/*/*")
   .count()
   ```
   ## Brief change log
   
   *(for example:)*
     - *Modify AnnotationLocation checkstyle rule in checkstyle.xml*
   
   ## Verify this pull request
   
   *(Please pick either of the following options)*
   
   This pull request is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This pull request is already covered by existing tests, such as *(please describe tests)*.
   
   (or)
   
   This change added tests and can be verified as follows:
   
   *(example:)*
   
     - *Added integration tests for end-to-end.*
     - *Added HoodieClientWriteTest to verify the change.*
     - *Manually verified the change by running a job locally.*
   
   ## Committer checklist
   
    - [ ] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.
   


-- 
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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala:
##########
@@ -104,14 +104,22 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
       val fileSlices = fileIndex.listFileSlices(convertedPartitionFilters)
       buildSplits(fileSlices.values.flatten.toSeq)
     } else {
-      // TODO refactor to avoid iterating over listed files multiple times
-      val partitions = listLatestBaseFiles(globPaths, convertedPartitionFilters, dataFilters)
-      val partitionPaths = partitions.keys.toSeq
+      val inMemoryFileIndex = HoodieInMemoryFileIndex.create(sparkSession, globPaths)

Review Comment:
   @boneanxs appreciate your effort of addressing this TODO.
   
   Although, the better way to refactor this would be to instead change `listLatestBaseFiles` to be `listLatestFileSlices` and return file-slices instead of just the base-files. Right now we've essentially duplicated this method implementation in here. I'd suggest we avoid the duplication by changing the method as noted above and keeping this code mostly intact.
   
   Keep in mind: one of the goals of the refactoring of COW/MOR relations was to bring implementations closer together making the bifurcate only at a points where it's necessary (for ex, in a way the actually read file-slices) and everywhere else keep them mostly identical.



-- 
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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/execution/datasources/HoodieInMemoryFileIndex.scala:
##########
@@ -34,7 +37,77 @@ class HoodieInMemoryFileIndex(sparkSession: SparkSession,
                               parameters: Map[String, String],
                               userSpecifiedSchema: Option[StructType],
                               fileStatusCache: FileStatusCache = NoopCache)
-  extends InMemoryFileIndex(sparkSession, rootPathsSpecified, parameters, userSpecifiedSchema, fileStatusCache) {
+  extends InMemoryFileIndex(sparkSession, rootPathsSpecified, parameters, userSpecifiedSchema, fileStatusCache)
+  with SparkAdapterSupport {
+
+  /**
+   * Returns all valid files grouped into partitions when the data is partitioned. If the data is unpartitioned,
+   * this will return a single partition with no partition values
+   *
+   * NOTE: This method replicates the one it overrides, however it uses custom method
+   * that accepts files starting with "."
+   */
+  override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = {

Review Comment:
   It makes sense now. Thanks for clarifying!



-- 
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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023",
       "triggerID" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143414292",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1143693845",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 3baf9d12d91a1375f37f679cd2f9b6205ff349cf Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028) 
   
   <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] boneanxs commented on pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   Gentle ping @codope, could you plz review this 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] hudi-bot commented on pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023",
       "triggerID" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143414292",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143693845",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 3baf9d12d91a1375f37f679cd2f9b6205ff349cf Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028) 
   
   <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] boneanxs commented on pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   @xushiyan @alexeykudinkin Could you plz review this?
   The CI error looks is not relate 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] codope commented on a diff in pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala:
##########
@@ -340,6 +340,21 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
     latestBaseFiles.groupBy(getPartitionPath)
   }
 
+  protected def getSelectedPartitionPaths(
+      globbedPaths: Seq[Path],
+      partitionFilters: Seq[Expression],
+      dataFilters: Seq[Expression]): Seq[Path] = {
+    val partitionDirs = if (globbedPaths.isEmpty) {
+      fileIndex.listFiles(partitionFilters, dataFilters)
+    } else {
+      val inMemoryFileIndex = HoodieInMemoryFileIndex.create(sparkSession, globbedPaths)
+      inMemoryFileIndex.listFiles(partitionFilters, dataFilters)
+    }
+
+    val fsView = new HoodieTableFileSystemView(metaClient, timeline, partitionDirs.flatMap(_.files).toArray)

Review Comment:
   it's being repeated in `listLatestBaseFiles`. Let's extract to a method?
   Also, I think it would be better if we keep the API signatures similar if possible. Currently, `listLatestBaseFiles` returns a Map while this method returns a Seq.



-- 
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] codope merged pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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


-- 
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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023",
       "triggerID" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143414292",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 3baf9d12d91a1375f37f679cd2f9b6205ff349cf Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028) 
   
   <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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023",
       "triggerID" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143414292",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143693845",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1144471085",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 3baf9d12d91a1375f37f679cd2f9b6205ff349cf Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028) 
   
   <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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023",
       "triggerID" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1143414292",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 3baf9d12d91a1375f37f679cd2f9b6205ff349cf Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028) 
   
   <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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023",
       "triggerID" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143414292",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143693845",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1144471085",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d67e7a8ddac236c3af0724a08c23e493d0d9e969",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9083",
       "triggerID" : "d67e7a8ddac236c3af0724a08c23e493d0d9e969",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3baf9d12d91a1375f37f679cd2f9b6205ff349cf Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028) 
   * d67e7a8ddac236c3af0724a08c23e493d0d9e969 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9083) 
   
   <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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023",
       "triggerID" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143414292",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143693845",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1144471085",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d67e7a8ddac236c3af0724a08c23e493d0d9e969",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9083",
       "triggerID" : "d67e7a8ddac236c3af0724a08c23e493d0d9e969",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6e3a4e36e60020c4e6601745c252536643180f2",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9155",
       "triggerID" : "c6e3a4e36e60020c4e6601745c252536643180f2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c6e3a4e36e60020c4e6601745c252536643180f2 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9155) 
   
   <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] codope commented on a diff in pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/execution/datasources/HoodieInMemoryFileIndex.scala:
##########
@@ -34,7 +37,77 @@ class HoodieInMemoryFileIndex(sparkSession: SparkSession,
                               parameters: Map[String, String],
                               userSpecifiedSchema: Option[StructType],
                               fileStatusCache: FileStatusCache = NoopCache)
-  extends InMemoryFileIndex(sparkSession, rootPathsSpecified, parameters, userSpecifiedSchema, fileStatusCache) {
+  extends InMemoryFileIndex(sparkSession, rootPathsSpecified, parameters, userSpecifiedSchema, fileStatusCache)
+  with SparkAdapterSupport {
+
+  /**
+   * Returns all valid files grouped into partitions when the data is partitioned. If the data is unpartitioned,
+   * this will return a single partition with no partition values
+   *
+   * NOTE: This method replicates the one it overrides, however it uses custom method
+   * that accepts files starting with "."
+   */
+  override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = {
+    val selectedPartitions = if (partitionSpec().partitionColumns.isEmpty) {
+      PartitionDirectory(InternalRow.empty, allFiles().filter(f => isDataPath(f.getPath))) :: Nil
+    } else {
+      prunePartitions(partitionFilters, partitionSpec()).map {
+        case PartitionPath(values, path) =>
+          val files: Seq[FileStatus] = leafDirToChildrenFiles.get(path) match {

Review Comment:
   even with partition pruning, this method is going to list files. FileStatus[] is already built in `AbstractTableFileSystemView`. So, now the same set of files are being listed twice i.e. once while building filesystem view and second time here. Is there a way to avoid that? 



-- 
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] codope commented on a diff in pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/execution/datasources/HoodieInMemoryFileIndex.scala:
##########
@@ -34,7 +37,77 @@ class HoodieInMemoryFileIndex(sparkSession: SparkSession,
                               parameters: Map[String, String],
                               userSpecifiedSchema: Option[StructType],
                               fileStatusCache: FileStatusCache = NoopCache)
-  extends InMemoryFileIndex(sparkSession, rootPathsSpecified, parameters, userSpecifiedSchema, fileStatusCache) {
+  extends InMemoryFileIndex(sparkSession, rootPathsSpecified, parameters, userSpecifiedSchema, fileStatusCache)
+  with SparkAdapterSupport {
+
+  /**
+   * Returns all valid files grouped into partitions when the data is partitioned. If the data is unpartitioned,
+   * this will return a single partition with no partition values
+   *
+   * NOTE: This method replicates the one it overrides, however it uses custom method
+   * that accepts files starting with "."
+   */
+  override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = {
+    val selectedPartitions = if (partitionSpec().partitionColumns.isEmpty) {
+      PartitionDirectory(InternalRow.empty, allFiles().filter(f => isDataPath(f.getPath))) :: Nil
+    } else {
+      prunePartitions(partitionFilters, partitionSpec()).map {
+        case PartitionPath(values, path) =>
+          val files: Seq[FileStatus] = leafDirToChildrenFiles.get(path) match {

Review Comment:
   Got it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [hudi] hudi-bot commented on pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023",
       "triggerID" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 27fd4aa08c38389f5205ce266cdf3f852933d87b Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023) 
   
   <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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023",
       "triggerID" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 27fd4aa08c38389f5205ce266cdf3f852933d87b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023) 
   
   <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] nsivabalan commented on pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   @codope : can you follow up on this when you get a chance. 


-- 
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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala:
##########
@@ -104,14 +104,22 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
       val fileSlices = fileIndex.listFileSlices(convertedPartitionFilters)
       buildSplits(fileSlices.values.flatten.toSeq)
     } else {
-      // TODO refactor to avoid iterating over listed files multiple times
-      val partitions = listLatestBaseFiles(globPaths, convertedPartitionFilters, dataFilters)
-      val partitionPaths = partitions.keys.toSeq
+      val inMemoryFileIndex = HoodieInMemoryFileIndex.create(sparkSession, globPaths)

Review Comment:
   cc @codope 



-- 
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] boneanxs commented on a diff in pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/execution/datasources/HoodieInMemoryFileIndex.scala:
##########
@@ -34,7 +37,77 @@ class HoodieInMemoryFileIndex(sparkSession: SparkSession,
                               parameters: Map[String, String],
                               userSpecifiedSchema: Option[StructType],
                               fileStatusCache: FileStatusCache = NoopCache)
-  extends InMemoryFileIndex(sparkSession, rootPathsSpecified, parameters, userSpecifiedSchema, fileStatusCache) {
+  extends InMemoryFileIndex(sparkSession, rootPathsSpecified, parameters, userSpecifiedSchema, fileStatusCache)
+  with SparkAdapterSupport {
+
+  /**
+   * Returns all valid files grouped into partitions when the data is partitioned. If the data is unpartitioned,
+   * this will return a single partition with no partition values
+   *
+   * NOTE: This method replicates the one it overrides, however it uses custom method
+   * that accepts files starting with "."
+   */
+  override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = {

Review Comment:
   This is needed, as we'll get partitions in `listLatestBaseFile` before,
   ```scala
   protected def listLatestBaseFiles(globbedPaths: Seq[Path], partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Map[Path, Seq[FileStatus]] = {
       val partitionDirs = if (globbedPaths.isEmpty) {
         fileIndex.listFiles(partitionFilters, dataFilters)
       } else {
         val inMemoryFileIndex = HoodieInMemoryFileIndex.create(sparkSession, globbedPaths)
         inMemoryFileIndex.listFiles(partitionFilters, dataFilters)
       }
   ```
   which will call `inMemoryFileIndex.listFiles` to get all partitionDirs, if we don't overwrite this method, log paths will be filtered
   
   ```scala
   val selectedPartitions = if (partitionSpec().partitionColumns.isEmpty) {
         // Method isDataPath will filter path if it is a log file
         PartitionDirectory(InternalRow.empty, allFiles().filter(f => isDataPath(f.getPath))) :: Nil
       }
   ```
   
   As `isDataPath` is a private method in `PartitioningAwareFileIndex`, we can't overwrite it directly, so we need to overwrite `listFiles`.



-- 
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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023",
       "triggerID" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 27fd4aa08c38389f5205ce266cdf3f852933d87b Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023) 
   * 3baf9d12d91a1375f37f679cd2f9b6205ff349cf Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028) 
   
   <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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023",
       "triggerID" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3baf9d12d91a1375f37f679cd2f9b6205ff349cf Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028) 
   
   <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] boneanxs commented on a diff in pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/execution/datasources/HoodieInMemoryFileIndex.scala:
##########
@@ -34,7 +37,77 @@ class HoodieInMemoryFileIndex(sparkSession: SparkSession,
                               parameters: Map[String, String],
                               userSpecifiedSchema: Option[StructType],
                               fileStatusCache: FileStatusCache = NoopCache)
-  extends InMemoryFileIndex(sparkSession, rootPathsSpecified, parameters, userSpecifiedSchema, fileStatusCache) {
+  extends InMemoryFileIndex(sparkSession, rootPathsSpecified, parameters, userSpecifiedSchema, fileStatusCache)
+  with SparkAdapterSupport {
+
+  /**
+   * Returns all valid files grouped into partitions when the data is partitioned. If the data is unpartitioned,
+   * this will return a single partition with no partition values
+   *
+   * NOTE: This method replicates the one it overrides, however it uses custom method
+   * that accepts files starting with "."
+   */
+  override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = {

Review Comment:
   Override this to accept files starting with "." to allow log files.



-- 
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] boneanxs commented on pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   @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] hudi-bot commented on pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023",
       "triggerID" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143414292",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143693845",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1144471085",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d67e7a8ddac236c3af0724a08c23e493d0d9e969",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d67e7a8ddac236c3af0724a08c23e493d0d9e969",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3baf9d12d91a1375f37f679cd2f9b6205ff349cf Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028) 
   * d67e7a8ddac236c3af0724a08c23e493d0d9e969 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] boneanxs commented on a diff in pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/execution/datasources/HoodieInMemoryFileIndex.scala:
##########
@@ -34,7 +37,77 @@ class HoodieInMemoryFileIndex(sparkSession: SparkSession,
                               parameters: Map[String, String],
                               userSpecifiedSchema: Option[StructType],
                               fileStatusCache: FileStatusCache = NoopCache)
-  extends InMemoryFileIndex(sparkSession, rootPathsSpecified, parameters, userSpecifiedSchema, fileStatusCache) {
+  extends InMemoryFileIndex(sparkSession, rootPathsSpecified, parameters, userSpecifiedSchema, fileStatusCache)
+  with SparkAdapterSupport {
+
+  /**
+   * Returns all valid files grouped into partitions when the data is partitioned. If the data is unpartitioned,
+   * this will return a single partition with no partition values
+   *
+   * NOTE: This method replicates the one it overrides, however it uses custom method
+   * that accepts files starting with "."
+   */
+  override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = {
+    val selectedPartitions = if (partitionSpec().partitionColumns.isEmpty) {
+      PartitionDirectory(InternalRow.empty, allFiles().filter(f => isDataPath(f.getPath))) :: Nil
+    } else {
+      prunePartitions(partitionFilters, partitionSpec()).map {
+        case PartitionPath(values, path) =>
+          val files: Seq[FileStatus] = leafDirToChildrenFiles.get(path) match {

Review Comment:
   Plz correct me if I'm wrong, I think here only being listed once, `HoodieInMemoryFileIndex` will call `refresh0` during construction, this will list all fileStatus and save them in `cachedLeafDirToChildrenFiles` and `cachedLeafFiles`. here `leafDirToChildrenFiles` will use cache to get fileStatus.
   
   As `HoodieTableFileSystemView` will reuse the result `HoodieInMemoryFileIndex` it returns, so I think here we will not add a new listing operation compare to previous codes.



-- 
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] boneanxs commented on pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   @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] boneanxs commented on a diff in pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala:
##########
@@ -104,14 +104,22 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
       val fileSlices = fileIndex.listFileSlices(convertedPartitionFilters)
       buildSplits(fileSlices.values.flatten.toSeq)
     } else {
-      // TODO refactor to avoid iterating over listed files multiple times
-      val partitions = listLatestBaseFiles(globPaths, convertedPartitionFilters, dataFilters)
-      val partitionPaths = partitions.keys.toSeq
+      val inMemoryFileIndex = HoodieInMemoryFileIndex.create(sparkSession, globPaths)

Review Comment:
   I'll address this in https://github.com/apache/hudi/pull/5722



-- 
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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/execution/datasources/HoodieInMemoryFileIndex.scala:
##########
@@ -34,7 +37,77 @@ class HoodieInMemoryFileIndex(sparkSession: SparkSession,
                               parameters: Map[String, String],
                               userSpecifiedSchema: Option[StructType],
                               fileStatusCache: FileStatusCache = NoopCache)
-  extends InMemoryFileIndex(sparkSession, rootPathsSpecified, parameters, userSpecifiedSchema, fileStatusCache) {
+  extends InMemoryFileIndex(sparkSession, rootPathsSpecified, parameters, userSpecifiedSchema, fileStatusCache)
+  with SparkAdapterSupport {
+
+  /**
+   * Returns all valid files grouped into partitions when the data is partitioned. If the data is unpartitioned,
+   * this will return a single partition with no partition values
+   *
+   * NOTE: This method replicates the one it overrides, however it uses custom method
+   * that accepts files starting with "."
+   */
+  override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = {

Review Comment:
   Why is there a need for that? That's exactly what `listLeafFiles` is overridden for 



-- 
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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023",
       "triggerID" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 27fd4aa08c38389f5205ce266cdf3f852933d87b Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023) 
   * 3baf9d12d91a1375f37f679cd2f9b6205ff349cf 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] boneanxs commented on a diff in pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala:
##########
@@ -340,6 +340,21 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
     latestBaseFiles.groupBy(getPartitionPath)
   }
 
+  protected def getSelectedPartitionPaths(
+      globbedPaths: Seq[Path],
+      partitionFilters: Seq[Expression],
+      dataFilters: Seq[Expression]): Seq[Path] = {
+    val partitionDirs = if (globbedPaths.isEmpty) {
+      fileIndex.listFiles(partitionFilters, dataFilters)
+    } else {
+      val inMemoryFileIndex = HoodieInMemoryFileIndex.create(sparkSession, globbedPaths)
+      inMemoryFileIndex.listFiles(partitionFilters, dataFilters)
+    }
+
+    val fsView = new HoodieTableFileSystemView(metaClient, timeline, partitionDirs.flatMap(_.files).toArray)

Review Comment:
   Here I move the codes to its caller `collectFileSplits` to reuse `HoodieTableFileSystemView` and `HoodieInMemoryFileIndex`. Also I think this can address the TODO that avoid iterating over listed files multiple times.



-- 
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] boneanxs commented on a diff in pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala:
##########
@@ -340,6 +340,21 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
     latestBaseFiles.groupBy(getPartitionPath)
   }
 
+  protected def getSelectedPartitionPaths(
+      globbedPaths: Seq[Path],
+      partitionFilters: Seq[Expression],
+      dataFilters: Seq[Expression]): Seq[Path] = {
+    val partitionDirs = if (globbedPaths.isEmpty) {
+      fileIndex.listFiles(partitionFilters, dataFilters)
+    } else {
+      val inMemoryFileIndex = HoodieInMemoryFileIndex.create(sparkSession, globbedPaths)
+      inMemoryFileIndex.listFiles(partitionFilters, dataFilters)
+    }
+
+    val fsView = new HoodieTableFileSystemView(metaClient, timeline, partitionDirs.flatMap(_.files).toArray)

Review Comment:
   Sure, will do



-- 
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] boneanxs commented on pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   > @boneanxs Left a comment. Do you also plan to fix it for other query engines? I think if we fix the issue in the `AbstractTableFileSystemView`, add an API to list file slices and give a merged view, that would help hive-compatible engines as well.
   
   @codope Could other hive-compatible engines also use glob paths to query HUDI? I'm not very familiar about these engines, If you could give me some hints, I'm willing to fix them as well :)


-- 
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] boneanxs commented on pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   @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] hudi-bot commented on pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 27fd4aa08c38389f5205ce266cdf3f852933d87b 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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023",
       "triggerID" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143414292",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143693845",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1144471085",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 3baf9d12d91a1375f37f679cd2f9b6205ff349cf Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028) 
   
   <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] boneanxs closed pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

Posted by GitBox <gi...@apache.org>.
boneanxs closed pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths
URL: https://github.com/apache/hudi/pull/5723


-- 
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] boneanxs commented on pull request #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   Gentle ping @alexeykudinkin @xushiyan @codope @nsivabalan...


-- 
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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023",
       "triggerID" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143414292",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143693845",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1144471085",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d67e7a8ddac236c3af0724a08c23e493d0d9e969",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9083",
       "triggerID" : "d67e7a8ddac236c3af0724a08c23e493d0d9e969",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d67e7a8ddac236c3af0724a08c23e493d0d9e969 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9083) 
   
   <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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023",
       "triggerID" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143414292",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143693845",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1144471085",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d67e7a8ddac236c3af0724a08c23e493d0d9e969",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9083",
       "triggerID" : "d67e7a8ddac236c3af0724a08c23e493d0d9e969",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6e3a4e36e60020c4e6601745c252536643180f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9155",
       "triggerID" : "c6e3a4e36e60020c4e6601745c252536643180f2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d67e7a8ddac236c3af0724a08c23e493d0d9e969 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9083) 
   * c6e3a4e36e60020c4e6601745c252536643180f2 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9155) 
   
   <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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9023",
       "triggerID" : "27fd4aa08c38389f5205ce266cdf3f852933d87b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143414292",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1143693845",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3baf9d12d91a1375f37f679cd2f9b6205ff349cf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9028",
       "triggerID" : "1144471085",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "d67e7a8ddac236c3af0724a08c23e493d0d9e969",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9083",
       "triggerID" : "d67e7a8ddac236c3af0724a08c23e493d0d9e969",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c6e3a4e36e60020c4e6601745c252536643180f2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c6e3a4e36e60020c4e6601745c252536643180f2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d67e7a8ddac236c3af0724a08c23e493d0d9e969 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9083) 
   * c6e3a4e36e60020c4e6601745c252536643180f2 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 #5723: [HUDI-4173] Fix wrong results if the user read no base files hudi table by glob paths

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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala:
##########
@@ -104,14 +104,22 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
       val fileSlices = fileIndex.listFileSlices(convertedPartitionFilters)
       buildSplits(fileSlices.values.flatten.toSeq)
     } else {
-      // TODO refactor to avoid iterating over listed files multiple times
-      val partitions = listLatestBaseFiles(globPaths, convertedPartitionFilters, dataFilters)
-      val partitionPaths = partitions.keys.toSeq
+      val inMemoryFileIndex = HoodieInMemoryFileIndex.create(sparkSession, globPaths)

Review Comment:
   @boneanxs appreciate your effort of addressing this TODO.
   
   Although, the better way to refactor this would be to instead change `listLatestBaseFiles` to be `listLatestFileSlices` and return file-slices instead of just the base-files. Right now we've essentially duplicated this method implementation in here. I'd suggest we avoid the duplication by changing the method as noted above and keeping this code mostly intact.
   
   Keep in mind: one of the goals of the refactoring of COW/MOR relations was to bring implementations closer together making them bifurcate only at a points where it's necessary (for ex, in a way they actually read file-slices) and everywhere else keep them mostly identical.



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