You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by Alexey Trenikhun <ye...@msn.com> on 2022/04/06 15:20:00 UTC

Error during shutdown of StandaloneApplicationClusterEntryPoint via JVM shutdown hook

Hello,

We are using Flink 1.13.6, Application Mode, k8s HA. To upgrade job, we use POST, url=http://gsp-jm:8081/jobs/00000000000000000000000000000000/savepoints, then we wait for up to 5 minutes for completion, periodically pulling status (GET, url=http://gsp-jm:8081/jobs/00000000000000000000000000000000/savepoints/{triggerId}). If savepoint is not complete in 5 minute, we cancel job (PATCH, url=http://gsp-jm:8081/jobs/0000000000000000000000000000000). Usually it works well, job stopped one way or another and we proceed with upgrade, but currently JM exits with code -2, and as result k8s restarts pod. We tried multiple times, but every time getting -2. JM log is below (newest messages on top):

2022-04-06T14:21:17.465Z Error during shutdown of StandaloneApplicationClusterEntryPoint via JVM shutdown hook.
java.util.concurrent.CompletionException: java.util.concurrent.TimeoutException
 at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
 at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
 at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:661)
 at java.util.concurrent.CompletableFuture.uniAcceptStage(CompletableFuture.java:683)
 at java.util.concurrent.CompletableFuture.thenAccept(CompletableFuture.java:2010)
 at org.apache.flink.runtime.entrypoint.ClusterEntrypoint.closeAsync(ClusterEntrypoint.java:379)
 at org.apache.flink.runtime.entrypoint.ClusterEntrypoint.lambda$new$0(ClusterEntrypoint.java:168)
 at org.apache.flink.util.ShutdownHookUtil.lambda$addShutdownHook$0(ShutdownHookUtil.java:39)
 at java.lang.Thread.run(Thread.java:750) Caused by: java.util.concurrent.TimeoutException: null
 at org.apache.flink.runtime.concurrent.FutureUtils$Timeout.run(FutureUtils.java:1255)
 at org.apache.flink.runtime.concurrent.DirectExecutorService.execute(DirectExecutorService.java:217)
 at org.apache.flink.runtime.concurrent.FutureUtils.lambda$orTimeout$15(FutureUtils.java:582)
 at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
 at java.util.concurrent.FutureTask.run(FutureTask.java:266)
 at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
 at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
 at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
 at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
 ... 1 common frames omitted
2022-04-06T14:21:17.464Z Terminating cluster entrypoint process StandaloneApplicationClusterEntryPoint with exit code 2.
java.util.concurrent.TimeoutException: null
 at org.apache.flink.runtime.concurrent.FutureUtils$Timeout.run(FutureUtils.java:1255)
 at org.apache.flink.runtime.concurrent.DirectExecutorService.execute(DirectExecutorService.java:217)
 at org.apache.flink.runtime.concurrent.FutureUtils.lambda$orTimeout$15(FutureUtils.java:582)
 at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
 at java.util.concurrent.FutureTask.run(FutureTask.java:266)
 at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
 at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
 at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
 at  java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
 at java.lang.Thread.run(Thread.java:750)
