[
https://issues.apache.org/jira/browse/FLINK-28265?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17566116#comment-17566116
]
Yang Wang commented on FLINK-28265:
-----------------------------------
Since we always add the new checkpoint first and then subsume the oldest one, I
am curious how it could happen we only have one checkpoint which is invalid. If
adding the new checkpoint failed, we should have the old successful checkpoint.
On the contrary, if subsuming the oldest one failed, we should still have the
newly added checkpoint.
Could you please verify the checkpoint 9701 or 9703 exists on the S3?
I believe the the logs of previous run(e.g. kubectl logs <pod-name> --previous)
and the Kubernetes APIServer audit log will help a lot to debug the root cause.
> Inconsistency in Kubernetes HA service: broken state handle
> -----------------------------------------------------------
>
> Key: FLINK-28265
> URL: https://issues.apache.org/jira/browse/FLINK-28265
> Project: Flink
> Issue Type: Bug
> Components: Runtime / Coordination
> Affects Versions: 1.14.4
> Reporter: Robert Metzger
> Priority: Major
> Attachments: flink_checkpoint_issue.txt
>
>
> I have a JobManager, which at some point failed to acknowledge a checkpoint:
> {code}
> Error while processing AcknowledgeCheckpoint message
> org.apache.flink.runtime.checkpoint.CheckpointException: Could not complete
> the pending checkpoint 193393. Failure reason: Failure to finalize checkpoint.
> at
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.completePendingCheckpoint(CheckpointCoordinator.java:1255)
> at
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.receiveAcknowledgeMessage(CheckpointCoordinator.java:1100)
> at
> org.apache.flink.runtime.scheduler.ExecutionGraphHandler.lambda$acknowledgeCheckpoint$1(ExecutionGraphHandler.java:89)
> at
> org.apache.flink.runtime.scheduler.ExecutionGraphHandler.lambda$processCheckpointCoordinatorMessage$3(ExecutionGraphHandler.java:119)
> at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown
> Source)
> at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown
> Source)
> at java.base/java.lang.Thread.run(Unknown Source)
> Caused by:
> org.apache.flink.runtime.persistence.StateHandleStore$AlreadyExistException:
> checkpointID-0000000000000193393 already exists in ConfigMap
> cm-00000000000000000000000000000000-jobmanager-leader
> at
> org.apache.flink.kubernetes.highavailability.KubernetesStateHandleStore.getKeyAlreadyExistException(KubernetesStateHandleStore.java:534)
> at
> org.apache.flink.kubernetes.highavailability.KubernetesStateHandleStore.lambda$addAndLock$0(KubernetesStateHandleStore.java:155)
> at
> org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient.lambda$attemptCheckAndUpdateConfigMap$11(Fabric8FlinkKubeClient.java:316)
> at
> java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(Unknown
> Source)
> ... 3 common frames omitted
> {code}
> the JobManager creates subsequent checkpoints successfully.
> Upon failure, it tries to recover this checkpoint (0000000000000193393), but
> fails to do so because of:
> {code}
> Caused by: org.apache.flink.util.FlinkException: Could not retrieve
> checkpoint 193393 from state handle under checkpointID-0000000000000193393.
> This indicates that the retrieved state handle is broken. Try cleaning the
> state handle store ... Caused by: java.io.FileNotFoundException: No such file
> or directory: s3://xxx/flink-ha/xxx/completedCheckpoint72e30229420c
> {code}
> I'm running Flink 1.14.4.
> Note: This issue has been first discussed here:
> https://github.com/apache/flink/pull/15832#pullrequestreview-1005973050
--
This message was sent by Atlassian Jira
(v8.20.10#820010)