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

Lin Liu edited comment on HUDI-5807 at 9/8/23 1:39 AM:
-------------------------------------------------------

Recompiled the Jar, and reran the script. But we still see the partition-path 
column correctly. I checked the .hoodie/hoodie.properties and confirmed the 
propertiy is set to true. All properties are listed as follows. 
{code:java}
#Updated at 2023-09-08T01:31:55.789Z
#Thu Sep 07 18:31:55 PDT 2023
hoodie.table.timeline.timezone=LOCAL
hoodie.table.keygenerator.class=org.apache.hudi.keygen.SimpleKeyGenerator
hoodie.table.precombine.field=ts
hoodie.table.version=6
hoodie.database.name=
hoodie.datasource.write.hive_style_partitioning=false
hoodie.table.metadata.partitions.inflight=
hoodie.table.checksum=581566219
hoodie.partition.metafile.use.base.format=false
hoodie.table.cdc.enabled=false
hoodie.archivelog.folder=archived
hoodie.table.name=uuid_test
hoodie.populate.meta.fields=true
hoodie.table.type=COPY_ON_WRITE
hoodie.datasource.write.partitionpath.urlencode=false
hoodie.table.base.file.format=PARQUET
hoodie.datasource.write.drop.partition.columns=true
hoodie.table.metadata.partitions=files
hoodie.timeline.layout.version=1
hoodie.table.recordkey.fields=uuid
hoodie.table.partition.fields=partitionpath {code}
 

And the query result still shows the partition-path column.
{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            |
+-------------------+---------------------+------------------------------------+-------------------------+------------------------------------------------------------------------+------------------+------------------+----------+------------------+------------------+-----------------+---------+-------------+------------------------------------+-------------------------+
|20230907183152377  
|20230907183152377_0_0|0f760d6c-16a7-4d25-9093-440119df0b3f|americas/brazil/sao_paulo|7942e0b7-66fb-4a7a-9883-174a83746814-0_0-24-23_20230907183152377.parquet|0.8946621240010502|0.7425723852244098|driver-213|0.9451772959802982|0.6526397615401383|30.98679973544559|rider-213|1693821831384|0f760d6c-16a7-4d25-9093-440119df0b3f|americas/brazil/sao_paulo|
+-------------------+---------------------+------------------------------------+-------------------------+------------------------------------------------------------------------+------------------+------------------+----------+------------------+------------------+-----------------+---------+-------------+------------------------------------+-------------------------+
 {code}


was (Author: JIRAUSER301185):
Recompiled the Jar, and reran the script. But we still see the partition-path 
column correctly. I checked the .hoodie/hoodie.properties and confirmed the 
propertiy is set to true. All properties are listed as follows. 
{code:java}
#Updated at 2023-09-08T01:31:55.789Z
#Thu Sep 07 18:31:55 PDT 2023
hoodie.table.timeline.timezone=LOCAL
hoodie.table.keygenerator.class=org.apache.hudi.keygen.SimpleKeyGenerator
hoodie.table.precombine.field=ts
hoodie.table.version=6
hoodie.database.name=
hoodie.datasource.write.hive_style_partitioning=false
hoodie.table.metadata.partitions.inflight=
hoodie.table.checksum=581566219
hoodie.partition.metafile.use.base.format=false
hoodie.table.cdc.enabled=false
hoodie.archivelog.folder=archived
hoodie.table.name=uuid_test
hoodie.populate.meta.fields=true
hoodie.table.type=COPY_ON_WRITE
hoodie.datasource.write.partitionpath.urlencode=false
hoodie.table.base.file.format=PARQUET
hoodie.datasource.write.drop.partition.columns=true
hoodie.table.metadata.partitions=files
hoodie.timeline.layout.version=1
hoodie.table.recordkey.fields=uuid
hoodie.table.partition.fields=partitionpath {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)

Reply via email to