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

Tsuyoshi Ozawa commented on YARN-4438:
--------------------------------------

[~jianhe] Thank you for taking the issue. +1 for the design. Could you check 
following comments?

1. In the code path of {{RMWebApp.getHAZookeeperConnectionState}}, 
{{embeddedElector}}, which is null since {{AdminService.embeddedElector}} is 
uninitialized, can be accessed directly. Can we use 
{{rmContext.getLeaderElectorService()}} instead?

{code:title=AdminService.java}
public class AdminService extends CompositeService implements
    HAServiceProtocol, ResourceManagerAdministrationProtocol {
  public String getHAZookeeperConnectionState() {
    if (!rmContext.isHAEnabled()) {
      return "ResourceManager HA is not enabled.";
    } else if (!autoFailoverEnabled) {
      return "Auto Failover is not enabled.";
    }
    return this.embeddedElector.getHAZookeeperConnectionState();
  }
}
{code}

{code:title=RMWebApp.java}
public class RMWebApp extends WebApp implements YarnWebParams {
  ...
  public String getHAZookeeperConnectionState() {
    return rm.getRMContext().getRMAdminService()
      .getHAZookeeperConnectionState();
  }
}
{code}

2. In {{TestLeaderElectorService.testKillZKInstance}}, should we check (rm1 is 
active and rm2 is standby) or  (rm1 is standby and rm2 is active) explicitly to 
detect split brain problem?

{code:title=TestLeaderElectorService.java}
    GenericTestUtils.waitFor(new Supplier<Boolean>() {
      @Override public Boolean get() {
        try {
          return rm1.getAdminService().getServiceStatus().getState()
              .equals(HAServiceState.ACTIVE) || rm2.getAdminService()
              .getServiceStatus().getState().equals(HAServiceState.ACTIVE);
        } catch (IOException e) {
        }
        return false;
      }
{code}

can be:

{code}
          return 
(rm1.getAdminService().getServiceStatus().getState().equals(HAServiceState.ACTIVE)
              && 
rm2.getAdminService().getServiceStatus().getState().equals(HAServiceState.STANDBY))
              || (rm1.getAdminService().getServiceStatus().getState()
              .equals(HAServiceState.STANDBY) && rm2.getAdminService()
              .getServiceStatus().getState().equals(HAServiceState.ACTIVE));
{code}       


Following comments are minor nits:

* LeaderElectorService and TestLeaderElectorService includes lines which exceed 
80 characters.
* Should we name a variable, {{TestingCluster cluster}} in 
TestLeaderElectorService, {{zkCluster}} to make it clear?
* Found a typo:
{code}
  public void testRMFailToTransitionToActive() throws Exception{
    ...
    Thread laucnRM = new Thread() {
    ...
  }
{code}

* We can remove unused imports in {{LeaderElectorService}}, 
{{TestLeaderElectorService}}, {{ResourceManager}}.

{code:title=LeaderElectorService.java}
...
import org.apache.hadoop.fs.CommonConfigurationKeys;
...
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
{code}


{code:title= TestLeaderElectorService.java}
...
import static org.mockito.Mockito.spy;
...
{code}


{code:title=ResourceManager.java}
...
import org.apache.curator.framework.recipes.leader.LeaderLatchListener;
...
{code}

* This is just a question - why did you change an argument of 
{{RMAuditLogger.logFailure}}, target, from {{RMHAProtocolService}} to {{RM}}?

{code:title=AdminService.java}
@@ -319,7 +323,7 @@ public synchronized void transitionToActive(
       rm.transitionToActive();
     } catch (Exception e) {
       RMAuditLogger.logFailure(user.getShortUserName(), "transitionToActive",
-          "", "RMHAProtocolService",
+          "", "RM",
           "Exception transitioning to active");
       throw new ServiceFailedException(
           "Error when transitioning to Active mode", e);
@@ -338,7 +342,7 @@ public synchronized void transitionToActive(
           "Error on refreshAll during transistion to Active", e);
     }
     RMAuditLogger.logSuccess(user.getShortUserName(), "transitionToActive",
-        "RMHAProtocolService");
+        "RM");
   }
{code}

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