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

Zakelly Lan commented on FLINK-35803:
-------------------------------------

[~fanrui] Thanks for you infomation!

After some debugging, I found it actually is an old bug about referencing the 
wrong file that is already cleaned by CLAIM mode. It happens when the thrid job 
tries to restore and read a cp file from the first job (which is already 
deleted by the second job of course). Before FLINK-35784, the files are not 
deleted properly, so this bug is covered up. And the reason way the AZP for 
FLINK-35784's PR passed is that, the cp files are deleted by async io threads, 
so it happens that the third job read the wrong files before their deletion.

I'll prepare the fix.

> ResumeCheckpointManuallyITCase fails with checkpoint file merging
> -----------------------------------------------------------------
>
>                 Key: FLINK-35803
>                 URL: https://issues.apache.org/jira/browse/FLINK-35803
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Checkpointing
>    Affects Versions: 2.0.0, 1.20.0
>            Reporter: Rui Fan
>            Priority: Blocker
>         Attachments: image-2024-07-10-17-19-00-916.png
>
>
> [https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=60807&view=logs&j=5c8e7682-d68f-54d1-16a2-a09310218a49&t=86f654fa-ab48-5c1a-25f4-7e7f6afb9bba&l=8905]
> Test: 
> ResumeCheckpointManuallyITCase.testExternalizedIncrementalRocksDBCheckpointsWithLocalRecoveryZookeeper
>  
> FileNotFoundException happens when restoring from checkpoint that enabled 
> file merging.
> I'm not sure the file is deleted unpectedly by flink code or CI is unstable.



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

Reply via email to