|
@@ -17,10 +17,13 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
|
|
import static org.junit.Assert.assertFalse;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
import static org.junit.Assert.fail;
|
|
|
import static org.mockito.Matchers.any;
|
|
|
+import static org.mockito.Matchers.anyLong;
|
|
|
+import static org.mockito.Matchers.anyInt;
|
|
|
import static org.mockito.Mockito.doThrow;
|
|
|
import static org.mockito.Mockito.spy;
|
|
|
|
|
@@ -34,6 +37,7 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
+import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
|
|
@@ -262,6 +266,46 @@ public class TestEditLogJournalFailures {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testMultipleRedundantFailedEditsDirOnStartLogSegment()
|
|
|
+ throws Exception {
|
|
|
+ // Set up 4 name/edits dirs.
|
|
|
+ shutDownMiniCluster();
|
|
|
+ Configuration conf = getConf();
|
|
|
+ String[] nameDirs = new String[4];
|
|
|
+ for (int i = 0; i < nameDirs.length; i++) {
|
|
|
+ File nameDir = new File(PathUtils.getTestDir(getClass()), "name-dir" + i);
|
|
|
+ nameDir.mkdirs();
|
|
|
+ nameDirs[i] = nameDir.getAbsolutePath();
|
|
|
+ }
|
|
|
+ conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
|
|
|
+ StringUtils.join(nameDirs, ","));
|
|
|
+ conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY,
|
|
|
+ StringUtils.join(nameDirs, ",", 0, 3));
|
|
|
+
|
|
|
+ setUpMiniCluster(conf, false);
|
|
|
+
|
|
|
+ // All journals active.
|
|
|
+ assertTrue(doAnEdit());
|
|
|
+ // The NN has not terminated (no ExitException thrown)
|
|
|
+ spyOnJASjournal(3);
|
|
|
+ RemoteException re = intercept(RemoteException.class,
|
|
|
+ "too few journals successfully started.",
|
|
|
+ () -> ((DistributedFileSystem) fs).rollEdits());
|
|
|
+ GenericTestUtils.assertExceptionContains("ExitException", re);
|
|
|
+ }
|
|
|
+
|
|
|
+ private JournalManager spyOnJASjournal(int index) throws Exception {
|
|
|
+ JournalAndStream jas = getJournalAndStream(index);
|
|
|
+ JournalManager manager = jas.getManager();
|
|
|
+ JournalManager spyManager = spy(manager);
|
|
|
+ jas.setJournalForTests(spyManager);
|
|
|
+ doThrow(new IOException("Unable to start log segment ")).when(spyManager)
|
|
|
+ .startLogSegment(anyLong(), anyInt());
|
|
|
+
|
|
|
+ return spyManager;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Replace the journal at index <code>index</code> with one that throws an
|
|
|
* exception on flush.
|