|
@@ -23,6 +23,7 @@ import java.util.Collection;
|
|
|
import java.util.Random;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.dfs.FSImage.NameNodeFile;
|
|
|
+import org.apache.hadoop.dfs.SecondaryNameNode.ErrorSimulator;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
@@ -132,18 +133,17 @@ public class TestCheckpoint extends TestCase {
|
|
|
try {
|
|
|
assertTrue(!fileSys.exists(file1));
|
|
|
//
|
|
|
- // Make the checkpoint fail after rolling the
|
|
|
- // edit log.
|
|
|
+ // Make the checkpoint fail after rolling the edits log.
|
|
|
//
|
|
|
SecondaryNameNode secondary = new SecondaryNameNode(conf);
|
|
|
- secondary.initializeErrorSimulationEvent(2);
|
|
|
- secondary.setErrorSimulation(0);
|
|
|
+ ErrorSimulator.setErrorSimulation(0);
|
|
|
|
|
|
try {
|
|
|
secondary.doCheckpoint(); // this should fail
|
|
|
- assertTrue(false);
|
|
|
+ assertTrue(false);
|
|
|
} catch (IOException e) {
|
|
|
}
|
|
|
+ ErrorSimulator.clearErrorSimulation(0);
|
|
|
secondary.shutdown();
|
|
|
|
|
|
//
|
|
@@ -173,15 +173,7 @@ public class TestCheckpoint extends TestCase {
|
|
|
assertFalse(image.getEditNewFile(idx).exists());
|
|
|
File edits = image.getEditFile(idx);
|
|
|
assertTrue(edits.exists()); // edits should exist and be empty
|
|
|
- long editsLen = -1;
|
|
|
- RandomAccessFile eF = null;
|
|
|
- try {
|
|
|
- eF = new RandomAccessFile(edits, "r");
|
|
|
- editsLen = eF.length();
|
|
|
- } finally {
|
|
|
- if(eF != null)
|
|
|
- eF.close();
|
|
|
- }
|
|
|
+ long editsLen = edits.length();
|
|
|
assertTrue(editsLen == Integer.SIZE/Byte.SIZE);
|
|
|
}
|
|
|
|
|
@@ -215,14 +207,14 @@ public class TestCheckpoint extends TestCase {
|
|
|
// Make the checkpoint fail after uploading the new fsimage.
|
|
|
//
|
|
|
SecondaryNameNode secondary = new SecondaryNameNode(conf);
|
|
|
- secondary.initializeErrorSimulationEvent(2);
|
|
|
- secondary.setErrorSimulation(1);
|
|
|
+ ErrorSimulator.setErrorSimulation(1);
|
|
|
|
|
|
try {
|
|
|
secondary.doCheckpoint(); // this should fail
|
|
|
- assertTrue(false);
|
|
|
+ assertTrue(false);
|
|
|
} catch (IOException e) {
|
|
|
}
|
|
|
+ ErrorSimulator.clearErrorSimulation(1);
|
|
|
secondary.shutdown();
|
|
|
|
|
|
//
|
|
@@ -273,14 +265,14 @@ public class TestCheckpoint extends TestCase {
|
|
|
// Make the checkpoint fail after rolling the edit log.
|
|
|
//
|
|
|
SecondaryNameNode secondary = new SecondaryNameNode(conf);
|
|
|
- secondary.initializeErrorSimulationEvent(2);
|
|
|
- secondary.setErrorSimulation(0);
|
|
|
+ ErrorSimulator.setErrorSimulation(0);
|
|
|
|
|
|
try {
|
|
|
secondary.doCheckpoint(); // this should fail
|
|
|
- assertTrue(false);
|
|
|
+ assertTrue(false);
|
|
|
} catch (IOException e) {
|
|
|
}
|
|
|
+ ErrorSimulator.clearErrorSimulation(0);
|
|
|
secondary.shutdown(); // secondary namenode crash!
|
|
|
|
|
|
// start new instance of secondary and verify that
|
|
@@ -322,6 +314,52 @@ public class TestCheckpoint extends TestCase {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Simulate a secondary node failure to transfer image
|
|
|
+ * back to the name-node.
|
|
|
+ * Used to truncate primary fsimage file.
|
|
|
+ */
|
|
|
+ void testSecondaryFailsToReturnImage(Configuration conf)
|
|
|
+ throws IOException {
|
|
|
+ System.out.println("Starting testSecondaryFailsToReturnImage");
|
|
|
+ Path file1 = new Path("checkpointRI.dat");
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster(conf, numDatanodes,
|
|
|
+ false, null);
|
|
|
+ cluster.waitActive();
|
|
|
+ FileSystem fileSys = cluster.getFileSystem();
|
|
|
+ FSImage image = cluster.getNameNode().getFSImage();
|
|
|
+ try {
|
|
|
+ assertTrue(!fileSys.exists(file1));
|
|
|
+ long fsimageLength = image.getImageFile(0, NameNodeFile.IMAGE).length();
|
|
|
+ //
|
|
|
+ // Make the checkpoint
|
|
|
+ //
|
|
|
+ SecondaryNameNode secondary = new SecondaryNameNode(conf);
|
|
|
+ ErrorSimulator.setErrorSimulation(2);
|
|
|
+
|
|
|
+ try {
|
|
|
+ secondary.doCheckpoint(); // this should fail
|
|
|
+ assertTrue(false);
|
|
|
+ } catch (IOException e) {
|
|
|
+ System.out.println("testSecondaryFailsToReturnImage: doCheckpoint() " +
|
|
|
+ "failed predictably - " + e);
|
|
|
+ }
|
|
|
+ ErrorSimulator.clearErrorSimulation(2);
|
|
|
+
|
|
|
+ // Verify that image file sizes did not change.
|
|
|
+ int nrDirs = image.getNumStorageDirs();
|
|
|
+ for(int idx = 0; idx < nrDirs; idx++) {
|
|
|
+ assertTrue(image.getImageFile(idx,
|
|
|
+ NameNodeFile.IMAGE).length() == fsimageLength);
|
|
|
+ }
|
|
|
+
|
|
|
+ secondary.shutdown();
|
|
|
+ } finally {
|
|
|
+ fileSys.close();
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Tests checkpoint in DFS.
|
|
|
*/
|
|
@@ -355,6 +393,7 @@ public class TestCheckpoint extends TestCase {
|
|
|
// Take a checkpoint
|
|
|
//
|
|
|
SecondaryNameNode secondary = new SecondaryNameNode(conf);
|
|
|
+ ErrorSimulator.initializeErrorSimulationEvent(3);
|
|
|
secondary.doCheckpoint();
|
|
|
secondary.shutdown();
|
|
|
} finally {
|
|
@@ -412,5 +451,6 @@ public class TestCheckpoint extends TestCase {
|
|
|
testSecondaryNamenodeError2(conf);
|
|
|
testSecondaryNamenodeError3(conf);
|
|
|
testNamedirError(conf, namedirs);
|
|
|
+ testSecondaryFailsToReturnImage(conf);
|
|
|
}
|
|
|
}
|