Selaa lähdekoodia

HADOOP-7938. HA: the FailoverController should optionally fence the active during failover. Contributed by Eli Collins

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1238058 13f79535-47bb-0310-9956-ffa450edef68
Eli Collins 13 vuotta sitten
vanhempi
commit
6884348444

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES.HDFS-1623.txt

@@ -36,3 +36,6 @@ HADOOP-7992. Add ZKClient library to facilitate leader election.
 (Bikas Saha via suresh).
 
 HADOOP-7983. HA: failover should be able to pass args to fencers. (eli)
+
+HADOOP-7938. HA: the FailoverController should optionally fence the
+active during failover. (eli)

+ 51 - 30
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/FailoverController.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.ha;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -26,6 +27,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 
+import com.google.common.base.Preconditions;
+
 /**
  * The FailOverController is responsible for electing an active service
  * on startup or when the current active is changing (eg due to failure),
@@ -48,13 +51,13 @@ public class FailoverController {
    * @throws FailoverFailedException if we should avoid failover
    */
   private static void preFailoverChecks(HAServiceProtocol toSvc,
-                                        String toSvcName)
+                                        InetSocketAddress toSvcAddr)
       throws FailoverFailedException {
     HAServiceState toSvcState;
     try {
       toSvcState = toSvc.getServiceState();
     } catch (IOException e) {
-      String msg = "Unable to get service state for " + toSvcName;
+      String msg = "Unable to get service state for " + toSvcAddr;
       LOG.error(msg, e);
       throw new FailoverFailedException(msg, e);
     }
@@ -69,7 +72,7 @@ public class FailoverController {
           "Can't failover to an unhealthy service", hce);
     } catch (IOException e) {
       throw new FailoverFailedException(
-          "Got an io exception", e);
+          "Got an IO exception", e);
     }
     // TODO(HA): ask toSvc if it's capable. Eg not in SM.
   }
