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

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1360329 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Eli Collins 2012-07-11 17:58:02 +00:00
parent 664ca6b75e
commit cdae6953e8
19 changed files with 276 additions and 310 deletions

View File

@ -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");
}
}

View File

@ -300,6 +300,8 @@ Branch-2 ( Unreleased changes )
HDFS-3611. NameNode prints unnecessary WARNs about edit log normally skipping a few bytes. (Colin Patrick McCabe via harsh) 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 OPTIMIZATIONS
HDFS-2982. Startup performance suffers when there are many edit log HDFS-2982. Startup performance suffers when there are many edit log

View File

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

View File

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

View File

@ -53,6 +53,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.BlockTokenSecretManager.AccessMode;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys; import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo; 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.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.Util; import org.apache.hadoop.hdfs.server.common.Util;
@ -2962,8 +2965,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
LOG.warn("ReplicationMonitor thread received InterruptedException.", ie); LOG.warn("ReplicationMonitor thread received InterruptedException.", ie);
break; break;
} catch (Throwable t) { } catch (Throwable t) {
LOG.warn("ReplicationMonitor thread received Runtime exception. ", t); LOG.fatal("ReplicationMonitor thread received Runtime exception. ", t);
Runtime.getRuntime().exit(-1); terminate(1);
} }
} }
} }

View File

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

View File

