|
@@ -42,7 +42,6 @@ import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
-import org.apache.hadoop.hdfs.DFSUtil.ErrorSimulator;
|
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
@@ -63,6 +62,7 @@ import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.log4j.Level;
|
|
|
+import org.mockito.ArgumentMatcher;
|
|
|
import org.mockito.Mockito;
|
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
|
import org.mockito.stubbing.Answer;
|
|
@@ -93,11 +93,15 @@ public class TestCheckpoint extends TestCase {
|
|
|
static final int fileSize = 8192;
|
|
|
static final int numDatanodes = 3;
|
|
|
short replication = 3;
|
|
|
+
|
|
|
+ private CheckpointFaultInjector faultInjector;
|
|
|
|
|
|
@Override
|
|
|
public void setUp() throws IOException {
|
|
|
FileUtil.fullyDeleteContents(new File(MiniDFSCluster.getBaseDirectory()));
|
|
|
- ErrorSimulator.initializeErrorSimulationEvent(5);
|
|
|
+
|
|
|
+ faultInjector = Mockito.mock(CheckpointFaultInjector.class);
|
|
|
+ CheckpointFaultInjector.instance = faultInjector;
|
|
|
}
|
|
|
|
|
|
static void writeFile(FileSystem fileSys, Path name, int repl)
|
|
@@ -222,14 +226,18 @@ public class TestCheckpoint extends TestCase {
|
|
|
// Make the checkpoint fail after rolling the edits log.
|
|
|
//
|
|
|
SecondaryNameNode secondary = startSecondaryNameNode(conf);
|
|
|
- ErrorSimulator.setErrorSimulation(0);
|
|
|
+
|
|
|
+ Mockito.doThrow(new IOException(
|
|
|
+ "Injecting failure after rolling edit logs"))
|
|
|
+ .when(faultInjector).afterSecondaryCallsRollEditLog();
|
|
|
|
|
|
try {
|
|
|
secondary.doCheckpoint(); // this should fail
|
|
|
assertTrue(false);
|
|
|
} catch (IOException e) {
|
|
|
}
|
|
|
- ErrorSimulator.clearErrorSimulation(0);
|
|
|
+
|
|
|
+ Mockito.reset(faultInjector);
|
|
|
secondary.shutdown();
|
|
|
|
|
|
//
|
|
@@ -282,14 +290,17 @@ public class TestCheckpoint extends TestCase {
|
|
|
// Make the checkpoint fail after uploading the new fsimage.
|
|
|
//
|
|
|
SecondaryNameNode secondary = startSecondaryNameNode(conf);
|
|
|
- ErrorSimulator.setErrorSimulation(1);
|
|
|
+
|
|
|
+ Mockito.doThrow(new IOException(
|
|
|
+ "Injecting failure after uploading new image"))
|
|
|
+ .when(faultInjector).afterSecondaryUploadsNewImage();
|
|
|
|
|
|
try {
|
|
|
secondary.doCheckpoint(); // this should fail
|
|
|
assertTrue(false);
|
|
|
} catch (IOException e) {
|
|
|
}
|
|
|
- ErrorSimulator.clearErrorSimulation(1);
|
|
|
+ Mockito.reset(faultInjector);
|
|
|
secondary.shutdown();
|
|
|
|
|
|
//
|
|
@@ -341,14 +352,17 @@ public class TestCheckpoint extends TestCase {
|
|
|
// Make the checkpoint fail after rolling the edit log.
|
|
|
//
|
|
|
SecondaryNameNode secondary = startSecondaryNameNode(conf);
|
|
|
- ErrorSimulator.setErrorSimulation(0);
|
|
|
+
|
|
|
+ Mockito.doThrow(new IOException(
|
|
|
+ "Injecting failure after rolling edit logs"))
|
|
|
+ .when(faultInjector).afterSecondaryCallsRollEditLog();
|
|
|
|
|
|
try {
|
|
|
secondary.doCheckpoint(); // this should fail
|
|
|
assertTrue(false);
|
|
|
} catch (IOException e) {
|
|
|
}
|
|
|
- ErrorSimulator.clearErrorSimulation(0);
|
|
|
+ Mockito.reset(faultInjector);
|
|
|
secondary.shutdown(); // secondary namenode crash!
|
|
|
|
|
|
// start new instance of secondary and verify that
|
|
@@ -395,6 +409,28 @@ public class TestCheckpoint extends TestCase {
|
|
|
* Used to truncate primary fsimage file.
|
|
|
*/
|
|
|
public void testSecondaryFailsToReturnImage() throws IOException {
|
|
|
+ Mockito.doThrow(new IOException("If this exception is not caught by the " +
|
|
|
+ "name-node, fs image will be truncated."))
|
|
|
+ .when(faultInjector).aboutToSendFile(filePathContaining("secondary"));
|
|
|
+
|
|
|
+ doSecondaryFailsToReturnImage();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Similar to above test, but uses an unchecked Error, and causes it
|
|
|
+ * before even setting the length header. This used to cause image
|
|
|
+ * truncation. Regression test for HDFS-3330.
|
|
|
+ */
|
|
|
+ public void testSecondaryFailsWithErrorBeforeSettingHeaders()
|
|
|
+ throws IOException {
|
|
|
+ Mockito.doThrow(new Error("If this exception is not caught by the " +
|
|
|
+ "name-node, fs image will be truncated."))
|
|
|
+ .when(faultInjector).beforeGetImageSetsHeaders();
|
|
|
+
|
|
|
+ doSecondaryFailsToReturnImage();
|
|
|
+ }
|
|
|
+
|
|
|
+ private void doSecondaryFailsToReturnImage() throws IOException {
|
|
|
LOG.info("Starting testSecondaryFailsToReturnImage");
|
|
|
Configuration conf = new HdfsConfiguration();
|
|
|
Path file1 = new Path("checkpointRI.dat");
|
|
@@ -414,7 +450,6 @@ public class TestCheckpoint extends TestCase {
|
|
|
// Make the checkpoint
|
|
|
//
|
|
|
SecondaryNameNode secondary = startSecondaryNameNode(conf);
|
|
|
- ErrorSimulator.setErrorSimulation(2);
|
|
|
|
|
|
try {
|
|
|
secondary.doCheckpoint(); // this should fail
|
|
@@ -424,7 +459,7 @@ public class TestCheckpoint extends TestCase {
|
|
|
GenericTestUtils.assertExceptionContains(
|
|
|
"If this exception is not caught", e);
|
|
|
}
|
|
|
- ErrorSimulator.clearErrorSimulation(2);
|
|
|
+ Mockito.reset(faultInjector);
|
|
|
|
|
|
// Verify that image file sizes did not change.
|
|
|
for (StorageDirectory sd2 :
|
|
@@ -442,6 +477,17 @@ public class TestCheckpoint extends TestCase {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private File filePathContaining(final String substring) {
|
|
|
+ return Mockito.<File>argThat(
|
|
|
+ new ArgumentMatcher<File>() {
|
|
|
+ @Override
|
|
|
+ public boolean matches(Object argument) {
|
|
|
+ String path = ((File)argument).getAbsolutePath();
|
|
|
+ return path.contains(substring);
|
|
|
+ }
|
|
|
+ });
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Simulate 2NN failing to send the whole file (error type 3)
|
|
|
* The length header in the HTTP transfer should prevent
|
|
@@ -450,7 +496,10 @@ public class TestCheckpoint extends TestCase {
|
|
|
public void testNameNodeImageSendFailWrongSize()
|
|
|
throws IOException {
|
|
|
LOG.info("Starting testNameNodeImageSendFailWrongSize");
|
|
|
- doSendFailTest(3, "is not of the advertised size");
|
|
|
+
|
|
|
+ Mockito.doReturn(true).when(faultInjector)
|
|
|
+ .shouldSendShortFile(filePathContaining("fsimage"));
|
|
|
+ doSendFailTest("is not of the advertised size");
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -461,19 +510,21 @@ public class TestCheckpoint extends TestCase {
|
|
|
public void testNameNodeImageSendFailWrongDigest()
|
|
|
throws IOException {
|
|
|
LOG.info("Starting testNameNodeImageSendFailWrongDigest");
|
|
|
- doSendFailTest(4, "does not match advertised digest");
|
|
|
+
|
|
|
+ Mockito.doReturn(true).when(faultInjector)
|
|
|
+ .shouldCorruptAByte(Mockito.any(File.class));
|
|
|
+ doSendFailTest("does not match advertised digest");
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Run a test where the 2NN runs into some kind of error when
|
|
|
* sending the checkpoint back to the NN.
|
|
|
- * @param errorType the ErrorSimulator type to trigger
|
|
|
* @param exceptionSubstring an expected substring of the triggered exception
|
|
|
*/
|
|
|
- private void doSendFailTest(int errorType, String exceptionSubstring)
|
|
|
+ private void doSendFailTest(String exceptionSubstring)
|
|
|
throws IOException {
|
|
|
Configuration conf = new HdfsConfiguration();
|
|
|
- Path file1 = new Path("checkpoint-doSendFailTest-" + errorType + ".dat");
|
|
|
+ Path file1 = new Path("checkpoint-doSendFailTest-" + getName() + ".dat");
|
|
|
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
|
|
.numDataNodes(numDatanodes)
|
|
|
.build();
|
|
@@ -485,7 +536,6 @@ public class TestCheckpoint extends TestCase {
|
|
|
// Make the checkpoint fail after rolling the edit log.
|
|
|
//
|
|
|
SecondaryNameNode secondary = startSecondaryNameNode(conf);
|
|
|
- ErrorSimulator.setErrorSimulation(errorType);
|
|
|
|
|
|
try {
|
|
|
secondary.doCheckpoint(); // this should fail
|
|
@@ -494,7 +544,7 @@ public class TestCheckpoint extends TestCase {
|
|
|
// We only sent part of the image. Have to trigger this exception
|
|
|
GenericTestUtils.assertExceptionContains(exceptionSubstring, e);
|
|
|
}
|
|
|
- ErrorSimulator.clearErrorSimulation(errorType);
|
|
|
+ Mockito.reset(faultInjector);
|
|
|
secondary.shutdown(); // secondary namenode crash!
|
|
|
|
|
|
// start new instance of secondary and verify that
|
|
@@ -1017,7 +1067,9 @@ public class TestCheckpoint extends TestCase {
|
|
|
|
|
|
secondary = startSecondaryNameNode(conf);
|
|
|
|
|
|
- ErrorSimulator.setErrorSimulation(1);
|
|
|
+ Mockito.doThrow(new IOException(
|
|
|
+ "Injecting failure after rolling edit logs"))
|
|
|
+ .when(faultInjector).afterSecondaryCallsRollEditLog();
|
|
|
|
|
|
// Fail to checkpoint once
|
|
|
try {
|
|
@@ -1025,7 +1077,7 @@ public class TestCheckpoint extends TestCase {
|
|
|
fail("Should have failed upload");
|
|
|
} catch (IOException ioe) {
|
|
|
LOG.info("Got expected failure", ioe);
|
|
|
- assertTrue(ioe.toString().contains("Simulating error1"));
|
|
|
+ assertTrue(ioe.toString().contains("Injecting failure"));
|
|
|
}
|
|
|
|
|
|
// Fail to checkpoint again
|
|
@@ -1034,9 +1086,9 @@ public class TestCheckpoint extends TestCase {
|
|
|
fail("Should have failed upload");
|
|
|
} catch (IOException ioe) {
|
|
|
LOG.info("Got expected failure", ioe);
|
|
|
- assertTrue(ioe.toString().contains("Simulating error1"));
|
|
|
+ assertTrue(ioe.toString().contains("Injecting failure"));
|
|
|
} finally {
|
|
|
- ErrorSimulator.clearErrorSimulation(1);
|
|
|
+ Mockito.reset(faultInjector);
|
|
|
}
|
|
|
|
|
|
// Now with the cleared error simulation, it should succeed
|