[ https://issues.apache.org/jira/browse/FLINK-23129?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17369182#comment-17369182 ]
Yang Wang commented on FLINK-23129: ----------------------------------- I am +1 for only updating the documentation. IIUC, it is an expected behavior when job is canceled or failed. If the job finished successfully, then it should work well. I remember some of our users are submitting multiple job to a same application in the following way. {code:java} // do some preparatory work if necessary if (xxxx) { env.execute("job1"); } // start the streaming job env.executeAsync("job2");{code} > When cancelling any running job of multiple jobs in an application cluster, > JobManager shuts down > ------------------------------------------------------------------------------------------------- > > Key: FLINK-23129 > URL: https://issues.apache.org/jira/browse/FLINK-23129 > Project: Flink > Issue Type: Bug > Components: Runtime / Coordination > Affects Versions: 1.14.0 > Reporter: Robert Metzger > Assignee: Robert Metzger > Priority: Major > Labels: pull-request-available > Fix For: 1.14.0, 1.13.2 > > > I have a jar with two jobs, both executeAsync() from the same main method. I > execute the main method in an Application Mode cluster. When I cancel one of > the two jobs, both jobs will stop executing. > I would expect that the JobManager shuts down once all jobs submitted from an > application are finished. > If this is a known limitation, we should document it. > {code} > 2021-06-23 21:29:53,123 INFO > org.apache.flink.runtime.executiongraph.ExecutionGraph [] - Job first > job (18181be02da272387354d093519b2359) switched from state RUNNING to > CANCELLING. > 2021-06-23 21:29:53,124 INFO > org.apache.flink.runtime.executiongraph.ExecutionGraph [] - Source: > Custom Source -> Sink: Unnamed (1/1) (5a69b1c19f8da23975f6961898ab50a2) > switched from RUNNING to CANCELING. > 2021-06-23 21:29:53,141 INFO > org.apache.flink.runtime.executiongraph.ExecutionGraph [] - Source: > Custom Source -> Sink: Unnamed (1/1) (5a69b1c19f8da23975f6961898ab50a2) > switched from CANCELING to CANCELED. > 2021-06-23 21:29:53,144 INFO > org.apache.flink.runtime.resourcemanager.slotmanager.DeclarativeSlotManager > [] - Clearing resource requirements of job 18181be02da272387354d093519b2359 > 2021-06-23 21:29:53,145 INFO > org.apache.flink.runtime.executiongraph.ExecutionGraph [] - Job first > job (18181be02da272387354d093519b2359) switched from state CANCELLING to > CANCELED. > 2021-06-23 21:29:53,145 INFO > org.apache.flink.runtime.checkpoint.CheckpointCoordinator [] - Stopping > checkpoint coordinator for job 18181be02da272387354d093519b2359. > 2021-06-23 21:29:53,147 INFO > org.apache.flink.runtime.checkpoint.StandaloneCompletedCheckpointStore [] - > Shutting down > 2021-06-23 21:29:53,150 INFO > org.apache.flink.runtime.dispatcher.StandaloneDispatcher [] - Job > 18181be02da272387354d093519b2359 reached terminal state CANCELED. > 2021-06-23 21:29:53,152 INFO org.apache.flink.runtime.jobmaster.JobMaster > [] - Stopping the JobMaster for job first > job(18181be02da272387354d093519b2359). > 2021-06-23 21:29:53,155 INFO > org.apache.flink.runtime.jobmaster.slotpool.DefaultDeclarativeSlotPool [] - > Releasing slot [c35b64879d6b02d383c825ea735ebba0]. > 2021-06-23 21:29:53,159 INFO > org.apache.flink.runtime.resourcemanager.slotmanager.DeclarativeSlotManager > [] - Clearing resource requirements of job 18181be02da272387354d093519b2359 > 2021-06-23 21:29:53,159 INFO org.apache.flink.runtime.jobmaster.JobMaster > [] - Close ResourceManager connection > 281b3fcf7ad0a6f7763fa90b8a5b9adb: Stopping JobMaster for job first > job(18181be02da272387354d093519b2359).. > 2021-06-23 21:29:53,160 INFO > org.apache.flink.runtime.resourcemanager.StandaloneResourceManager [] - > Disconnect job manager > 00000000000000000000000000000...@akka.tcp://flink@localhost:6123/user/rpc/jobmanager_2 > for job 18181be02da272387354d093519b2359 from the resource manager. > 2021-06-23 21:29:53,225 INFO > org.apache.flink.client.deployment.application.ApplicationDispatcherBootstrap > [] - Application CANCELED: > java.util.concurrent.CompletionException: > org.apache.flink.client.deployment.application.UnsuccessfulExecutionException: > Application Status: CANCELED > at > org.apache.flink.client.deployment.application.ApplicationDispatcherBootstrap.lambda$unwrapJobResultException$4(ApplicationDispatcherBootstrap.java:304) > ~[flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:616) > ~[?:1.8.0_252] > at > java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:591) > ~[?:1.8.0_252] > at > java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488) > ~[?:1.8.0_252] > at > java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975) > ~[?:1.8.0_252] > at > org.apache.flink.client.deployment.application.JobStatusPollingUtils.lambda$null$2(JobStatusPollingUtils.java:101) > ~[flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774) > ~[?:1.8.0_252] > at > java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750) > ~[?:1.8.0_252] > at > java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488) > ~[?:1.8.0_252] > at > java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975) > ~[?:1.8.0_252] > at > org.apache.flink.runtime.rpc.akka.AkkaInvocationHandler.lambda$invokeRpc$0(AkkaInvocationHandler.java:237) > ~[flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774) > [?:1.8.0_252] > at > java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750) > [?:1.8.0_252] > at > java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488) > [?:1.8.0_252] > at > java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975) > [?:1.8.0_252] > at > org.apache.flink.runtime.concurrent.FutureUtils$1.onComplete(FutureUtils.java:1081) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at akka.dispatch.OnComplete.internal(Future.scala:264) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at akka.dispatch.OnComplete.internal(Future.scala:261) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at akka.dispatch.japi$CallbackBridge.apply(Future.scala:191) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at akka.dispatch.japi$CallbackBridge.apply(Future.scala:188) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:36) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > org.apache.flink.runtime.concurrent.Executors$DirectExecutionContext.execute(Executors.java:73) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:572) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > akka.pattern.PipeToSupport$PipeableFuture$$anonfun$pipeTo$1.applyOrElse(PipeToSupport.scala:22) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > akka.pattern.PipeToSupport$PipeableFuture$$anonfun$pipeTo$1.applyOrElse(PipeToSupport.scala:21) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at scala.concurrent.Future$$anonfun$andThen$1.apply(Future.scala:436) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at scala.concurrent.Future$$anonfun$andThen$1.apply(Future.scala:435) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:36) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > akka.dispatch.BatchingExecutor$AbstractBatch.processBatch(BatchingExecutor.scala:55) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply$mcV$sp(BatchingExecutor.scala:91) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply(BatchingExecutor.scala:91) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply(BatchingExecutor.scala:91) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:72) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > akka.dispatch.BatchingExecutor$BlockableBatch.run(BatchingExecutor.scala:90) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:44) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > Caused by: > org.apache.flink.client.deployment.application.UnsuccessfulExecutionException: > Application Status: CANCELED > at > org.apache.flink.client.deployment.application.UnsuccessfulExecutionException.fromJobResult(UnsuccessfulExecutionException.java:71) > ~[flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > ... 42 more > Caused by: org.apache.flink.runtime.client.JobCancellationException: Job was > cancelled. > at > org.apache.flink.runtime.jobmaster.JobResult.toJobExecutionResult(JobResult.java:146) > ~[flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > org.apache.flink.client.deployment.application.UnsuccessfulExecutionException.fromJobResult(UnsuccessfulExecutionException.java:60) > ~[flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > ... 42 more > 2021-06-23 21:29:53,238 INFO > org.apache.flink.runtime.entrypoint.ClusterEntrypoint [] - Shutting > StandaloneApplicationClusterEntryPoint down with application status CANCELED. > Diagnostics null. > 2021-06-23 21:29:53,239 INFO > org.apache.flink.runtime.jobmaster.MiniDispatcherRestEndpoint [] - Shutting > down rest endpoint. > 2021-06-23 21:29:53,257 INFO > org.apache.flink.runtime.jobmaster.MiniDispatcherRestEndpoint [] - Removing > cache directory > /var/folders/js/yfk_y2450q7559kygttykwk00000gn/T/flink-web-a0d034d2-da2b-4d72-9ece-ec00c9ae032b/flink-web-ui > 2021-06-23 21:29:53,307 INFO > org.apache.flink.runtime.jobmaster.MiniDispatcherRestEndpoint [] - > http://localhost:8081 lost leadership > 2021-06-23 21:29:53,307 INFO > org.apache.flink.runtime.jobmaster.MiniDispatcherRestEndpoint [] - Shut down > complete. > 2021-06-23 21:29:53,307 INFO > org.apache.flink.runtime.resourcemanager.StandaloneResourceManager [] - Shut > down cluster because application is in CANCELED, diagnostics null. > 2021-06-23 21:29:53,307 INFO > org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponent > [] - Closing components. > 2021-06-23 21:29:53,308 INFO > org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess [] > - Stopping SessionDispatcherLeaderProcess. > 2021-06-23 21:29:53,308 INFO > org.apache.flink.runtime.dispatcher.StandaloneDispatcher [] - Stopping > dispatcher akka.tcp://flink@localhost:6123/user/rpc/dispatcher_0. > 2021-06-23 21:29:53,308 INFO > org.apache.flink.runtime.dispatcher.StandaloneDispatcher [] - Stopping > all currently running jobs of dispatcher > akka.tcp://flink@localhost:6123/user/rpc/dispatcher_0. > 2021-06-23 21:29:53,308 INFO > org.apache.flink.runtime.resourcemanager.ResourceManagerServiceImpl [] - > Stopping resource manager service. > 2021-06-23 21:29:53,308 INFO org.apache.flink.runtime.jobmaster.JobMaster > [] - Stopping the JobMaster for job second > job(e4ff65c30754648cf114232c07ef903e). > 2021-06-23 21:29:53,309 INFO > org.apache.flink.runtime.resourcemanager.slotmanager.DeclarativeSlotManager > [] - Closing the slot manager. > 2021-06-23 21:29:53,309 INFO > org.apache.flink.runtime.dispatcher.StandaloneDispatcher [] - Job > e4ff65c30754648cf114232c07ef903e reached terminal state SUSPENDED. > 2021-06-23 21:29:53,309 INFO > org.apache.flink.runtime.resourcemanager.slotmanager.DeclarativeSlotManager > [] - Suspending the slot manager. > 2021-06-23 21:29:53,309 INFO > org.apache.flink.runtime.resourcemanager.ResourceManagerServiceImpl [] - > Resource manager service is not running. Ignore revoking leadership. > 2021-06-23 21:29:53,309 INFO > org.apache.flink.runtime.executiongraph.ExecutionGraph [] - Job second > job (e4ff65c30754648cf114232c07ef903e) switched from state RUNNING to > SUSPENDED. > org.apache.flink.util.FlinkException: Scheduler is being stopped. > at > org.apache.flink.runtime.scheduler.SchedulerBase.closeAsync(SchedulerBase.java:604) > ~[flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > org.apache.flink.runtime.jobmaster.JobMaster.stopScheduling(JobMaster.java:962) > ~[flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > org.apache.flink.runtime.jobmaster.JobMaster.stopJobExecution(JobMaster.java:926) > ~[flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > org.apache.flink.runtime.jobmaster.JobMaster.onStop(JobMaster.java:398) > ~[flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > org.apache.flink.runtime.rpc.RpcEndpoint.internalCallOnStop(RpcEndpoint.java:214) > ~[flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor$StartedState.terminate(AkkaRpcActor.java:563) > ~[flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleControlMessage(AkkaRpcActor.java:186) > ~[flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at akka.actor.Actor$class.aroundReceive(Actor.scala:517) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at akka.actor.ActorCell.invoke(ActorCell.scala:561) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at akka.dispatch.Mailbox.run(Mailbox.scala:225) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at akka.dispatch.Mailbox.exec(Mailbox.scala:235) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > at > akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) > [flink-dist_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] > 2021-06-23 21:29:53,311 INFO > org.apache.flink.runtime.executiongraph.ExecutionGraph [] - Source: > Custom Source -> Sink: Unnamed (1/1) (b08fac5184817c72f73a0b3fff0afbd3) > switched from RUNNING to CANCELING. > 2021-06-23 21:29:53,312 INFO > org.apache.flink.runtime.executiongraph.ExecutionGraph [] - Source: > Custom Source -> Sink: Unnamed (1/1) (b08fac5184817c72f73a0b3fff0afbd3) > switched from CANCELING to CANCELED. > 2021-06-23 21:29:53,313 INFO > org.apache.flink.runtime.executiongraph.ExecutionGraph [] - Discarding > the results produced by task execution b08fac5184817c72f73a0b3fff0afbd3. > 2021-06-23 21:29:53,314 INFO > org.apache.flink.runtime.checkpoint.CheckpointCoordinator [] - Stopping > checkpoint coordinator for job e4ff65c30754648cf114232c07ef903e. > 2021-06-23 21:29:53,314 INFO > org.apache.flink.runtime.checkpoint.StandaloneCompletedCheckpointStore [] - > Shutting down > 2021-06-23 21:29:53,314 INFO > org.apache.flink.runtime.executiongraph.ExecutionGraph [] - Job > e4ff65c30754648cf114232c07ef903e has been suspended. > 2021-06-23 21:29:53,314 INFO > org.apache.flink.runtime.jobmaster.slotpool.DefaultDeclarativeSlotPool [] - > Releasing slot [30b64fc00bc2c8e83e80567e4f984ae9]. > 2021-06-23 21:29:53,315 INFO org.apache.flink.runtime.jobmaster.JobMaster > [] - Close ResourceManager connection > 281b3fcf7ad0a6f7763fa90b8a5b9adb: Stopping JobMaster for job second > job(e4ff65c30754648cf114232c07ef903e).. > 2021-06-23 21:29:53,318 INFO > org.apache.flink.runtime.dispatcher.StandaloneDispatcher [] - Stopped > dispatcher akka.tcp://flink@localhost:6123/user/rpc/dispatcher_0. > 2021-06-23 21:29:53,323 INFO org.apache.flink.runtime.blob.BlobServer > [] - Stopped BLOB server at 0.0.0.0:61498 > 2021-06-23 21:29:53,323 INFO > org.apache.flink.runtime.rpc.akka.AkkaRpcService [] - Stopping > Akka RPC service. > 2021-06-23 21:29:53,326 INFO > org.apache.flink.runtime.rpc.akka.AkkaRpcService [] - Stopping > Akka RPC service. > 2021-06-23 21:29:53,331 INFO > akka.remote.RemoteActorRefProvider$RemotingTerminator [] - Shutting > down remote daemon. > 2021-06-23 21:29:53,331 INFO > akka.remote.RemoteActorRefProvider$RemotingTerminator [] - Shutting > down remote daemon. > 2021-06-23 21:29:53,332 INFO > akka.remote.RemoteActorRefProvider$RemotingTerminator [] - Remote > daemon shut down; proceeding with flushing remote transports. > 2021-06-23 21:29:53,332 INFO > akka.remote.RemoteActorRefProvider$RemotingTerminator [] - Remote > daemon shut down; proceeding with flushing remote transports. > 2021-06-23 21:29:53,348 INFO > akka.remote.RemoteActorRefProvider$RemotingTerminator [] - Remoting > shut down. > 2021-06-23 21:29:53,348 INFO > akka.remote.RemoteActorRefProvider$RemotingTerminator [] - Remoting > shut down. > 2021-06-23 21:29:53,359 INFO > org.apache.flink.runtime.rpc.akka.AkkaRpcService [] - Stopped > Akka RPC service. > 2021-06-23 21:29:53,366 INFO > org.apache.flink.runtime.rpc.akka.AkkaRpcService [] - Stopped > Akka RPC service. > 2021-06-23 21:29:53,366 INFO > org.apache.flink.runtime.entrypoint.ClusterEntrypoint [] - Terminating > cluster entrypoint process StandaloneApplicationClusterEntryPoint with exit > code 0. > {code} -- This message was sent by Atlassian Jira (v8.3.4#803005)