@ -37,6 +37,9 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; 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.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp;
@ -136,10 +139,6 @@ public class FSEditLog {
// is an automatic sync scheduled? // is an automatic sync scheduled?
private volatile boolean isAutoSyncScheduled = false; 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. // these are statistics counters.
private long numTransactions; // number of transactions private long numTransactions; // number of transactions
private long numTransactionsBatchedInSync; private long numTransactionsBatchedInSync;
@ -232,9 +231,6 @@ public class FSEditLog {
DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_MINIMUM_DEFAULT); DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_MINIMUM_DEFAULT);
journalSet = new JournalSet(minimumRedundantJournals); 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) { for (URI u : dirs) {
boolean required = FSNamesystem.getRequiredNamespaceEditsDirs(conf) boolean required = FSNamesystem.getRequiredNamespaceEditsDirs(conf)
@ -547,10 +543,11 @@ public class FSEditLog {
} }
editLogStream.setReadyToFlush(); editLogStream.setReadyToFlush();
} catch (IOException e) { } catch (IOException e) {
LOG.fatal("Could not sync enough journals to persistent storage. " final String msg =
+ "Unsynced transactions: " + (txid - synctxid), "Could not sync enough journals to persistent storage. "
new Exception()); + "Unsynced transactions: " + (txid - synctxid);
runtime.exit(1); LOG.fatal(msg, new Exception());
terminate(1, msg);
} }
} finally { } finally {
// Prevent RuntimeException from blocking other log edit write // Prevent RuntimeException from blocking other log edit write
@ -569,9 +566,11 @@ public class FSEditLog {
} }
} catch (IOException ex) { } catch (IOException ex) {
synchronized (this) { synchronized (this) {
LOG.fatal("Could not sync enough journals to persistent storage. " final String msg =
+ "Unsynced transactions: " + (txid - synctxid), new Exception()); "Could not sync enough journals to persistent storage. "
runtime.exit(1); + "Unsynced transactions: " + (txid - synctxid);
LOG.fatal(msg, new Exception());
terminate(1, msg);
} }
} }
long elapsed = now() - start; long elapsed = now() - start;
@ -843,15 +842,6 @@ public class FSEditLog {
return journalSet; 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. * Used only by tests.
*/ */

View File

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

View File

@ -81,6 +81,9 @@ import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
import org.apache.hadoop.tools.GetUserMappingsProtocol; import org.apache.hadoop.tools.GetUserMappingsProtocol;
import org.apache.hadoop.util.ServicePlugin; import org.apache.hadoop.util.ServicePlugin;
import org.apache.hadoop.util.StringUtils; 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 static org.apache.hadoop.util.ToolRunner.confirmPrompt;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
@ -226,7 +229,6 @@ public class NameNode {
private final boolean haEnabled; private final boolean haEnabled;
private final HAContext haContext; private final HAContext haContext;
protected boolean allowStaleStandbyReads; protected boolean allowStaleStandbyReads;
private Runtime runtime = Runtime.getRuntime();
/** httpServer */ /** httpServer */
@ -1089,29 +1091,29 @@ public class NameNode {
case FORMAT: { case FORMAT: {
boolean aborted = format(conf, startOpt.getForceFormat(), boolean aborted = format(conf, startOpt.getForceFormat(),
startOpt.getInteractiveFormat()); startOpt.getInteractiveFormat());
System.exit(aborted ? 1 : 0); terminate(aborted ? 1 : 0);
return null; // avoid javac warning return null; // avoid javac warning
} }
case GENCLUSTERID: { case GENCLUSTERID: {
System.err.println("Generating new cluster id:"); System.err.println("Generating new cluster id:");
System.out.println(NNStorage.newClusterID()); System.out.println(NNStorage.newClusterID());
System.exit(0); terminate(0);
return null; return null;
} }
case FINALIZE: { case FINALIZE: {
boolean aborted = finalize(conf, true); boolean aborted = finalize(conf, true);
System.exit(aborted ? 1 : 0); terminate(aborted ? 1 : 0);
return null; // avoid javac warning return null; // avoid javac warning
} }
case BOOTSTRAPSTANDBY: { case BOOTSTRAPSTANDBY: {
String toolArgs[] = Arrays.copyOfRange(argv, 1, argv.length); String toolArgs[] = Arrays.copyOfRange(argv, 1, argv.length);
int rc = BootstrapStandby.run(toolArgs, conf); int rc = BootstrapStandby.run(toolArgs, conf);
System.exit(rc); terminate(rc);
return null; // avoid warning return null; // avoid warning
} }
case INITIALIZESHAREDEDITS: { case INITIALIZESHAREDEDITS: {
boolean aborted = initializeSharedEdits(conf, false, true); boolean aborted = initializeSharedEdits(conf, false, true);
System.exit(aborted ? 1 : 0); terminate(aborted ? 1 : 0);
return null; // avoid warning return null; // avoid warning
} }
case BACKUP: case BACKUP:
@ -1124,9 +1126,10 @@ public class NameNode {
NameNode.doRecovery(startOpt, conf); NameNode.doRecovery(startOpt, conf);
return null; return null;
} }
default: default: {
DefaultMetricsSystem.initialize("NameNode"); DefaultMetricsSystem.initialize("NameNode");
return new NameNode(conf); return new NameNode(conf);
}
} }
} }
@ -1189,8 +1192,8 @@ public class NameNode {
if (namenode != null) if (namenode != null)
namenode.join(); namenode.join();
} catch (Throwable e) { } catch (Throwable e) {
LOG.error("Exception in namenode join", e); LOG.fatal("Exception in namenode join", e);
System.exit(-1); terminate(1);
} }
} }
@ -1260,11 +1263,6 @@ public class NameNode {
return state.getServiceState(); 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 * Shutdown the NN immediately in an ungraceful way. Used when it would be
* unsafe for the NN to continue operating, e.g. during a failed HA state * unsafe for the NN to continue operating, e.g. during a failed HA state
@ -1272,10 +1270,10 @@ public class NameNode {
* *
* @param t exception which warrants the shutdown. Printed to the NN log * @param t exception which warrants the shutdown. Printed to the NN log
* before exit. * before exit.
* @throws ServiceFailedException thrown only for testing. * @throws ExitException thrown only for testing.
*/ */
private synchronized void doImmediateShutdown(Throwable t) private synchronized void doImmediateShutdown(Throwable t)
throws ServiceFailedException { throws ExitException {
String message = "Error encountered requiring NN shutdown. " + String message = "Error encountered requiring NN shutdown. " +
"Shutting down immediately."; "Shutting down immediately.";
try { try {
@ -1283,9 +1281,7 @@ public class NameNode {
} catch (Throwable ignored) { } catch (Throwable ignored) {
// This is unlikely to happen, but there's nothing we can do if it does. // This is unlikely to happen, but there's nothing we can do if it does.
} }
runtime.exit(1); terminate(1, t.getMessage());
// This code is only reached during testing, when runtime is stubbed out.
throw new ServiceFailedException(message, t);
} }
/** /**

View File

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

View File

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

View File

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

View File

@ -30,7 +30,6 @@ import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.PrintStream; import java.io.PrintStream;
import java.net.URI; import java.net.URI;
import java.security.Permission;
import java.util.Collection; import java.util.Collection;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; 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.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; 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.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -69,7 +70,7 @@ public class TestClusterId {
@Before @Before
public void setUp() throws IOException { public void setUp() throws IOException {
System.setSecurityManager(new NoExitSecurityManager()); ExitUtil.disableSystemExit();
String baseDir = System.getProperty("test.build.data", "build/test/data"); String baseDir = System.getProperty("test.build.data", "build/test/data");
@ -90,8 +91,6 @@ public class TestClusterId {
@After @After
public void tearDown() throws IOException { public void tearDown() throws IOException {
System.setSecurityManager(null);
if (hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir)) { if (hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir)) {
throw new IOException("Could not tearDown test directory '" + hdfsDir throw new IOException("Could not tearDown test directory '" + hdfsDir
+ "'"); + "'");
@ -446,32 +445,4 @@ public class TestClusterId {
File version = new File(hdfsDir, "current/VERSION"); File version = new File(hdfsDir, "current/VERSION");
assertFalse("Check version should not exist", version.exists()); 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);
}
}
} }

View File

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

View File

@ -28,8 +28,6 @@ import java.net.URISyntaxException;
import java.util.Collection; import java.util.Collection;
import java.util.List; 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.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
@ -41,15 +39,13 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NNStorage; import org.apache.hadoop.hdfs.server.namenode.NNStorage;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.ExitUtil.ExitException;
import org.junit.Test; import org.junit.Test;
import org.mockito.Mockito;
import com.google.common.base.Joiner; import com.google.common.base.Joiner;
public class TestFailureOfSharedDir { 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 * Test that the shared edits dir is automatically added to the list of edits
* dirs that are marked required. * dirs that are marked required.
@ -138,6 +134,7 @@ public class TestFailureOfSharedDir {
cluster = new MiniDFSCluster.Builder(conf) cluster = new MiniDFSCluster.Builder(conf)
.nnTopology(MiniDFSNNTopology.simpleHATopology()) .nnTopology(MiniDFSNNTopology.simpleHATopology())
.numDataNodes(0) .numDataNodes(0)
.checkExitOnShutdown(false)
.build(); .build();
cluster.waitActive(); cluster.waitActive();
@ -148,7 +145,6 @@ public class TestFailureOfSharedDir {
assertTrue(fs.mkdirs(new Path("/test1"))); assertTrue(fs.mkdirs(new Path("/test1")));
// Blow away the shared edits dir. // Blow away the shared edits dir.
Runtime mockRuntime = Mockito.mock(Runtime.class);
URI sharedEditsUri = cluster.getSharedEditsDir(0, 1); URI sharedEditsUri = cluster.getSharedEditsDir(0, 1);
sharedEditsDir = new File(sharedEditsUri); sharedEditsDir = new File(sharedEditsUri);
assertEquals(0, FileUtil.chmod(sharedEditsDir.getAbsolutePath(), "-w", assertEquals(0, FileUtil.chmod(sharedEditsDir.getAbsolutePath(), "-w",
@ -164,23 +160,13 @@ public class TestFailureOfSharedDir {
nn1.isInSafeMode()); nn1.isInSafeMode());
NameNode nn0 = cluster.getNameNode(0); NameNode nn0 = cluster.getNameNode(0);
nn0.getNamesystem().getFSImage().getEditLog().getJournalSet()
.setRuntimeForTesting(mockRuntime);
try { try {
// Make sure that subsequent operations on the NN fail. // Make sure that subsequent operations on the NN fail.
nn0.getRpcServer().rollEditLog(); nn0.getRpcServer().rollEditLog();
fail("Succeeded in rolling edit log despite shared dir being deleted"); fail("Succeeded in rolling edit log despite shared dir being deleted");
} catch (IOException ioe) { } catch (ExitException ee) {
GenericTestUtils.assertExceptionContains( GenericTestUtils.assertExceptionContains(
"Unable to start log segment 4: too few journals successfully started", "finalize log segment 1, 3 failed for required journal", ee);
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);
} }
// Check that none of the edits dirs rolled, since the shared edits // Check that none of the edits dirs rolled, since the shared edits

View File

@ -21,36 +21,30 @@ import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import static org.mockito.Matchers.anyBoolean; import static org.mockito.Matchers.anyBoolean;
import static org.mockito.Matchers.anyInt;
import static org.mockito.Matchers.anyLong; import static org.mockito.Matchers.anyLong;
import static org.mockito.Matchers.anyObject; import static org.mockito.Matchers.anyObject;
import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy; import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import java.io.IOException; import java.io.IOException;
import java.util.Collection; import java.util.Collection;
import java.util.LinkedList; 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.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.ha.ServiceFailedException;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HAUtil; import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology; 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.EditLogInputStream;
import org.apache.hadoop.hdfs.server.namenode.FSEditLog; import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
import org.apache.hadoop.hdfs.server.namenode.MetaRecoveryContext; import org.apache.hadoop.hdfs.server.namenode.MetaRecoveryContext;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; 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.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -61,14 +55,11 @@ import com.google.common.collect.ImmutableList;
public class TestFailureToReadEdits { public class TestFailureToReadEdits {
private static final Log LOG = LogFactory.getLog(TestFailureToReadEdits.class);
private static final String TEST_DIR1 = "/test1"; private static final String TEST_DIR1 = "/test1";
private static final String TEST_DIR2 = "/test2"; private static final String TEST_DIR2 = "/test2";
private static final String TEST_DIR3 = "/test3"; private static final String TEST_DIR3 = "/test3";
private Configuration conf; private Configuration conf;
private Runtime mockRuntime = mock(Runtime.class);
private MiniDFSCluster cluster; private MiniDFSCluster cluster;
private NameNode nn0; private NameNode nn0;
private NameNode nn1; private NameNode nn1;
@ -90,13 +81,13 @@ public class TestFailureToReadEdits {
cluster = new MiniDFSCluster.Builder(conf) cluster = new MiniDFSCluster.Builder(conf)
.nnTopology(topology) .nnTopology(topology)
.numDataNodes(0) .numDataNodes(0)
.checkExitOnShutdown(false)
.build(); .build();
cluster.waitActive(); cluster.waitActive();
nn0 = cluster.getNameNode(0); nn0 = cluster.getNameNode(0);
nn1 = cluster.getNameNode(1); nn1 = cluster.getNameNode(1);
nn1.getNamesystem().getEditLogTailer().setRuntime(mockRuntime);
cluster.transitionToActive(0); cluster.transitionToActive(0);
fs = HATestUtil.configureFailoverFs(cluster, conf); fs = HATestUtil.configureFailoverFs(cluster, conf);
@ -139,7 +130,7 @@ public class TestFailureToReadEdits {
HATestUtil.waitForStandbyToCatchUp(nn0, nn1); HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
fail("Standby fully caught up, but should not have been able to"); fail("Standby fully caught up, but should not have been able to");
} catch (HATestUtil.CouldNotCatchUpException e) { } catch (HATestUtil.CouldNotCatchUpException e) {
verify(mockRuntime, times(0)).exit(anyInt()); // Expected. The NN did not exit.
} }
// Null because it was deleted. // Null because it was deleted.
@ -200,7 +191,7 @@ public class TestFailureToReadEdits {
HATestUtil.waitForStandbyToCatchUp(nn0, nn1); HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
fail("Standby fully caught up, but should not have been able to"); fail("Standby fully caught up, but should not have been able to");
} catch (HATestUtil.CouldNotCatchUpException e) { } 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 // 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); HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
fail("Standby fully caught up, but should not have been able to"); fail("Standby fully caught up, but should not have been able to");
} catch (HATestUtil.CouldNotCatchUpException e) { } catch (HATestUtil.CouldNotCatchUpException e) {
verify(mockRuntime, times(0)).exit(anyInt()); // Expected. The NN did not exit.
} }
// Shutdown the active NN. // Shutdown the active NN.
cluster.shutdownNameNode(0); cluster.shutdownNameNode(0);
Runtime mockRuntime = mock(Runtime.class);
cluster.getNameNode(1).setRuntimeForTesting(mockRuntime);
verify(mockRuntime, times(0)).exit(anyInt());
try { try {
// Transition the standby to active. // Transition the standby to active.
cluster.transitionToActive(1); cluster.transitionToActive(1);
fail("Standby transitioned to active, but should not have been able to"); fail("Standby transitioned to active, but should not have been able to");
} catch (ServiceFailedException sfe) { } catch (ExitException ee) {
Throwable sfeCause = sfe.getCause(); GenericTestUtils.assertExceptionContains("Error replaying edit log", ee);
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()));
} }
verify(mockRuntime, times(1)).exit(anyInt());
} }
private LimitedEditLogAnswer causeFailureOnEditLogRead() throws IOException { private LimitedEditLogAnswer causeFailureOnEditLogRead() throws IOException {

View File

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

View File

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

View File

@ -19,20 +19,14 @@ package org.apache.hadoop.hdfs.server.namenode.ha;
import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains; import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
import static org.junit.Assert.fail; 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 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.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.ha.ServiceFailedException;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology; import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.util.ExitUtil.ExitException;
import org.junit.Test; import org.junit.Test;
/** /**
@ -40,8 +34,6 @@ import org.junit.Test;
*/ */
public class TestStateTransitionFailure { 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 * Ensure that a failure to fully transition to the active state causes a
* shutdown of the NameNode. * shutdown of the NameNode.
@ -57,20 +49,16 @@ public class TestStateTransitionFailure {
cluster = new MiniDFSCluster.Builder(conf) cluster = new MiniDFSCluster.Builder(conf)
.nnTopology(MiniDFSNNTopology.simpleHATopology()) .nnTopology(MiniDFSNNTopology.simpleHATopology())
.numDataNodes(0) .numDataNodes(0)
.checkExitOnShutdown(false)
.build(); .build();
cluster.waitActive(); cluster.waitActive();
Runtime mockRuntime = mock(Runtime.class);
cluster.getNameNode(0).setRuntimeForTesting(mockRuntime);
verify(mockRuntime, times(0)).exit(anyInt());
try { try {
cluster.transitionToActive(0); cluster.transitionToActive(0);
fail("Transitioned to active but should not have been able to."); fail("Transitioned to active but should not have been able to.");
} catch (ServiceFailedException sfe) { } catch (ExitException ee) {
assertExceptionContains("Error encountered requiring NN shutdown. " + assertExceptionContains(
"Shutting down immediately.", sfe.getCause()); "Cannot start tresh emptier with negative interval", ee);
LOG.info("got expected exception", sfe.getCause());
} }
verify(mockRuntime, times(1)).exit(anyInt());
} finally { } finally {
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();