2022-04-06T14:21:17.463Z Stopped Akka RPC service.
2022-04-06T14:21:17.46Z Stopped Akka RPC service.
2022-04-06T14:21:17.373Z Remoting shut down.
2022-04-06T14:21:17.373Z Remoting shut down.
2022-04-06T14:21:17.297Z Remote daemon shut down; proceeding with flushing remote transports.
2022-04-06T14:21:17.297Z Remote daemon shut down; proceeding with flushing remote transports.
2022-04-06T14:21:17.296Z Shutting down remote daemon.
2022-04-06T14:21:17.296Z Shutting down remote daemon.
2022-04-06T14:21:17.286Z Stopping Akka RPC service.
2022-04-06T14:21:17.281Z Stopping Akka RPC service.
2022-04-06T14:21:17.277Z Finished cleaning up the high availability data.
2022-04-06T14:21:14.04Z Close and clean up all data for KubernetesHaServices.
2022-04-06T14:21:14.04Z Stopped BLOB server at 0.0.0.0:6124
2022-04-06T14:21:14.039Z Stopping DefaultJobGraphStore.
2022-04-06T14:21:14.037Z The watcher is closing.
2022-04-06T14:21:14.037Z Closing KubernetesLeaderElectionDriver{configMapName='gsp-resourcemanager-leader'}.
2022-04-06T14:21:14.037Z Stopping DefaultLeaderElectionService.
2022-04-06T14:21:14.036Z Stopped dispatcher akka.tcp://flink@10.204.19.239:6123/user/rpc/dispatcher_1.
2022-04-06T14:21:14.035Z Suspending the slot manager.
2022-04-06T14:21:14.035Z Closing the slot manager.
2022-04-06T14:21:14.035Z Stopping all currently running jobs of dispatcher akka.tcp://flink@10.204.19.239:6123/user/rpc/dispatcher_1.
2022-04-06T14:21:14.034Z Stopping dispatcher akka.tcp://flink@10.204.19.239:6123/user/rpc/dispatcher_1.
2022-04-06T14:21:14.034Z Stopping SessionDispatcherLeaderProcess.
2022-04-06T14:21:14.033Z The watcher is closing.
2022-04-06T14:21:14.033Z Closing KubernetesLeaderElectionDriver{configMapName='gsp-dispatcher-leader'}.
2022-04-06T14:21:14.033Z Stopping DefaultLeaderElectionService.
2022-04-06T14:21:14.032Z The watcher is closing.
2022-04-06T14:21:14.032Z Stopping KubernetesLeaderRetrievalDriver{configMapName='gsp-resourcemanager-leader'}.
2022-04-06T14:21:14.032Z Stopping DefaultLeaderRetrievalService.
2022-04-06T14:21:14.031Z The watcher is closing.
2022-04-06T14:21:14.031Z Stopping KubernetesLeaderRetrievalDriver{configMapName='gsp-dispatcher-leader'}.
2022-04-06T14:21:14.031Z Stopping DefaultLeaderRetrievalService.
2022-04-06T14:21:14.031Z Closing components.
2022-04-06T14:21:14.031Z Shut down cluster because application is in CANCELED, diagnostics null.
2022-04-06T14:21:14.03Z Shut down complete.
2022-04-06T14:21:14.03Z The watcher is closing.
2022-04-06T14:21:14.03Z Closing KubernetesLeaderElectionDriver{configMapName='gsp-restserver-leader'}.
2022-04-06T14:21:14.03Z Stopping DefaultLeaderElectionService.
2022-04-06T14:21:14.029Z Removing cache directory /tmp/flink-web-3034722d-bd81-4ef7-8292-1d4e2de40d32/flink-web-ui
2022-04-06T14:16:20.145Z Time taken for Delete operation is: 1 ms with threads: 0
2022-04-06T14:16:20.145Z Disabling threads for Delete operation as thread count 0 is <= 1
2022-04-06T14:16:20.119Z Finished cleaning up the high availability data for job 00000000000000000000000000000000.
2022-04-06T14:16:19.349Z Clean up the high availability data for job 00000000000000000000000000000000.
2022-04-06T14:16:15.313Z Removed job graph 00000000000000000000000000000000 from KubernetesStateHandleStore{configMapName='gsp-dispatcher-leader'}.
2022-04-06T14:16:14.026Z The watcher is closing.
2022-04-06T14:16:14.026Z Closing KubernetesLeaderElectionDriver{configMapName='gsp-00000000000000000000000000000000-jobmanager-leader'}.
2022-04-06T14:16:14.026Z Stopping DefaultLeaderElectionService.
2022-04-06T14:16:14.021Z Disconnect job manager b3883fc8d6b2f800d573dba9fd7a4aa7@akka.tcp://flink@10.204.19.239:6123/user/rpc/jobmanager_2 for job 00000000000000000000000000000000 from the resource manager.
2022-04-06T14:16:14.021Z The watcher is closing.
2022-04-06T14:16:14.02Z Stopping KubernetesLeaderRetrievalDriver{configMapName='gsp-00000000000000000000000000000000-jobmanager-leader'}.
2022-04-06T14:16:14.02Z The watcher is closing.
2022-04-06T14:16:14.02Z Stopping DefaultLeaderRetrievalService.
2022-04-06T14:16:14.02Z Stopping KubernetesLeaderRetrievalDriver{configMapName='gsp-resourcemanager-leader'}.
2022-04-06T14:16:14.02Z Stopping DefaultLeaderRetrievalService.
2022-04-06T14:16:14.02Z Close ResourceManager connection 56a3c21df209479ff40e42efd6e4f933: Stopping JobMaster for job gim(00000000000000000000000000000000)..
2022-04-06T14:16:14.019Z Releasing slot [9f08529035c5c24d6023210c045c4558].
2022-04-06T14:16:14.017Z Shutting down rest endpoint.
2022-04-06T14:16:14.016Z Shutting StandaloneApplicationClusterEntryPoint down with application status CANCELED. Diagnostics null.
2022-04-06T14:16:14.014Z Application CANCELED:
2022-04-06T14:16:14.014Z Stopping the JobMaster for job gim(00000000000000000000000000000000).
2022-04-06T14:16:14.012Z Job 00000000000000000000000000000000 reached terminal state CANCELED.
2022-04-06T14:15:52.621Z Removing counter from ConfigMap gsp-00000000000000000000000000000000-jobmanager-leader
2022-04-06T14:15:52.62Z Shutting down.
2022-04-06T14:15:51.36Z Time taken for Delete operation is: 0 ms with threads: 0
2022-04-06T14:15:51.36Z Disabling threads for Delete operation as thread count 0 is <= 1
2022-04-06T14:15:49.982Z Shutting down
2022-04-06T14:15:49.982Z Stopping checkpoint coordinator for job 00000000000000000000000000000000.
2022-04-06T14:15:49.981Z Clearing resource requirements of job 00000000000000000000000000000000
2022-04-06T14:15:49.981Z Job gim (00000000000000000000000000000000) switched from state CANCELLING to CANCELED.
...

Is it known issue?

Thanks,
Alexey

Re: Error during shutdown of StandaloneApplicationClusterEntryPoint via JVM shutdown hook

Posted by Roman Khachatryan <ro...@apache.org>.
I'd try to increase the value, so that the timeout doesn't happen
during the shutdown.

Regards,
Roman

