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
This commit is contained in:
parent
786aaca2ab
commit
ef21cbf70a
|
@ -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");
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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.
|
||||
*/
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
Loading…
Reference in New Issue