Author: atm
Date: Mon Jan 16 22:16:15 2012
New Revision: 1232197
URL: http://svn.apache.org/viewvc?rev=1232197&view=rev
Log:
HDFS-2772. On transition to active, standby should not swallow ELIE.
Contributed by Aaron T. Myers
Modified:
hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-1623.txt
hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
Modified:
hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-1623.txt
URL:
http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-1623.txt?rev=1232197&r1=1232196&r2=1232197&view=diff
==============================================================================
---
hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-1623.txt
(original)
+++
hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-1623.txt
Mon Jan 16 22:16:15 2012
@@ -107,3 +107,5 @@ HDFS-2738. FSEditLog.selectinputStreams
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)
Modified:
hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
URL:
http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java?rev=1232197&r1=1232196&r2=1232197&view=diff
==============================================================================
---
hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
(original)
+++
hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
Mon Jan 16 22:16:15 2012
@@ -144,11 +144,13 @@ public class EditLogTailer {
try {
editsLoaded = image.loadEdits(streams, namesystem);
} catch (EditLogInputException elie) {
- LOG.warn("Error while reading edits from disk. Will try again.", elie);
editsLoaded = elie.getNumEditsLoaded();
- }
- if (LOG.isDebugEnabled()) {
- LOG.debug("editsLoaded: " + editsLoaded);
+ throw elie;
+ } finally {
+ if (editsLoaded > 0) {
+ LOG.info(String.format("Loaded %d edits starting from txid %d ",
+ editsLoaded, lastTxnId));
+ }
}
} finally {
namesystem.writeUnlock();
@@ -180,12 +182,14 @@ public class EditLogTailer {
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);
}
Modified:
hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
URL:
http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java?rev=1232197&r1=1232196&r2=1232197&view=diff
==============================================================================
---
hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
(original)
+++
hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
Mon Jan 16 22:16:15 2012
@@ -30,17 +30,20 @@ import static org.mockito.Mockito.times;
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 org.mockito.stubbing.Answer;
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";
@@ -221,6 +227,47 @@ public class TestFailureToReadEdits {
fs0.close();
}
}
+
+ /**
+ * 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()