yihua commented on code in PR #7744:
URL: https://github.com/apache/hudi/pull/7744#discussion_r1085798921


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala:
##########
@@ -223,7 +223,7 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
       //       we might not be able to properly parse partition-values from 
the listed partition-paths.
       //       In that case, we simply could not apply partition pruning and 
will have to regress to scanning
       //       the whole table
-      if (haveProperPartitionValues(partitionPaths)) {
+      if (haveProperPartitionValues(partitionPaths) && 
partitionSchema.nonEmpty) {

Review Comment:
   This fix is needed for the case where the partition column value has slashes 
(partition column values are not supposed to be filled), and the query 
predicates cover all the partition columns, so the exact matching of a 
partition path happens and partition values are filled inside `partitionPaths` 
(see below code snippet from 
`SparkHoodieTableFileIndex#tryListByPartitionPathPrefix`).
   
   ```
     if (staticPartitionColumnNameValuePairs.length == 
partitionColumnNames.length) {
           // In case composed partition path is complete, we can return it 
directly avoiding extra listing operation
           Seq(new PartitionPath(relativePartitionPathPrefix, 
staticPartitionColumnNameValuePairs.map(_._2.asInstanceOf[AnyRef]).toArray))
         }
   ```
   
   Without this fix, the test case 3 (with predicate `"dt = '2023/01/01' and 
region_code = '1'"`) in `testFileListingWithPartitionPrefixPruning` would fail.



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