浏览代码

HDFS-5159. Secondary NameNode fails to checkpoint if error occurs downloading edits on first checkpoint. Contributed by Aaron T. Myers.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1520366 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers 11 年之前
父节点
当前提交
c79ca11fed

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -177,6 +177,9 @@ Release 2.1.1-beta - UNRELEASED
     HDFS-5140. Too many safemode monitor threads being created in the standby 
     namenode causing it to fail with out of memory error. (jing9)
 
+    HDFS-5159. Secondary NameNode fails to checkpoint if error occurs
+    downloading edits on first checkpoint. (atm)
+
 Release 2.1.0-beta - 2013-08-22
 
   INCOMPATIBLE CHANGES

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

@@ -429,10 +429,8 @@ public class SecondaryNameNode implements Runnable {
             dstImage.getStorage().cTime = sig.cTime;
 
             // get fsimage
-            boolean downloadImage = true;
             if (sig.mostRecentCheckpointTxId ==
                 dstImage.getStorage().getMostRecentCheckpointTxId()) {
-              downloadImage = false;
               LOG.info("Image has not changed. Will not download image.");
             } else {
               LOG.info("Image has changed. Downloading updated image from NN.");
@@ -448,7 +446,9 @@ public class SecondaryNameNode implements Runnable {
                   nnHostPort, log, dstImage.getStorage());
             }
         
-            return Boolean.valueOf(downloadImage);
+            // true if we haven't loaded all the transactions represented by the
+            // downloaded fsimage.
+            return dstImage.getLastAppliedTxId() < sig.mostRecentCheckpointTxId;
           }
         });
         return b.booleanValue();

+ 54 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -40,7 +40,6 @@ import java.util.List;
 import java.util.Random;
 
 import org.apache.commons.cli.ParseException;
-import org.apache.commons.io.filefilter.FileFilterUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
@@ -1226,7 +1225,6 @@ public class TestCheckpoint {
   }
   
   /* Test case to test CheckpointSignature */
-  @SuppressWarnings("deprecation")
   @Test
   public void testCheckpointSignature() throws IOException {
 
@@ -1565,12 +1563,65 @@ public class TestCheckpoint {
       Mockito.reset(faultInjector);
     }
   }
+  
+  /**
+   * Test that a fault while downloading edits the first time after the 2NN
+   * starts up does not prevent future checkpointing.
+   */
+  @Test(timeout = 30000)
+  public void testEditFailureOnFirstCheckpoint() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    SecondaryNameNode secondary = null;
+    MiniDFSCluster cluster = null;
+    FileSystem fs = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
+          .build();
+      cluster.waitActive();
+      fs = cluster.getFileSystem();
+      fs.mkdirs(new Path("test-file-1"));
+      
+      // Make sure the on-disk fsimage on the NN has txid > 0.
+      FSNamesystem fsns = cluster.getNamesystem();
+      fsns.enterSafeMode(false);
+      fsns.saveNamespace();
+      fsns.leaveSafeMode();
+      
+      secondary = startSecondaryNameNode(conf);
+
+      // Cause edit rename to fail during next checkpoint
+      Mockito.doThrow(new IOException("Injecting failure before edit rename"))
+          .when(faultInjector).beforeEditsRename();
+      
+      try {
+        secondary.doCheckpoint();
+        fail("Fault injection failed.");
+      } catch (IOException ioe) {
+        GenericTestUtils.assertExceptionContains(
+            "Injecting failure before edit rename", ioe);
+      }
+      Mockito.reset(faultInjector);
+      
+      // Next checkpoint should succeed
+      secondary.doCheckpoint();
+    } finally {
+      if (secondary != null) {
+        secondary.shutdown();
+      }
+      if (fs != null) {
+        fs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+      Mockito.reset(faultInjector);
+    }
+  }
 
   /**
    * Test that the secondary namenode correctly deletes temporary edits
    * on startup.
    */
-
   @Test(timeout = 30000)
   public void testDeleteTemporaryEditsOnStartup() throws IOException {
     Configuration conf = new HdfsConfiguration();
@@ -1946,7 +1997,6 @@ public class TestCheckpoint {
    * Test that, if a storage directory is failed when a checkpoint occurs,
    * the non-failed storage directory receives the checkpoint.
    */
-  @SuppressWarnings("deprecation")
   @Test
   public void testCheckpointWithFailedStorageDir() throws Exception {
     MiniDFSCluster cluster = null;
@@ -2009,7 +2059,6 @@ public class TestCheckpoint {
    * should function correctly.
    * @throws Exception
    */
-  @SuppressWarnings("deprecation")
   @Test
   public void testCheckpointWithSeparateDirsAfterNameFails() throws Exception {
     MiniDFSCluster cluster = null;