[ https://issues.apache.org/jira/browse/FLINK-28515?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17566293#comment-17566293 ]
Jinzhong Li commented on FLINK-28515: ------------------------------------- Thanks for your reply [~roman] . For your concerns, i think: 1. Because the taskStateSnapshot of aborted checkpoint may not be registered into TaskLocalStateStoreImpl, it's difficult to call taskStateSnapshot.discardState() in TaskLocalStateStoreImpl for unregistered checkpoint. But i think we can catch exception in [AsyncCheckpointRunnable.finalizeNonFinishedSnapshots|https://github.com/apache/flink/blob/7e494c91d1e340a1f9438785e2a56242fb5894c8/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncCheckpointRunnable.java#L177], and call jobManagerTaskOperatorSubtaskStates.discardState()/localTaskOperatorSubtaskStates.discardState() after catching a exception. WDYT? 2. I agree that we could register localSnapshot to TaskLocalStateStore early for the missing abort notification case. we can solve it independently if needed. Do I need to create another jira ticket to keep track of this? > The files in local recovery directory hasn't be clean up properly after > checkpoint abort > ---------------------------------------------------------------------------------------- > > Key: FLINK-28515 > URL: https://issues.apache.org/jira/browse/FLINK-28515 > Project: Flink > Issue Type: Bug > Components: Runtime / Checkpointing, Runtime / State Backends > Affects Versions: 1.15.1, 1.16.0 > Reporter: Jinzhong Li > Assignee: Jinzhong Li > Priority: Major > Labels: pull-request-available > Attachments: C7245668-CE31-4F56-B9CB-12E2F1E900C5.png, image.png > > > In my case, i found that some files in local recovery directory hasn't be > clean up properly after checkpoint abort(as shown in the attached picture). > By analyzing flink log, I found that when stateBackend completes the local > snapshot but the task has not completed the whole snapshot, > then checkpoint is aborted (caused by checkpoint timeout or netword-error), > files in the local directory directory may not be cleaned up properly. > I think the reason for local snapshot file residual is: > (1) In the org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable, > the comleted localSnapshot info can be registered into > org.apache.flink.runtime.state.TaskLocalStateStoreImpl only after task has > completed the whole snapshot. > ([AsyncCheckpointRunnable.java#L136|https://github.com/apache/flink/blob/3ec376601f836df6314e771b243ca6f896a7f642/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncCheckpointRunnable.java#L136]). > (2) If stateBackend completes the local snapshot but the task has not > completed the entire snapshot, when checkpoint-aborting is triggered, the > TaskLocalStateStore can't clean up the unregistered localSnapshot files. > ([TaskLocalStateStoreImpl.java#L301|https://github.com/apache/flink/blob/3ec376601f836df6314e771b243ca6f896a7f642/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStoreImpl.java#L301]) > To fix this problem, I think when TaskLocalStateStoreImpl abort Checkpoint, > we can try to delete the corresponding localRecovery directory, even if the > checkpoint is not unregistered into TaskLocalStateStoreImpl. > > -- This message was sent by Atlassian Jira (v8.20.10#820010)