HDFS-2772. On transition to active, standby should not swallow ELIE. Contributed by Aaron T. Myers
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1232197 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
940eeb866c
commit
d880c7cc78
@ -107,3 +107,5 @@ HDFS-2738. FSEditLog.selectinputStreams is reading through in-progress streams e
|
||||
HDFS-2789. TestHAAdmin.testFailover is failing (eli)
|
||||
|
||||
HDFS-2747. Entering safe mode after starting SBN can NPE. (Uma Maheswara Rao G via todd)
|
||||
|
||||
HDFS-2772. On transition to active, standby should not swallow ELIE. (atm)
|
||||
|
@ -144,11 +144,13 @@ private void doTailEdits() throws IOException, InterruptedException {
|
||||
try {
|
||||
editsLoaded = image.loadEdits(streams, namesystem);
|
||||
} catch (EditLogInputException elie) {
|
||||
LOG.warn("Error while reading edits from disk. Will try again.", elie);
|
||||
editsLoaded = elie.getNumEditsLoaded();
|
||||
throw elie;
|
||||
} finally {
|
||||
if (editsLoaded > 0) {
|
||||
LOG.info(String.format("Loaded %d edits starting from txid %d ",
|
||||
editsLoaded, lastTxnId));
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("editsLoaded: " + editsLoaded);
|
||||
}
|
||||
} finally {
|
||||
namesystem.writeUnlock();
|
||||
@ -180,12 +182,14 @@ public void run() {
|
||||
while (shouldRun) {
|
||||
try {
|
||||
doTailEdits();
|
||||
} catch (EditLogInputException elie) {
|
||||
LOG.warn("Error while reading edits from disk. Will try again.", elie);
|
||||
} catch (InterruptedException ie) {
|
||||
// interrupter should have already set shouldRun to false
|
||||
continue;
|
||||
} catch (Throwable t) {
|
||||
LOG.error("Error encountered while tailing edits. Shutting down " +
|
||||
"standby NN.", t);
|
||||
LOG.error("Unknown error encountered while tailing edits. " +
|
||||
"Shutting down standby NN.", t);
|
||||
runtime.exit(1);
|
||||
}
|
||||
|
||||
|
@ -30,17 +30,20 @@
|
||||
import static org.mockito.Mockito.verify;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.util.Collection;
|
||||
import java.util.LinkedList;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.ha.ServiceFailedException;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.HAUtil;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
||||
import org.apache.hadoop.hdfs.server.namenode.EditLogInputException;
|
||||
import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
|
||||
@ -55,6 +58,9 @@
|
||||
import com.google.common.collect.ImmutableList;
|
||||
|
||||
public class TestFailureToReadEdits {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(TestFailureToReadEdits.class);
|
||||
|
||||
private static final String TEST_DIR1 = "/test1";
|
||||
private static final String TEST_DIR2 = "/test2";
|
||||
private static final String TEST_DIR3 = "/test3";
|
||||
@ -222,6 +228,47 @@ public void testCheckpointStartingMidEditsFile() throws Exception {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensure that the standby fails to become active if it cannot read all
|
||||
* available edits in the shared edits dir when it is transitioning to active
|
||||
* state.
|
||||
*/
|
||||
@Test
|
||||
public void testFailureToReadEditsOnTransitionToActive() throws Exception {
|
||||
assertTrue(fs.mkdirs(new Path(TEST_DIR1)));
|
||||
|
||||
HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
|
||||
|
||||
// It should also upload it back to the active.
|
||||
HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(0, 3));
|
||||
|
||||
causeFailureOnEditLogRead();
|
||||
|
||||
assertTrue(fs.mkdirs(new Path(TEST_DIR2)));
|
||||
assertTrue(fs.mkdirs(new Path(TEST_DIR3)));
|
||||
|
||||
try {
|
||||
HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
|
||||
fail("Standby fully caught up, but should not have been able to");
|
||||
} catch (HATestUtil.CouldNotCatchUpException e) {
|
||||
verify(mockRuntime, times(0)).exit(anyInt());
|
||||
}
|
||||
|
||||
// Shutdown the active NN.
|
||||
cluster.shutdownNameNode(0);
|
||||
|
||||
try {
|
||||
// Transition the standby to active.
|
||||
cluster.transitionToActive(1);
|
||||
fail("Standby transitioned to active, but should not have been able to");
|
||||
} catch (ServiceFailedException sfe) {
|
||||
LOG.info("got expected exception: " + sfe.toString(), sfe);
|
||||
assertTrue("Standby failed to catch up for some reason other than "
|
||||
+ "failure to read logs", sfe.toString().contains(
|
||||
EditLogInputException.class.getName()));
|
||||
}
|
||||
}
|
||||
|
||||
private LimitedEditLogAnswer causeFailureOnEditLogRead() throws IOException {
|
||||
FSEditLog spyEditLog = spy(nn1.getNamesystem().getEditLogTailer()
|
||||
.getEditLog());
|
||||
|
Loading…
Reference in New Issue
Block a user