wecharyu commented on code in PR #9889:
URL: https://github.com/apache/hudi/pull/9889#discussion_r1369866606


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala:
##########
@@ -149,27 +152,10 @@ case class BaseFileOnlyRelation(override val sqlContext: 
SQLContext,
     val enableFileIndex = HoodieSparkConfUtils.getConfigValue(optParams, 
sparkSession.sessionState.conf,
       ENABLE_HOODIE_FILE_INDEX.key, 
ENABLE_HOODIE_FILE_INDEX.defaultValue.toString).toBoolean
     if (enableFileIndex && globPaths.isEmpty) {
-      // NOTE: There are currently 2 ways partition values could be fetched:
-      //          - Source columns (producing the values used for physical 
partitioning) will be read
-      //          from the data file
-      //          - Values parsed from the actual partition path would be 
appended to the final dataset
-      //
-      //        In the former case, we don't need to provide the 
partition-schema to the relation,
-      //        therefore we simply stub it w/ empty schema and use full 
table-schema as the one being
-      //        read from the data file.

Review Comment:
   @danny0405 @yihua This PR followed HUDI-4161, the only difference is that 
when `hasSchemaOnRead` is true it migrates the 
`shouldExtractPartitionValuesFromPartitionPath` value from super class while 
the value is false before this pr. So the optimization in HUDI-4161 still works.
   
   Also we do not aim to make `shouldExtractPartitionValuesFromPartitionPath` 
alway be false after schema evolution, after all hudi supports read partition 
values from both path and files. It should be determined by configurations and 
get from super class.
   



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestGetPartitionValuesFromPath.scala:
##########
@@ -90,4 +90,37 @@ class TestGetPartitionValuesFromPath extends 
HoodieSparkSqlTestBase {
       }
     }
   }
+
+  test("Test get partition values from path when schema evolution applied") {
+    withTable(generateTableName) { tableName =>
+      spark.sql(
+        s"""
+           |create table $tableName (
+           | id int,
+           | name string,
+           | ts bigint,
+           | region string,
+           | dt date
+           |) using hudi
+           |tblproperties (
+           | primaryKey = 'id',
+           | type = 'cow',
+           | preCombineField = 'ts',
+           | hoodie.datasource.write.drop.partition.columns = 'true'
+           |)
+           |partitioned by (region, dt)""".stripMargin)
+
+      spark.sql(s"insert into $tableName partition (region='reg1', 
dt='2023-10-01') select 1, 'name1', 1000")
+      checkAnswer(s"select id, name, ts, region, cast(dt as string) from 
$tableName")(
+        Seq(1, "name1", 1000, "reg1", "2023-10-01")
+      )

Review Comment:
   Addressed.



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