hadoop-yarn-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jian He (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (YARN-4438) Implement RM leader election with curator
Date Wed, 16 Dec 2015 23:31:46 GMT

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

Jian He commented on YARN-4438:
-------------------------------

Thanks for detailed review !

bq. All that is needed is store the CuratorFramework instance in RMContext.
Actually, I need to refactor out the zkClient creation logic from ZKRMStateStore as the zkClient
is requiring a bunch of other configs. And because ZKRMStateStore is currently in active service,
it cannot be simply moved to AlwaysOn service. So, I'd like to do it separately to minimize
the core change in this jira.
bq. The instance, rm, is not used anywhere. Why even pass it?
I was ealier directly calling rm.transitionToActive instead of calling AdminService#transitionToActive.
But just to minimize the change and keep it consistent with EmbeddedElectorService, I changed
to call AdminService#transitionToActive. 
The only extra thing AdminService does is to refresh the ACLs. Suppose the shared storage
based configraion provider is not enabled(which is the most usual case), why do we need to
call refresh the configs? It cannot read the remote RM's config anyway. Without calling these
refresh calls, we can avoid bugs like YARN-3893. Also, RM itself does not need to depend on
the AdminACl for it to transition to active/standby. It should always has the permission to
do that. I'd like to change this part for RM to not refresh the configs if shared storage
based config provider is not enabled. 

bq. why sleep for 1 second
To avoid a busy loop and rejoining immediately. That's what ActiveStandbyElector does too.
It could be more than 1s. I don't think we need one more config for this.

bq. If it is due to close(), don't we want to force give-up so the other RM becomes active?
 If it is on initAndStartLeaderLatch(), this RM will never become active; don't we want to
just die?
What do you mean by force give-up ? exit RM ?
The underlying curator implementation will retry the connection in background, even though
the exception is thrown. See Guaranteeable interface in Curator. I think exit RM is too harsh
here. Even though RM remains at standby, all services should be already shutdown, so there's
no harm to the end users ?

I have one question about ActiveStandbyCheckThread.  if we make zkStateStore and elector to
share the same zkClient, do we still need the ActiveStandbyCheckThread ? the elector itself
should get notification when the connection is lost.

bq. notLeader: Again, we should likely do more than just logging.
This is currently what EmbeddedElectorService is doing. If the leadership is already lost
from zk's perspective, the other RM should take up the leadership
 
bq. How about adding a method called closeLeaderLatch to complement initAndStart? That would
help us avoid cases like the null check missing in rejoinElection?
I think leaderLatch could never be null ?
 
bq.  may be we should have a config to use embedded-elector instead of curator-elector e.g.
yarn.resourcemanager.ha.use-active-standby-elector
This flag is just a temporary thing, a lot of test cases need to be changed without this flag.
I plan to remove this flag and the embeddedElector code too in followup.

bq. Why change the argument to transitionToStandby from true to false? in the following method,
reinitialize(initialize) should be called outside the if. No?
Why does it need to be called outside of {{if (state == HAServiceProtocol.HAServiceState.ACTIVE)}}
? This is a fresh start, it does not need to call reinitiialize.

bq. still feel the AdminService should be the one handling the LeaderElectorService. Also,
the LeaderElectorService talks to AdminService for transitions to active/standby.
Currently, AdminService does not depend on EmbeddedLeaderElector at all. All it does is to
initialize EmbeddedElectorService. May be the elector does not need to depend on AdminService
too, i.e. not need to refresh the acls if shared storage based config provider is not enabled.

Will update other comments accordingly.

> Implement RM leader election with curator
> -----------------------------------------
>
>                 Key: YARN-4438
>                 URL: https://issues.apache.org/jira/browse/YARN-4438
>             Project: Hadoop YARN
>          Issue Type: Improvement
>            Reporter: Jian He
>            Assignee: Jian He
>         Attachments: YARN-4438.1.patch, YARN-4438.2.patch, YARN-4438.3.patch
>
>
> This is to implement the leader election with curator instead of the ActiveStandbyElector
from common package,  this also avoids adding more configs in common to suit RM's own needs.




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message