Browse Source

HDFS-3582. Hook daemon process exit for testing. Contributed by Eli Collins

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1360331 13f79535-47bb-0310-9956-ffa450edef68
Eli Collins 13 years ago
parent
commit
ef21cbf70a
19 changed files with 275 additions and 309 deletions
  1. 83 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java
  2. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  3. 21 36
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperAsHASharedDir.java
  4. 0 5
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogTestUtil.java
  5. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  6. 10 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  7. 13 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  8. 4 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
  9. 15 19
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  10. 12 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  11. 3 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
  12. 24 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  13. 3 32
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java
  14. 62 76
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java
  15. 4 18
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureOfSharedDir.java
  16. 9 26
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
  17. 0 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java
  18. 0 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyIsHot.java
  19. 5 17
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStateTransitionFailure.java

+ 83 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java

@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.util;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Facilitates hooking process termination for tests and debugging.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Unstable
+public final class ExitUtil {
+  private final static Log LOG = LogFactory.getLog(ExitUtil.class.getName());
+  private static volatile boolean systemExitDisabled = false;
+  private static volatile boolean terminateCalled = false;
+
+  public static class ExitException extends RuntimeException {
+    private static final long serialVersionUID = 1L;
+    public final int status;
+
+    public ExitException(int status, String msg) {
+      super(msg);
+      this.status = status;
+    }
+  }
+
+  /**
+   * Disable the use of System.exit for testing.
+   */
+  public static void disableSystemExit() {
+    systemExitDisabled = true;
+  }
+
+  /**
+   * @return true if terminate has been called
+   */
+  public static boolean terminateCalled() {
+    return terminateCalled;
+  }
+
+  /**
+   * Terminate the current process. Note that terminate is the *only* method
+   * that should be used to terminate the daemon processes.
+   * @param status exit code
+   * @param msg message used to create the ExitException
+   * @throws ExitException if System.exit is disabled for test purposes
+   */
+  public static void terminate(int status, String msg) throws ExitException {
+    LOG.info("Exiting with status " + status);
+    terminateCalled = true;
+    if (systemExitDisabled) {
+      throw new ExitException(status, msg);
+    }
+    System.exit(status);
+  }
+
+  /**
+   * Like {@link terminate(int, String)} without a message.
+   * @param status
+   * @throws ExitException
+   */
+  public static void terminate(int status) throws ExitException {
+    terminate(status, "ExitException");
+  }
+}

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

@@ -123,6 +123,8 @@ Release 2.0.1-alpha - UNRELEASED
 
     HDFS-3611. NameNode prints unnecessary WARNs about edit log normally skipping a few bytes. (Colin Patrick McCabe via harsh)
 
+    HDFS-3582. Hook daemon process exit for testing. (eli)
+
   OPTIMIZATIONS
 
     HDFS-2982. Startup performance suffers when there are many edit log

+ 21 - 36
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperAsHASharedDir.java

@@ -32,7 +32,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 
-import org.apache.hadoop.ha.ServiceFailedException;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogTestUtil;
@@ -42,6 +41,8 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
+import org.apache.hadoop.util.ExitUtil.ExitException;
+
 import org.apache.bookkeeper.proto.BookieServer;
 
 import org.apache.commons.logging.Log;
