codope commented on code in PR #9345:
URL: https://github.com/apache/hudi/pull/9345#discussion_r1284972391


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala:
##########
@@ -122,86 +141,132 @@ case class HoodieFileIndex(spark: SparkSession,
    * @return list of PartitionDirectory containing partition to base files 
mapping
    */
   override def listFiles(partitionFilters: Seq[Expression], dataFilters: 
Seq[Expression]): Seq[PartitionDirectory] = {
-    // Look up candidate files names in the col-stats index, if all of the 
following conditions are true
-    //    - Data-skipping is enabled
-    //    - Col-Stats Index is present
-    //    - List of predicates (filters) is present
-    val candidateFilesNamesOpt: Option[Set[String]] =
+    // Prune the partition path by the partition filters
+    // NOTE: Non-partitioned tables are assumed to consist from a single 
partition
+    //       encompassing the whole table
+    val partitionsAndFileSlices = 
getFileSlicesForPrunedPartitions(partitionFilters)
+    val partitionsAndFilteredFileSlices = filterFileSlices(dataFilters, 
partitionsAndFileSlices).map {
+      case (partitionOpt, fileSlices) =>
+        val allCandidateFiles: Seq[FileStatus] = fileSlices.flatMap(fs => {
+          val baseFileStatusOpt = 
getBaseFileStatus(Option.apply(fs.getBaseFile.orElse(null)))
+          val logFilesStatus = if (includeLogFiles) {
+            
fs.getLogFiles.map[FileStatus](JFunction.toJavaFunction[HoodieLogFile, 
FileStatus](lf => lf.getFileStatus))
+          } else {
+            java.util.stream.Stream.empty()
+          }
+          val files = 
logFilesStatus.collect(Collectors.toList[FileStatus]).asScala
+          baseFileStatusOpt.foreach(f => files.append(f))
+          files
+        })
+
+        PartitionDirectory(InternalRow.fromSeq(partitionOpt.get.values), 
allCandidateFiles)
+    }
+
+    hasPushedDownPartitionPredicates = true
+
+    if (shouldReadAsPartitionedTable()) {
+      partitionsAndFilteredFileSlices
+    } else {
+      Seq(PartitionDirectory(InternalRow.empty, 
partitionsAndFilteredFileSlices.flatMap(_.files)))
+    }
+  }
+
+  def filterFileSlices(dataFilters: Seq[Expression], partitionAndFileSlices: 
Seq[(Option[BaseHoodieTableFileIndex.PartitionPath], Seq[FileSlice])])
+  : Seq[(Option[BaseHoodieTableFileIndex.PartitionPath], Seq[FileSlice])] = {
+    // If there are no data filters, return all the file slices.
+    // If there are no file slices, return empty list.
+    if (partitionAndFileSlices.isEmpty || dataFilters.isEmpty) {
+      partitionAndFileSlices
+    } else {
+      // Look up candidate files names in the col-stats index, if all of the 
following conditions are true
+      //    - Data-skipping is enabled
+      //    - Col-Stats Index is present
+      //    - List of predicates (filters) is present
+      val candidateFilesNamesOpt: Option[Set[String]] =

Review Comment:
   Can you point me where are the file slices being filtered based on column 
stats? The `prunedCandidateFileNames` in `lookupCandidateFilesInMetadataTable` 
will return a set of base files and log files.. Then my understanding is that 
we further need to check whether any of these pruned files are part of file 
slices returned by `getAllFiles()`. If so, then take the complete file slice as 
a candidate, if none, then skip that file slice.



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

Reply via email to