You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by sh...@apache.org on 2017/05/03 01:36:08 UTC

hadoop git commit: HDFS-11717. Add unit test for HDFS-11709 StandbyCheckpointer should handle non-existing legacyOivImageDir gracefully. Contributed by Erik Krogen.

Repository: hadoop
Updated Branches:
  refs/heads/trunk 8b82317fa -> d9014bda9


HDFS-11717. Add unit test for HDFS-11709 StandbyCheckpointer should handle non-existing legacyOivImageDir gracefully. Contributed by Erik Krogen.

Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d9014bda
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d9014bda
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d9014bda

Branch: refs/heads/trunk
Commit: d9014bda93760f223789d2ec9f5e35f40de157d4
Parents: 8b82317
Author: Erik Krogen <ek...@linkedin.com>
Authored: Tue May 2 17:56:19 2017 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Tue May 2 18:34:11 2017 -0700

----------------------------------------------------------------------
 .../server/namenode/ha/StandbyCheckpointer.java |  2 +-
 .../namenode/ha/TestStandbyCheckpoints.java     | 20 ++++++++++++++++++++
 2 files changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d9014bda/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
index 2196caa..753447b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
@@ -200,7 +200,7 @@ public class StandbyCheckpointer {
         try {
           img.saveLegacyOIVImage(namesystem, outputDir, canceler);
         } catch (IOException ioe) {
-          LOG.error("Exception encountered while saving legacy OIV image; "
+          LOG.warn("Exception encountered while saving legacy OIV image; "
                   + "continuing with other checkpointing steps", ioe);
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d9014bda/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
index ada62ba..2af373f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
@@ -27,6 +27,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -545,6 +546,25 @@ public class TestStandbyCheckpoints {
     HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(23));
   }
 
+  /**
+   * Test that checkpointing is still successful even if an issue
+   * was encountered while writing the legacy OIV image.
+   */
+  @Test(timeout=300000)
+  public void testCheckpointSucceedsWithLegacyOIVException() throws Exception {
+    // Delete the OIV image dir to cause an IOException while saving
+    FileUtil.fullyDelete(tmpOivImgDir);
+
+    doEdits(0, 10);
+    HATestUtil.waitForStandbyToCatchUp(nns[0], nns[1]);
+    // Once the standby catches up, it should notice that it needs to
+    // do a checkpoint and save one to its local directories.
+    HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(12));
+
+    // It should also upload it back to the active.
+    HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(12));
+  }
+
   private void doEdits(int start, int stop) throws IOException {
     for (int i = start; i < stop; i++) {
       Path p = new Path("/test" + i);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org