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

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
 ##########
 @@ -894,17 +900,21 @@ public void grantLeadership(final UUID 
newLeaderSessionID) {
 
                                // clear the state if we've been the leader 
before
                                if (getFencingToken() != null) {
-                                       clearState();
+                                       clearStateInternal();
                                }
 
                                setFencingToken(newResourceManagerId);
 
                                slotManager.start(getFencingToken(), 
getMainThreadExecutor(), new ResourceActionsImpl());
 
-                               getRpcService().execute(
-                                       () ->
+                               prepareLeadershipAsync()
+                                       .exceptionally(t -> {
+                                               onFatalError(t);
+                                               return null;
+                                       })
+                                       .thenRunAsync(() ->
                                                // confirming the leader 
session ID might be blocking,
-                                               
leaderElectionService.confirmLeaderSessionID(newLeaderSessionID));
+                                               
leaderElectionService.confirmLeaderSessionID(newLeaderSessionID), 
getRpcService().getExecutor());
 
 Review comment:
   Maybe we could refactor the `grantLeadership` the following way:
   ```
   /**
         * Callback method when current resourceManager is granted leadership.
         *
         * @param newLeaderSessionID unique leadershipID
         */
        @Override
        public void grantLeadership(final UUID newLeaderSessionID) {
                final CompletableFuture<Boolean> acceptLeadershipFuture = 
CompletableFuture.supplyAsync(
                        () -> tryAcceptLeadership(newLeaderSessionID),
                        
getUnfencedMainThreadExecutor()).thenCompose(Function.identity());
   
                final CompletableFuture<Void> confirmationFuture = 
acceptLeadershipFuture.thenAcceptAsync(
                        (Boolean acceptLeadership) -> {
                                if (acceptLeadership) {
                                        // confirming the leader session ID 
might be blocking,
                                        
leaderElectionService.confirmLeaderSessionID(newLeaderSessionID);
                                }
                        },
                        getRpcService().getExecutor());
   
                confirmationFuture.whenComplete(
                        (Void ignored, Throwable throwable) -> {
                                if (throwable != null) {
                                        
onFatalError(ExceptionUtils.stripCompletionException(throwable));
                                }
                        });
        }
   
        private CompletableFuture<Boolean> tryAcceptLeadership(UUID 
newLeaderSessionID) {
                if (leaderElectionService.hasLeadership(newLeaderSessionID)) {
                        final ResourceManagerId newResourceManagerId = 
ResourceManagerId.fromUuid(newLeaderSessionID);
   
                        log.info("ResourceManager {} was granted leadership 
with fencing token {}", getAddress(), newResourceManagerId);
   
                        // clear the state if we've been the leader before
                        if (getFencingToken() != null) {
                                clearStateInternal();
                        }
   
                        setFencingToken(newResourceManagerId);
   
                        slotManager.start(getFencingToken(), 
getMainThreadExecutor(), new ResourceActionsImpl());
   
                        return prepareLeadershipAsync().thenApply(ignored -> 
true);
                } else {
                        return CompletableFuture.completedFuture(false);
                }
        }
   ```

----------------------------------------------------------------
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:
[email protected]


With regards,
Apache Git Services

Reply via email to