소스 검색

HDFS-2949. Add check to active state transition to prevent operator-induced split brain. Contributed by Rushabh S Shah.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1594709 13f79535-47bb-0310-9956-ffa450edef68
Kihwal Lee 11 년 전
부모
커밋
88e76f9c45

+ 65 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.ha;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Map;
 
 import org.apache.commons.cli.Options;
@@ -33,6 +34,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
 import org.apache.hadoop.ha.HAServiceProtocol.RequestSource;
 import org.apache.hadoop.util.Tool;
@@ -66,7 +68,7 @@ public abstract class HAAdmin extends Configured implements Tool {
   protected final static Map<String, UsageInfo> USAGE =
     ImmutableMap.<String, UsageInfo>builder()
     .put("-transitionToActive",
-        new UsageInfo("<serviceId>", "Transitions the service into Active state"))
+        new UsageInfo(" <serviceId> [--"+FORCEACTIVE+"]", "Transitions the service into Active state"))
     .put("-transitionToStandby",
         new UsageInfo("<serviceId>", "Transitions the service into Standby state"))
     .put("-failover",
@@ -100,6 +102,10 @@ public abstract class HAAdmin extends Configured implements Tool {
   }
 
   protected abstract HAServiceTarget resolveTarget(String string);
+  
+  protected Collection<String> getTargetIds(String targetNodeToActivate) {
+    return Arrays.asList(new String[]{targetNodeToActivate});
+  }
 
   protected String getUsageString() {
     return "Usage: HAAdmin";
@@ -133,6 +139,11 @@ public abstract class HAAdmin extends Configured implements Tool {
       printUsage(errOut, "-transitionToActive");
       return -1;
     }
+    /*  returns true if other target node is active or some exception occurred 
+        and forceActive was not set  */
+    if(isOtherTargetNodeActive(argv[0], cmd.hasOption(FORCEACTIVE))) {
+      return -1;
+    }
     HAServiceTarget target = resolveTarget(argv[0]);
     if (!checkManualStateManagementOK(target)) {
       return -1;
@@ -142,7 +153,48 @@ public abstract class HAAdmin extends Configured implements Tool {
     HAServiceProtocolHelper.transitionToActive(proto, createReqInfo());
     return 0;
   }
-
+  
+  /**
+   * Checks whether other target node is active or not
+   * @param targetNodeToActivate
+   * @return true if other target node is active or some other exception 
+   * occurred and forceActive was set otherwise false
+   * @throws IOException
+   */
+  private boolean isOtherTargetNodeActive(String targetNodeToActivate, boolean forceActive)
+      throws IOException  {
+    Collection<String> targetIds = getTargetIds(targetNodeToActivate);
+    if(targetIds == null) {
+      errOut.println("transitionToActive: No target node in the "
+          + "current configuration");
+      printUsage(errOut, "-transitionToActive");
+      return true;
+    }
+    targetIds.remove(targetNodeToActivate);
+    for(String targetId : targetIds) {
+      HAServiceTarget target = resolveTarget(targetId);
+      if (!checkManualStateManagementOK(target)) {
+        return true;
+      }
+      try {
+        HAServiceProtocol proto = target.getProxy(getConf(), 5000);
+        if(proto.getServiceStatus().getState() == HAServiceState.ACTIVE) {
+          errOut.println("transitionToActive: Node " +  targetId +" is already active");
+          printUsage(errOut, "-transitionToActive");
+          return true;
+        }
+      } catch (Exception e) {
+        //If forceActive switch is false then return true
+        if(!forceActive) {
+          errOut.println("Unexpected error occurred  " + e.getMessage());
+          printUsage(errOut, "-transitionToActive");
+          return true; 
+        }
+      }
+    }
+    return false;
+  }
+  
   private int transitionToStandby(final CommandLine cmd)
       throws IOException, ServiceFailedException {
     String[] argv = cmd.getArgs();
@@ -364,6 +416,9 @@ public abstract class HAAdmin extends Configured implements Tool {
     if ("-failover".equals(cmd)) {
       addFailoverCliOpts(opts);
     }
+    if("-transitionToActive".equals(cmd)) {
+      addTransitionToActiveCliOpts(opts);
+    }
     // Mutative commands take FORCEMANUAL option
     if ("-transitionToActive".equals(cmd) ||
         "-transitionToStandby".equals(cmd) ||
@@ -433,6 +488,14 @@ public abstract class HAAdmin extends Configured implements Tool {
     // that change state.
   }
   
+  /**
+   * Add CLI options which are specific to the transitionToActive command and
+   * no others.
+   */
+  private void addTransitionToActiveCliOpts(Options transitionToActiveCliOpts) {
+    transitionToActiveCliOpts.addOption(FORCEACTIVE, false, "force active");
+  }
+  
   private CommandLine parseOpts(String cmdName, Options opts, String[] argv) {
     try {
       // Strip off the first arg, since that's just the command name

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -358,6 +358,9 @@ Release 2.5.0 - UNRELEASED
 
     HDFS-6293. Issues with OIV processing PB-based fsimages. (kihwal)
 
+    HDFS-2949. Add check to active state transition to prevent operator-induced
+    split brain. (Rushabh S Shah via kihwal)
+
   OPTIMIZATIONS
 
     HDFS-6214. Webhdfs has poor throughput for files >2GB (daryn)

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -1677,7 +1677,11 @@ public class NameNode implements NameNodeStatusMXBean {
   public boolean isStandbyState() {
     return (state.equals(STANDBY_STATE));
   }
-
+  
+  public boolean isActiveState() {
+    return (state.equals(ACTIVE_STATE));
+  }
+  
   /**
    * Check that a request to change this node's HA state is valid.
    * In particular, verifies that, if auto failover is enabled, non-forced

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSHAAdmin.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.tools;
 
 import java.io.PrintStream;
 import java.util.Arrays;
+import java.util.Collection;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -27,6 +28,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.ha.HAAdmin;
 import org.apache.hadoop.ha.HAServiceTarget;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -117,7 +119,15 @@ public class DFSHAAdmin extends HAAdmin {
     
     return super.runCmd(argv);
   }
-
+  
+  /**
+   * returns the list of all namenode ids for the given configuration 
+   */
+  @Override
+  protected Collection<String> getTargetIds(String namenodeToActivate) {
+    return DFSUtil.getNameNodeIds(getConf(), (nameserviceId != null)? nameserviceId : DFSUtil.getNamenodeNameServiceId(getConf()));
+  }
+  
   public static void main(String[] argv) throws Exception {
     int res = ToolRunner.run(new DFSHAAdmin(), argv);
     System.exit(res);

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java

@@ -176,6 +176,7 @@ public class TestDFSHAAdmin {
   
   @Test
   public void testTransitionToActive() throws Exception {
+    Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol).getServiceStatus();
     assertEquals(0, runTool("-transitionToActive", "nn1"));
     Mockito.verify(mockProtocol).transitionToActive(
         reqInfoCaptor.capture());

+ 65 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java

@@ -31,6 +31,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ha.HAAdmin;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
@@ -204,6 +205,70 @@ public class TestDFSHAAdminMiniCluster {
     assertEquals(0, runTool("-checkHealth", "nn2"));
   }
   
+  /**
+   * Test case to check whether both the name node is active or not
+   * @throws Exception
+   */
+  @Test
+  public void testTransitionToActiveWhenOtherNamenodeisActive() 
+      throws Exception {
+    NameNode nn1 = cluster.getNameNode(0);
+    NameNode nn2 = cluster.getNameNode(1);
+    if(nn1.getState() != null && !nn1.getState().
+        equals(HAServiceState.STANDBY.name()) ) {
+      cluster.transitionToStandby(0);
+    }
+    if(nn2.getState() != null && !nn2.getState().
+        equals(HAServiceState.STANDBY.name()) ) {
+      cluster.transitionToStandby(1);
+    }
+    //Making sure both the namenode are in standby state
+    assertTrue(nn1.isStandbyState());
+    assertTrue(nn2.isStandbyState());
+    // Triggering the transition for both namenode to Active
+    runTool("-transitionToActive", "nn1");
+    runTool("-transitionToActive", "nn2");
+
+    assertFalse("Both namenodes cannot be active", nn1.isActiveState() 
+        && nn2.isActiveState());
+   
+    /*  This test case doesn't allow nn2 to transition to Active even with
+        forceActive switch since nn1 is already active  */
+    if(nn1.getState() != null && !nn1.getState().
+        equals(HAServiceState.STANDBY.name()) ) {
+      cluster.transitionToStandby(0);
+    }
+    if(nn2.getState() != null && !nn2.getState().
+        equals(HAServiceState.STANDBY.name()) ) {
+      cluster.transitionToStandby(1);
+    }
+    //Making sure both the namenode are in standby state
+    assertTrue(nn1.isStandbyState());
+    assertTrue(nn2.isStandbyState());
+    
+    runTool("-transitionToActive", "nn1");
+    runTool("-transitionToActive", "nn2","--forceactive");
+    
+    assertFalse("Both namenodes cannot be active even though with forceActive",
+        nn1.isActiveState() && nn2.isActiveState());
+
+    /*  In this test case, we have deliberately shut down nn1 and this will
+        cause HAAAdmin#isOtherTargetNodeActive to throw an Exception 
+        and transitionToActive for nn2 with  forceActive switch will succeed 
+        even with Exception  */
+    cluster.shutdownNameNode(0);
+    if(nn2.getState() != null && !nn2.getState().
+        equals(HAServiceState.STANDBY.name()) ) {
+      cluster.transitionToStandby(1);
+    }
+    //Making sure both the namenode (nn2) is in standby state
+    assertTrue(nn2.isStandbyState());
+    assertFalse(cluster.isNameNodeUp(0));
+    
+    runTool("-transitionToActive", "nn2", "--forceactive");
+    assertTrue("Namenode nn2 should be active", nn2.isActiveState());
+  }
+  
   private int runTool(String ... args) throws Exception {
     errOutBytes.reset();
     LOG.info("Running: DFSHAAdmin " + Joiner.on(" ").join(args));