[ 
https://issues.apache.org/jira/browse/HUDI-2107?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17375614#comment-17375614
 ] 

ASF GitHub Bot commented on HUDI-2107:
--------------------------------------

xiarixiaoyao commented on a change in pull request #3193:
URL: https://github.com/apache/hudi/pull/3193#discussion_r664363268



##########
File path: 
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala
##########
@@ -137,11 +136,14 @@ class MergeOnReadSnapshotRelation(val sqlContext: 
SQLContext,
   }
 
   def buildFileIndex(filters: Array[Filter]): List[HoodieMergeOnReadFileSplit] 
= {
-
-    val fileStatuses = if (globPaths.isDefined) {
+    // Get all partition paths
+    val partitionPaths = if (globPaths.isDefined) {
       // Load files from the global paths if it has defined to be compatible 
with the original mode
       val inMemoryFileIndex = 
HoodieSparkUtils.createInMemoryFileIndex(sqlContext.sparkSession, globPaths.get)
-      inMemoryFileIndex.allFiles()
+      val fsView = new HoodieTableFileSystemView(metaClient,
+        metaClient.getActiveTimeline.getCommitsTimeline
+          .filterCompletedInstants, inMemoryFileIndex.allFiles().toArray)
+      
fsView.getLatestBaseFiles.iterator().asScala.toList.map(_.getFileStatus.getPath.getParent)

Review comment:
       if we only have log files.  inMemoryFileIndex.allfiles will be empty, 
since spark will filter .log
   
fsView.getLatestBaseFiles.iterator().asScala.toList.map(_.getFileStatus.getPath.getParent)
  will return a empty partitionPaths.  then buildFileIndex will return 
HoodieMergeOnReadFileSplit, nothings will be read




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

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

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


> Support Read Log Only MOR Table For Spark
> -----------------------------------------
>
>                 Key: HUDI-2107
>                 URL: https://issues.apache.org/jira/browse/HUDI-2107
>             Project: Apache Hudi
>          Issue Type: Bug
>          Components: Spark Integration
>            Reporter: pengzhiwei
>            Assignee: pengzhiwei
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 0.9.0
>
>
> Currently we cannot support read log-only mor table(which is generated by 
> index like InMemeoryIndex, HbaseIndex and FlinkIndex which support indexing 
> log file) for spark.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to