Преглед изворни кода

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

Surendra Singh Lilhore пре 6 година
родитељ
комит
042c8ef593

+ 8 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

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

+ 8 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 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.
    */
   static class JournalAndStream implements CheckableNameNodeResource {
-    private final JournalManager journal;
+    private JournalManager journal;
     private boolean disabled = false;
     private EditLogOutputStream stream;
     private final boolean required;
@@ -146,7 +147,12 @@ public class JournalSet implements JournalManager {
     void setCurrentStreamForTests(EditLogOutputStream stream) {
       this.stream = stream;
     }
-    
+
+    @VisibleForTesting
+    void setJournalForTests(JournalManager jm) {
+      this.journal = jm;
+    }
+
     JournalManager getManager() {
       return journal;
     }

+ 8 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.util.ExitUtil.ExitException;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -191,15 +192,15 @@ public class TestNNWithQJM {
     // Start the NN - should fail because the JNs are still formatted
     // with the old namespace ID.
     try {
-      cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(0)
-        .manageNameDfsDirs(false)
-        .format(false)
-        .build();
+      ExitUtil.disableSystemExit();
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+          .manageNameDfsDirs(false).format(false).checkExitOnShutdown(false)
+          .build();
       fail("New NN with different namespace should have been rejected");
-    } catch (IOException ioe) {
+    } catch (ExitException ee) {
       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());
     }
   }
 }

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java

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

+ 44 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java

@@ -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.