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

Gyula Fora commented on FLINK-34451:
------------------------------------

I took a closer look at this and it also happens with the default restart 
strategy. 

The relevant log segment during shutdown (if we simply delete the deployment 
object as the last-state suspend does in the operator)
{code:java}
2024-02-26 07:05:04,412 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Map -> Sink: 
Unnamed (1/1) 
(6a8685c5160d149204dd115f396dcb38_90bea66de1c231edf33913ecd54406c1_0_1) 
switched from RUNNING to FAILED on autoscaling-example-taskmanager-1-1 @ 
10.244.0.54 (dataPort=35905).
org.apache.flink.util.FlinkExpectedException: The TaskExecutor is shutting down.
    at 
org.apache.flink.runtime.taskexecutor.TaskExecutor.onStop(TaskExecutor.java:481)
 ~[flink-dist-1.18.1.jar:1.18.1]
    at 
org.apache.flink.runtime.rpc.RpcEndpoint.internalCallOnStop(RpcEndpoint.java:239)
 ~[flink-dist-1.18.1.jar:1.18.1]
    ...
2024-02-26 07:05:04,415 INFO  
org.apache.flink.runtime.jobmaster.MiniDispatcherRestEndpoint [] - Shutting 
down rest endpoint.
2024-02-26 07:05:04,415 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Discarding 
the results produced by task execution 
6a8685c5160d149204dd115f396dcb38_90bea66de1c231edf33913ecd54406c1_0_1.
2024-02-26 07:05:04,416 DEBUG 
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Sending out 
cancel request, to remove task execution from TaskManager.
2024-02-26 07:05:04,417 WARN  
org.apache.flink.runtime.taskmanager.TaskManagerLocation     [] - No hostname 
could be resolved for the IP address 10.244.0.54, using IP address as host 
name. Local input split assignment (such as for HDFS files) may be impacted.
2024-02-26 07:05:04,417 INFO  
org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler [] - Restarting 
job.
org.apache.flink.util.FlinkExpectedException: The TaskExecutor is shutting down.
    at 
org.apache.flink.runtime.taskexecutor.TaskExecutor.onStop(TaskExecutor.java:481)
 ~[flink-dist-1.18.1.jar:1.18.1]
    at 
org.apache.flink.runtime.rpc.RpcEndpoint.internalCallOnStop(RpcEndpoint.java:239)
 ~[flink-dist-1.18.1.jar:1.18.1]
    ...
2024-02-26 07:05:04,417 DEBUG 
org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler [] - Transition 
from state Executing to Restarting.
2024-02-26 07:05:04,417 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Job 
Autoscaling Example (5ddd0b1ba346d3bfd5ef53a63772e43c) switched from state 
RUNNING to CANCELLING.
2024-02-26 07:05:04,417 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Source: 
Sequence Source -> Filter (1/1) 
(6a8685c5160d149204dd115f396dcb38_cbc357ccb763df2852fee8c4fc7d55f2_0_1) 
switched from RUNNING to CANCELING.
2024-02-26 07:05:04,418 DEBUG 
org.apache.flink.runtime.scheduler.adaptive.allocator.SharedSlot [] - Returning 
logical slot to shared slot (SlotRequestId{cda2d4d04521eed8b88245bb0eb497e0})
2024-02-26 07:05:04,418 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Source: 
Sequence Source -> Filter (1/1) 
(6a8685c5160d149204dd115f396dcb38_cbc357ccb763df2852fee8c4fc7d55f2_0_1) 
switched from CANCELING to CANCELED.
2024-02-26 07:05:04,418 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Discarding 
the results produced by task execution 
6a8685c5160d149204dd115f396dcb38_cbc357ccb763df2852fee8c4fc7d55f2_0_1.
2024-02-26 07:05:04,419 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Discarding 
the results produced by task execution 
6a8685c5160d149204dd115f396dcb38_cbc357ccb763df2852fee8c4fc7d55f2_0_1.
2024-02-26 07:05:04,419 DEBUG 
org.apache.flink.runtime.scheduler.adaptive.allocator.SharedSlot [] - Returning 
logical slot to shared slot (SlotRequestId{cda2d4d04521eed8b88245bb0eb497e0})
2024-02-26 07:05:04,419 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Job 
Autoscaling Example (5ddd0b1ba346d3bfd5ef53a63772e43c) switched from state 
CANCELLING to CANCELED.
2024-02-26 07:05:04,420 DEBUG 
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - 
ExecutionGraph 5ddd0b1ba346d3bfd5ef53a63772e43c reached terminal state CANCELED.
2024-02-26 07:05:04,420 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Stopping 
checkpoint coordinator for job 5ddd0b1ba346d3bfd5ef53a63772e43c.
2024-02-26 07:05:04,420 INFO  
org.apache.flink.runtime.jobmaster.slotpool.DefaultDeclarativeSlotPool [] - 
Releasing slot [d3be5ab8662c10de36088fddeb531b59].
org.apache.flink.util.FlinkExpectedException: The TaskExecutor is shutting down.
    at 