@@ -79,26 +82,42 @@ public class FailoverController {
    * then try to failback.
    *
    * @param fromSvc currently active service
-   * @param fromSvcName name of currently active service
+   * @param fromSvcAddr addr of the currently active service
    * @param toSvc service to make active
-   * @param toSvcName name of service to make active
+   * @param toSvcAddr addr of the service to make active
+   * @param fencer for fencing fromSvc
+   * @param forceFence to fence fromSvc even if not strictly necessary
    * @throws FailoverFailedException if the failover fails
    */
-  public static void failover(HAServiceProtocol fromSvc, String fromSvcName,
-                              HAServiceProtocol toSvc, String toSvcName)
+  public static void failover(HAServiceProtocol fromSvc,
+                              InetSocketAddress fromSvcAddr,
+                              HAServiceProtocol toSvc,
+                              InetSocketAddress toSvcAddr,
+                              NodeFencer fencer, boolean forceFence)
       throws FailoverFailedException {
-    preFailoverChecks(toSvc, toSvcName);
+    Preconditions.checkArgument(fencer != null, "failover requires a fencer");
+    preFailoverChecks(toSvc, toSvcAddr);
 
     // Try to make fromSvc standby
+    boolean tryFence = true;
     try {
       HAServiceProtocolHelper.transitionToStandby(fromSvc);
+      // We should try to fence if we failed or it was forced
+      tryFence = forceFence ? true : false;
     } catch (ServiceFailedException sfe) {
-      LOG.warn("Unable to make " + fromSvcName + " standby (" +
+      LOG.warn("Unable to make " + fromSvcAddr + " standby (" +
           sfe.getMessage() + ")");
-    } catch (Exception e) {
-      LOG.warn("Unable to make " + fromSvcName +
-          " standby (unable to connect)", e);
-      // TODO(HA): fence fromSvc and unfence on failback
+    } catch (IOException ioe) {
+      LOG.warn("Unable to make " + fromSvcAddr +
+          " standby (unable to connect)", ioe);
+    }
+
+    // Fence fromSvc if it's required or forced by the user
+    if (tryFence) {
+      if (!fencer.fence(fromSvcAddr)) {
+        throw new FailoverFailedException("Unable to fence " +
+            fromSvcAddr + ". Fencing failed.");
+      }
     }
 
     // Try to make toSvc active
@@ -107,29 +126,31 @@ public class FailoverController {
     try {
       HAServiceProtocolHelper.transitionToActive(toSvc);
     } catch (ServiceFailedException sfe) {
-      LOG.error("Unable to make " + toSvcName + " active (" +
-          sfe.getMessage() + "). Failing back");
+      LOG.error("Unable to make " + toSvcAddr + " active (" +
+          sfe.getMessage() + "). Failing back.");
       failed = true;
       cause = sfe;
-    } catch (Exception e) {
-      LOG.error("Unable to make " + toSvcName +
-          " active (unable to connect). Failing back", e);
+    } catch (IOException ioe) {
+      LOG.error("Unable to make " + toSvcAddr +
+          " active (unable to connect). Failing back.", ioe);
       failed = true;
-      cause = e;
+      cause = ioe;
     }
 
-    // Try to failback if we failed to make toSvc active
+    // We failed to make toSvc active
     if (failed) {
-      String msg = "Unable to failover to " + toSvcName;
-      try {
-        HAServiceProtocolHelper.transitionToActive(fromSvc);
-      } catch (ServiceFailedException sfe) {
-        msg = "Failback to " + fromSvcName + " failed (" +
-              sfe.getMessage() + ")";
-        LOG.fatal(msg);
-      } catch (Exception e) {
-        msg = "Failback to " + fromSvcName + " failed (unable to connect)";
-        LOG.fatal(msg);
+      String msg = "Unable to failover to " + toSvcAddr;
+      // Only try to failback if we didn't fence fromSvc
+      if (!tryFence) {
+        try {
+          // Unconditionally fence toSvc in case it is still trying to
+          // become active, eg we timed out waiting for its response.
+          failover(toSvc, toSvcAddr, fromSvc, fromSvcAddr, fencer, true);
+        } catch (FailoverFailedException ffe) {
+          msg += ". Failback to " + fromSvcAddr +
+            " failed (" + ffe.getMessage() + ")";
+          LOG.fatal(msg);
+        }
       }
       throw new FailoverFailedException(msg, cause);
     }

+ 63 - 11
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java

@@ -22,6 +22,13 @@ import java.io.PrintStream;
 import java.net.InetSocketAddress;
 import java.util.Map;
 
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.ParseException;
+
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.ipc.RPC;
@@ -37,8 +44,11 @@ import com.google.common.collect.ImmutableMap;
  * mode, or to trigger a health-check.
  */
 @InterfaceAudience.Private
+
 public abstract class HAAdmin extends Configured implements Tool {
   
+  private static final String FORCEFENCE = "forcefence";
+
   private static Map<String, UsageInfo> USAGE =
     ImmutableMap.<String, UsageInfo>builder()
     .put("-transitionToActive",
@@ -46,8 +56,9 @@ public abstract class HAAdmin extends Configured implements Tool {
     .put("-transitionToStandby",
         new UsageInfo("<host:port>", "Transitions the daemon into Standby state"))
     .put("-failover",
-        new UsageInfo("<host:port> <host:port>",
-            "Failover from the first daemon to the second"))
+        new UsageInfo("[--"+FORCEFENCE+"] <host:port> <host:port>",
+            "Failover from the first daemon to the second.\n" +
+            "Unconditionally fence services if the "+FORCEFENCE+" option is used."))
     .put("-getServiceState",
         new UsageInfo("<host:port>", "Returns the state of the daemon"))
     .put("-checkHealth",
@@ -111,20 +122,61 @@ public abstract class HAAdmin extends Configured implements Tool {
 
   private int failover(final String[] argv)
       throws IOException, ServiceFailedException {
-    if (argv.length != 3) {
-      errOut.println("failover: incorrect number of arguments");
+    Configuration conf = getConf();
+    boolean forceFence = false;
+
+    Options failoverOpts = new Options();
+    // "-failover" isn't really an option but we need to add
+    // it to appease CommandLineParser
+    failoverOpts.addOption("failover", false, "failover");
+    failoverOpts.addOption(FORCEFENCE, false, "force fencing");
+
+    CommandLineParser parser = new GnuParser();
+    CommandLine cmd;
+
+    try {
+      cmd = parser.parse(failoverOpts, argv);
+      forceFence = cmd.hasOption(FORCEFENCE);
+    } catch (ParseException pe) {
+      errOut.println("failover: incorrect arguments");
+      printUsage(errOut, "-failover");
+      return -1;
+    }
+    
+    int numOpts = cmd.getOptions() == null ? 0 : cmd.getOptions().length;
+    final String[] args = cmd.getArgs();
+
+    if (numOpts > 2 || args.length != 2) {
+      errOut.println("failover: incorrect arguments");
       printUsage(errOut, "-failover");
       return -1;
     }
 
-    HAServiceProtocol proto1 = getProtocol(argv[1]);
-    HAServiceProtocol proto2 = getProtocol(argv[2]);
+    NodeFencer fencer;
     try {
-      FailoverController.failover(proto1, argv[1], proto2, argv[2]);
-      out.println("Failover from "+argv[1]+" to "+argv[2]+" successful");
+      fencer = NodeFencer.create(conf);
+    } catch (BadFencingConfigurationException bfce) {
+      errOut.println("failover: incorrect fencing configuration: " + 
+          bfce.getLocalizedMessage());
+      return -1;
+    }
+    if (fencer == null) {
+      errOut.println("failover: no fencer configured");
+      return -1;
+    }
+
+    InetSocketAddress addr1 = NetUtils.createSocketAddr(args[0]);
+    InetSocketAddress addr2 = NetUtils.createSocketAddr(args[1]);
+    HAServiceProtocol proto1 = getProtocol(args[0]);
+    HAServiceProtocol proto2 = getProtocol(args[1]);
+
+    try {
+      FailoverController.failover(proto1, addr1, proto2, addr2,
+          fencer, forceFence); 
+      out.println("Failover from "+args[0]+" to "+args[1]+" successful");
     } catch (FailoverFailedException ffe) {
       errOut.println("Failover failed: " + ffe.getLocalizedMessage());
-      return 1;
+      return -1;
     }
     return 0;
   }
@@ -142,7 +194,7 @@ public abstract class HAAdmin extends Configured implements Tool {
       HAServiceProtocolHelper.monitorHealth(proto);
     } catch (HealthCheckFailedException e) {
       errOut.println("Health check failed: " + e.getLocalizedMessage());
-      return 1;
+      return -1;
     }
     return 0;
   }
@@ -223,7 +275,7 @@ public abstract class HAAdmin extends Configured implements Tool {
     }
     
     errOut.println(cmd + " [" + usageInfo.args + "]: " + usageInfo.help);
-    return 1;
+    return 0;
   }
   
   private static class UsageInfo {

+ 0 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceProtocolHelper.java

@@ -19,18 +19,13 @@ package org.apache.hadoop.ha;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ipc.RemoteException;
 
 /**
  * Helper for making {@link HAServiceProtocol} RPC calls. This helper
  * unwraps the {@link RemoteException} to specific exceptions.
- * 
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving

+ 10 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/NodeFencer.java

@@ -82,6 +82,15 @@ public class NodeFencer {
     this.methods = parseMethods(conf);
   }
   
+  public static NodeFencer create(Configuration conf)
+      throws BadFencingConfigurationException {
+    String confStr = conf.get(CONF_METHODS_KEY);
+    if (confStr == null) {
+      return null;
+    }
+    return new NodeFencer(conf);
+  }
+
   public boolean fence(InetSocketAddress serviceAddr) {
     LOG.info("====== Beginning NameNode Fencing Process... ======");
     int i = 0;
@@ -108,7 +117,7 @@ public class NodeFencer {
   }
 
   private static List<FenceMethodWithArg> parseMethods(Configuration conf)
-  throws BadFencingConfigurationException {
+      throws BadFencingConfigurationException {
     String confStr = conf.get(CONF_METHODS_KEY);
     String[] lines = confStr.split("\\s*\n\\s*");
     
@@ -130,7 +139,6 @@ public class NodeFencer {
     if ((m = CLASS_WITH_ARGUMENT.matcher(line)).matches()) {
       String className = m.group(1);
       String arg = m.group(2);
-      
       return createFenceMethod(conf, className, arg);
     } else if ((m = CLASS_WITHOUT_ARGUMENT.matcher(line)).matches()) {
       String className = m.group(1);

+ 167 - 19
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestFailoverController.java

@@ -26,6 +26,9 @@ import static org.mockito.Mockito.verify;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.ha.TestNodeFencer.AlwaysSucceedFencer;
+import org.apache.hadoop.ha.TestNodeFencer.AlwaysFailFencer;
+import static org.apache.hadoop.ha.TestNodeFencer.setupFencer;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
@@ -35,6 +38,9 @@ import static org.junit.Assert.*;
 
 public class TestFailoverController {
 
+  private InetSocketAddress svc1Addr = new InetSocketAddress("svc1", 1234); 
+  private InetSocketAddress svc2Addr = new InetSocketAddress("svc2", 5678); 
+
   private class DummyService implements HAServiceProtocol {
     HAServiceState state;
 
@@ -55,36 +61,41 @@ public class TestFailoverController {
     }
 
     @Override
-    public void monitorHealth() throws HealthCheckFailedException {
+    public void monitorHealth() throws HealthCheckFailedException, IOException {
       // Do nothing
     }
 
     @Override
-    public void transitionToActive() throws ServiceFailedException {
+    public void transitionToActive() throws ServiceFailedException, IOException {
       state = HAServiceState.ACTIVE;
     }
 
     @Override
-    public void transitionToStandby() throws ServiceFailedException {
+    public void transitionToStandby() throws ServiceFailedException, IOException {
       state = HAServiceState.STANDBY;
     }
 
     @Override
-    public HAServiceState getServiceState() {
+    public HAServiceState getServiceState() throws IOException {
       return state;
     }
   }
-
+  
   @Test
   public void testFailoverAndFailback() throws Exception {
     DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
     DummyService svc2 = new DummyService(HAServiceState.STANDBY);
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
 
-    FailoverController.failover(svc1, "svc1",  svc2,  "svc2");
+    AlwaysSucceedFencer.fenceCalled = 0;
+    FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false);
+    assertEquals(0, TestNodeFencer.AlwaysSucceedFencer.fenceCalled);
     assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
     assertEquals(HAServiceState.ACTIVE, svc2.getServiceState());
 
-    FailoverController.failover(svc2, "svc2", svc1, "svc1");
+    AlwaysSucceedFencer.fenceCalled = 0;
+    FailoverController.failover(svc2, svc2Addr, svc1, svc1Addr, fencer, false);
+    assertEquals(0, TestNodeFencer.AlwaysSucceedFencer.fenceCalled);
     assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
     assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
   }
@@ -93,8 +104,9 @@ public class TestFailoverController {
   public void testFailoverFromStandbyToStandby() throws Exception {
     DummyService svc1 = new DummyService(HAServiceState.STANDBY);
     DummyService svc2 = new DummyService(HAServiceState.STANDBY);
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
 
-    FailoverController.failover(svc1, "svc1",  svc2,  "svc2");
+    FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false);
     assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
     assertEquals(HAServiceState.ACTIVE, svc2.getServiceState());
   }
@@ -103,9 +115,10 @@ public class TestFailoverController {
   public void testFailoverFromActiveToActive() throws Exception {
     DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
     DummyService svc2 = new DummyService(HAServiceState.ACTIVE);
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
 
     try {
-      FailoverController.failover(svc1, "svc1",  svc2,  "svc2");
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false);
       fail("Can't failover to an already active service");
     } catch (FailoverFailedException ffe) {
       // Expected
@@ -116,7 +129,7 @@ public class TestFailoverController {
   }
 
   @Test
-  public void testFailoverToUnhealthyServiceFails() throws Exception {
+  public void testFailoverToUnhealthyServiceFailsAndFailsback() throws Exception {
     DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
     DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
       @Override
@@ -124,9 +137,10 @@ public class TestFailoverController {
         throw new HealthCheckFailedException("Failed!");
       }
     };
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
 
     try {
-      FailoverController.failover(svc1, "svc1",  svc2,  "svc2");
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false);
       fail("Failover to unhealthy service");
     } catch (FailoverFailedException ffe) {
       // Expected
@@ -144,17 +158,69 @@ public class TestFailoverController {
       }
     };
     DummyService svc2 = new DummyService(HAServiceState.STANDBY);
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
 
+    AlwaysSucceedFencer.fenceCalled = 0;
     try {
-      FailoverController.failover(svc1, "svc1",  svc2,  "svc2");
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false);
     } catch (FailoverFailedException ffe) {
       fail("Faulty active prevented failover");
     }
-    // svc1 still thinks they're active, that's OK, we'll fence them
+
+    // svc1 still thinks it's active, that's OK, it was fenced
+    assertEquals(1, AlwaysSucceedFencer.fenceCalled);
+    assertEquals("svc1:1234", AlwaysSucceedFencer.fencedSvc);
     assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
     assertEquals(HAServiceState.ACTIVE, svc2.getServiceState());
   }
 
+  @Test
+  public void testFailoverFromFaultyServiceFencingFailure() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.ACTIVE) {
+      @Override
+      public void transitionToStandby() throws ServiceFailedException {
+        throw new ServiceFailedException("Failed!");
+      }
+    };
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY);
+    NodeFencer fencer = setupFencer(AlwaysFailFencer.class.getName());
+
+    AlwaysFailFencer.fenceCalled = 0;
+    try {
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false);
+      fail("Failed over even though fencing failed");
+    } catch (FailoverFailedException ffe) {
+      // Expected
+    }
+
+    assertEquals(1, AlwaysFailFencer.fenceCalled);
+    assertEquals("svc1:1234", AlwaysFailFencer.fencedSvc);
+    assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
+    assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
+  }
+
+  @Test
+  public void testFencingFailureDuringFailover() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY);
+    NodeFencer fencer = setupFencer(AlwaysFailFencer.class.getName());
+
+    AlwaysFailFencer.fenceCalled = 0;
+    try {
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, true);
+      fail("Failed over even though fencing requested and failed");
+    } catch (FailoverFailedException ffe) {
+      // Expected
+    }
+
+    // If fencing was requested and it failed we don't try to make
+    // svc2 active anyway, and we don't failback to svc1.
+    assertEquals(1, AlwaysFailFencer.fenceCalled);
+    assertEquals("svc1:1234", AlwaysFailFencer.fencedSvc);
+    assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
+    assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
+  }
+  
   private HAServiceProtocol getProtocol(String target)
       throws IOException {
     InetSocketAddress addr = NetUtils.createSocketAddr(target);
@@ -166,17 +232,18 @@ public class TestFailoverController {
   }
 
   @Test
-  public void testFailoverFromNonExistantServiceSucceeds() throws Exception {
+  public void testFailoverFromNonExistantServiceWithFencer() throws Exception {
     HAServiceProtocol svc1 = getProtocol("localhost:1234");
     DummyService svc2 = new DummyService(HAServiceState.STANDBY);
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
 
     try {
-      FailoverController.failover(svc1, "svc1",  svc2,  "svc2");
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false);
     } catch (FailoverFailedException ffe) {
       fail("Non-existant active prevented failover");
     }
 
-    // Don't check svc1 (we can't reach it, but that's OK, we'll fence)
+    // Don't check svc1 because we can't reach it, but that's OK, it's been fenced.
     assertEquals(HAServiceState.ACTIVE, svc2.getServiceState());
   }
 
@@ -184,9 +251,10 @@ public class TestFailoverController {
   public void testFailoverToNonExistantServiceFails() throws Exception {
     DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
     HAServiceProtocol svc2 = getProtocol("localhost:1234");
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
 
     try {
-      FailoverController.failover(svc1, "svc1",  svc2,  "svc2");
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false);
       fail("Failed over to a non-existant standby");
     } catch (FailoverFailedException ffe) {
       // Expected
@@ -204,9 +272,10 @@ public class TestFailoverController {
         throw new ServiceFailedException("Failed!");
       }
     };
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
 
     try {
-      FailoverController.failover(svc1, "svc1",  svc2,  "svc2");
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false);
       fail("Failover to already active service");
     } catch (FailoverFailedException ffe) {
       // Expected
@@ -219,6 +288,84 @@ public class TestFailoverController {
     assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
   }
 
+  @Test
+  public void testWeDontFailbackIfActiveWasFenced() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
+      @Override
+      public void transitionToActive() throws ServiceFailedException {
+        throw new ServiceFailedException("Failed!");
+      }
+    };
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
+
+    try {
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, true);
+      fail("Failed over to service that won't transition to active");
+    } catch (FailoverFailedException ffe) {
+      // Expected
+    }
+
+    // We failed to failover and did not failback because we fenced
+    // svc1 (we forced it), therefore svc1 and svc2 should be standby.
+    assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
+    assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
+  }
+
+  @Test
+  public void testWeFenceOnFailbackIfTransitionToActiveFails() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
+      @Override
+      public void transitionToActive() throws ServiceFailedException, IOException {
+        throw new IOException("Failed!");
+      }
+    };
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
+    AlwaysSucceedFencer.fenceCalled = 0;
+
+    try {
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false);
+      fail("Failed over to service that won't transition to active");
+    } catch (FailoverFailedException ffe) {
+      // Expected
+    }
+
+    // We failed to failover. We did not fence svc1 because it cooperated
+    // and we didn't force it, so we failed back to svc1 and fenced svc2.
+    // Note svc2 still thinks it's active, that's OK, we fenced it.
+    assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
+    assertEquals(1, AlwaysSucceedFencer.fenceCalled);
+    assertEquals("svc2:5678", AlwaysSucceedFencer.fencedSvc);
+  }
+
+  @Test
+  public void testFailureToFenceOnFailbackFailsTheFailback() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
+      @Override
+      public void transitionToActive() throws ServiceFailedException, IOException {
+        throw new IOException("Failed!");
+      }
+    };
+    NodeFencer fencer = setupFencer(AlwaysFailFencer.class.getName());
+    AlwaysFailFencer.fenceCalled = 0;
+
+    try {
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false);
+      fail("Failed over to service that won't transition to active");
+    } catch (FailoverFailedException ffe) {
+      // Expected
+    }
+
+    // We did not fence svc1 because it cooperated and we didn't force it, 
+    // we failed to failover so we fenced svc2, we failed to fence svc2
+    // so we did not failback to svc1, ie it's still standby.
+    assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
+    assertEquals(1, AlwaysFailFencer.fenceCalled);
+    assertEquals("svc2:5678", AlwaysFailFencer.fencedSvc);
+  }
+
   @Test
   public void testFailbackToFaultyServiceFails() throws Exception {
     DummyService svc1 = new DummyService(HAServiceState.ACTIVE) {
@@ -233,9 +380,10 @@ public class TestFailoverController {
         throw new ServiceFailedException("Failed!");
       }
     };
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
 
     try {
-      FailoverController.failover(svc1, "svc1", svc2, "svc2");
+      FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, false);
       fail("Failover to already active service");
     } catch (FailoverFailedException ffe) {
       // Expected

+ 64 - 8
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestHAAdmin.java

@@ -80,47 +80,103 @@ public class TestHAAdmin {
     assertOutputContains("transitionToActive: incorrect number of arguments");
     assertEquals(-1, runTool("-transitionToActive", "x", "y"));
     assertOutputContains("transitionToActive: incorrect number of arguments");
+    assertEquals(-1, runTool("-failover"));
+    assertOutputContains("failover: incorrect arguments");
+    assertOutputContains("failover: incorrect arguments");    
+    assertEquals(-1, runTool("-failover", "foo:1234"));
+    assertOutputContains("failover: incorrect arguments");
   }
   
   @Test
   public void testHelp() throws Exception {
     assertEquals(-1, runTool("-help"));
-    assertEquals(1, runTool("-help", "transitionToActive"));
+    assertEquals(0, runTool("-help", "transitionToActive"));
     assertOutputContains("Transitions the daemon into Active");
   }
   
   @Test
   public void testTransitionToActive() throws Exception {
-    assertEquals(0, runTool("-transitionToActive", "xxx"));
+    assertEquals(0, runTool("-transitionToActive", "foo:1234"));
     Mockito.verify(mockProtocol).transitionToActive();
   }
 
   @Test
   public void testTransitionToStandby() throws Exception {
-    assertEquals(0, runTool("-transitionToStandby", "xxx"));
+    assertEquals(0, runTool("-transitionToStandby", "foo:1234"));
     Mockito.verify(mockProtocol).transitionToStandby();
   }
 
   @Test
-  public void testFailover() throws Exception {
+  public void testFailoverWithNoFencerConfigured() throws Exception {
     Mockito.doReturn(HAServiceState.STANDBY).when(mockProtocol).getServiceState();
-    assertEquals(0, runTool("-failover", "xxx", "yyy"));
+    assertEquals(-1, runTool("-failover", "foo:1234", "bar:5678"));
+  }
+
+  @Test
+  public void testFailoverWithFencerConfigured() throws Exception {
+    Mockito.doReturn(HAServiceState.STANDBY).when(mockProtocol).getServiceState();
+    Configuration conf = new Configuration();
+    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    tool.setConf(conf);
+    assertEquals(0, runTool("-failover", "foo:1234", "bar:5678"));
+  }
+
+  @Test
+  public void testFailoverWithFencerConfiguredAndForce() throws Exception {
+    Mockito.doReturn(HAServiceState.STANDBY).when(mockProtocol).getServiceState();
+    Configuration conf = new Configuration();
+    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    tool.setConf(conf);
+    assertEquals(0, runTool("-failover", "foo:1234", "bar:5678", "--forcefence"));
+  }
+
+  @Test
+  public void testFailoverWithInvalidFenceArg() throws Exception {
+    Mockito.doReturn(HAServiceState.STANDBY).when(mockProtocol).getServiceState();
+    Configuration conf = new Configuration();
+    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    tool.setConf(conf);
+    assertEquals(-1, runTool("-failover", "foo:1234", "bar:5678", "notforcefence"));
+  }
+
+  @Test
+  public void testFailoverWithFenceButNoFencer() throws Exception {
+    Mockito.doReturn(HAServiceState.STANDBY).when(mockProtocol).getServiceState();
+    assertEquals(-1, runTool("-failover", "foo:1234", "bar:5678", "--forcefence"));
+  }
+
+  @Test
+  public void testFailoverWithFenceAndBadFencer() throws Exception {
+    Mockito.doReturn(HAServiceState.STANDBY).when(mockProtocol).getServiceState();
+    Configuration conf = new Configuration();
+    conf.set(NodeFencer.CONF_METHODS_KEY, "foobar!");
+    tool.setConf(conf);
+    assertEquals(-1, runTool("-failover", "foo:1234", "bar:5678", "--forcefence"));
+  }
+
+  @Test
+  public void testForceFenceOptionListedBeforeArgs() throws Exception {
+    Mockito.doReturn(HAServiceState.STANDBY).when(mockProtocol).getServiceState();
+    Configuration conf = new Configuration();
+    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    tool.setConf(conf);
+    assertEquals(0, runTool("-failover", "--forcefence", "foo:1234", "bar:5678"));
   }
 
   @Test
   public void testGetServiceState() throws Exception {
-    assertEquals(0, runTool("-getServiceState", "xxx"));
+    assertEquals(0, runTool("-getServiceState", "foo:1234"));
     Mockito.verify(mockProtocol).getServiceState();
   }
 
   @Test
   public void testCheckHealth() throws Exception {
-    assertEquals(0, runTool("-checkHealth", "xxx"));
+    assertEquals(0, runTool("-checkHealth", "foo:1234"));
     Mockito.verify(mockProtocol).monitorHealth();
     
     Mockito.doThrow(new HealthCheckFailedException("fake health check failure"))
       .when(mockProtocol).monitorHealth();
-    assertEquals(1, runTool("-checkHealth", "xxx"));
+    assertEquals(-1, runTool("-checkHealth", "foo:1234"));
     assertOutputContains("Health check failed: fake health check failure");
   }
 

+ 2 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestNodeFencer.java

@@ -119,12 +119,11 @@ public class TestNodeFencer {
     assertFalse(fencer.fence(new InetSocketAddress("host", 1234)));
   }
 
-  private NodeFencer setupFencer(String confStr)
+  public static NodeFencer setupFencer(String confStr)
       throws BadFencingConfigurationException {
     System.err.println("Testing configuration:\n" + confStr);
     Configuration conf = new Configuration();
-    conf.set(NodeFencer.CONF_METHODS_KEY,
-        confStr);
+    conf.set(NodeFencer.CONF_METHODS_KEY, confStr);
     return new NodeFencer(conf);
   }
   

+ 23 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAStateTransitions.java

@@ -105,7 +105,29 @@ public class TestHAStateTransitions {
       cluster.shutdown();
     }
   }
-  
+
+  /**
+   * Test that transitioning a service to the state that it is already
+   * in is a nop, specifically, an exception is not thrown.
+   */
+  @Test
+  public void testTransitionToCurrentStateIsANop() throws Exception {
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(1)
+      .build();
+    try {
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+      cluster.transitionToActive(0);
+      cluster.transitionToStandby(0);
+      cluster.transitionToStandby(0);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
   /**
    * Test manual failover failback for one namespace
    * @param cluster single process test cluster