flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-9936) Mesos resource manager unable to connect to master after failover
Date Fri, 03 Aug 2018 12:01:00 GMT

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

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

GJL commented on a change in pull request #6464: [FLINK-9936][mesos] Mesos resource manager
unable to connect to master after failover 
URL: https://github.com/apache/flink/pull/6464#discussion_r207522805
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
 ##########
 @@ -892,30 +896,48 @@ protected void onFatalError(Throwable t) {
 	 */
 	@Override
 	public void grantLeadership(final UUID newLeaderSessionID) {
-		runAsyncWithoutFencing(
-			() -> {
-				final ResourceManagerId newResourceManagerId = ResourceManagerId.fromUuid(newLeaderSessionID);
-
-				log.info("ResourceManager {} was granted leadership with fencing token {}", getAddress(),
newResourceManagerId);
+		final CompletableFuture<Boolean> acceptLeadershipFuture = CompletableFuture.supplyAsync(
+			() -> tryAcceptLeadership(newLeaderSessionID),
+			getUnfencedMainThreadExecutor()).thenCompose(Function.identity());
+
+		final CompletableFuture<Void> confirmationFuture = acceptLeadershipFuture.thenAcceptAsync(
+			(acceptLeadership) -> {
+				if (acceptLeadership) {
+					// confirming the leader session ID might be blocking,
+					leaderElectionService.confirmLeaderSessionID(newLeaderSessionID);
+				}
+			},
+			getRpcService().getExecutor());
 
-				// clear the state if we've been the leader before
-				if (getFencingToken() != null) {
-					clearStateInternal();
+		confirmationFuture.whenComplete(
+			(Void ignored, Throwable throwable) -> {
+				if (throwable != null) {
+					onFatalError(ExceptionUtils.stripCompletionException(throwable));
 				}
+			});
+	}
 
-				setFencingToken(newResourceManagerId);
+	private CompletableFuture<Boolean> tryAcceptLeadership(final UUID newLeaderSessionID)
{
+		if (leaderElectionService.hasLeadership(newLeaderSessionID)) {
+			final ResourceManagerId newResourceManagerId = ResourceManagerId.fromUuid(newLeaderSessionID);
 
-				slotManager.start(getFencingToken(), getMainThreadExecutor(), new ResourceActionsImpl());
+			log.info("ResourceManager {} was granted leadership with fencing token {}", getAddress(),
newResourceManagerId);
 
-				prepareLeadershipAsync()
-					.thenRunAsync(() ->
-						// confirming the leader session ID might be blocking,
-						leaderElectionService.confirmLeaderSessionID(newLeaderSessionID), getRpcService().getExecutor())
-					.exceptionally(t -> {
-						onFatalError(t);
-						return null;
-					});
-			});
+			// clear the state if we've been the leader before
+			if (getFencingToken() != null) {
+				clearStateInternal();
+			}
+
+			setFencingToken(newResourceManagerId);
+
+			slotManager.start(getFencingToken(), getMainThreadExecutor(), new ResourceActionsImpl());
+
+			return clearStateFuture
 
 Review comment:
   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:
users@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)

Mime
View raw message