On Fri, Apr 8, 2022 at 7:50 PM Alexey Trenikhun <ye...@msn.com> wrote:
>
> Hi Roman,
> Currently rest.async.store-duration is not set. Are you suggesting to try to decrease value from default or vice-versa?
>
> Thanks,
> Alexey
> ________________________________
> From: Roman Khachatryan <ro...@apache.org>
> Sent: Friday, April 8, 2022 5:32:45 AM
> To: Alexey Trenikhun <ye...@msn.com>
> Cc: Flink User Mail List <us...@flink.apache.org>
> Subject: Re: Error during shutdown of StandaloneApplicationClusterEntryPoint via JVM shutdown hook
>
> Hello,
>
> Unfortunately, it's difficult to name the exact reason why the timeout
> happens because there's no message logged.
> I've opened a ticket to improve the logging [1].
> There, I also listed some code paths that might lead to this situation.
>
> From the described scenario, I'd suppose that it's
> CompletedOperationCache.closeAsync()  that times out. It can be
> verified or maybe mitigated by changing rest.async.store-duration  [2]
> (the default is 5 minutes).
> Could you check that?
>
> [1]
> https://issues.apache.org/jira/browse/FLINK-27144
> [2]
> https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/config/#rest-async-store-duration
>
> Regards,
> Roman
>
> On Wed, Apr 6, 2022 at 5:21 PM Alexey Trenikhun <ye...@msn.com> wrote:
> >
> > Hello,
> >
> > We are using Flink 1.13.6, Application Mode, k8s HA. To upgrade job, we use POST, url=http://gsp-jm:8081/jobs/00000000000000000000000000000000/savepoints, then we wait for up to 5 minutes for completion, periodically pulling status (GET, url=http://gsp-jm:8081/jobs/00000000000000000000000000000000/savepoints/{triggerId}). If savepoint is not complete in 5 minute, we cancel job (PATCH, url=http://gsp-jm:8081/jobs/0000000000000000000000000000000). Usually it works well, job stopped one way or another and we proceed with upgrade, but currently JM exits with code -2, and as result k8s restarts pod. We tried multiple times, but every time getting -2. JM log is below (newest messages on top):
> >
> > 2022-04-06T14:21:17.465Z Error during shutdown of StandaloneApplicationClusterEntryPoint via JVM shutdown hook.
> > java.util.concurrent.CompletionException: java.util.concurrent.TimeoutException
> >  at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
> >  at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
> >  at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:661)
> >  at java.util.concurrent.CompletableFuture.uniAcceptStage(CompletableFuture.java:683)
> >  at java.util.concurrent.CompletableFuture.thenAccept(CompletableFuture.java:2010)
> >  at org.apache.flink.runtime.entrypoint.ClusterEntrypoint.closeAsync(ClusterEntrypoint.java:379)
> >  at org.apache.flink.runtime.entrypoint.ClusterEntrypoint.lambda$new$0(ClusterEntrypoint.java:168)
> >  at org.apache.flink.util.ShutdownHookUtil.lambda$addShutdownHook$0(ShutdownHookUtil.java:39)
> >  at java.lang.Thread.run(Thread.java:750) Caused by: java.util.concurrent.TimeoutException: null
> >  at org.apache.flink.runtime.concurrent.FutureUtils$Timeout.run(FutureUtils.java:1255)
> >  at org.apache.flink.runtime.concurrent.DirectExecutorService.execute(DirectExecutorService.java:217)
> >  at org.apache.flink.runtime.concurrent.FutureUtils.lambda$orTimeout$15(FutureUtils.java:582)
> >  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> >  at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> >  at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
> >  at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> >  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> >  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> >  ... 1 common frames omitted
> > 2022-04-06T14:21:17.464Z Terminating cluster entrypoint process StandaloneApplicationClusterEntryPoint with exit code 2.
> > java.util.concurrent.TimeoutException: null
> >  at org.apache.flink.runtime.concurrent.FutureUtils$Timeout.run(FutureUtils.java:1255)
> >  at org.apache.flink.runtime.concurrent.DirectExecutorService.execute(DirectExecutorService.java:217)
> >  at org.apache.flink.runtime.concurrent.FutureUtils.lambda$orTimeout$15(FutureUtils.java:582)
> >  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> >  at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> >  at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
> >  at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> >  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> >  at  java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> >  at java.lang.Thread.run(Thread.java:750)
> > 2022-04-06T14:21:17.463Z Stopped Akka RPC service.
> > 2022-04-06T14:21:17.46Z Stopped Akka RPC service.
> > 2022-04-06T14:21:17.373Z Remoting shut down.
> > 2022-04-06T14:21:17.373Z Remoting shut down.
> > 2022-04-06T14:21:17.297Z Remote daemon shut down; proceeding with flushing remote transports.
> > 2022-04-06T14:21:17.297Z Remote daemon shut down; proceeding with flushing remote transports.
> > 2022-04-06T14:21:17.296Z Shutting down remote daemon.
> > 2022-04-06T14:21:17.296Z Shutting down remote daemon.
> > 2022-04-06T14:21:17.286Z Stopping Akka RPC service.
> > 2022-04-06T14:21:17.281Z Stopping Akka RPC service.
> > 2022-04-06T14:21:17.277Z Finished cleaning up the high availability data.
> > 2022-04-06T14:21:14.04Z Close and clean up all data for KubernetesHaServices.
> > 2022-04-06T14:21:14.04Z Stopped BLOB server at 0.0.0.0:6124
> > 2022-04-06T14:21:14.039Z Stopping DefaultJobGraphStore.
> > 2022-04-06T14:21:14.037Z The watcher is closing.
> > 2022-04-06T14:21:14.037Z Closing KubernetesLeaderElectionDriver{configMapName='gsp-resourcemanager-leader'}.
> > 2022-04-06T14:21:14.037Z Stopping DefaultLeaderElectionService.
> > 2022-04-06T14:21:14.036Z Stopped dispatcher akka.tcp://flink@10.204.19.239:6123/user/rpc/dispatcher_1.
> > 2022-04-06T14:21:14.035Z Suspending the slot manager.
> > 2022-04-06T14:21:14.035Z Closing the slot manager.
> > 2022-04-06T14:21:14.035Z Stopping all currently running jobs of dispatcher akka.tcp://flink@10.204.19.239:6123/user/rpc/dispatcher_1.
> > 2022-04-06T14:21:14.034Z Stopping dispatcher akka.tcp://flink@10.204.19.239:6123/user/rpc/dispatcher_1.
> > 2022-04-06T14:21:14.034Z Stopping SessionDispatcherLeaderProcess.
> > 2022-04-06T14:21:14.033Z The watcher is closing.
> > 2022-04-06T14:21:14.033Z Closing KubernetesLeaderElectionDriver{configMapName='gsp-dispatcher-leader'}.
> > 2022-04-06T14:21:14.033Z Stopping DefaultLeaderElectionService.
> > 2022-04-06T14:21:14.032Z The watcher is closing.
> > 2022-04-06T14:21:14.032Z Stopping KubernetesLeaderRetrievalDriver{configMapName='gsp-resourcemanager-leader'}.
> > 2022-04-06T14:21:14.032Z Stopping DefaultLeaderRetrievalService.
> > 2022-04-06T14:21:14.031Z The watcher is closing.
> > 2022-04-06T14:21:14.031Z Stopping KubernetesLeaderRetrievalDriver{configMapName='gsp-dispatcher-leader'}.
> > 2022-04-06T14:21:14.031Z Stopping DefaultLeaderRetrievalService.
> > 2022-04-06T14:21:14.031Z Closing components.
> > 2022-04-06T14:21:14.031Z Shut down cluster because application is in CANCELED, diagnostics null.
> > 2022-04-06T14:21:14.03Z Shut down complete.
> > 2022-04-06T14:21:14.03Z The watcher is closing.
> > 2022-04-06T14:21:14.03Z Closing KubernetesLeaderElectionDriver{configMapName='gsp-restserver-leader'}.
> > 2022-04-06T14:21:14.03Z Stopping DefaultLeaderElectionService.
> > 2022-04-06T14:21:14.029Z Removing cache directory /tmp/flink-web-3034722d-bd81-4ef7-8292-1d4e2de40d32/flink-web-ui
> > 2022-04-06T14:16:20.145Z Time taken for Delete operation is: 1 ms with threads: 0
> > 2022-04-06T14:16:20.145Z Disabling threads for Delete operation as thread count 0 is <= 1
> > 2022-04-06T14:16:20.119Z Finished cleaning up the high availability data for job 00000000000000000000000000000000.
> > 2022-04-06T14:16:19.349Z Clean up the high availability data for job 00000000000000000000000000000000.
> > 2022-04-06T14:16:15.313Z Removed job graph 00000000000000000000000000000000 from KubernetesStateHandleStore{configMapName='gsp-dispatcher-leader'}.
> > 2022-04-06T14:16:14.026Z The watcher is closing.
> > 2022-04-06T14:16:14.026Z Closing KubernetesLeaderElectionDriver{configMapName='gsp-00000000000000000000000000000000-jobmanager-leader'}.
> > 2022-04-06T14:16:14.026Z Stopping DefaultLeaderElectionService.
> > 2022-04-06T14:16:14.021Z Disconnect job manager b3883fc8d6b2f800d573dba9fd7a4aa7@akka.tcp://flink@10.204.19.239:6123/user/rpc/jobmanager_2 for job 00000000000000000000000000000000 from the resource manager.
> > 2022-04-06T14:16:14.021Z The watcher is closing.
> > 2022-04-06T14:16:14.02Z Stopping KubernetesLeaderRetrievalDriver{configMapName='gsp-00000000000000000000000000000000-jobmanager-leader'}.
> > 2022-04-06T14:16:14.02Z The watcher is closing.
> > 2022-04-06T14:16:14.02Z Stopping DefaultLeaderRetrievalService.
> > 2022-04-06T14:16:14.02Z Stopping KubernetesLeaderRetrievalDriver{configMapName='gsp-resourcemanager-leader'}.
> > 2022-04-06T14:16:14.02Z Stopping DefaultLeaderRetrievalService.
> > 2022-04-06T14:16:14.02Z Close ResourceManager connection 56a3c21df209479ff40e42efd6e4f933: Stopping JobMaster for job gim(00000000000000000000000000000000)..
> > 2022-04-06T14:16:14.019Z Releasing slot [9f08529035c5c24d6023210c045c4558].
> > 2022-04-06T14:16:14.017Z Shutting down rest endpoint.
> > 2022-04-06T14:16:14.016Z Shutting StandaloneApplicationClusterEntryPoint down with application status CANCELED. Diagnostics null.
> > 2022-04-06T14:16:14.014Z Application CANCELED:
> > 2022-04-06T14:16:14.014Z Stopping the JobMaster for job gim(00000000000000000000000000000000).
> > 2022-04-06T14:16:14.012Z Job 00000000000000000000000000000000 reached terminal state CANCELED.
> > 2022-04-06T14:15:52.621Z Removing counter from ConfigMap gsp-00000000000000000000000000000000-jobmanager-leader
> > 2022-04-06T14:15:52.62Z Shutting down.
> > 2022-04-06T14:15:51.36Z Time taken for Delete operation is: 0 ms with threads: 0
> > 2022-04-06T14:15:51.36Z Disabling threads for Delete operation as thread count 0 is <= 1
> > 2022-04-06T14:15:49.982Z Shutting down
> > 2022-04-06T14:15:49.982Z Stopping checkpoint coordinator for job 00000000000000000000000000000000.
> > 2022-04-06T14:15:49.981Z Clearing resource requirements of job 00000000000000000000000000000000
> > 2022-04-06T14:15:49.981Z Job gim (00000000000000000000000000000000) switched from state CANCELLING to CANCELED.
> > ...
> >
> > Is it known issue?
> >
> > Thanks,
> > Alexey

