[
https://issues.apache.org/jira/browse/HUDI-2243?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17394842#comment-17394842
]
ASF GitHub Bot commented on HUDI-2243:
--------------------------------------
pengzhiwei2018 commented on a change in pull request #3360:
URL: https://github.com/apache/hudi/pull/3360#discussion_r684327143
##########
File path:
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala
##########
@@ -214,15 +217,23 @@ case class HoodieFileIndex(
metaClient.reloadActiveTimeline()
val activeInstants =
metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants
+ val latestInstant = activeInstants.lastInstant()
fileSystemView = new HoodieTableFileSystemView(metaClient, activeInstants,
allFiles)
+ val queryInstant = if (specifiedQueryInstant.isDefined) {
+ specifiedQueryInstant
+ } else if (latestInstant.isPresent) {
+ Some(latestInstant.get.getTimestamp)
+ } else {
+ None
+ }
(tableType, queryType) match {
case (MERGE_ON_READ, QUERY_TYPE_SNAPSHOT_OPT_VAL) =>
// Fetch and store latest base and log files, and their sizes
cachedAllInputFileSlices = partitionFiles.map(p => {
val latestSlices = if (activeInstants.lastInstant().isPresent) {
-
fileSystemView.getLatestMergedFileSlicesBeforeOrOn(p._1.partitionPath,
-
activeInstants.lastInstant().get().getTimestamp).iterator().asScala.toSeq
+
fileSystemView.getLatestMergedFileSlicesBeforeOrOn(p._1.partitionPath,
queryInstant.get)
Review comment:
Good catch!
##########
File path:
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala
##########
@@ -238,7 +249,11 @@ case class HoodieFileIndex(
case (_, _) =>
// Fetch and store latest base files and its sizes
cachedAllInputFileSlices = partitionFiles.map(p => {
- (p._1,
fileSystemView.getLatestFileSlices(p._1.partitionPath).iterator().asScala.toSeq)
+ val fileSlices = queryInstant
Review comment:
I will change the code to keep the origin behavior for non-time travel
query.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
> Support Time Travel Query For Hoodie Table
> ------------------------------------------
>
> Key: HUDI-2243
> URL: https://issues.apache.org/jira/browse/HUDI-2243
> Project: Apache Hudi
> Issue Type: Bug
> Components: Spark Integration
> Reporter: pengzhiwei
> Assignee: pengzhiwei
> Priority: Major
> Labels: pull-request-available
>
> Support time travel query for hoodie table for both COW and MOR table.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)