You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Roman Khachatryan (Jira)" <ji...@apache.org> on 2022/01/17 10:51:00 UTC

[jira] [Resolved] (FLINK-25395) FileNotFoundException during recovery caused by Incremental shared state being discarded by TM

     [ https://issues.apache.org/jira/browse/FLINK-25395?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Roman Khachatryan resolved FLINK-25395.
---------------------------------------
    Resolution: Fixed

Merged into master as:

4691b66545010ed812624a259869c7a522663720 (rocksDB - revert to re-upload again)

e28f4e2c5d4d54f5f727b9557024c537becfd054 (don't discard state)

> FileNotFoundException during recovery caused by Incremental shared state being discarded by TM
> ----------------------------------------------------------------------------------------------
>
>                 Key: FLINK-25395
>                 URL: https://issues.apache.org/jira/browse/FLINK-25395
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Checkpointing, Runtime / State Backends
>    Affects Versions: 1.15.0
>            Reporter: Roman Khachatryan
>            Assignee: Roman Khachatryan
>            Priority: Blocker
>              Labels: pull-request-available
>             Fix For: 1.15.0
>
>
> Extracting from FLINK-25185 discussion
> On checkpoint abortion or any failure in AsyncCheckpointRunnable,
> it discards the state, in particular shared (incremental) state.
> Since FLINK-24611, this creates a problem because shared state can be re-used for future checkpoints.
>  
> A similar case is in PeriodicMaterializationManager (uploaded SST files will be deleted on failure without notifying the wrapped RocksDB state backend).
>  
> Symptom of this failure is a following exception during recovery:
> {noformat}
> Caused by: java.io.FileNotFoundException: /tmp/junit3146957979516280339/junit1602669867129285236/d6a6dbdd-3fd7-4786-9dc1-9ccc161740da (No such file or directory)
>         at java.io.FileInputStream.open0(Native Method) ~[?:1.8.0_292]
>         at java.io.FileInputStream.open(FileInputStream.java:195) ~[?:1.8.0_292]
>         at java.io.FileInputStream.<init>(FileInputStream.java:138) ~[?:1.8.0_292]
>         at org.apache.flink.core.fs.local.LocalDataInputStream.<init>(LocalDataInputStream.java:50) ~[flink-core-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
>         at org.apache.flink.core.fs.local.LocalFileSystem.open(LocalFileSystem.java:134) ~[flink-core-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
>         at org.apache.flink.core.fs.SafetyNetWrapperFileSystem.open(SafetyNetWrapperFileSystem.java:87) ~[flink-core-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
>         at org.apache.flink.runtime.state.filesystem.FileStateHandle.openInputStream(FileStateHandle.java:68) ~[flink-runtime-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
>         at org.apache.flink.changelog.fs.StateChangeFormat.read(StateChangeFormat.java:92) ~[flink-dstl-dfs-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
>         at org.apache.flink.runtime.state.changelog.StateChangelogHandleStreamHandleReader$1.advance(StateChangelogHandleStreamHandleReader.java:85) ~[flink-runtime-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
> {noformat}



--
This message was sent by Atlassian Jira
(v8.20.1#820001)