|
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
|
|
import java.io.File;
|
|
import java.io.File;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.lang.management.ManagementFactory;
|
|
import java.lang.management.ManagementFactory;
|
|
|
|
+import java.util.concurrent.CountDownLatch;
|
|
import java.util.concurrent.ThreadLocalRandom;
|
|
import java.util.concurrent.ThreadLocalRandom;
|
|
|
|
|
|
import javax.management.AttributeNotFoundException;
|
|
import javax.management.AttributeNotFoundException;
|
|
@@ -46,6 +47,7 @@ import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster;
|
|
import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
|
|
import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.CheckpointFaultInjector;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSImage;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSImage;
|
|
import org.apache.hadoop.hdfs.server.namenode.NNStorage;
|
|
import org.apache.hadoop.hdfs.server.namenode.NNStorage;
|
|
import org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode;
|
|
import org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode;
|
|
@@ -568,6 +570,52 @@ public class TestRollingUpgrade {
|
|
testCheckpoint(3);
|
|
testCheckpoint(3);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Test(timeout = 60000)
|
|
|
|
+ public void testRollBackImage() throws Exception {
|
|
|
|
+ final Configuration conf = new Configuration();
|
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 10);
|
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
|
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY, 2);
|
|
|
|
+ MiniQJMHACluster cluster = null;
|
|
|
|
+ CheckpointFaultInjector old = CheckpointFaultInjector.getInstance();
|
|
|
|
+ try {
|
|
|
|
+ cluster = new MiniQJMHACluster.Builder(conf).setNumNameNodes(2).build();
|
|
|
|
+ MiniDFSCluster dfsCluster = cluster.getDfsCluster();
|
|
|
|
+ dfsCluster.waitActive();
|
|
|
|
+ dfsCluster.transitionToActive(0);
|
|
|
|
+ DistributedFileSystem dfs = dfsCluster.getFileSystem(0);
|
|
|
|
+ for (int i = 0; i <= 10; i++) {
|
|
|
|
+ Path foo = new Path("/foo" + i);
|
|
|
|
+ dfs.mkdirs(foo);
|
|
|
|
+ }
|
|
|
|
+ cluster.getDfsCluster().getNameNodeRpc(0).rollEdits();
|
|
|
|
+ CountDownLatch ruEdit = new CountDownLatch(1);
|
|
|
|
+ CheckpointFaultInjector.set(new CheckpointFaultInjector() {
|
|
|
|
+ @Override
|
|
|
|
+ public void duringUploadInProgess()
|
|
|
|
+ throws IOException, InterruptedException {
|
|
|
|
+ if (ruEdit.getCount() == 1) {
|
|
|
|
+ ruEdit.countDown();
|
|
|
|
+ Thread.sleep(180000);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ });
|
|
|
|
+ ruEdit.await();
|
|
|
|
+ RollingUpgradeInfo info = dfs
|
|
|
|
+ .rollingUpgrade(RollingUpgradeAction.PREPARE);
|
|
|
|
+ Assert.assertTrue(info.isStarted());
|
|
|
|
+ FSImage fsimage = dfsCluster.getNamesystem(0).getFSImage();
|
|
|
|
+ queryForPreparation(dfs);
|
|
|
|
+ // The NN should have a copy of the fsimage in case of rollbacks.
|
|
|
|
+ Assert.assertTrue(fsimage.hasRollbackFSImage());
|
|
|
|
+ } finally {
|
|
|
|
+ CheckpointFaultInjector.set(old);
|
|
|
|
+ if (cluster != null) {
|
|
|
|
+ cluster.shutdown();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
public void testCheckpoint(int nnCount) throws IOException, InterruptedException {
|
|
public void testCheckpoint(int nnCount) throws IOException, InterruptedException {
|
|
final Configuration conf = new Configuration();
|
|
final Configuration conf = new Configuration();
|
|
conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
|
|
conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
|