@@ -49,12 +50,6 @@ import org.apache.commons.logging.LogFactory;
 
 import java.io.IOException;
 
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.atLeastOnce;
-import static org.mockito.Mockito.verify;
-
 /**
  * Integration test to ensure that the BookKeeper JournalManager
  * works for HDFS Namenode HA
@@ -83,8 +78,6 @@ public class TestBookKeeperAsHASharedDir {
    */
   @Test
   public void testFailoverWithBK() throws Exception {
-    Runtime mockRuntime1 = mock(Runtime.class);
-    Runtime mockRuntime2 = mock(Runtime.class);
     MiniDFSCluster cluster = null;
     try {
       Configuration conf = new Configuration();
@@ -100,8 +93,6 @@ public class TestBookKeeperAsHASharedDir {
         .build();
       NameNode nn1 = cluster.getNameNode(0);
       NameNode nn2 = cluster.getNameNode(1);
-      FSEditLogTestUtil.setRuntimeForEditLog(nn1, mockRuntime1);
-      FSEditLogTestUtil.setRuntimeForEditLog(nn2, mockRuntime2);
 
       cluster.waitActive();
       cluster.transitionToActive(0);
@@ -117,9 +108,6 @@ public class TestBookKeeperAsHASharedDir {
 
       assertTrue(fs.exists(p));
     } finally {
-      verify(mockRuntime1, times(0)).exit(anyInt());
-      verify(mockRuntime2, times(0)).exit(anyInt());
-
       if (cluster != null) {
         cluster.shutdown();
       }
@@ -141,9 +129,6 @@ public class TestBookKeeperAsHASharedDir {
 
     BookieServer replacementBookie = null;
 
-    Runtime mockRuntime1 = mock(Runtime.class);
-    Runtime mockRuntime2 = mock(Runtime.class);
-
     MiniDFSCluster cluster = null;
 
     try {
@@ -161,11 +146,10 @@ public class TestBookKeeperAsHASharedDir {
         .nnTopology(MiniDFSNNTopology.simpleHATopology())
         .numDataNodes(0)
         .manageNameDfsSharedDirs(false)
+        .checkExitOnShutdown(false)
         .build();
       NameNode nn1 = cluster.getNameNode(0);
       NameNode nn2 = cluster.getNameNode(1);
-      FSEditLogTestUtil.setRuntimeForEditLog(nn1, mockRuntime1);
-      FSEditLogTestUtil.setRuntimeForEditLog(nn2, mockRuntime2);
 
       cluster.waitActive();
       cluster.transitionToActive(0);
@@ -180,20 +164,22 @@ public class TestBookKeeperAsHASharedDir {
       assertEquals("New bookie didn't stop",
                    numBookies, bkutil.checkBookiesUp(numBookies, 10));
 
-      // mkdirs will "succeed", but nn have called runtime.exit
-      fs.mkdirs(p2);
-      verify(mockRuntime1, atLeastOnce()).exit(anyInt());
-      verify(mockRuntime2, times(0)).exit(anyInt());
+      try {
+        fs.mkdirs(p2);
+        fail("mkdirs should result in the NN exiting");
+      } catch (RemoteException re) {
+        assertTrue(re.getClassName().contains("ExitException"));
+      }
       cluster.shutdownNameNode(0);
 
       try {
         cluster.transitionToActive(1);
         fail("Shouldn't have been able to transition with bookies down");
-      } catch (ServiceFailedException e) {
-        assertTrue("Wrong exception",
-            e.getMessage().contains("Failed to start active services"));
+      } catch (ExitException ee) {
+        assertTrue("Should shutdown due to required journal failure",
+            ee.getMessage().contains(
+                "starting log segment 3 failed for required journal"));
       }
-      verify(mockRuntime2, atLeastOnce()).exit(anyInt());
 
       replacementBookie = bkutil.newBookie();
       assertEquals("Replacement bookie didn't start",
@@ -219,8 +205,6 @@ public class TestBookKeeperAsHASharedDir {
    */
   @Test
   public void testMultiplePrimariesStarted() throws Exception {
-    Runtime mockRuntime1 = mock(Runtime.class);
-    Runtime mockRuntime2 = mock(Runtime.class);
     Path p1 = new Path("/testBKJMMultiplePrimary");
 
     MiniDFSCluster cluster = null;
@@ -235,11 +219,10 @@ public class TestBookKeeperAsHASharedDir {
         .nnTopology(MiniDFSNNTopology.simpleHATopology())
         .numDataNodes(0)
         .manageNameDfsSharedDirs(false)
+        .checkExitOnShutdown(false)
         .build();
       NameNode nn1 = cluster.getNameNode(0);
       NameNode nn2 = cluster.getNameNode(1);
-      FSEditLogTestUtil.setRuntimeForEditLog(nn1, mockRuntime1);
-      FSEditLogTestUtil.setRuntimeForEditLog(nn2, mockRuntime2);
       cluster.waitActive();
       cluster.transitionToActive(0);
 
@@ -248,11 +231,13 @@ public class TestBookKeeperAsHASharedDir {
       nn1.getRpcServer().rollEditLog();
       cluster.transitionToActive(1);
       fs = cluster.getFileSystem(0); // get the older active server.
-      // This edit log updation on older active should make older active
-      // shutdown.
-      fs.delete(p1, true);
-      verify(mockRuntime1, atLeastOnce()).exit(anyInt());
-      verify(mockRuntime2, times(0)).exit(anyInt());
+
+      try {
+        fs.delete(p1, true);
+        fail("Log update on older active should cause it to exit");
+      } catch (RemoteException re) {
+        assertTrue(re.getClassName().contains("ExitException"));
+      }
     } finally {
       if (cluster != null) {
         cluster.shutdown();

+ 0 - 5
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogTestUtil.java

@@ -36,9 +36,4 @@ public class FSEditLogTestUtil {
     FSEditLogLoader.EditLogValidation validation = FSEditLogLoader.validateEditLog(in);
     return (validation.getEndTxId() - in.getFirstTxId()) + 1;
   }
-
-  public static void setRuntimeForEditLog(NameNode nn, Runtime rt) {
-    nn.setRuntimeForTesting(rt);
-    nn.getFSImage().getEditLog().setRuntimeForTesting(rt);
-  }
 }

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -54,6 +54,9 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
+
+import static org.apache.hadoop.util.ExitUtil.terminate;
+
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.Util;
@@ -2991,8 +2994,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
           LOG.warn("ReplicationMonitor thread received InterruptedException.", ie);
           break;
         } catch (Throwable t) {
-          LOG.warn("ReplicationMonitor thread received Runtime exception. ", t);
-          Runtime.getRuntime().exit(-1);
+          LOG.fatal("ReplicationMonitor thread received Runtime exception. ", t);
+          terminate(1);
         }
       }
     }

+ 10 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -123,6 +123,9 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.Util;
+
+import static org.apache.hadoop.util.ExitUtil.terminate;
+
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
@@ -1646,7 +1649,7 @@ public class DataNode extends Configured
       if ("-r".equalsIgnoreCase(cmd) || "--rack".equalsIgnoreCase(cmd)) {
         LOG.error("-r, --rack arguments are not supported anymore. RackID " +
             "resolution is handled by the NameNode.");
-        System.exit(-1);
+        terminate(1);
       } else if ("-rollback".equalsIgnoreCase(cmd)) {
         startOpt = StartupOption.ROLLBACK;
       } else if ("-regular".equalsIgnoreCase(cmd)) {
@@ -1701,15 +1704,15 @@ public class DataNode extends Configured
       if (datanode != null)
         datanode.join();
     } catch (Throwable e) {
-      LOG.error("Exception in secureMain", e);
-      System.exit(-1);
+      LOG.fatal("Exception in secureMain", e);
+      terminate(1);
     } finally {
-      // We need to add System.exit here because either shutdown was called or
-      // some disk related conditions like volumes tolerated or volumes required
+      // We need to terminate the process here because either shutdown was called
+      // or some disk related conditions like volumes tolerated or volumes required
       // condition was not met. Also, In secure mode, control will go to Jsvc
-      // and Datanode process hangs without System.exit.
+      // and Datanode process hangs if it does not exit.
       LOG.warn("Exiting Datanode");
-      System.exit(0);
+      terminate(0);
     }
   }
   

+ 13 - 23
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -35,6 +35,9 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+
+import static org.apache.hadoop.util.ExitUtil.terminate;
+
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.*;
@@ -114,10 +117,6 @@ public class FSEditLog  {
   // is an automatic sync scheduled?
   private volatile boolean isAutoSyncScheduled = false;
   
-  // Used to exit in the event of a failure to sync to all journals. It's a
-  // member variable so it can be swapped out for testing.
-  private Runtime runtime = Runtime.getRuntime();
-
   // these are statistics counters.
   private long numTransactions;        // number of transactions
   private long numTransactionsBatchedInSync;
@@ -210,9 +209,6 @@ public class FSEditLog  {
         DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_MINIMUM_DEFAULT);
 
     journalSet = new JournalSet(minimumRedundantJournals);
-    // set runtime so we can test starting with a faulty or unavailable
-    // shared directory
-    this.journalSet.setRuntimeForTesting(runtime);
 
     for (URI u : dirs) {
       boolean required = FSNamesystem.getRequiredNamespaceEditsDirs(conf)
@@ -525,10 +521,11 @@ public class FSEditLog  {
             }
             editLogStream.setReadyToFlush();
           } catch (IOException e) {
-            LOG.fatal("Could not sync enough journals to persistent storage. "
-                + "Unsynced transactions: " + (txid - synctxid),
-                new Exception());
-            runtime.exit(1);
+            final String msg =
+                "Could not sync enough journals to persistent storage. "
+                + "Unsynced transactions: " + (txid - synctxid);
+            LOG.fatal(msg, new Exception());
+            terminate(1, msg);
           }
         } finally {
           // Prevent RuntimeException from blocking other log edit write 
@@ -547,9 +544,11 @@ public class FSEditLog  {
         }
       } catch (IOException ex) {
         synchronized (this) {
-          LOG.fatal("Could not sync enough journals to persistent storage. "
-              + "Unsynced transactions: " + (txid - synctxid), new Exception());
-          runtime.exit(1);
+          final String msg =
+              "Could not sync enough journals to persistent storage. "
+              + "Unsynced transactions: " + (txid - synctxid);
+          LOG.fatal(msg, new Exception());
+          terminate(1, msg);
         }
       }
       long elapsed = now() - start;
@@ -821,15 +820,6 @@ public class FSEditLog  {
     return journalSet;
   }
   
-  /**
-   * Used only by unit tests.
-   */
-  @VisibleForTesting
-  synchronized public void setRuntimeForTesting(Runtime runtime) {
-    this.runtime = runtime;
-    this.journalSet.setRuntimeForTesting(runtime);
-  }
-
   /**
    * Used only by tests.
    */

+ 4 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java

@@ -32,6 +32,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 
+import static org.apache.hadoop.util.ExitUtil.terminate;
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ComparisonChain;
@@ -163,17 +165,11 @@ public class JournalSet implements JournalManager {
   
   private List<JournalAndStream> journals = Lists.newArrayList();
   final int minimumRedundantJournals;
-  private volatile Runtime runtime = Runtime.getRuntime();
   
   JournalSet(int minimumRedundantResources) {
     this.minimumRedundantJournals = minimumRedundantResources;
   }
   
-  @VisibleForTesting
-  public void setRuntimeForTesting(Runtime runtime) {
-    this.runtime = runtime;
-  }
-  
   @Override
   public EditLogOutputStream startLogSegment(final long txId) throws IOException {
     mapJournalsAndReportErrors(new JournalClosure() {
@@ -319,7 +315,7 @@ public class JournalSet implements JournalManager {
         closure.apply(jas);
       } catch (Throwable t) {
         if (jas.isRequired()) {
-          String msg = "Error: " + status + " failed for required journal ("
+          final String msg = "Error: " + status + " failed for required journal ("
             + jas + ")";
           LOG.fatal(msg, t);
           // If we fail on *any* of the required journals, then we must not
@@ -331,8 +327,7 @@ public class JournalSet implements JournalManager {
           // roll of edits etc. All of them go through this common function 
           // where the isRequired() check is made. Applying exit policy here 
           // to catch all code paths.
-          runtime.exit(1);
-          throw new IOException(msg);
+          terminate(1, msg);
         } else {
           LOG.error("Error: " + status + " failed for (journal " + jas + ")", t);
           badJAS.add(jas);          

+ 15 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -82,6 +82,9 @@ import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
 import org.apache.hadoop.util.ServicePlugin;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.ExitUtil.ExitException;
+
+import static org.apache.hadoop.util.ExitUtil.terminate;
 import static org.apache.hadoop.util.ToolRunner.confirmPrompt;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -227,7 +230,6 @@ public class NameNode {
   private final boolean haEnabled;
   private final HAContext haContext;
   protected boolean allowStaleStandbyReads;
-  private Runtime runtime = Runtime.getRuntime();
 
   
   /** httpServer */
@@ -1095,29 +1097,29 @@ public class NameNode {
       case FORMAT: {
         boolean aborted = format(conf, startOpt.getForceFormat(),
             startOpt.getInteractiveFormat());
-        System.exit(aborted ? 1 : 0);
+        terminate(aborted ? 1 : 0);
         return null; // avoid javac warning
       }
       case GENCLUSTERID: {
         System.err.println("Generating new cluster id:");
         System.out.println(NNStorage.newClusterID());
-        System.exit(0);
+        terminate(0);
         return null;
       }
       case FINALIZE: {
         boolean aborted = finalize(conf, true);
-        System.exit(aborted ? 1 : 0);
+        terminate(aborted ? 1 : 0);
         return null; // avoid javac warning
       }
       case BOOTSTRAPSTANDBY: {
         String toolArgs[] = Arrays.copyOfRange(argv, 1, argv.length);
         int rc = BootstrapStandby.run(toolArgs, conf);
-        System.exit(rc);
+        terminate(rc);
         return null; // avoid warning
       }
       case INITIALIZESHAREDEDITS: {
         boolean aborted = initializeSharedEdits(conf, false, true);
-        System.exit(aborted ? 1 : 0);
+        terminate(aborted ? 1 : 0);
         return null; // avoid warning
       }
       case BACKUP:
@@ -1130,9 +1132,10 @@ public class NameNode {
         NameNode.doRecovery(startOpt, conf);
         return null;
       }
-      default:
+      default: {
         DefaultMetricsSystem.initialize("NameNode");
         return new NameNode(conf);
+      }
     }
   }
 
@@ -1195,8 +1198,8 @@ public class NameNode {
       if (namenode != null)
         namenode.join();
     } catch (Throwable e) {
-      LOG.error("Exception in namenode join", e);
-      System.exit(-1);
+      LOG.fatal("Exception in namenode join", e);
+      terminate(1);
     }
   }
 
@@ -1265,11 +1268,6 @@ public class NameNode {
     }
     return state.getServiceState();
   }
-  
-  @VisibleForTesting
-  public synchronized void setRuntimeForTesting(Runtime runtime) {
-    this.runtime = runtime;
-  }
 
   /**
    * Shutdown the NN immediately in an ungraceful way. Used when it would be
@@ -1278,10 +1276,10 @@ public class NameNode {
    * 
    * @param t exception which warrants the shutdown. Printed to the NN log
    *          before exit.
-   * @throws ServiceFailedException thrown only for testing.
+   * @throws ExitException thrown only for testing.
    */
   private synchronized void doImmediateShutdown(Throwable t)
-      throws ServiceFailedException {
+      throws ExitException {
     String message = "Error encountered requiring NN shutdown. " +
         "Shutting down immediately.";
     try {
@@ -1289,9 +1287,7 @@ public class NameNode {
     } catch (Throwable ignored) {
       // This is unlikely to happen, but there's nothing we can do if it does.
     }
-    runtime.exit(1);
-    // This code is only reached during testing, when runtime is stubbed out.
-    throw new ServiceFailedException(message, t);
+    terminate(1, t.getMessage());
   }
   
   /**

+ 12 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -55,6 +55,9 @@ import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageState;
+
+import static org.apache.hadoop.util.ExitUtil.terminate;
+
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
@@ -323,9 +326,9 @@ public class SecondaryNameNode implements Runnable {
         LOG.error("Exception in doCheckpoint", e);
         e.printStackTrace();
       } catch (Throwable e) {
-        LOG.error("Throwable Exception in doCheckpoint", e);
+        LOG.fatal("Throwable Exception in doCheckpoint", e);
         e.printStackTrace();
-        Runtime.getRuntime().exit(-1);
+        terminate(1);
       }
     }
   }
@@ -517,7 +520,7 @@ public class SecondaryNameNode implements Runnable {
       //
       // This is a error returned by hadoop server. Print
       // out the first line of the error mesage, ignore the stack trace.
-      exitCode = -1;
+      exitCode = 1;
       try {
         String[] content;
         content = e.getLocalizedMessage().split("\n");
@@ -529,7 +532,7 @@ public class SecondaryNameNode implements Runnable {
       //
       // IO exception encountered locally.
       //
-      exitCode = -1;
+      exitCode = 1;
       LOG.error(cmd + ": " + e.getLocalizedMessage());
     } finally {
       // Does the RPC connection need to be closed?
@@ -557,7 +560,8 @@ public class SecondaryNameNode implements Runnable {
   public static void main(String[] argv) throws Exception {
     CommandLineOpts opts = SecondaryNameNode.parseArgs(argv);
     if (opts == null) {
-      System.exit(-1);
+      LOG.fatal("Failed to parse options");
+      terminate(1);
     }
     
     StringUtils.startupShutdownMessage(SecondaryNameNode.class, argv, LOG);
@@ -567,12 +571,12 @@ public class SecondaryNameNode implements Runnable {
       secondary = new SecondaryNameNode(tconf, opts);
     } catch (IOException ioe) {
       LOG.fatal("Failed to start secondary namenode", ioe);
-      System.exit(-1);
+      terminate(1);
     }
 
-    if (opts.getCommand() != null) {
+    if (opts != null && opts.getCommand() != null) {
       int ret = secondary.processStartupCommand(opts);
-      System.exit(ret);
+      terminate(ret);
     }
 
     // Create a never ending deamon

+ 3 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java

@@ -44,6 +44,7 @@ import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.SecurityUtil;
 
 import static org.apache.hadoop.hdfs.server.common.Util.now;
+import static org.apache.hadoop.util.ExitUtil.terminate;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -64,8 +65,6 @@ public class EditLogTailer {
   private final Configuration conf;
   private final FSNamesystem namesystem;
   private FSEditLog editLog;
-  
-  private volatile Runtime runtime = Runtime.getRuntime();
 
   private InetSocketAddress activeAddr;
   private NamenodeProtocol cachedActiveProxy = null;
@@ -169,11 +168,6 @@ public class EditLogTailer {
     this.editLog = editLog;
   }
   
-  @VisibleForTesting
-  synchronized void setRuntime(Runtime runtime) {
-    this.runtime = runtime;
-  }
-  
   public void catchupDuringFailover() throws IOException {
     Preconditions.checkState(tailerThread == null ||
         !tailerThread.isAlive(),
@@ -320,9 +314,9 @@ public class EditLogTailer {
           // interrupter should have already set shouldRun to false
           continue;
         } catch (Throwable t) {
-          LOG.error("Unknown error encountered while tailing edits. " +
+          LOG.fatal("Unknown error encountered while tailing edits. " +
               "Shutting down standby NN.", t);
-          runtime.exit(1);
+          terminate(1, t.getMessage());
         }
 
         try {

+ 24 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -95,6 +95,7 @@ import org.apache.hadoop.net.StaticMapping;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.ProxyUsers;
+import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -140,6 +141,7 @@ public class MiniDFSCluster {
     private boolean waitSafeMode = true;
     private boolean setupHostsFile = false;
     private MiniDFSNNTopology nnTopology = null;
+    private boolean checkExitOnShutdown = true;
     
     public Builder(Configuration conf) {
       this.conf = conf;
@@ -240,7 +242,15 @@ public class MiniDFSCluster {
       this.waitSafeMode = val;
       return this;
     }
-    
+
+    /**
+     * Default: true
+     */
+    public Builder checkExitOnShutdown(boolean val) {
+      this.checkExitOnShutdown = val;
+      return this;
+    }
+
     /**
      * Default: null
      */
@@ -303,7 +313,8 @@ public class MiniDFSCluster {
                        builder.clusterId,
                        builder.waitSafeMode,
                        builder.setupHostsFile,
-                       builder.nnTopology);
+                       builder.nnTopology,
+                       builder.checkExitOnShutdown);
   }
   
   public class DataNodeProperties {
@@ -327,6 +338,7 @@ public class MiniDFSCluster {
   private File data_dir;
   private boolean waitSafeMode = true;
   private boolean federation;
+  private boolean checkExitOnShutdown = true;
   
   /**
    * A unique instance identifier for the cluster. This
@@ -536,7 +548,7 @@ public class MiniDFSCluster {
     initMiniDFSCluster(conf, numDataNodes, format,
         manageNameDfsDirs, true, manageDataDfsDirs, operation, racks, hosts,
         simulatedCapacities, null, true, false,
-        MiniDFSNNTopology.simpleSingleNN(nameNodePort, 0));
+        MiniDFSNNTopology.simpleSingleNN(nameNodePort, 0), true);
   }
 
   private void initMiniDFSCluster(
@@ -546,8 +558,10 @@ public class MiniDFSCluster {
       StartupOption operation, String[] racks,
       String[] hosts, long[] simulatedCapacities, String clusterId,
       boolean waitSafeMode, boolean setupHostsFile,
-      MiniDFSNNTopology nnTopology)
+      MiniDFSNNTopology nnTopology, boolean checkExitOnShutdown)
   throws IOException {
+    ExitUtil.disableSystemExit();
+
     synchronized (MiniDFSCluster.class) {
       instanceId = instanceCount++;
     }
@@ -556,6 +570,7 @@ public class MiniDFSCluster {
     base_dir = new File(determineDfsBaseDir());
     data_dir = new File(base_dir, "data");
     this.waitSafeMode = waitSafeMode;
+    this.checkExitOnShutdown = checkExitOnShutdown;
     
     int replication = conf.getInt(DFS_REPLICATION_KEY, 3);
     conf.setInt(DFS_REPLICATION_KEY, Math.min(replication, numDataNodes));
@@ -1275,6 +1290,11 @@ public class MiniDFSCluster {
    */
   public void shutdown() {
     LOG.info("Shutting down the Mini HDFS Cluster");
+    if (checkExitOnShutdown)  {
+     if (ExitUtil.terminateCalled()) {
+       throw new AssertionError("Test resulted in an unexpected exit");
+     }
+    }
     shutdownDataNodes();
     for (NameNodeInfo nnInfo : nameNodes) {
       if (nnInfo == null) continue;

+ 3 - 32
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java

@@ -30,7 +30,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintStream;
 import java.net.URI;
-import java.security.Permission;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
@@ -43,6 +42,8 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.util.ExitUtil.ExitException;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -69,7 +70,7 @@ public class TestClusterId {
 
   @Before
   public void setUp() throws IOException {
-    System.setSecurityManager(new NoExitSecurityManager());
+    ExitUtil.disableSystemExit();
 
     String baseDir = System.getProperty("test.build.data", "build/test/data");
 
@@ -90,8 +91,6 @@ public class TestClusterId {
 
   @After
   public void tearDown() throws IOException {
-    System.setSecurityManager(null);
-
     if (hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir)) {
       throw new IOException("Could not tearDown test directory '" + hdfsDir
           + "'");
@@ -446,32 +445,4 @@ public class TestClusterId {
     File version = new File(hdfsDir, "current/VERSION");
     assertFalse("Check version should not exist", version.exists());
   }
-
-  private static class ExitException extends SecurityException {
-    private static final long serialVersionUID = 1L;
-    public final int status;
-
-    public ExitException(int status) {
-      super("There is no escape!");
-      this.status = status;
-    }
-  }
-
-  private static class NoExitSecurityManager extends SecurityManager {
-    @Override
-    public void checkPermission(Permission perm) {
-      // allow anything.
-    }
-
-    @Override
-    public void checkPermission(Permission perm, Object context) {
-      // allow anything.
-    }
-
-    @Override
-    public void checkExit(int status) {
-      super.checkExit(status);
-      throw new ExitException(status);
-    }
-  }
 }

+ 62 - 76
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java

@@ -19,14 +19,11 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.anyInt;
+import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.atLeast;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
 
 import java.io.File;
 import java.io.IOException;
@@ -39,18 +36,19 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.ExitUtil.ExitException;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
-import org.mockito.verification.VerificationMode;
 
 public class TestEditLogJournalFailures {
 
   private int editsPerformed = 0;
   private MiniDFSCluster cluster;
   private FileSystem fs;
-  private Runtime runtime;
 
   /**
    * Create the mini cluster for testing and sub in a custom runtime so that
@@ -64,23 +62,23 @@ public class TestEditLogJournalFailures {
   public void setUpMiniCluster(Configuration conf, boolean manageNameDfsDirs)
       throws IOException {
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
-        .manageNameDfsDirs(manageNameDfsDirs).build();
+        .manageNameDfsDirs(manageNameDfsDirs).checkExitOnShutdown(false).build();
     cluster.waitActive();
     fs = cluster.getFileSystem();
-    
-    runtime = Runtime.getRuntime();
-    runtime = spy(runtime);
-    doNothing().when(runtime).exit(anyInt());
-    
-    cluster.getNameNode().getFSImage().getEditLog().setRuntimeForTesting(runtime);
   }
   
   @After
   public void shutDownMiniCluster() throws IOException {
     if (fs != null)
       fs.close();
-    if (cluster != null)
-      cluster.shutdown();
+    if (cluster != null) {
+      try {
+        cluster.shutdown();
+      } catch (ExitException ee) {
+        // Ignore ExitExceptions as the tests may result in the
+        // NameNode doing an immediate shutdown.
+      }
+    }
   }
    
   @Test
@@ -88,11 +86,9 @@ public class TestEditLogJournalFailures {
     assertTrue(doAnEdit());
     // Invalidate one edits journal.
     invalidateEditsDirAtIndex(0, true, false);
-    // Make sure runtime.exit(...) hasn't been called at all yet.
-    assertExitInvocations(0);
+    // The NN has not terminated (no ExitException thrown)
     assertTrue(doAnEdit());
-    // A single journal failure should not result in a call to runtime.exit(...).
-    assertExitInvocations(0);
+    // A single journal failure should not result in a call to terminate
     assertFalse(cluster.getNameNode().isInSafeMode());
   }
    
@@ -102,12 +98,17 @@ public class TestEditLogJournalFailures {
     // Invalidate both edits journals.
     invalidateEditsDirAtIndex(0, true, false);
     invalidateEditsDirAtIndex(1, true, false);
-    // Make sure runtime.exit(...) hasn't been called at all yet.
-    assertExitInvocations(0);
-    assertTrue(doAnEdit());
-    // The previous edit could not be synced to any persistent storage, should
-    // have halted the NN.
-    assertExitInvocations(1);
+    // The NN has not terminated (no ExitException thrown)
+    try {
+      doAnEdit();
+      fail("The previous edit could not be synced to any persistent storage, "
+          + "should have halted the NN");
+    } catch (RemoteException re) {
+      assertTrue(re.getClassName().contains("ExitException"));
+      GenericTestUtils.assertExceptionContains(
+          "Could not sync enough journals to persistent storage. " +
+          "Unsynced transactions: 1", re);
+    }
   }
   
   @Test
@@ -116,12 +117,17 @@ public class TestEditLogJournalFailures {
     // Invalidate both edits journals.
     invalidateEditsDirAtIndex(0, true, true);
     invalidateEditsDirAtIndex(1, true, true);
-    // Make sure runtime.exit(...) hasn't been called at all yet.
-    assertExitInvocations(0);
-    assertTrue(doAnEdit());
-    // The previous edit could not be synced to any persistent storage, should
-    // have halted the NN.
-    assertExitInvocations(atLeast(1));
+    // The NN has not terminated (no ExitException thrown)
+    try {
+      doAnEdit();
+      fail("The previous edit could not be synced to any persistent storage, "
+          + " should have halted the NN");
+    } catch (RemoteException re) {
+      assertTrue(re.getClassName().contains("ExitException"));
+      GenericTestUtils.assertExceptionContains(
+          "Could not sync enough journals to persistent storage. " +
+          "Unsynced transactions: 1", re);
+    }
   }
   
   @Test
@@ -129,11 +135,9 @@ public class TestEditLogJournalFailures {
     assertTrue(doAnEdit());
     // Invalidate one edits journal.
     invalidateEditsDirAtIndex(0, false, false);
-    // Make sure runtime.exit(...) hasn't been called at all yet.
-    assertExitInvocations(0);
+    // The NN has not terminated (no ExitException thrown)
     assertTrue(doAnEdit());
-    // A single journal failure should not result in a call to runtime.exit(...).
-    assertExitInvocations(0);
+    // A single journal failure should not result in a call to terminate
     assertFalse(cluster.getNameNode().isInSafeMode());
   }
   
@@ -157,15 +161,19 @@ public class TestEditLogJournalFailures {
     EditLogFileOutputStream nonRequiredSpy =
       spyOnStream(nonRequiredJas);
     
-    // Make sure runtime.exit(...) hasn't been called at all yet.
-    assertExitInvocations(0);
+    // The NN has not terminated (no ExitException thrown)
     
     // ..and that the other stream is active.
     assertTrue(nonRequiredJas.isActive());
     
-    // This will actually return true in the tests, since the NN will not in
-    // fact call Runtime.exit();
-    doAnEdit();
+    try {
+      doAnEdit();
+      fail("A single failure of a required journal should have halted the NN");
+    } catch (RemoteException re) {
+      assertTrue(re.getClassName().contains("ExitException"));
+      GenericTestUtils.assertExceptionContains(
+          "setReadyToFlush failed for required journal", re);
+    }
     
     // Since the required directory failed setReadyToFlush, and that
     // directory was listed prior to the non-required directory,
@@ -173,10 +181,6 @@ public class TestEditLogJournalFailures {
     // directory. Regression test for HDFS-2874.
     Mockito.verify(nonRequiredSpy, Mockito.never()).setReadyToFlush();
     assertFalse(nonRequiredJas.isActive());
-    
-    // A single failure of a required journal should result in a call to
-    // runtime.exit(...).
-    assertExitInvocations(atLeast(1));
   }
   
   @Test
@@ -201,28 +205,31 @@ public class TestEditLogJournalFailures {
     
     // All journals active.
     assertTrue(doAnEdit());
-    assertExitInvocations(0);
+    // The NN has not terminated (no ExitException thrown)
     
     // Invalidate 1/4 of the redundant journals.
     invalidateEditsDirAtIndex(0, false, false);
     assertTrue(doAnEdit());
-    assertExitInvocations(0);
+    // The NN has not terminated (no ExitException thrown)
 
     // Invalidate 2/4 of the redundant journals.
     invalidateEditsDirAtIndex(1, false, false);
     assertTrue(doAnEdit());
-    assertExitInvocations(0);
+    // The NN has not terminated (no ExitException thrown)
     
     // Invalidate 3/4 of the redundant journals.
     invalidateEditsDirAtIndex(2, false, false);
-    
-    // This will actually return true in the tests, since the NN will not in
-    // fact call Runtime.exit();
-    doAnEdit();
-    
-    // A failure of more than the minimum number of redundant journals should
-    // result in a call to runtime.exit(...).
-    assertExitInvocations(atLeast(1));
+
+    try {
+      doAnEdit();
+      fail("A failure of more than the minimum number of redundant journals "
+          + "should have halted ");
+    } catch (RemoteException re) {
+      assertTrue(re.getClassName().contains("ExitException"));
+      GenericTestUtils.assertExceptionContains(
+          "Could not sync enough journals to persistent storage. " +
+          "Unsynced transactions: 1", re);
+    }
   }
 
   /**
@@ -275,25 +282,4 @@ public class TestEditLogJournalFailures {
   private boolean doAnEdit() throws IOException {
     return fs.mkdirs(new Path("/tmp", Integer.toString(editsPerformed++)));
   }
-  
-  /**
-   * Make sure that Runtime.exit(...) has been called exactly
-   * <code>expectedExits<code> number of times.
-   * 
-   * @param expectedExits the exact number of times Runtime.exit(...) should
-   *                      have been called.
-   */
-  private void assertExitInvocations(int expectedExits) {
-    assertExitInvocations(times(expectedExits));
-  }
-
-  /**
-   * Make sure that Runtime.exit(...) has been called
-   * <code>expectedExits<code> number of times.
-   * 
-   * @param expectedExits the number of times Runtime.exit(...) should have been called.
-   */
-  private void assertExitInvocations(VerificationMode expectedExits) {
-    verify(runtime, expectedExits).exit(anyInt());
-  }
 }

+ 4 - 18
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureOfSharedDir.java

@@ -28,8 +28,6 @@ import java.net.URISyntaxException;
 import java.util.Collection;
 import java.util.List;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
@@ -41,14 +39,12 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.ExitUtil.ExitException;
 import org.junit.Test;
-import org.mockito.Mockito;
 
 import com.google.common.base.Joiner;
 
 public class TestFailureOfSharedDir {
-  
-  private static final Log LOG = LogFactory.getLog(TestFailureOfSharedDir.class);
 
   /**
    * Test that the shared edits dir is automatically added to the list of edits
@@ -138,6 +134,7 @@ public class TestFailureOfSharedDir {
       cluster = new MiniDFSCluster.Builder(conf)
         .nnTopology(MiniDFSNNTopology.simpleHATopology())
         .numDataNodes(0)
+        .checkExitOnShutdown(false)
         .build();
       
       cluster.waitActive();
@@ -148,7 +145,6 @@ public class TestFailureOfSharedDir {
       assertTrue(fs.mkdirs(new Path("/test1")));
       
       // Blow away the shared edits dir.
-      Runtime mockRuntime = Mockito.mock(Runtime.class);
       URI sharedEditsUri = cluster.getSharedEditsDir(0, 1);
       sharedEditsDir = new File(sharedEditsUri);
       assertEquals(0, FileUtil.chmod(sharedEditsDir.getAbsolutePath(), "-w",
@@ -164,23 +160,13 @@ public class TestFailureOfSharedDir {
           nn1.isInSafeMode());
 
       NameNode nn0 = cluster.getNameNode(0);
-      nn0.getNamesystem().getFSImage().getEditLog().getJournalSet()
-          .setRuntimeForTesting(mockRuntime);
       try {
         // Make sure that subsequent operations on the NN fail.
         nn0.getRpcServer().rollEditLog();
         fail("Succeeded in rolling edit log despite shared dir being deleted");
-      } catch (IOException ioe) {
+      } catch (ExitException ee) {
         GenericTestUtils.assertExceptionContains(
-            "Unable to start log segment 4: too few journals successfully started",
-            ioe);
-        // By current policy the NN should exit upon this error.
-        // exit() should be called once, but since it is mocked, exit gets
-        // called once during FSEditsLog.endCurrentLogSegment() and then after
-        // that during FSEditsLog.startLogSegment(). So the check is atLeast(1)
-        Mockito.verify(mockRuntime, Mockito.atLeastOnce()).exit(
-            Mockito.anyInt());
-        LOG.info("Got expected exception", ioe);
+            "finalize log segment 1, 3 failed for required journal", ee);
       }
       
       // Check that none of the edits dirs rolled, since the shared edits

+ 9 - 26
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java

@@ -21,36 +21,30 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Matchers.anyInt;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.anyObject;
 import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
 
 import java.io.IOException;
 import java.util.Collection;
 import java.util.LinkedList;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.ha.ServiceFailedException;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.apache.hadoop.hdfs.server.namenode.EditLogInputException;
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
 import org.apache.hadoop.hdfs.server.namenode.MetaRecoveryContext;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.ExitUtil.ExitException;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -60,15 +54,12 @@ import org.mockito.stubbing.Answer;
 import com.google.common.collect.ImmutableList;
 
 public class TestFailureToReadEdits {
-  
-  private static final Log LOG = LogFactory.getLog(TestFailureToReadEdits.class);
-  
+
   private static final String TEST_DIR1 = "/test1";
   private static final String TEST_DIR2 = "/test2";
   private static final String TEST_DIR3 = "/test3";
   
   private Configuration conf;
-  private Runtime mockRuntime = mock(Runtime.class);
   private MiniDFSCluster cluster;
   private NameNode nn0;
   private NameNode nn1;
@@ -90,13 +81,13 @@ public class TestFailureToReadEdits {
     cluster = new MiniDFSCluster.Builder(conf)
       .nnTopology(topology)
       .numDataNodes(0)
+      .checkExitOnShutdown(false)
       .build();
     
     cluster.waitActive();
     
     nn0 = cluster.getNameNode(0);
     nn1 = cluster.getNameNode(1);
-    nn1.getNamesystem().getEditLogTailer().setRuntime(mockRuntime);
     
     cluster.transitionToActive(0);
     fs = HATestUtil.configureFailoverFs(cluster, conf);
@@ -139,7 +130,7 @@ public class TestFailureToReadEdits {
       HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
       fail("Standby fully caught up, but should not have been able to");
     } catch (HATestUtil.CouldNotCatchUpException e) {
-      verify(mockRuntime, times(0)).exit(anyInt());
+      // Expected. The NN did not exit.
     }
     
     // Null because it was deleted.
@@ -200,7 +191,7 @@ public class TestFailureToReadEdits {
       HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
       fail("Standby fully caught up, but should not have been able to");
     } catch (HATestUtil.CouldNotCatchUpException e) {
-      verify(mockRuntime, times(0)).exit(anyInt());
+      // Expected. The NN did not exit.
     }
     
     // 5 because we should get OP_START_LOG_SEGMENT and one successful OP_MKDIR
@@ -252,27 +243,19 @@ public class TestFailureToReadEdits {
       HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
       fail("Standby fully caught up, but should not have been able to");
     } catch (HATestUtil.CouldNotCatchUpException e) {
-      verify(mockRuntime, times(0)).exit(anyInt());
+      // Expected. The NN did not exit.
     }
     
     // Shutdown the active NN.
     cluster.shutdownNameNode(0);
     
-    Runtime mockRuntime = mock(Runtime.class);
-    cluster.getNameNode(1).setRuntimeForTesting(mockRuntime);
-    verify(mockRuntime, times(0)).exit(anyInt());
     try {
       // Transition the standby to active.
       cluster.transitionToActive(1);
       fail("Standby transitioned to active, but should not have been able to");
-    } catch (ServiceFailedException sfe) {
-      Throwable sfeCause = sfe.getCause();
-      LOG.info("got expected exception: " + sfeCause.toString(), sfeCause);
-      assertTrue("Standby failed to catch up for some reason other than "
-          + "failure to read logs", sfeCause.getCause().toString().contains(
-              EditLogInputException.class.getName()));
+    } catch (ExitException ee) {
+      GenericTestUtils.assertExceptionContains("Error replaying edit log", ee);
     }
-    verify(mockRuntime, times(1)).exit(anyInt());
   }
   
   private LimitedEditLogAnswer causeFailureOnEditLogRead() throws IOException {

+ 0 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java

@@ -18,11 +18,6 @@
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
 import static org.junit.Assert.*;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
 
 import java.io.IOException;
 import java.util.List;
@@ -38,7 +33,6 @@ import org.apache.hadoop.ha.HAServiceProtocol.RequestSource;
 import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
@@ -67,7 +61,6 @@ public class TestHASafeMode {
   private NameNode nn1;
   private FileSystem fs;
   private MiniDFSCluster cluster;
-  private Runtime mockRuntime = mock(Runtime.class);
   
   static {
     ((Log4JLogger)LogFactory.getLog(FSImage.class)).getLogger().setLevel(Level.ALL);
@@ -92,8 +85,6 @@ public class TestHASafeMode {
     nn0 = cluster.getNameNode(0);
     nn1 = cluster.getNameNode(1);
     fs = HATestUtil.configureFailoverFs(cluster, conf);
-    
-    nn0.getNamesystem().getEditLogTailer().setRuntime(mockRuntime);
 
     cluster.transitionToActive(0);
   }
@@ -101,7 +92,6 @@ public class TestHASafeMode {
   @After
   public void shutdownCluster() throws IOException {
     if (cluster != null) {
-      verify(mockRuntime, times(0)).exit(anyInt());
       cluster.shutdown();
     }
   }

+ 0 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyIsHot.java

@@ -18,10 +18,6 @@
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
 import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
 
 import java.io.IOException;
 
@@ -79,7 +75,6 @@ public class TestStandbyIsHot {
       .nnTopology(MiniDFSNNTopology.simpleHATopology())
       .numDataNodes(3)
       .build();
-    Runtime mockRuntime = mock(Runtime.class);
     try {
       cluster.waitActive();
       cluster.transitionToActive(0);
@@ -87,8 +82,6 @@ public class TestStandbyIsHot {
       NameNode nn1 = cluster.getNameNode(0);
       NameNode nn2 = cluster.getNameNode(1);
       
-      nn2.getNamesystem().getEditLogTailer().setRuntime(mockRuntime);
-      
       FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
       
       Thread.sleep(1000);
@@ -130,7 +123,6 @@ public class TestStandbyIsHot {
       waitForBlockLocations(cluster, nn2, TEST_FILE, 3);
       
     } finally {
-      verify(mockRuntime, times(0)).exit(anyInt());
       cluster.shutdown();
     }
   }

+ 5 - 17
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStateTransitionFailure.java

@@ -19,28 +19,20 @@ package org.apache.hadoop.hdfs.server.namenode.ha;
 
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.fail;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.ha.ServiceFailedException;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.util.ExitUtil.ExitException;
 import org.junit.Test;
 
 /**
  * Tests to verify the behavior of failing to fully start transition HA states.
  */
 public class TestStateTransitionFailure {
-  
-  public static final Log LOG = LogFactory.getLog(TestStateTransitionFailure.class);
 
   /**
    * Ensure that a failure to fully transition to the active state causes a
@@ -57,20 +49,16 @@ public class TestStateTransitionFailure {
       cluster = new MiniDFSCluster.Builder(conf)
           .nnTopology(MiniDFSNNTopology.simpleHATopology())
           .numDataNodes(0)
+          .checkExitOnShutdown(false)
           .build();
       cluster.waitActive();
-      Runtime mockRuntime = mock(Runtime.class);
-      cluster.getNameNode(0).setRuntimeForTesting(mockRuntime);
-      verify(mockRuntime, times(0)).exit(anyInt());
       try {
         cluster.transitionToActive(0);
         fail("Transitioned to active but should not have been able to.");
-      } catch (ServiceFailedException sfe) {
-        assertExceptionContains("Error encountered requiring NN shutdown. " +
-            "Shutting down immediately.", sfe.getCause());
-        LOG.info("got expected exception", sfe.getCause());
+      } catch (ExitException ee) {
+        assertExceptionContains(
+            "Cannot start tresh emptier with negative interval", ee);
       }
-      verify(mockRuntime, times(1)).exit(anyInt());
     } finally {
       if (cluster != null) {
         cluster.shutdown();