Re: Error during shutdown of StandaloneApplicationClusterEntryPoint via JVM shutdown hook

Posted by Alexey Trenikhun <ye...@msn.com>.
Hi Roman,
Currently rest.async.store-duration is not set. Are you suggesting to try to decrease value from default or vice-versa?

Thanks,
Alexey
________________________________
From: Roman Khachatryan <ro...@apache.org>
Sent: Friday, April 8, 2022 5:32:45 AM
To: Alexey Trenikhun <ye...@msn.com>
Cc: Flink User Mail List <us...@flink.apache.org>
Subject: Re: Error during shutdown of StandaloneApplicationClusterEntryPoint via JVM shutdown hook

Hello,

Unfortunately, it's difficult to name the exact reason why the timeout
happens because there's no message logged.
I've opened a ticket to improve the logging [1].
There, I also listed some code paths that might lead to this situation.

From the described scenario, I'd suppose that it's
CompletedOperationCache.closeAsync()  that times out. It can be
verified or maybe mitigated by changing rest.async.store-duration  [2]
(the default is 5 minutes).
Could you check that?

[1]
https://issues.apache.org/jira/browse/FLINK-27144
[2]
https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/config/#rest-async-store-duration

Regards,
Roman

On Wed, Apr 6, 2022 at 5:21 PM Alexey Trenikhun <ye...@msn.com> wrote:
>
> Hello,
>
> We are using Flink 1.13.6, Application Mode, k8s HA. To upgrade job, we use POST, url=http://gsp-jm:8081/jobs/00000000000000000000000000000000/savepoints, then we wait for up to 5 minutes for completion, periodically pulling status (GET, url=http://gsp-jm:8081/jobs/00000000000000000000000000000000/savepoints/{triggerId}). If savepoint is not complete in 5 minute, we cancel job (PATCH, url=http://gsp-jm:8081/jobs/0000000000000000000000000000000). Usually it works well, job stopped one way or another and we proceed with upgrade, but currently JM exits with code -2, and as result k8s restarts pod. We tried multiple times, but every time getting -2. JM log is below (newest messages on top):
>
> 2022-04-06T14:21:17.465Z Error during shutdown of StandaloneApplicationClusterEntryPoint via JVM shutdown hook.
> java.util.concurrent.CompletionException: java.util.concurrent.TimeoutException
>  at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>  at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>  at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:661)
>  at java.util.concurrent.CompletableFuture.uniAcceptStage(CompletableFuture.java:683)
>  at java.util.concurrent.CompletableFuture.thenAccept(CompletableFuture.java:2010)
>  at org.apache.flink.runtime.entrypoint.ClusterEntrypoint.closeAsync(ClusterEntrypoint.java:379)
>  at org.apache.flink.runtime.entrypoint.ClusterEntrypoint.lambda$new$0(ClusterEntrypoint.java:168)
>  at org.apache.flink.util.ShutdownHookUtil.lambda$addShutdownHook$0(ShutdownHookUtil.java:39)
>  at java.lang.Thread.run(Thread.java:750) Caused by: java.util.concurrent.TimeoutException: null
>  at org.apache.flink.runtime.concurrent.FutureUtils$Timeout.run(FutureUtils.java:1255)
>  at org.apache.flink.runtime.concurrent.DirectExecutorService.execute(DirectExecutorService.java:217)
>  at org.apache.flink.runtime.concurrent.FutureUtils.lambda$orTimeout$15(FutureUtils.java:582)
>  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>  at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>  at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>  at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  ... 1 common frames omitted
> 2022-04-06T14:21:17.464Z Terminating cluster entrypoint process StandaloneApplicationClusterEntryPoint with exit code 2.
> java.util.concurrent.TimeoutException: null
>  at org.apache.flink.runtime.concurrent.FutureUtils$Timeout.run(FutureUtils.java:1255)
>  at org.apache.flink.runtime.concurrent.DirectExecutorService.execute(DirectExecutorService.java:217)
>  at org.apache.flink.runtime.concurrent.FutureUtils.lambda$orTimeout$15(FutureUtils.java:582)
>  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>  at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>  at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>  at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at  java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at java.lang.Thread.run(Thread.java:750)
> 2022-04-06T14:21:17.463Z Stopped Akka RPC service.
> 2022-04-06T14:21:17.46Z Stopped Akka RPC service.
> 2022-04-06T14:21:17.373Z Remoting shut down.
> 2022-04-06T14:21:17.373Z Remoting shut down.
> 2022-04-06T14:21:17.297Z Remote daemon shut down; proceeding with flushing remote transports.
> 2022-04-06T14:21:17.297Z Remote daemon shut down; proceeding with flushing remote transports.
> 2022-04-06T14:21:17.296Z Shutting down remote daemon.
> 2022-04-06T14:21:17.296Z Shutting down remote daemon.
> 2022-04-06T14:21:17.286Z Stopping Akka RPC service.
> 2022-04-06T14:21:17.281Z Stopping Akka RPC service.
> 2022-04-06T14:21:17.277Z Finished cleaning up the high availability data.
> 2022-04-06T14:21:14.04Z Close and clean up all data for KubernetesHaServices.
> 2022-04-06T14:21:14.04Z Stopped BLOB server at 0.0.0.0:6124
> 2022-04-06T14:21:14.039Z Stopping DefaultJobGraphStore.
> 2022-04-06T14:21:14.037Z The watcher is closing.
> 2022-04-06T14:21:14.037Z Closing KubernetesLeaderElectionDriver{configMapName='gsp-resourcemanager-leader'}.
> 2022-04-06T14:21:14.037Z Stopping DefaultLeaderElectionService.
> 2022-04-06T14:21:14.036Z Stopped dispatcher akka.tcp://flink@10.204.19.239:6123/user/rpc/dispatcher_1.
> 2022-04-06T14:21:14.035Z Suspending the slot manager.
> 2022-04-06T14:21:14.035Z Closing the slot manager.
> 2022-04-06T14:21:14.035Z Stopping all currently running jobs of dispatcher akka.tcp://flink@10.204.19.239:6123/user/rpc/dispatcher_1.
> 2022-04-06T14:21:14.034Z Stopping dispatcher akka.tcp://flink@10.204.19.239:6123/user/rpc/dispatcher_1.
> 2022-04-06T14:21:14.034Z Stopping SessionDispatcherLeaderProcess.
> 2022-04-06T14:21:14.033Z The watcher is closing.
> 2022-04-06T14:21:14.033Z Closing KubernetesLeaderElectionDriver{configMapName='gsp-dispatcher-leader'}.
> 2022-04-06T14:21:14.033Z Stopping DefaultLeaderElectionService.
> 2022-04-06T14:21:14.032Z The watcher is closing.
> 2022-04-06T14:21:14.032Z Stopping KubernetesLeaderRetrievalDriver{configMapName='gsp-resourcemanager-leader'}.
> 2022-04-06T14:21:14.032Z Stopping DefaultLeaderRetrievalService.
> 2022-04-06T14:21:14.031Z The watcher is closing.
> 2022-04-06T14:21:14.031Z Stopping KubernetesLeaderRetrievalDriver{configMapName='gsp-dispatcher-leader'}.
> 2022-04-06T14:21:14.031Z Stopping DefaultLeaderRetrievalService.
> 2022-04-06T14:21:14.031Z Closing components.
> 2022-04-06T14:21:14.031Z Shut down cluster because application is in CANCELED, diagnostics null.
> 2022-04-06T14:21:14.03Z Shut down complete.
> 2022-04-06T14:21:14.03Z The watcher is closing.
> 2022-04-06T14:21:14.03Z Closing KubernetesLeaderElectionDriver{configMapName='gsp-restserver-leader'}.
> 2022-04-06T14:21:14.03Z Stopping DefaultLeaderElectionService.
> 2022-04-06T14:21:14.029Z Removing cache directory /tmp/flink-web-3034722d-bd81-4ef7-8292-1d4e2de40d32/flink-web-ui
> 2022-04-06T14:16:20.145Z Time taken for Delete operation is: 1 ms with threads: 0
> 2022-04-06T14:16:20.145Z Disabling threads for Delete operation as thread count 0 is <= 1
> 2022-04-06T14:16:20.119Z Finished cleaning up the high availability data for job 00000000000000000000000000000000.
> 2022-04-06T14:16:19.349Z Clean up the high availability data for job 00000000000000000000000000000000.
> 2022-04-06T14:16:15.313Z Removed job graph 00000000000000000000000000000000 from KubernetesStateHandleStore{configMapName='gsp-dispatcher-leader'}.
> 2022-04-06T14:16:14.026Z The watcher is closing.
> 2022-04-06T14:16:14.026Z Closing KubernetesLeaderElectionDriver{configMapName='gsp-00000000000000000000000000000000-jobmanager-leader'}.
> 2022-04-06T14:16:14.026Z Stopping DefaultLeaderElectionService.
> 2022-04-06T14:16:14.021Z Disconnect job manager b3883fc8d6b2f800d573dba9fd7a4aa7@akka.tcp://flink@10.204.19.239:6123/user/rpc/jobmanager_2 for job 00000000000000000000000000000000 from the resource manager.
> 2022-04-06T14:16:14.021Z The watcher is closing.
> 2022-04-06T14:16:14.02Z Stopping KubernetesLeaderRetrievalDriver{configMapName='gsp-00000000000000000000000000000000-jobmanager-leader'}.
> 2022-04-06T14:16:14.02Z The watcher is closing.
> 2022-04-06T14:16:14.02Z Stopping DefaultLeaderRetrievalService.
> 2022-04-06T14:16:14.02Z Stopping KubernetesLeaderRetrievalDriver{configMapName='gsp-resourcemanager-leader'}.
> 2022-04-06T14:16:14.02Z Stopping DefaultLeaderRetrievalService.
> 2022-04-06T14:16:14.02Z Close ResourceManager connection 56a3c21df209479ff40e42efd6e4f933: Stopping JobMaster for job gim(00000000000000000000000000000000)..
> 2022-04-06T14:16:14.019Z Releasing slot [9f08529035c5c24d6023210c045c4558].
> 2022-04-06T14:16:14.017Z Shutting down rest endpoint.
> 2022-04-06T14:16:14.016Z Shutting StandaloneApplicationClusterEntryPoint down with application status CANCELED. Diagnostics null.
> 2022-04-06T14:16:14.014Z Application CANCELED:
> 2022-04-06T14:16:14.014Z Stopping the JobMaster for job gim(00000000000000000000000000000000).
> 2022-04-06T14:16:14.012Z Job 00000000000000000000000000000000 reached terminal state CANCELED.
> 2022-04-06T14:15:52.621Z Removing counter from ConfigMap gsp-00000000000000000000000000000000-jobmanager-leader
> 2022-04-06T14:15:52.62Z Shutting down.
> 2022-04-06T14:15:51.36Z Time taken for Delete operation is: 0 ms with threads: 0
> 2022-04-06T14:15:51.36Z Disabling threads for Delete operation as thread count 0 is <= 1
> 2022-04-06T14:15:49.982Z Shutting down
> 2022-04-06T14:15:49.982Z Stopping checkpoint coordinator for job 00000000000000000000000000000000.
> 2022-04-06T14:15:49.981Z Clearing resource requirements of job 00000000000000000000000000000000
> 2022-04-06T14:15:49.981Z Job gim (00000000000000000000000000000000) switched from state CANCELLING to CANCELED.
> ...
>
> Is it known issue?
>
> Thanks,
> Alexey

