HDFS-11717. Add unit test for HDFS-11709 StandbyCheckpointer should handle non-existing legacyOivImageDir gracefully. Contributed by Erik Krogen.
(cherry picked from commit d9014bda93760f223789d2ec9f5e35f40de157d4) Conflicts: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
This commit is contained in:
parent
9e9c3a511c
commit
9d6b873d3c
@ -193,7 +193,7 @@ private void doCheckpoint() throws InterruptedException, IOException {
|
|||||||
try {
|
try {
|
||||||
img.saveLegacyOIVImage(namesystem, outputDir, canceler);
|
img.saveLegacyOIVImage(namesystem, outputDir, canceler);
|
||||||
} catch (IOException ioe) {
|
} 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);
|
+ "continuing with other checkpointing steps", ioe);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -27,6 +27,7 @@
|
|||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
|
import org.apache.hadoop.fs.FileUtil;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||||
@ -485,6 +486,25 @@ public void run() {
|
|||||||
t.join();
|
t.join();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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(nn0, nn1);
|
||||||
|
// 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 {
|
private void doEdits(int start, int stop) throws IOException {
|
||||||
for (int i = start; i < stop; i++) {
|
for (int i = start; i < stop; i++) {
|
||||||
Path p = new Path("/test" + i);
|
Path p = new Path("/test" + i);
|
||||||
|
Loading…
x
Reference in New Issue
Block a user