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