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

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

bq. Not sure I understand why ZKRMStateStore needs to be an AlwaysOn service.
It does not need to be always on, just the zkClient in ZKRMStateStore needs to 
be always on.
bq. How would this change look? 
At first glance, in AdminService#transitionToStandby and transitionToActive, 
not call refreshAll if the shared-storage-config-provider is not enabled.
bq. Is the concern that Curator may be biased in picking an RM in certain 
conditions?
Yeah, that's just my guess. Immediately rejonning may have more chance to take 
leadership again.  
ActiveStandbyElector#reJoinElectionAfterFailureToBecomeActive has similar 
comments. 
bq.If leaderLatch.close() throws an exception, when does Curator realize the RM 
is not participating in the election anymore? 
Based on my understanding, I think curator will realize when it does not hear 
RM for the zkSessionTimeout period. Essentially, the zkClient at RM side will 
keep retrying to notify zk quorum that this client is closed. If close 
successds, zk quorum will get notified immediately and re-selects a leader. If 
close is kept retrying for beyond zkSessionTimeout, zk quorum will assume this 
client dies and re-selects a leader.
bq.  we might not need that thread.
Then, we can remove this thread ? I'll do separately if you agree.
bq. What happens if this RM is subsequently elected leader? Does the transition 
to Active succeed just fine?
I think it can transition to active next time it's selected as leader. The 
previous failure will most likely happen on refreshAcl.
bq.  I would like for us to err on the side of caution and do null-checks.
will do

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

Reply via email to