HDFS-14075. Terminate the namenode when failed to start log segment. Contributed by Ayush Saxena.

This commit is contained in:
Surendra Singh Lilhore 2018-12-02 12:31:08 +05:30
parent 8f3e12ff07
commit 042c8ef593
5 changed files with 74 additions and 14 deletions

View File

@ -1382,10 +1382,15 @@ private void startLogSegment(final long segmentTxId, int layoutVersion)
try { try {
editLogStream = journalSet.startLogSegment(segmentTxId, layoutVersion); editLogStream = journalSet.startLogSegment(segmentTxId, layoutVersion);
} catch (IOException ex) { } catch (IOException ex) {
throw new IOException("Unable to start log segment " + final String msg = "Unable to start log segment " + segmentTxId
segmentTxId + ": too few journals successfully started.", ex); + ": too few journals successfully started.";
LOG.error(msg, ex);
synchronized (journalSetLock) {
IOUtils.cleanupWithLogger(LOG, journalSet);
}
terminate(1, msg);
} }
curSegmentTxId = segmentTxId; curSegmentTxId = segmentTxId;
state = State.IN_SEGMENT; state = State.IN_SEGMENT;
} }

View File

@ -39,6 +39,7 @@
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog; import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest; import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableListMultimap; import com.google.common.collect.ImmutableListMultimap;
@ -76,7 +77,7 @@ public class JournalSet implements JournalManager {
* stream, then the stream will be aborted and set to null. * stream, then the stream will be aborted and set to null.
*/ */
static class JournalAndStream implements CheckableNameNodeResource { static class JournalAndStream implements CheckableNameNodeResource {
private final JournalManager journal; private JournalManager journal;
private boolean disabled = false; private boolean disabled = false;
private EditLogOutputStream stream; private EditLogOutputStream stream;
private final boolean required; private final boolean required;
@ -146,7 +147,12 @@ public String toString() {
void setCurrentStreamForTests(EditLogOutputStream stream) { void setCurrentStreamForTests(EditLogOutputStream stream) {
this.stream = stream; this.stream = stream;
} }
@VisibleForTesting
void setJournalForTests(JournalManager jm) {
this.journal = jm;
}
JournalManager getManager() { JournalManager getManager() {
return journal; return journal;
} }

View File

@ -33,6 +33,7 @@
import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.util.ExitUtil.ExitException;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -191,15 +192,15 @@ public void testMismatchedNNIsRejected() throws Exception {
// Start the NN - should fail because the JNs are still formatted // Start the NN - should fail because the JNs are still formatted
// with the old namespace ID. // with the old namespace ID.
try { try {
cluster = new MiniDFSCluster.Builder(conf) ExitUtil.disableSystemExit();
.numDataNodes(0) cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
.manageNameDfsDirs(false) .manageNameDfsDirs(false).format(false).checkExitOnShutdown(false)
.format(false) .build();
.build();
fail("New NN with different namespace should have been rejected"); fail("New NN with different namespace should have been rejected");
} catch (IOException ioe) { } catch (ExitException ee) {
GenericTestUtils.assertExceptionContains( GenericTestUtils.assertExceptionContains(
"Unable to start log segment 1: too few journals", ioe); "Unable to start log segment 1: too few journals", ee);
assertTrue("Didn't terminate properly ", ExitUtil.terminateCalled());
} }
} }
} }

View File

@ -81,6 +81,8 @@
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.test.PathUtils;
import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.util.ExitUtil.ExitException;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.log4j.Level; import org.apache.log4j.Level;
@ -974,17 +976,19 @@ public void setMaxOpSize(int maxOpSize) {
public void testFailedOpen() throws Exception { public void testFailedOpen() throws Exception {
File logDir = new File(TEST_DIR, "testFailedOpen"); File logDir = new File(TEST_DIR, "testFailedOpen");
logDir.mkdirs(); logDir.mkdirs();
ExitUtil.disableSystemExit();
FSEditLog log = FSImageTestUtil.createStandaloneEditLog(logDir); FSEditLog log = FSImageTestUtil.createStandaloneEditLog(logDir);
try { try {
FileUtil.setWritable(logDir, false); FileUtil.setWritable(logDir, false);
log.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION); log.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
fail("Did no throw exception on only having a bad dir"); fail("Did no throw exception on only having a bad dir");
} catch (IOException ioe) { } catch (ExitException ee) {
GenericTestUtils.assertExceptionContains( GenericTestUtils.assertExceptionContains(
"too few journals successfully started", ioe); "too few journals successfully started", ee);
} finally { } finally {
FileUtil.setWritable(logDir, true); FileUtil.setWritable(logDir, true);
log.close(); log.close();
ExitUtil.resetFirstExitException();
} }
} }

View File

@ -17,10 +17,13 @@
*/ */
package org.apache.hadoop.hdfs.server.namenode; 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.assertFalse;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import static org.mockito.Matchers.any; 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.doThrow;
import static org.mockito.Mockito.spy; import static org.mockito.Mockito.spy;
@ -34,6 +37,7 @@
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream; import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
@ -262,6 +266,46 @@ public void testMultipleRedundantFailedEditsDirOnSetReadyToFlush()
} }
} }
@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 * Replace the journal at index <code>index</code> with one that throws an
* exception on flush. * exception on flush.