org.apache.flink.runtime.taskexecutor.TaskExecutor.onStop(TaskExecutor.java:481)
 ~[flink-dist-1.18.1.jar:1.18.1]
    at 
org.apache.flink.runtime.rpc.RpcEndpoint.internalCallOnStop(RpcEndpoint.java:239)
 ~[flink-dist-1.18.1.jar:1.18.1]
    ...
2024-02-26 07:05:04,420 DEBUG 
org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler [] - Ignoring 
scheduled action because expected state 
org.apache.flink.runtime.scheduler.adaptive.Executing@310e697d is not the 
actual state org.apache.flink.runtime.scheduler.adaptive.Restarting@5654b461.
2024-02-26 07:05:04,619 DEBUG 
org.apache.pekko.remote.transport.netty.NettyTransport       [] - Remote 
connection to [/10.244.0.54:33105] was disconnected because of [id: 0x6431709f, 
/10.244.0.53:55610 :> /10.244.0.54:33105] DISCONNECTED
2024-02-26 07:05:04,620 DEBUG 
org.apache.pekko.remote.transport.ProtocolStateActor         [] - Association 
between local [tcp://flink-metrics@10.244.0.53:55610] and remote 
[tcp://flink-metrics@10.244.0.54:33105] was disassociated because the 
ProtocolStateActor failed: Shutdown
2024-02-26 07:05:04,619 DEBUG 
org.apache.pekko.remote.transport.netty.NettyTransport       [] - Remote 
connection to [/10.244.0.54:49696] was disconnected because of [id: 0x0e459e2e, 
/10.244.0.54:49696 :> /10.244.0.53:6123] DISCONNECTED
2024-02-26 07:05:04,700 DEBUG 
org.apache.pekko.remote.transport.ProtocolStateActor         [] - Association 
between local [tcp://flink@10.244.0.53:6123] and remote 
[tcp://flink@10.244.0.54:49696] was disassociated because the 
ProtocolStateActor failed: Shutdown
2024-02-26 07:05:04,700 DEBUG org.apache.pekko.remote.Remoting                  
           [] - Remote system with address 
[pekko.tcp://flink-metrics@10.244.0.54:33105] has shut down. Address is now 
gated for 50 ms, all messages to this address will be delivered to dead letters.
2024-02-26 07:05:04,707 DEBUG org.apache.pekko.remote.Remoting                  
           [] - Remote system with address [pekko.tcp://flink@10.244.0.54:6122] 
has shut down. Address is now gated for 50 ms, all messages to this address 
will be delivered to dead letters.
2024-02-26 07:05:04,841 DEBUG 
org.apache.flink.kubernetes.shaded.io.fabric8.kubernetes.client.Watcher [] - 
Watcher closed
2024-02-26 07:05:05,002 WARN  org.apache.pekko.actor.CoordinatedShutdown        
           [] - Could not addJvmShutdownHook, due to: Shutdown in progress
2024-02-26 07:05:05,002 WARN  org.apache.pekko.actor.CoordinatedShutdown        
           [] - Could not addJvmShutdownHook, due to: Shutdown in progress 
{code}
For some reason seems like the job actually goes into a globally terminal 
CANCELED state and HA metadata is cleaned up / lost. I wonder why this happens 
with the AdaptiveScheduler and not with the default scheduler. 

cc [~chesnay] [~dmvk] do you guys have any clue why there is a completely 
different behaviour in this case for the adaptive scheduler? Why does the job 
end up in a globally terminal state?

Here are also the logs for the default scheduler which clearly doesn't do any 
cleanup / cancelling as expected (by me):


{code:java}
2024-02-26 07:13:04,449 INFO  
org.apache.flink.runtime.resourcemanager.active.ActiveResourceManager [] - 
Closing TaskExecutor connection autoscaling-example-taskmanager-1-1 because: 
The TaskExecutor is shutting down.
2024-02-26 07:13:04,449 INFO  org.apache.flink.runtime.jobmaster.JobMaster      
           [] - Disconnect TaskExecutor autoscaling-example-taskmanager-1-1 
because: The TaskExecutor is shutting down.
2024-02-26 07:13:04,449 INFO  
org.apache.flink.runtime.resourcemanager.slotmanager.FineGrainedSlotManager [] 
- Unregistering task executor 24ea20d5af9f88b43319c53118f5ce3c from the slot 
manager.
2024-02-26 07:13:04,449 INFO  
org.apache.flink.runtime.resourcemanager.slotmanager.DefaultSlotStatusSyncer [] 
- Freeing slot 9f25210e811654cf75f0a813c65fb1b8.
2024-02-26 07:13:04,451 DEBUG org.apache.flink.runtime.scheduler.SharedSlot     
           [] - Release shared slot 
(SlotRequestId{09ea3d193e173b8cc62a6c50274926b5})
2024-02-26 07:13:04,454 DEBUG org.apache.flink.runtime.scheduler.SharedSlot     
           [] - Release logical slot 
(SlotRequestId{241da77c2fc4f7d840ffa12641bc135f}) for execution vertex (id 
cbc357ccb763df2852fee8c4fc7d55f2_0) from the physical slot 
(SlotRequestId{09ea3d193e173b8cc62a6c50274926b5})
2024-02-26 07:13:04,458 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint        [] - RECEIVED 
SIGNAL 15: SIGTERM. Shutting down as requested.
2024-02-26 07:13:04,459 INFO  org.apache.flink.runtime.blob.BlobServer          
           [] - Stopped BLOB server at 0.0.0.0:6124
2024-02-26 07:13:04,462 DEBUG 
org.apache.flink.runtime.resourcemanager.slotmanager.FineGrainedTaskManagerTracker
 [] - Free allocated slot with allocationId 9f25210e811654cf75f0a813c65fb1b8.
2024-02-26 07:13:04,463 DEBUG 
org.apache.flink.runtime.resourcemanager.slotmanager.FineGrainedTaskManagerTracker
 [] - Remove task manager 24ea20d5af9f88b43319c53118f5ce3c.
2024-02-26 07:13:04,463 DEBUG 
org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerImpl
 [] - Processing shutdown of task executor autoscaling-example-taskmanager-1-1.
2024-02-26 07:13:04,463 INFO  
org.apache.flink.runtime.resourcemanager.active.ActiveResourceManager [] - 
Stopping worker autoscaling-example-taskmanager-1-1.
2024-02-26 07:13:04,459 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:13:04,456 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Source: 
Sequence Source -> Filter (1/1) 
(a6d599cf375af90bdbffdbbb5363b13a_cbc357ccb763df2852fee8c4fc7d55f2_0_0) 
switched from RUNNING to FAILED on autoscaling-example-taskmanager-1-1 @ 
10.244.0.56 (dataPort=46729).
org.apache.flink.util.FlinkExpectedException: The TaskExecutor is shutting down.
at 
org.apache.flink.runtime.taskexecutor.TaskExecutor.onStop(TaskExecutor.java:481)
 ~[flink-dist-1.18.1.jar:1.18.1]
at 
org.apache.flink.runtime.rpc.RpcEndpoint.internalCallOnStop(RpcEndpoint.java:239)
 ~[flink-dist-1.18.1.jar:1.18.1]
at 
org.apache.flink.runtime.rpc.pekko.PekkoRpcActor$StartedState.lambda$terminate$0(PekkoRpcActor.java:574)
 ~[flink-rpc-akka44a70f6e-66c7-4ad0-98eb-a736d52e5378.jar:1.18.1]
at 
org.apache.flink.runtime.concurrent.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:83)
 ~[flink-dist-1.18.1.jar:1.18.1]
at 
org.apache.flink.runtime.rpc.pekko.PekkoRpcActor$StartedState.terminate(PekkoRpcActor.java:573)
 ~[flink-rpc-akka44a70f6e-66c7-4ad0-98eb-a736d52e5378.jar:1.18.1]
at 
org.apache.flink.runtime.rpc.pekko.PekkoRpcActor.handleControlMessage(PekkoRpcActor.java:196)
 ~[flink-rpc-akka44a70f6e-66c7-4ad0-98eb-a736d52e5378.jar:1.18.1]
at org.apache.pekko.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:33) 
~[flink-rpc-akka44a70f6e-66c7-4ad0-98eb-a736d52e5378.jar:1.18.1]
at org.apache.pekko.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:29) 
~[flink-rpc-akka44a70f6e-66c7-4ad0-98eb-a736d52e5378.jar:1.18.1]
at scala.PartialFunction.applyOrElse(PartialFunction.scala:127) 
~[flink-rpc-akka44a70f6e-66c7-4ad0-98eb-a736d52e5378.jar:1.18.1]
at scala.PartialFunction.applyOrElse$(PartialFunction.scala:126) 
~[flink-rpc-akka44a70f6e-66c7-4ad0-98eb-a736d52e5378.jar:1.18.1]
at 
org.apache.pekko.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:29) 
~[flink-rpc-akka44a70f6e-66c7-4ad0-98eb-a736d52e5378.jar:1.18.1]
at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:175) 
[flink-rpc-akka44a70f6e-66c7-4ad0-98eb-a736d52e5378.jar:1.18.1]
at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:176) 
[flink-rpc-akka44a70f6e-66c7-4ad0-98eb-a736d52e5378.jar:1.18.1]
at org.apache.pekko.actor.Actor.aroundReceive(Actor.scala:547) 
[flink-rpc-akka44a70f6e-66c7-4ad0-98eb-a736d52e5378.jar:1.18.1]
at org.apache.pekko.actor.Actor.aroundReceive$(Actor.scala:545) 
[flink-rpc-akka44a70f6e-66c7-4ad0-98eb-a736d52e5378.jar:1.18.1]
at org.apache.pekko.actor.AbstractActor.aroundReceive(AbstractActor.scala:229) 
[flink-rpc-akka44a70f6e-66c7-4ad0-98eb-a736d52e5378.jar:1.18.1]
at org.apache.pekko.actor.ActorCell.receiveMessage(ActorCell.scala:590) 
[flink-rpc-akka44a70f6e-66c7-4ad0-98eb-a736d52e5378.jar:1.18.1]
at org.apache.pekko.actor.ActorCell.invoke(ActorCell.scala:557) 
[flink-rpc-akka44a70f6e-66c7-4ad0-98eb-a736d52e5378.jar:1.18.1]
at org.apache.pekko.dispatch.Mailbox.processMailbox(Mailbox.scala:280) 
[flink-rpc-akka44a70f6e-66c7-4ad0-98eb-a736d52e5378.jar:1.18.1]
at org.apache.pekko.dispatch.Mailbox.run(Mailbox.scala:241) 
[flink-rpc-akka44a70f6e-66c7-4ad0-98eb-a736d52e5378.jar:1.18.1]
at org.apache.pekko.dispatch.Mailbox.exec(Mailbox.scala:253) 
[flink-rpc-akka44a70f6e-66c7-4ad0-98eb-a736d52e5378.jar:1.18.1]
at java.util.concurrent.ForkJoinTask.doExec(Unknown Source) [?:?]
at java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(Unknown Source) 
[?:?]
at java.util.concurrent.ForkJoinPool.scan(Unknown Source) [?:?]
at java.util.concurrent.ForkJoinPool.runWorker(Unknown Source) [?:?]
at java.util.concurrent.ForkJoinWorkerThread.run(Unknown Source) [?:?]
2024-02-26 07:13:04,709 DEBUG 
org.apache.pekko.remote.transport.netty.NettyTransport       [] - Remote 
connection to [/10.244.0.56:35175] was disconnected because of [id: 0xa827118c, 
/10.244.0.55:51996 :> /10.244.0.56:35175] DISCONNECTED
2024-02-26 07:13:04,712 DEBUG 
org.apache.pekko.remote.transport.ProtocolStateActor         [] - Association 
between local [tcp://flink-metrics@10.244.0.55:51996] and remote 
[tcp://flink-metrics@10.244.0.56:35175] was disassociated because the 
ProtocolStateActor failed: Shutdown
2024-02-26 07:13:04,709 DEBUG 
org.apache.pekko.remote.transport.ProtocolStateActor         [] - Association 
between local [tcp://flink@10.244.0.55:6123] and remote 
[tcp://flink@10.244.0.56:57044] was disassociated because the 
ProtocolStateActor failed: Shutdown
2024-02-26 07:13:04,716 DEBUG org.apache.pekko.remote.Remoting                  
           [] - Remote system with address 
[pekko.tcp://flink-metrics@10.244.0.56:35175] has shut down. Address is now 
gated for 50 ms, all messages to this address will be delivered to dead letters.
2024-02-26 07:13:04,718 DEBUG 
org.apache.pekko.remote.transport.netty.NettyTransport       [] - Remote 
connection to [/10.244.0.56:57044] was disconnected because of [id: 0xb1bfb799, 
/10.244.0.56:57044 :> /10.244.0.55:6123] DISCONNECTED
2024-02-26 07:13:04,719 DEBUG org.apache.pekko.remote.Remoting                  
           [] - Remote system with address [pekko.tcp://flink@10.244.0.56:6122] 
has shut down. Address is now gated for 50 ms, all messages to this address 
will be delivered to dead letters.
2024-02-26 07:13:04,804 WARN  
org.apache.pekko.remote.transport.netty.NettyTransport       [] - Remote 
connection to [null] failed with java.net.ConnectException: Connection refused: 
/10.244.0.56:6122
2024-02-26 07:13:04,804 WARN  
org.apache.pekko.remote.ReliableDeliverySupervisor           [] - Association 
with remote system [pekko.tcp://flink@10.244.0.56:6122] has failed, address is 
now gated for [50] ms. Reason: [Association failed with 
[pekko.tcp://flink@10.244.0.56:6122]] Caused by: [java.net.ConnectException: 
Connection refused: /10.244.0.56:6122]
2024-02-26 07:13:04,840 DEBUG 
org.apache.flink.kubernetes.shaded.io.fabric8.kubernetes.client.Watcher [] - 
Watcher closed {code}

> [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)

Reply via email to