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

Geser Dugarov commented on HUDI-8785:
-------------------------------------

Currently, there is exception in logs:

 
{code:java}
2024-12-19 21:19:06,042 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Source: 
hudi_debug[3] -> ConstraintEnforcer[4] (6/8) 
(d0fe7e1ca63f7d71cbbd2d8d243d713c_cbc357ccb763df2852fee8c4fc7d55f2_5_0) 
switched from RUNNING to FAILED on localhost:33699-00d6b0 @ localhost 
(dataPort=45539).
org.apache.hudi.exception.HoodieException: Exception when reading log file 
    at 
org.apache.hudi.common.table.log.AbstractHoodieLogRecordScanner.scanInternalV1(AbstractHoodieLogRecordScanner.java:388)
 ~[hudi-flink1.17-bundle-1.1.0-populate-config.jar:1.1.0-SNAPSHOT]
    at 
org.apache.hudi.common.table.log.AbstractHoodieLogRecordScanner.scanInternal(AbstractHoodieLogRecordScanner.java:250)
 ~[hudi-flink1.17-bundle-1.1.0-populate-config.jar:1.1.0-SNAPSHOT]
    at 
org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner.performScan(HoodieMergedLogRecordScanner.java:204)
 ~[hudi-flink1.17-bundle-1.1.0-populate-config.jar:1.1.0-SNAPSHOT]
    at 
org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner.<init>(HoodieMergedLogRecordScanner.java:120)
 ~[hudi-flink1.17-bundle-1.1.0-populate-config.jar:1.1.0-SNAPSHOT]
    at 
org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner$Builder.build(HoodieMergedLogRecordScanner.java:481)
 ~[hudi-flink1.17-bundle-1.1.0-populate-config.jar:1.1.0-SNAPSHOT]
    at 
org.apache.hudi.table.format.FormatUtils.logScanner(FormatUtils.java:174) 
~[hudi-flink1.17-bundle-1.1.0-populate-config.jar:1.1.0-SNAPSHOT]
    at 
org.apache.hudi.table.format.mor.MergeOnReadInputFormat.getLogFileIterator(MergeOnReadInputFormat.java:355)
 ~[hudi-flink1.17-bundle-1.1.0-populate-config.jar:1.1.0-SNAPSHOT]
    at 
org.apache.hudi.table.format.mor.MergeOnReadInputFormat.initIterator(MergeOnReadInputFormat.java:211)
 ~[hudi-flink1.17-bundle-1.1.0-populate-config.jar:1.1.0-SNAPSHOT]
    at 
org.apache.hudi.table.format.mor.MergeOnReadInputFormat.open(MergeOnReadInputFormat.java:189)
 ~[hudi-flink1.17-bundle-1.1.0-populate-config.jar:1.1.0-SNAPSHOT]
    at 
org.apache.hudi.table.format.mor.MergeOnReadInputFormat.open(MergeOnReadInputFormat.java:90)
 ~[hudi-flink1.17-bundle-1.1.0-populate-config.jar:1.1.0-SNAPSHOT]
    at 
org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction.run(InputFormatSourceFunction.java:84)
 ~[flink-dist-1.17.1.jar:1.17.1]
    at 
org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:142)
 ~[flink-dist-1.17.1.jar:1.17.1]
    at 
org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:88) 
~[flink-dist-1.17.1.jar:1.17.1]
    at 
org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:358)
 ~[flink-dist-1.17.1.jar:1.17.1]
Caused by: java.lang.NullPointerException
    at 
org.apache.hudi.common.util.SpillableMapUtils.convertToHoodieRecordPayload(SpillableMapUtils.java:135)
 ~[hudi-flink1.17-bundle-1.1.0-populate-config.jar:1.1.0-SNAPSHOT]
    at 
org.apache.hudi.common.util.SpillableMapUtils.convertToHoodieRecordPayload(SpillableMapUtils.java:111)
 ~[hudi-flink1.17-bundle-1.1.0-populate-config.jar:1.1.0-SNAPSHOT]
    at 
org.apache.hudi.avro.HoodieAvroUtils.createHoodieRecordFromAvro(HoodieAvroUtils.java:1340)
 ~[hudi-flink1.17-bundle-1.1.0-populate-config.jar:1.1.0-SNAPSHOT]
    at 
org.apache.hudi.common.model.HoodieAvroIndexedRecord.wrapIntoHoodieRecordPayloadWithParams(HoodieAvroIndexedRecord.java:167)
 ~[hudi-flink1.17-bundle-1.1.0-populate-config.jar:1.1.0-SNAPSHOT]
    at 
org.apache.hudi.common.table.log.AbstractHoodieLogRecordScanner.processDataBlock(AbstractHoodieLogRecordScanner.java:635)
 ~[hudi-flink1.17-bundle-1.1.0-populate-config.jar:1.1.0-SNAPSHOT]
    at 
org.apache.hudi.common.table.log.AbstractHoodieLogRecordScanner.processQueuedBlocksForInstant(AbstractHoodieLogRecordScanner.java:675)
 ~[hudi-flink1.17-bundle-1.1.0-populate-config.jar:1.1.0-SNAPSHOT]
    at 
org.apache.hudi.common.table.log.AbstractHoodieLogRecordScanner.scanInternalV1(AbstractHoodieLogRecordScanner.java:378)
 ~[hudi-flink1.17-bundle-1.1.0-populate-config.jar:1.1.0-SNAPSHOT]
    ... 13 more {code}

> MOR table after upsert with disabled hoodie.populate.meta.fields couldn't be 
> read by Flink
> ------------------------------------------------------------------------------------------
>
>                 Key: HUDI-8785
>                 URL: https://issues.apache.org/jira/browse/HUDI-8785
>             Project: Apache Hudi
>          Issue Type: Bug
>            Reporter: Geser Dugarov
>            Assignee: Geser Dugarov
>            Priority: Major
>
> SQL queries:
> {code:java}
> CREATE TABLE hudi_debug (
>     id INT,
>     part INT,
>     desc STRING,
>     PRIMARY KEY (id) NOT ENFORCED
> ) 
> WITH (
>     'connector' = 'hudi',
>     'path' = '...',
>     'table.type' = 'MERGE_ON_READ',
>     'write.operation' = 'upsert',
>     'hoodie.populate.meta.fields' = 'false'
> ); {code}
> {code:java}
> INSERT INTO hudi_debug VALUES 
>     (1,100,'aaa'),
>     (2,200,'bbb'); {code}
>  
> Check that there is no exceptions during:
> {code:java}
> SELECT * FROM hudi_debug; {code}
> and corresponding log files in HDFS don't not contain columns with metadata.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to