[ https://issues.apache.org/jira/browse/FLINK-34451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17820622#comment-17820622 ]
Gyula Fora commented on FLINK-34451: ------------------------------------ It looks like there is a race condition between handling the TaskManager failure and the JobManager shutdown in the adaptive scheduler which leads to a terminal failed state. If I only terminate the JobManager (scale down the k8s Deployment replicas to 0) then basically nothing happens to the HA metadata. (which is expected): {noformat} 2024-02-26 07:59:07,413 DEBUG org.apache.flink.kubernetes.shaded.io.fabric8.kubernetes.client.informers.impl.cache.Reflector [] - Event received MODIFIED ConfigMap resourceVersion v6719805 for v1/namespaces/default/configmaps 2024-02-26 07:59:08,096 INFO org.apache.flink.runtime.entrypoint.ClusterEntrypoint [] - RECEIVED SIGNAL 15: SIGTERM. Shutting down as requested. 2024-02-26 07:59:08,097 INFO org.apache.flink.runtime.entrypoint.ClusterEntrypoint [] - Shutting KubernetesApplicationClusterEntrypoint down with application status UNKNOWN. Diagnostics Cluster entrypoint has been closed externally.. 2024-02-26 07:59:08,098 INFO org.apache.flink.runtime.blob.BlobServer [] - Stopped BLOB server at 0.0.0.0:6124 2024-02-26 07:59:08,308 DEBUG org.apache.flink.kubernetes.shaded.io.fabric8.kubernetes.client.Watcher [] - Watcher closed 2024-02-26 07:59:08,508 WARN org.apache.pekko.actor.CoordinatedShutdown [] - Could not addJvmShutdownHook, due to: Shutdown in progress{noformat} > [Kubernetes Operator] Job with restarting TaskManagers uses wrong/misleading > fallback approach > ---------------------------------------------------------------------------------------------- > > Key: FLINK-34451 > URL: https://issues.apache.org/jira/browse/FLINK-34451 > Project: Flink > Issue Type: Bug > Components: Kubernetes Operator > Affects Versions: kubernetes-operator-1.6.1 > Environment: Operator version: 1.7.1 > Flink version 1.18.0 > HA JobManagers > Adaptive scheduler mode using the operator's autoscaler > Checkpointing at an interval of 60s > Upgrade mode savepoint > Reporter: Alex Hoffer > Priority: Major > > > We had a situation where TaskManagers were constantly restarting from OOM. > We're using the Adaptive scheduler with the Kubernetes Operator, and a > restart strategy of exponential backoff, and so the JobManagers remained > alive. We're also using savepoint upgrade mode. > When we tried to remedy the situation by raising the direct memory allocation > to the pods, we were surprised that Flink used the last savepoint taken, > rather than the checkpoint. This was unfortunate for us because we are on > adaptive scheduler and the job hasn't changed in some time, so this last > savepoint was 6 days old! Meanwhile, checkpoints were taken every minute up > until failure. I can confirm the HA metadata existed in the configmaps, and > the corresponding checkpoints existed in remote storage for it to access. > Plus, no Flink version changes were in the deployment. > The Operator logs reported that it was using last-state recovery in this > situation: > {code:java} > 2024-02-15 19:38:38,252 o.a.f.k.o.l.AuditUtils [INFO ][job-name] >>> > Event | Info | SPECCHANGED | UPGRADE change(s) detected (Diff: > FlinkDeploymentSpec[image : image:0a7c41b -> image:ebebc53, restartNonce : > null -> 100]), starting reconciliation. > 2024-02-15 19:38:38,252 o.a.f.k.o.r.d.AbstractJobReconciler [INFO ][job-name] > Upgrading/Restarting running job, suspending first... > 2024-02-15 19:38:38,260 o.a.f.k.o.r.d.ApplicationReconciler [INFO ][job-name] > Job is not running but HA metadata is available for last state restore, ready > for upgrade > 2024-02-15 19:38:38,270 o.a.f.k.o.l.AuditUtils [INFO ][job-name] >>> > Event | Info | SUSPENDED | Suspending existing deployment. > 2024-02-15 19:38:38,270 o.a.f.k.o.s.NativeFlinkService [INFO ][job-name] > Deleting JobManager deployment while preserving HA metadata. > 2024-02-15 19:38:40,431 o.a.f.k.o.l.AuditUtils [INFO ][job-name] >>> > Status | Info | UPGRADING | The resource is being upgraded > 2024-02-15 19:38:40,532 o.a.f.k.o.l.AuditUtils [INFO ][job-name] >>> > Event | Info | SUBMIT | Starting deployment > 2024-02-15 19:38:40,532 o.a.f.k.o.s.AbstractFlinkService [INFO ][job-name] > Deploying application cluster requiring last-state from HA metadata > 2024-02-15 19:38:40,538 o.a.f.k.o.u.FlinkUtils [INFO ][job-name] Job > graph in ConfigMap job-name-cluster-config-map is deleted {code} > But when the job booted up, it reported restoring from savepoint: > {code:java} > 2024-02-15 19:39:03,887 INFO > org.apache.flink.runtime.checkpoint.CheckpointCoordinator [] - Restoring > job 522b3c363499d81ed7922aa30b13e237 from Savepoint 20207 @ 0 for > 522b3c363499d81ed7922aa30b13e237 located at > abfss://savepoi...@storageaccount.dfs.core.windows.net/job-name/savepoint-522b3c-8836a1edc709. > {code} > Our expectation was that the Operator logs were true, and that it would be > restoring from checkpoint. We had to scramble and manually restore from the > checkpoint to restore function. > > > It's also worth noting I can recreate this issue in a testing environment. > The process for doing so is: > - Boot up HA JobManagers with checkpoints on and savepoint upgrade mode, > using adaptive scheduler > - Make a dummy change to trigger a savepoint. > - Allow the TaskManagers to process some data and hit the checkpoint interval. > - Cause the TaskManagers to crash. In our case, we could use up a bunch of > memory in the pods and cause it to crash. > - Observe the Operator logs saying it is restoring from last-state, but watch > as the pods instead use the last savepoint. -- This message was sent by Atlassian Jira (v8.20.10#820010)