Re: Error during shutdown of StandaloneApplicationClusterEntryPoint via JVM shutdown hook

Posted by Roman Khachatryan <ro...@apache.org>.
Hello,

Unfortunately, it's difficult to name the exact reason why the timeout
happens because there's no message logged.
I've opened a ticket to improve the logging [1].
There, I also listed some code paths that might lead to this situation.

From the described scenario, I'd suppose that it's
CompletedOperationCache.closeAsync()  that times out. It can be
verified or maybe mitigated by changing rest.async.store-duration  [2]
(the default is 5 minutes).
Could you check that?

[1]
https://issues.apache.org/jira/browse/FLINK-27144
[2]
https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/config/#rest-async-store-duration

Regards,
Roman

On Wed, Apr 6, 2022 at 5:21 PM Alexey Trenikhun <ye...@msn.com> wrote:
>
> Hello,
>
> We are using Flink 1.13.6, Application Mode, k8s HA. To upgrade job, we use POST, url=http://gsp-jm:8081/jobs/00000000000000000000000000000000/savepoints, then we wait for up to 5 minutes for completion, periodically pulling status (GET, url=http://gsp-jm:8081/jobs/00000000000000000000000000000000/savepoints/{triggerId}). If savepoint is not complete in 5 minute, we cancel job (PATCH, url=http://gsp-jm:8081/jobs/0000000000000000000000000000000). Usually it works well, job stopped one way or another and we proceed with upgrade, but currently JM exits with code -2, and as result k8s restarts pod. We tried multiple times, but every time getting -2. JM log is below (newest messages on top):
>
> 2022-04-06T14:21:17.465Z Error during shutdown of StandaloneApplicationClusterEntryPoint via JVM shutdown hook.
> java.util.concurrent.CompletionException: java.util.concurrent.TimeoutException
>  at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
>  at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
>  at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:661)
>  at java.util.concurrent.CompletableFuture.uniAcceptStage(CompletableFuture.java:683)
>  at java.util.concurrent.CompletableFuture.thenAccept(CompletableFuture.java:2010)
>  at org.apache.flink.runtime.entrypoint.ClusterEntrypoint.closeAsync(ClusterEntrypoint.java:379)
>  at org.apache.flink.runtime.entrypoint.ClusterEntrypoint.lambda$new$0(ClusterEntrypoint.java:168)
>  at org.apache.flink.util.ShutdownHookUtil.lambda$addShutdownHook$0(ShutdownHookUtil.java:39)
>  at java.lang.Thread.run(Thread.java:750) Caused by: java.util.concurrent.TimeoutException: null
>  at org.apache.flink.runtime.concurrent.FutureUtils$Timeout.run(FutureUtils.java:1255)
>  at org.apache.flink.runtime.concurrent.DirectExecutorService.execute(DirectExecutorService.java:217)
>  at org.apache.flink.runtime.concurrent.FutureUtils.lambda$orTimeout$15(FutureUtils.java:582)
>  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>  at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>  at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>  at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  ... 1 common frames omitted
> 2022-04-06T14:21:17.464Z Terminating cluster entrypoint process StandaloneApplicationClusterEntryPoint with exit code 2.
> java.util.concurrent.TimeoutException: null
>  at org.apache.flink.runtime.concurrent.FutureUtils$Timeout.run(FutureUtils.java:1255)
>  at org.apache.flink.runtime.concurrent.DirectExecutorService.execute(DirectExecutorService.java:217)
>  at org.apache.flink.runtime.concurrent.FutureUtils.lambda$orTimeout$15(FutureUtils.java:582)
>  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>  at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>  at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>  at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at  java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at java.lang.Thread.run(Thread.java:750)
> 2022-04-06T14:21:17.463Z Stopped Akka RPC service.
> 2022-04-06T14:21:17.46Z Stopped Akka RPC service.
> 2022-04-06T14:21:17.373Z Remoting shut down.
> 2022-04-06T14:21:17.373Z Remoting shut down.
> 2022-04-06T14:21:17.297Z Remote daemon shut down; proceeding with flushing remote transports.
> 2022-04-06T14:21:17.297Z Remote daemon shut down; proceeding with flushing remote transports.
> 2022-04-06T14:21:17.296Z Shutting down remote daemon.
> 2022-04-06T14:21:17.296Z Shutting down remote daemon.
> 2022-04-06T14:21:17.286Z Stopping Akka RPC service.
> 2022-04-06T14:21:17.281Z Stopping Akka RPC service.
> 2022-04-06T14:21:17.277Z Finished cleaning up the high availability data.
> 2022-04-06T14:21:14.04Z Close and clean up all data for KubernetesHaServices.
> 2022-04-06T14:21:14.04Z Stopped BLOB server at 0.0.0.0:6124
> 2022-04-06T14:21:14.039Z Stopping DefaultJobGraphStore.
> 2022-04-06T14:21:14.037Z The watcher is closing.
> 2022-04-06T14:21:14.037Z Closing KubernetesLeaderElectionDriver{configMapName='gsp-resourcemanager-leader'}.
> 2022-04-06T14:21:14.037Z Stopping DefaultLeaderElectionService.
> 2022-04-06T14:21:14.036Z Stopped dispatcher akka.tcp://flink@10.204.19.239:6123/user/rpc/dispatcher_1.
> 2022-04-06T14:21:14.035Z Suspending the slot manager.
> 2022-04-06T14:21:14.035Z Closing the slot manager.
> 2022-04-06T14:21:14.035Z Stopping all currently running jobs of dispatcher akka.tcp://flink@10.204.19.239:6123/user/rpc/dispatcher_1.
> 2022-04-06T14:21:14.034Z Stopping dispatcher akka.tcp://flink@10.204.19.239:6123/user/rpc/dispatcher_1.
> 2022-04-06T14:21:14.034Z Stopping SessionDispatcherLeaderProcess.
> 2022-04-06T14:21:14.033Z The watcher is closing.
> 2022-04-06T14:21:14.033Z Closing KubernetesLeaderElectionDriver{configMapName='gsp-dispatcher-leader'}.
> 2022-04-06T14:21:14.033Z Stopping DefaultLeaderElectionService.
> 2022-04-06T14:21:14.032Z The watcher is closing.
> 2022-04-06T14:21:14.032Z Stopping KubernetesLeaderRetrievalDriver{configMapName='gsp-resourcemanager-leader'}.
> 2022-04-06T14:21:14.032Z Stopping DefaultLeaderRetrievalService.
> 2022-04-06T14:21:14.031Z The watcher is closing.
> 2022-04-06T14:21:14.031Z Stopping KubernetesLeaderRetrievalDriver{configMapName='gsp-dispatcher-leader'}.
> 2022-04-06T14:21:14.031Z Stopping DefaultLeaderRetrievalService.
> 2022-04-06T14:21:14.031Z Closing components.
> 2022-04-06T14:21:14.031Z Shut down cluster because application is in CANCELED, diagnostics null.
> 2022-04-06T14:21:14.03Z Shut down complete.
> 2022-04-06T14:21:14.03Z The watcher is closing.
> 2022-04-06T14:21:14.03Z Closing KubernetesLeaderElectionDriver{configMapName='gsp-restserver-leader'}.
> 2022-04-06T14:21:14.03Z Stopping DefaultLeaderElectionService.
> 2022-04-06T14:21:14.029Z Removing cache directory /tmp/flink-web-3034722d-bd81-4ef7-8292-1d4e2de40d32/flink-web-ui
> 2022-04-06T14:16:20.145Z Time taken for Delete operation is: 1 ms with threads: 0
> 2022-04-06T14:16:20.145Z Disabling threads for Delete operation as thread count 0 is <= 1
> 2022-04-06T14:16:20.119Z Finished cleaning up the high availability data for job 00000000000000000000000000000000.
> 2022-04-06T14:16:19.349Z Clean up the high availability data for job 00000000000000000000000000000000.
> 2022-04-06T14:16:15.313Z Removed job graph 00000000000000000000000000000000 from KubernetesStateHandleStore{configMapName='gsp-dispatcher-leader'}.
> 2022-04-06T14:16:14.026Z The watcher is closing.
> 2022-04-06T14:16:14.026Z Closing KubernetesLeaderElectionDriver{configMapName='gsp-00000000000000000000000000000000-jobmanager-leader'}.
> 2022-04-06T14:16:14.026Z Stopping DefaultLeaderElectionService.
> 2022-04-06T14:16:14.021Z Disconnect job manager b3883fc8d6b2f800d573dba9fd7a4aa7@akka.tcp://flink@10.204.19.239:6123/user/rpc/jobmanager_2 for job 00000000000000000000000000000000 from the resource manager.
> 2022-04-06T14:16:14.021Z The watcher is closing.
> 2022-04-06T14:16:14.02Z Stopping KubernetesLeaderRetrievalDriver{configMapName='gsp-00000000000000000000000000000000-jobmanager-leader'}.
> 2022-04-06T14:16:14.02Z The watcher is closing.
> 2022-04-06T14:16:14.02Z Stopping DefaultLeaderRetrievalService.
> 2022-04-06T14:16:14.02Z Stopping KubernetesLeaderRetrievalDriver{configMapName='gsp-resourcemanager-leader'}.
> 2022-04-06T14:16:14.02Z Stopping DefaultLeaderRetrievalService.
> 2022-04-06T14:16:14.02Z Close ResourceManager connection 56a3c21df209479ff40e42efd6e4f933: Stopping JobMaster for job gim(00000000000000000000000000000000)..
> 2022-04-06T14:16:14.019Z Releasing slot [9f08529035c5c24d6023210c045c4558].
> 2022-04-06T14:16:14.017Z Shutting down rest endpoint.
> 2022-04-06T14:16:14.016Z Shutting StandaloneApplicationClusterEntryPoint down with application status CANCELED. Diagnostics null.
> 2022-04-06T14:16:14.014Z Application CANCELED:
> 2022-04-06T14:16:14.014Z Stopping the JobMaster for job gim(00000000000000000000000000000000).
> 2022-04-06T14:16:14.012Z Job 00000000000000000000000000000000 reached terminal state CANCELED.
> 2022-04-06T14:15:52.621Z Removing counter from ConfigMap gsp-00000000000000000000000000000000-jobmanager-leader
> 2022-04-06T14:15:52.62Z Shutting down.
> 2022-04-06T14:15:51.36Z Time taken for Delete operation is: 0 ms with threads: 0
> 2022-04-06T14:15:51.36Z Disabling threads for Delete operation as thread count 0 is <= 1
> 2022-04-06T14:15:49.982Z Shutting down
> 2022-04-06T14:15:49.982Z Stopping checkpoint coordinator for job 00000000000000000000000000000000.
> 2022-04-06T14:15:49.981Z Clearing resource requirements of job 00000000000000000000000000000000
> 2022-04-06T14:15:49.981Z Job gim (00000000000000000000000000000000) switched from state CANCELLING to CANCELED.
> ...
>
> Is it known issue?
>
> Thanks,
> Alexey