[ 
https://issues.apache.org/jira/browse/FLINK-23129?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Robert Metzger reassigned FLINK-23129:
--------------------------------------

    Assignee: Robert Metzger

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

Reply via email to