|
@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSImage;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.JournalSet;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NNStorage;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
@@ -66,6 +67,12 @@ public class TestStandbyCheckpoints {
|
|
|
conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY, 1);
|
|
|
conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 5);
|
|
|
conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
|
|
|
+
|
|
|
+ // Dial down the retention of extra edits and checkpoints. This is to
|
|
|
+ // help catch regressions of HDFS-4238 (SBN should not purge shared edits)
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_KEY, 1);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_KEY, 0);
|
|
|
+
|
|
|
conf.setBoolean(DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY, true);
|
|
|
conf.set(DFSConfigKeys.DFS_IMAGE_COMPRESSION_CODEC_KEY,
|
|
|
SlowCodec.class.getCanonicalName());
|
|
@@ -99,15 +106,20 @@ public class TestStandbyCheckpoints {
|
|
|
|
|
|
@Test
|
|
|
public void testSBNCheckpoints() throws Exception {
|
|
|
- doEdits(0, 10);
|
|
|
+ JournalSet standbyJournalSet = NameNodeAdapter.spyOnJournalSet(nn1);
|
|
|
|
|
|
+ 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(0, 12));
|
|
|
+ HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(12));
|
|
|
|
|
|
// It should also upload it back to the active.
|
|
|
- HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(0, 12));
|
|
|
+ HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(12));
|
|
|
+
|
|
|
+ // The standby should never try to purge edit logs on shared storage.
|
|
|
+ Mockito.verify(standbyJournalSet, Mockito.never()).
|
|
|
+ purgeLogsOlderThan(Mockito.anyLong());
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -129,8 +141,8 @@ public class TestStandbyCheckpoints {
|
|
|
// so the standby will catch up. Then, both will be in standby mode
|
|
|
// with enough uncheckpointed txns to cause a checkpoint, and they
|
|
|
// will each try to take a checkpoint and upload to each other.
|
|
|
- HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(0, 12));
|
|
|
- HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(0, 12));
|
|
|
+ HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(12));
|
|
|
+ HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(12));
|
|
|
|
|
|
assertEquals(12, nn0.getNamesystem().getFSImage()
|
|
|
.getMostRecentCheckpointTxId());
|