[ https://issues.apache.org/jira/browse/HUDI-5807?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17762918#comment-17762918 ]
Lin Liu commented on HUDI-5807: ------------------------------- Tried to reproduce the issue, and run the following script in local, and the partition_path column showed. Not sure if this issue has been fixed already. My script is as follows. {code:java} spark-shell --jars /Users/linliu/projects/hudi/packaging/hudi-spark-bundle/target/hudi-spark3.4-bundle_2.12-1.0.0-SNAPSHOT.jar,/Users/linliu/.m2/repository/com/github/f4b6a3/uuid-creator/5.3.2/uuid-creator-5.3.2.jar \ --conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer' \ --conf 'spark.sql.catalog.spark_catalog=org.apache.spark.sql.hudi.catalog.HoodieCatalog' \ --conf 'spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension' \ --conf 'hoodie.datasource.write.drop.partition.columns=true' import org.apache.hudi.QuickstartUtils._ import scala.collection.JavaConversions._ import org.apache.spark.sql.SaveMode._ import org.apache.hudi.DataSourceReadOptions._ import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.config.HoodieWriteConfig._ import org.apache.hudi.common.model.HoodieRecordval tableName = "uuid_test" val basePath = "file:///tmp/uuid_test1" val dataGen = new DataGenerator val inserts = convertToStringList(dataGen.generateInserts(10000)) val df = spark.read.json(spark.sparkContext.parallelize(inserts, 1)) df.write.format("hudi"). options(getQuickstartWriteConfigs). option(PRECOMBINE_FIELD_OPT_KEY, "ts"). option(RECORDKEY_FIELD_OPT_KEY, "uuid"). option(PARTITIONPATH_FIELD_OPT_KEY, "partitionpath"). option(TABLE_NAME, tableName). mode(Overwrite). save(basePath)val uuidv_DF = spark. read. format("hudi"). load(basePath) uuidv_DF.createOrReplaceTempView("uuid_test_view")spark.sql("select * from uuid_test_view limit 10").show(false) {code} The result is as follows. {code:java} scala> spark.sql("select * from uuid_test_view limit 1").show(false) +-------------------+---------------------+------------------------------------+-------------------------+------------------------------------------------------------------------+-----------------+-------------------+----------+--------------------+-------------------+-----------------+---------+-------------+------------------------------------+-------------------------+ |_hoodie_commit_time|_hoodie_commit_seqno |_hoodie_record_key |_hoodie_partition_path |_hoodie_file_name |begin_lat |begin_lon |driver |end_lat |end_lon |fare |rider |ts |uuid |partitionpath | +-------------------+---------------------+------------------------------------+-------------------------+------------------------------------------------------------------------+-----------------+-------------------+----------+--------------------+-------------------+-----------------+---------+-------------+------------------------------------+-------------------------+ |20230907173941752 |20230907173941752_0_0|5911f8ea-94e5-4803-b3c4-aa574d8bab06|americas/brazil/sao_paulo|ea97ba95-dcfd-43f0-953b-9aac94a70474-0_0-24-23_20230907173941752.parquet|0.864670567383501|0.07253990487069728|driver-213|0.024148074339951364|0.12390571197157785|49.37822483824809|rider-213|1693834352147|5911f8ea-94e5-4803-b3c4-aa574d8bab06|americas/brazil/sao_paulo| +-------------------+---------------------+------------------------------------+-------------------------+------------------------------------------------------------------------+-----------------+-------------------+----------+--------------------+-------------------+-----------------+---------+-------------+------------------------------------+-------------------------+{code} > HoodieSparkParquetReader is not appending partition-path values > --------------------------------------------------------------- > > Key: HUDI-5807 > URL: https://issues.apache.org/jira/browse/HUDI-5807 > Project: Apache Hudi > Issue Type: Bug > Components: spark > Affects Versions: 0.13.0 > Reporter: Alexey Kudinkin > Assignee: Lin Liu > Priority: Blocker > Fix For: 1.0.0 > > > Current implementation of HoodieSparkParquetReader isn't supporting the case > when "hoodie.datasource.write.drop.partition.columns" is set to true. > In that case partition-path values are expected to be parsed from > partition-path and be injected w/in the File Reader (this is behavior of > Spark's own readers) -- This message was sent by Atlassian Jira (v8.20.10#820010)