Skip to content

Commit d9014bd

Browse files
xkrogenshvachko
authored andcommitted
HDFS-11717. Add unit test for HDFS-11709 StandbyCheckpointer should handle non-existing legacyOivImageDir gracefully. Contributed by Erik Krogen.
1 parent 8b82317 commit d9014bd

2 files changed

Lines changed: 21 additions & 1 deletion

File tree

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -200,7 +200,7 @@ private void doCheckpoint(boolean sendCheckpoint) throws InterruptedException, I
200200
try {
201201
img.saveLegacyOIVImage(namesystem, outputDir, canceler);
202202
} catch (IOException ioe) {
203-
LOG.error("Exception encountered while saving legacy OIV image; "
203+
LOG.warn("Exception encountered while saving legacy OIV image; "
204204
+ "continuing with other checkpointing steps", ioe);
205205
}
206206
}

hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
import org.apache.hadoop.conf.Configuration;
2828
import org.apache.hadoop.fs.FileSystem;
2929
import org.apache.hadoop.fs.FSDataOutputStream;
30+
import org.apache.hadoop.fs.FileUtil;
3031
import org.apache.hadoop.fs.Path;
3132
import org.apache.hadoop.hdfs.DFSConfigKeys;
3233
import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -545,6 +546,25 @@ public void testNonPrimarySBNUploadFSImage() throws Exception {
545546
HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(23));
546547
}
547548

549+
/**
550+
* Test that checkpointing is still successful even if an issue
551+
* was encountered while writing the legacy OIV image.
552+
*/
553+
@Test(timeout=300000)
554+
public void testCheckpointSucceedsWithLegacyOIVException() throws Exception {
555+
// Delete the OIV image dir to cause an IOException while saving
556+
FileUtil.fullyDelete(tmpOivImgDir);
557+
558+
doEdits(0, 10);
559+
HATestUtil.waitForStandbyToCatchUp(nns[0], nns[1]);
560+
// Once the standby catches up, it should notice that it needs to
561+
// do a checkpoint and save one to its local directories.
562+
HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(12));
563+
564+
// It should also upload it back to the active.
565+
HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(12));
566+
}
567+
548568
private void doEdits(int start, int stop) throws IOException {
549569
for (int i = start; i < stop; i++) {
550570
Path p = new Path("/test" + i);

0 commit comments

Comments
 (0)