|
@@ -28,7 +28,9 @@ import static org.junit.Assert.assertTrue;
|
|
|
import static org.junit.Assert.fail;
|
|
|
|
|
|
import java.io.File;
|
|
|
+import java.io.FilenameFilter;
|
|
|
import java.io.IOException;
|
|
|
+import java.io.RandomAccessFile;
|
|
|
import java.lang.management.ManagementFactory;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.net.URI;
|
|
@@ -38,6 +40,7 @@ import java.util.List;
|
|
|
import java.util.Random;
|
|
|
|
|
|
import org.apache.commons.cli.ParseException;
|
|
|
+import org.apache.commons.io.filefilter.FileFilterUtils;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.commons.logging.impl.Log4JLogger;
|
|
@@ -51,6 +54,7 @@ import org.apache.hadoop.fs.FileUtil;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
+import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
@@ -62,6 +66,7 @@ import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
|
|
import org.apache.hadoop.hdfs.server.common.StorageInfo;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode.CheckpointStorage;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
|
@@ -109,6 +114,13 @@ public class TestCheckpoint {
|
|
|
static final int numDatanodes = 3;
|
|
|
short replication = 3;
|
|
|
|
|
|
+ static FilenameFilter tmpEditsFilter = new FilenameFilter() {
|
|
|
+ @Override
|
|
|
+ public boolean accept(File dir, String name) {
|
|
|
+ return name.startsWith(NameNodeFile.EDITS_TMP.getName());
|
|
|
+ }
|
|
|
+ };
|
|
|
+
|
|
|
private CheckpointFaultInjector faultInjector;
|
|
|
|
|
|
@Before
|
|
@@ -278,7 +290,7 @@ public class TestCheckpoint {
|
|
|
/*
|
|
|
* Simulate 2NN exit due to too many merge failures.
|
|
|
*/
|
|
|
- @Test(timeout=10000)
|
|
|
+ @Test(timeout=30000)
|
|
|
public void testTooManyEditReplayFailures() throws IOException {
|
|
|
Configuration conf = new HdfsConfiguration();
|
|
|
conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_MAX_RETRIES_KEY, "1");
|
|
@@ -1486,6 +1498,134 @@ public class TestCheckpoint {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Test that a fault while downloading edits does not prevent future
|
|
|
+ * checkpointing
|
|
|
+ */
|
|
|
+ @Test(timeout = 30000)
|
|
|
+ public void testEditFailureBeforeRename() throws IOException {
|
|
|
+ Configuration conf = new HdfsConfiguration();
|
|
|
+ SecondaryNameNode secondary = null;
|
|
|
+ MiniDFSCluster cluster = null;
|
|
|
+ FileSystem fs = null;
|
|
|
+ try {
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
|
|
|
+ .build();
|
|
|
+ cluster.waitActive();
|
|
|
+ fs = cluster.getFileSystem();
|
|
|
+ secondary = startSecondaryNameNode(conf);
|
|
|
+ DFSTestUtil.createFile(fs, new Path("tmpfile0"), 1024, (short) 1, 0l);
|
|
|
+ secondary.doCheckpoint();
|
|
|
+
|
|
|
+ // Cause edit rename to fail during next checkpoint
|
|
|
+ Mockito.doThrow(new IOException("Injecting failure before edit rename"))
|
|
|
+ .when(faultInjector).beforeEditsRename();
|
|
|
+ DFSTestUtil.createFile(fs, new Path("tmpfile1"), 1024, (short) 1, 0l);
|
|
|
+
|
|
|
+ try {
|
|
|
+ secondary.doCheckpoint();
|
|
|
+ fail("Fault injection failed.");
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ GenericTestUtils.assertExceptionContains(
|
|
|
+ "Injecting failure before edit rename", ioe);
|
|
|
+ }
|
|
|
+ Mockito.reset(faultInjector);
|
|
|
+ // truncate the tmp edits file to simulate a partial download
|
|
|
+ for (StorageDirectory sd : secondary.getFSImage().getStorage()
|
|
|
+ .dirIterable(NameNodeDirType.EDITS)) {
|
|
|
+ File[] tmpEdits = sd.getCurrentDir().listFiles(tmpEditsFilter);
|
|
|
+ assertTrue(
|
|
|
+ "Expected a single tmp edits file in directory " + sd.toString(),
|
|
|
+ tmpEdits.length == 1);
|
|
|
+ RandomAccessFile randFile = new RandomAccessFile(tmpEdits[0], "rw");
|
|
|
+ randFile.setLength(0);
|
|
|
+ randFile.close();
|
|
|
+ }
|
|
|
+ // Next checkpoint should succeed
|
|
|
+ secondary.doCheckpoint();
|
|
|
+ } finally {
|
|
|
+ if (secondary != null) {
|
|
|
+ secondary.shutdown();
|
|
|
+ }
|
|
|
+ if (fs != null) {
|
|
|
+ fs.close();
|
|
|
+ }
|
|
|
+ if (cluster != null) {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ Mockito.reset(faultInjector);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test that the secondary namenode correctly deletes temporary edits
|
|
|
+ * on startup.
|
|
|
+ */
|
|
|
+
|
|
|
+ @Test(timeout = 30000)
|
|
|
+ public void testDeleteTemporaryEditsOnStartup() throws IOException {
|
|
|
+ Configuration conf = new HdfsConfiguration();
|
|
|
+ SecondaryNameNode secondary = null;
|
|
|
+ MiniDFSCluster cluster = null;
|
|
|
+ FileSystem fs = null;
|
|
|
+
|
|
|
+ try {
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
|
|
|
+ .build();
|
|
|
+ cluster.waitActive();
|
|
|
+ fs = cluster.getFileSystem();
|
|
|
+ secondary = startSecondaryNameNode(conf);
|
|
|
+ DFSTestUtil.createFile(fs, new Path("tmpfile0"), 1024, (short) 1, 0l);
|
|
|
+ secondary.doCheckpoint();
|
|
|
+
|
|
|
+ // Cause edit rename to fail during next checkpoint
|
|
|
+ Mockito.doThrow(new IOException("Injecting failure before edit rename"))
|
|
|
+ .when(faultInjector).beforeEditsRename();
|
|
|
+ DFSTestUtil.createFile(fs, new Path("tmpfile1"), 1024, (short) 1, 0l);
|
|
|
+
|
|
|
+ try {
|
|
|
+ secondary.doCheckpoint();
|
|
|
+ fail("Fault injection failed.");
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ GenericTestUtils.assertExceptionContains(
|
|
|
+ "Injecting failure before edit rename", ioe);
|
|
|
+ }
|
|
|
+ Mockito.reset(faultInjector);
|
|
|
+ // Verify that a temp edits file is present
|
|
|
+ for (StorageDirectory sd : secondary.getFSImage().getStorage()
|
|
|
+ .dirIterable(NameNodeDirType.EDITS)) {
|
|
|
+ File[] tmpEdits = sd.getCurrentDir().listFiles(tmpEditsFilter);
|
|
|
+ assertTrue(
|
|
|
+ "Expected a single tmp edits file in directory " + sd.toString(),
|
|
|
+ tmpEdits.length == 1);
|
|
|
+ }
|
|
|
+ // Restart 2NN
|
|
|
+ secondary.shutdown();
|
|
|
+ secondary = startSecondaryNameNode(conf);
|
|
|
+ // Verify that tmp files were deleted
|
|
|
+ for (StorageDirectory sd : secondary.getFSImage().getStorage()
|
|
|
+ .dirIterable(NameNodeDirType.EDITS)) {
|
|
|
+ File[] tmpEdits = sd.getCurrentDir().listFiles(tmpEditsFilter);
|
|
|
+ assertTrue(
|
|
|
+ "Did not expect a tmp edits file in directory " + sd.toString(),
|
|
|
+ tmpEdits.length == 0);
|
|
|
+ }
|
|
|
+ // Next checkpoint should succeed
|
|
|
+ secondary.doCheckpoint();
|
|
|
+ } finally {
|
|
|
+ if (secondary != null) {
|
|
|
+ secondary.shutdown();
|
|
|
+ }
|
|
|
+ if (fs != null) {
|
|
|
+ fs.close();
|
|
|
+ }
|
|
|
+ if (cluster != null) {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ Mockito.reset(faultInjector);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Test case where two secondary namenodes are checkpointing the same
|
|
|
* NameNode. This differs from {@link #testMultipleSecondaryNamenodes()}
|