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

Lin Liu edited comment on HUDI-5807 at 9/12/23 10:28 PM:
---------------------------------------------------------

So far I have found the cause of the bug:
 # During the write no partition_path is added to the base_file and log files.
 # During the read, the base file reader can append the partition path into 
record; but the log file reader does not have the mechanism to add the 
partition_path field into the payload.
 # During merging, the record from log file is output, whose partition_path 
field is NULL.

 

I have checked the solution to add the partition path to the underlying payload 
InternalRow. That is, the HoodieInternalRowUtils.RowUpdater can help set one 
field, with function `update(field_index, value)`. There is no efficient way to 
find the field index; search for the field index is linear, which is bad for 
large number of records.

We should discuss the possible solutions.

 


was (Author: JIRAUSER301185):
So far I have found the cause of the bug:
 # During the write no partition_path is added to the base_file and log files.
 # During the read, the base file reader can append the partition path into 
record; but the log file reader does not have the mechanism to add the 
partition_path field into the payload.
 # During merging, the record from log file is output, whose partition_path 
field is NULL.

 

I have checked the solution to add the partition path to the underlying payload 
InternalRow; but I haven't found an efficient solution. We should discuss the 
possible solutions.

 

> 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