Browse Source

HDFS-3891. Make selectInputStreams throw IOE instead of RTE. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-3077@1381481 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 13 years ago
parent
commit
437948ea1c

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-3077.txt

@@ -44,3 +44,5 @@ HDFS-3869. Expose non-file journal manager details in web UI (todd)
 HDFS-3884. Journal format() should reset cached values (todd)
 
 HDFS-3870. Add metrics to JournalNode (todd)
+
+HDFS-3891. Make selectInputStreams throw IOE instead of RTE (todd)

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java

@@ -158,9 +158,11 @@ class AsyncLoggerSet {
           timeoutMs);
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
-      throw new IOException("Interrupted waiting for quorum results");
+      throw new IOException("Interrupted waiting " + timeoutMs + "ms for a " +
+          "quorum of nodes to respond.");
     } catch (TimeoutException e) {
-      throw new IOException("Timed out waiting " + timeoutMs + " for write quorum");
+      throw new IOException("Timed out waiting " + timeoutMs + "ms for a " +
+          "quorum of nodes to respond.");
     }
     
     if (q.countSuccesses() < majority) {

+ 3 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java

@@ -394,17 +394,12 @@ public class QuorumJournalManager implements JournalManager {
 
   @Override
   public void selectInputStreams(Collection<EditLogInputStream> streams,
-      long fromTxnId, boolean inProgressOk) {
+      long fromTxnId, boolean inProgressOk) throws IOException {
 
     QuorumCall<AsyncLogger, RemoteEditLogManifest> q =
         loggers.getEditLogManifest(fromTxnId);
-    Map<AsyncLogger, RemoteEditLogManifest> resps;
-    try {
-      resps = loggers.waitForWriteQuorum(q, selectInputStreamsTimeoutMs);
-    } catch (IOException ioe) {
-      // TODO: can we do better here?
-      throw new RuntimeException(ioe);
-    }
+    Map<AsyncLogger, RemoteEditLogManifest> resps =
+        loggers.waitForWriteQuorum(q, selectInputStreamsTimeoutMs);
     
     LOG.debug("selectInputStream manifests:\n" +
         Joiner.on("\n").withKeyValueSeparator(": ").join(resps));

+ 2 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java

@@ -242,15 +242,8 @@ public class FileJournalManager implements JournalManager {
   @Override
   synchronized public void selectInputStreams(
       Collection<EditLogInputStream> streams, long fromTxId,
-      boolean inProgressOk) {
-    List<EditLogFile> elfs;
-    try {
-      elfs = matchEditLogs(sd.getCurrentDir());
-    } catch (IOException e) {
-      LOG.error("error listing files in " + this + ". " +
-          "Skipping all edit logs in this directory.", e);
-      return;
-    }
+      boolean inProgressOk) throws IOException {
+    List<EditLogFile> elfs = matchEditLogs(sd.getCurrentDir());
     LOG.debug(this + ": selecting input streams starting at " + fromTxId + 
         (inProgressOk ? " (inProgress ok) " : " (excluding inProgress) ") +
         "from among " + elfs.size() + " candidate file(s)");

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java

@@ -65,9 +65,11 @@ public interface JournalManager extends Closeable, FormatConfirmable,
    * @param inProgressOk whether or not in-progress streams should be returned
    *
    * @return a list of streams
+   * @throws IOException if the underlying storage has an error or is otherwise
+   * inaccessible
    */
   void selectInputStreams(Collection<EditLogInputStream> streams,
-      long fromTxnId, boolean inProgressOk);
+      long fromTxnId, boolean inProgressOk) throws IOException;
 
   /**
    * Set the amount of memory that this stream should use to buffer edits

+ 6 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java

@@ -247,7 +247,12 @@ public class JournalSet implements JournalManager {
         LOG.info("Skipping jas " + jas + " since it's disabled");
         continue;
       }
-      jas.getManager().selectInputStreams(allStreams, fromTxId, inProgressOk);
+      try {
+        jas.getManager().selectInputStreams(allStreams, fromTxId, inProgressOk);
+      } catch (IOException ioe) {
+        LOG.warn("Unable to determine input streams from " + jas.getManager() +
+            ". Skipping.", ioe);
+      }
     }
     chainAndMakeRedundantStreams(streams, allStreams, fromTxId, inProgressOk);
   }

+ 20 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java

@@ -211,6 +211,26 @@ public class TestQuorumJournalManager {
     }
   }
   
+  /**
+   * Regression test for HDFS-3891: selectInputStreams should throw
+   * an exception when a majority of journalnodes have crashed.
+   */
+  @Test
+  public void testSelectInputStreamsMajorityDown() throws Exception {
+    // Shut down all of the JNs.
+    cluster.shutdown();
+
+    List<EditLogInputStream> streams = Lists.newArrayList();
+    try {
+      qjm.selectInputStreams(streams, 0, false);
+      fail("Did not throw IOE");
+    } catch (QuorumException ioe) {
+      GenericTestUtils.assertExceptionContains(
+          "Got too many exceptions", ioe);
+      assertTrue(streams.isEmpty());
+    }
+  }
+  
   /**
    * Test the case where the NN crashes after starting a new segment
    * on all nodes, but before writing the first transaction to it.