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

ASF GitHub Bot commented on FLINK-9936:
---------------------------------------

GJL commented on a change in pull request #6464: [FLINK-9936][mesos] WIP 
URL: https://github.com/apache/flink/pull/6464#discussion_r207461579
 
 

 ##########
 File path: 
flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManager.java
 ##########
 @@ -278,22 +267,76 @@ protected void initialize() throws 
ResourceManagerException {
                catch (IOException e) {
                        throw new ResourceManagerException("Unable to configure 
the artifact server with TaskManager artifacts.", e);
                }
+       }
 
-               // begin scheduling
-               connectionMonitor.tell(new ConnectionMonitor.Start(), 
selfActor);
-               schedulerDriver.start();
+       @Override
+       protected CompletableFuture<Void> prepareLeadershipAsync() {
+               Preconditions.checkState(initializedMesosConfig != null);
+
+               return clearStateFuture
+                       .thenRunAsync(() -> {
+                               schedulerDriver = 
initializedMesosConfig.createDriver(
+                                       new 
MesosResourceManagerSchedulerCallback(),
+                                       false);
+
+                               // create supporting actors
+                               connectionMonitor = createConnectionMonitor();
+                               launchCoordinator = 
createLaunchCoordinator(schedulerDriver, selfActor);
+                               reconciliationCoordinator = 
createReconciliationCoordinator(schedulerDriver);
+                               taskMonitor = 
createTaskMonitor(schedulerDriver);
+                       }, getMainThreadExecutor())
+                       .thenCombineAsync(getWorkersAsync(), (ignored, 
tasksFromPreviousAttempts) -> {
+                               // recover state
+                               recoverWorkers(tasksFromPreviousAttempts);
+
+                               // begin scheduling
+                               connectionMonitor.tell(new 
ConnectionMonitor.Start(), selfActor);
+                               schedulerDriver.start();
+
+                               LOG.info("Mesos resource manager started.");
+                               return null;
+                       }, getMainThreadExecutor());
+       }
 
-               LOG.info("Mesos resource manager initialized.");
+       @Override
+       protected void clearState() {
+               schedulerDriver.stop(true);
+
+               clearStateFuture = stopSupportingActorsAsync().thenRunAsync(() 
-> {
+                       workersInNew.clear();
+                       workersInLaunch.clear();
+                       workersBeingReturned.clear();
 
 Review comment:
   That should be safe to do. Fixed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Mesos resource manager unable to connect to master after failover
> -----------------------------------------------------------------
>
>                 Key: FLINK-9936
>                 URL: https://issues.apache.org/jira/browse/FLINK-9936
>             Project: Flink
>          Issue Type: Bug
>          Components: Mesos, Scheduler
>    Affects Versions: 1.5.0, 1.5.1, 1.6.0
>            Reporter: Renjie Liu
>            Assignee: Gary Yao
>            Priority: Blocker
>              Labels: pull-request-available
>             Fix For: 1.5.3, 1.6.0
>
>
> When deployed in mesos session cluster mode, the connector monitor keeps 
> reporting unable to connect to mesos after restart. In fact, scheduler driver 
> already connected to mesos master, but when the connected message is lost. 
> This is because leadership is not granted yet and fence id is not set, the 
> rpc service ignores the connected message. So we should connect to mesos 
> master after leadership is granted.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to