|
@@ -24,13 +24,17 @@ import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.spy;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
+import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
+import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.protocol.SnapshotException;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.INode;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
|
|
|
+import org.apache.hadoop.test.LambdaTestUtils;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
import org.junit.Assert;
|
|
@@ -65,7 +69,7 @@ public class TestSnapshotManager {
|
|
|
conf.setInt(DFSConfigKeys.
|
|
|
DFS_NAMENODE_SNAPSHOT_MAX_LIMIT,
|
|
|
testMaxSnapshotIDLimit);
|
|
|
- testMaxSnapshotLimit(testMaxSnapshotIDLimit,"max snapshot limit" ,
|
|
|
+ testMaxSnapshotLimit(testMaxSnapshotIDLimit,"file system snapshot limit" ,
|
|
|
conf, testMaxSnapshotIDLimit * 2);
|
|
|
}
|
|
|
|
|
@@ -80,6 +84,7 @@ public class TestSnapshotManager {
|
|
|
SnapshotManager sm = spy(new SnapshotManager(conf, fsdir));
|
|
|
doReturn(ids).when(sm).getSnapshottableRoot(any());
|
|
|
doReturn(maxSnapID).when(sm).getMaxSnapshotID();
|
|
|
+ doReturn(true).when(fsdir).isImageLoaded();
|
|
|
|
|
|
// Create testMaxSnapshotLimit snapshots. These should all succeed.
|
|
|
//
|
|
@@ -133,4 +138,63 @@ public class TestSnapshotManager {
|
|
|
getMaxSnapshotID() < Snapshot.CURRENT_STATE_ID);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testSnapshotLimitOnRestart() throws Exception {
|
|
|
+ final Configuration conf = new Configuration();
|
|
|
+ final Path snapshottableDir
|
|
|
+ = new Path("/" + getClass().getSimpleName());
|
|
|
+ int numSnapshots = 5;
|
|
|
+ conf.setInt(DFSConfigKeys.
|
|
|
+ DFS_NAMENODE_SNAPSHOT_MAX_LIMIT, numSnapshots);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT,
|
|
|
+ numSnapshots * 2);
|
|
|
+ MiniDFSCluster cluster = null;
|
|
|
+ try {
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).
|
|
|
+ numDataNodes(0).build();
|
|
|
+ cluster.waitActive();
|
|
|
+ DistributedFileSystem hdfs = cluster.getFileSystem();
|
|
|
+ hdfs.mkdirs(snapshottableDir);
|
|
|
+ hdfs.allowSnapshot(snapshottableDir);
|
|
|
+ for (int i = 0; i < numSnapshots; i++) {
|
|
|
+ hdfs.createSnapshot(snapshottableDir, "s" + i);
|
|
|
+ }
|
|
|
+ LambdaTestUtils.intercept(SnapshotException.class,
|
|
|
+ "snapshot limit",
|
|
|
+ () -> hdfs.createSnapshot(snapshottableDir, "s5"));
|
|
|
+
|
|
|
+ // now change max snapshot directory limit to 2 and restart namenode
|
|
|
+ cluster.getNameNode().getConf().setInt(DFSConfigKeys.
|
|
|
+ DFS_NAMENODE_SNAPSHOT_MAX_LIMIT, 2);
|
|
|
+ cluster.restartNameNodes();
|
|
|
+ SnapshotManager snapshotManager = cluster.getNamesystem().
|
|
|
+ getSnapshotManager();
|
|
|
+
|
|
|
+ // make sure edits of all previous 5 create snapshots are replayed
|
|
|
+ Assert.assertEquals(numSnapshots, snapshotManager.getNumSnapshots());
|
|
|
+
|
|
|
+ // make sure namenode has the new snapshot limit configured as 2
|
|
|
+ Assert.assertEquals(2, snapshotManager.getMaxSnapshotLimit());
|
|
|
+
|
|
|
+ // Any new snapshot creation should still fail
|
|
|
+ LambdaTestUtils.intercept(SnapshotException.class,
|
|
|
+ "snapshot limit", () -> hdfs.createSnapshot(snapshottableDir, "s5"));
|
|
|
+ // now change max snapshot FS limit to 2 and restart namenode
|
|
|
+ cluster.getNameNode().getConf().setInt(DFSConfigKeys.
|
|
|
+ DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT, 2);
|
|
|
+ cluster.restartNameNodes();
|
|
|
+ snapshotManager = cluster.getNamesystem().
|
|
|
+ getSnapshotManager();
|
|
|
+ // make sure edits of all previous 5 create snapshots are replayed
|
|
|
+ Assert.assertEquals(numSnapshots, snapshotManager.getNumSnapshots());
|
|
|
+
|
|
|
+ // make sure namenode has the new snapshot limit configured as 2
|
|
|
+ Assert.assertEquals(2, snapshotManager.getMaxSnapshotLimit());
|
|
|
+ } finally {
|
|
|
+ if (cluster != null) {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
}
|