HDFS-15497. Make snapshot limit on global as well per snapshot root directory configurable (#2175)
This commit is contained in:
parent
ab2b3df2de
commit
e072d33327
@ -501,6 +501,12 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
|||||||
public static final String DFS_NAMENODE_SNAPSHOT_MAX_LIMIT =
|
public static final String DFS_NAMENODE_SNAPSHOT_MAX_LIMIT =
|
||||||
"dfs.namenode.snapshot.max.limit";
|
"dfs.namenode.snapshot.max.limit";
|
||||||
public static final int DFS_NAMENODE_SNAPSHOT_MAX_LIMIT_DEFAULT = 65536;
|
public static final int DFS_NAMENODE_SNAPSHOT_MAX_LIMIT_DEFAULT = 65536;
|
||||||
|
public static final String
|
||||||
|
DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT =
|
||||||
|
"dfs.namenode.snapshot.filesystem.limit";
|
||||||
|
// default value is same as snapshot quota set for a snapshottable directory
|
||||||
|
public static final int
|
||||||
|
DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT_DEFAULT = 65536;
|
||||||
|
|
||||||
public static final String DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL =
|
public static final String DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL =
|
||||||
"dfs.namenode.snapshot.skiplist.interval";
|
"dfs.namenode.snapshot.skiplist.interval";
|
||||||
|
@ -120,12 +120,14 @@ public class SnapshotManager implements SnapshotStatsMXBean {
|
|||||||
private final boolean snapshotDeletionOrdered;
|
private final boolean snapshotDeletionOrdered;
|
||||||
private int snapshotCounter = 0;
|
private int snapshotCounter = 0;
|
||||||
private final int maxSnapshotLimit;
|
private final int maxSnapshotLimit;
|
||||||
|
private final int maxSnapshotFSLimit;
|
||||||
|
|
||||||
/** All snapshottable directories in the namesystem. */
|
/** All snapshottable directories in the namesystem. */
|
||||||
private final Map<Long, INodeDirectory> snapshottables =
|
private final Map<Long, INodeDirectory> snapshottables =
|
||||||
new ConcurrentHashMap<>();
|
new ConcurrentHashMap<>();
|
||||||
|
|
||||||
public SnapshotManager(final Configuration conf, final FSDirectory fsdir) {
|
public SnapshotManager(final Configuration conf, final FSDirectory fsdir)
|
||||||
|
throws SnapshotException {
|
||||||
this.fsdir = fsdir;
|
this.fsdir = fsdir;
|
||||||
this.captureOpenFiles = conf.getBoolean(
|
this.captureOpenFiles = conf.getBoolean(
|
||||||
DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES,
|
DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES,
|
||||||
@ -138,13 +140,20 @@ public SnapshotManager(final Configuration conf, final FSDirectory fsdir) {
|
|||||||
DFSConfigKeys.
|
DFSConfigKeys.
|
||||||
DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT_DEFAULT);
|
DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT_DEFAULT);
|
||||||
this.maxSnapshotLimit = conf.getInt(
|
this.maxSnapshotLimit = conf.getInt(
|
||||||
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_MAX_LIMIT,
|
DFSConfigKeys.
|
||||||
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_MAX_LIMIT_DEFAULT);
|
DFS_NAMENODE_SNAPSHOT_MAX_LIMIT,
|
||||||
|
DFSConfigKeys.
|
||||||
|
DFS_NAMENODE_SNAPSHOT_MAX_LIMIT_DEFAULT);
|
||||||
|
this.maxSnapshotFSLimit = conf.getInt(
|
||||||
|
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT,
|
||||||
|
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT_DEFAULT);
|
||||||
LOG.info("Loaded config captureOpenFiles: " + captureOpenFiles
|
LOG.info("Loaded config captureOpenFiles: " + captureOpenFiles
|
||||||
+ ", skipCaptureAccessTimeOnlyChange: "
|
+ ", skipCaptureAccessTimeOnlyChange: "
|
||||||
+ skipCaptureAccessTimeOnlyChange
|
+ skipCaptureAccessTimeOnlyChange
|
||||||
+ ", snapshotDiffAllowSnapRootDescendant: "
|
+ ", snapshotDiffAllowSnapRootDescendant: "
|
||||||
+ snapshotDiffAllowSnapRootDescendant
|
+ snapshotDiffAllowSnapRootDescendant
|
||||||
|
+ ", maxSnapshotFSLimit: "
|
||||||
|
+ maxSnapshotFSLimit
|
||||||
+ ", maxSnapshotLimit: "
|
+ ", maxSnapshotLimit: "
|
||||||
+ maxSnapshotLimit);
|
+ maxSnapshotLimit);
|
||||||
|
|
||||||
@ -160,6 +169,13 @@ public SnapshotManager(final Configuration conf, final FSDirectory fsdir) {
|
|||||||
final int skipInterval = conf.getInt(
|
final int skipInterval = conf.getInt(
|
||||||
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL,
|
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL,
|
||||||
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL_DEFAULT);
|
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL_DEFAULT);
|
||||||
|
if (maxSnapshotLimit > maxSnapshotFSLimit) {
|
||||||
|
final String errMsg = DFSConfigKeys.
|
||||||
|
DFS_NAMENODE_SNAPSHOT_MAX_LIMIT
|
||||||
|
+ " cannot be greater than " +
|
||||||
|
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT;
|
||||||
|
throw new SnapshotException(errMsg);
|
||||||
|
}
|
||||||
DirectoryDiffListFactory.init(skipInterval, maxLevels, LOG);
|
DirectoryDiffListFactory.init(skipInterval, maxLevels, LOG);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -405,6 +421,14 @@ public String createSnapshot(final LeaseManager leaseManager,
|
|||||||
"Failed to create the snapshot. The FileSystem has run out of " +
|
"Failed to create the snapshot. The FileSystem has run out of " +
|
||||||
"snapshot IDs and ID rollover is not supported.");
|
"snapshot IDs and ID rollover is not supported.");
|
||||||
}
|
}
|
||||||
|
int n = numSnapshots.get();
|
||||||
|
if (n >= maxSnapshotFSLimit) {
|
||||||
|
// We have reached the maximum snapshot limit
|
||||||
|
throw new SnapshotException(
|
||||||
|
"Failed to create snapshot: there are already " + (n + 1)
|
||||||
|
+ " snapshot(s) and the max snapshot limit is "
|
||||||
|
+ maxSnapshotFSLimit);
|
||||||
|
}
|
||||||
|
|
||||||
srcRoot.addSnapshot(snapshotCounter, snapshotName, leaseManager,
|
srcRoot.addSnapshot(snapshotCounter, snapshotName, leaseManager,
|
||||||
this.captureOpenFiles, maxSnapshotLimit, mtime);
|
this.captureOpenFiles, maxSnapshotLimit, mtime);
|
||||||
|
@ -5101,6 +5101,15 @@
|
|||||||
for maximum no of snapshots allowed is 65536.
|
for maximum no of snapshots allowed is 65536.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>dfs.namenode.snapshot.filesystem.limit</name>
|
||||||
|
<value>65536</value>
|
||||||
|
<description>
|
||||||
|
Limits the maximum number of snapshots allowed on the entire filesystem.
|
||||||
|
If the configuration is not set, the default limit
|
||||||
|
for maximum no of snapshots allowed is 65536.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
|
||||||
<property>
|
<property>
|
||||||
<name>dfs.namenode.snapshot.skiplist.max.levels</name>
|
<name>dfs.namenode.snapshot.skiplist.max.levels</name>
|
||||||
|
@ -24,6 +24,7 @@
|
|||||||
import static org.mockito.Mockito.spy;
|
import static org.mockito.Mockito.spy;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.hdfs.protocol.SnapshotException;
|
import org.apache.hadoop.hdfs.protocol.SnapshotException;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
|
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.INode;
|
import org.apache.hadoop.hdfs.server.namenode.INode;
|
||||||
@ -35,32 +36,54 @@
|
|||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Testing snapshot manager functionality.
|
* Testing snapshot manager functionality.
|
||||||
*/
|
*/
|
||||||
public class TestSnapshotManager {
|
public class TestSnapshotManager {
|
||||||
private static final int testMaxSnapshotLimit = 7;
|
private static final int testMaxSnapshotIDLimit = 7;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test that the global limit on snapshots is honored.
|
* Test that the global limit on snapshot Ids is honored.
|
||||||
*/
|
*/
|
||||||
@Test (timeout=10000)
|
@Test (timeout=10000)
|
||||||
public void testSnapshotLimits() throws Exception {
|
public void testSnapshotIDLimits() throws Exception {
|
||||||
// Setup mock objects for SnapshotManager.createSnapshot.
|
testMaxSnapshotLimit(testMaxSnapshotIDLimit, "rollover",
|
||||||
//
|
new Configuration(), testMaxSnapshotIDLimit);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests that the global limit on snapshots is honored.
|
||||||
|
*/
|
||||||
|
@Test (timeout=10000)
|
||||||
|
public void testMaxSnapshotLimit() throws Exception {
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
conf.setInt(DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT,
|
||||||
|
testMaxSnapshotIDLimit);
|
||||||
|
conf.setInt(DFSConfigKeys.
|
||||||
|
DFS_NAMENODE_SNAPSHOT_MAX_LIMIT,
|
||||||
|
testMaxSnapshotIDLimit);
|
||||||
|
testMaxSnapshotLimit(testMaxSnapshotIDLimit,"max snapshot limit" ,
|
||||||
|
conf, testMaxSnapshotIDLimit * 2);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testMaxSnapshotLimit(int maxSnapshotLimit, String errMsg,
|
||||||
|
Configuration conf, int maxSnapID)
|
||||||
|
throws IOException {
|
||||||
LeaseManager leaseManager = mock(LeaseManager.class);
|
LeaseManager leaseManager = mock(LeaseManager.class);
|
||||||
INodeDirectory ids = mock(INodeDirectory.class);
|
INodeDirectory ids = mock(INodeDirectory.class);
|
||||||
FSDirectory fsdir = mock(FSDirectory.class);
|
FSDirectory fsdir = mock(FSDirectory.class);
|
||||||
INodesInPath iip = mock(INodesInPath.class);
|
INodesInPath iip = mock(INodesInPath.class);
|
||||||
|
|
||||||
SnapshotManager sm = spy(new SnapshotManager(new Configuration(), fsdir));
|
SnapshotManager sm = spy(new SnapshotManager(conf, fsdir));
|
||||||
doReturn(ids).when(sm).getSnapshottableRoot(any());
|
doReturn(ids).when(sm).getSnapshottableRoot(any());
|
||||||
doReturn(testMaxSnapshotLimit).when(sm).getMaxSnapshotID();
|
doReturn(maxSnapID).when(sm).getMaxSnapshotID();
|
||||||
|
|
||||||
// Create testMaxSnapshotLimit snapshots. These should all succeed.
|
// Create testMaxSnapshotLimit snapshots. These should all succeed.
|
||||||
//
|
//
|
||||||
for (Integer i = 0; i < testMaxSnapshotLimit; ++i) {
|
for (Integer i = 0; i < maxSnapshotLimit; ++i) {
|
||||||
sm.createSnapshot(leaseManager, iip, "dummy", i.toString(), Time.now());
|
sm.createSnapshot(leaseManager, iip, "dummy", i.toString(), Time.now());
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -73,7 +96,7 @@ public void testSnapshotLimits() throws Exception {
|
|||||||
Assert.fail("Expected SnapshotException not thrown");
|
Assert.fail("Expected SnapshotException not thrown");
|
||||||
} catch (SnapshotException se) {
|
} catch (SnapshotException se) {
|
||||||
Assert.assertTrue(
|
Assert.assertTrue(
|
||||||
StringUtils.toLowerCase(se.getMessage()).contains("rollover"));
|
StringUtils.toLowerCase(se.getMessage()).contains(errMsg));
|
||||||
}
|
}
|
||||||
|
|
||||||
// Delete a snapshot to free up a slot.
|
// Delete a snapshot to free up a slot.
|
||||||
@ -83,22 +106,26 @@ public void testSnapshotLimits() throws Exception {
|
|||||||
// Attempt to create a snapshot again. It should still fail due
|
// Attempt to create a snapshot again. It should still fail due
|
||||||
// to snapshot ID rollover.
|
// to snapshot ID rollover.
|
||||||
//
|
//
|
||||||
|
|
||||||
try {
|
try {
|
||||||
sm.createSnapshot(leaseManager, iip, "dummy", "shouldFailSnapshot2",
|
sm.createSnapshot(leaseManager, iip, "dummy", "shouldFailSnapshot2",
|
||||||
Time.now());
|
Time.now());
|
||||||
Assert.fail("Expected SnapshotException not thrown");
|
// in case the snapshot ID limit is hit, further creation of snapshots
|
||||||
|
// even post deletions of snapshots won't succeed
|
||||||
|
if (maxSnapID < maxSnapshotLimit) {
|
||||||
|
Assert.fail("CreateSnapshot should succeed");
|
||||||
|
}
|
||||||
} catch (SnapshotException se) {
|
} catch (SnapshotException se) {
|
||||||
Assert.assertTrue(
|
Assert.assertTrue(
|
||||||
StringUtils.toLowerCase(se.getMessage()).contains("rollover"));
|
StringUtils.toLowerCase(se.getMessage()).contains(errMsg));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Snapshot is identified by INODE CURRENT_STATE_ID.
|
* Snapshot is identified by INODE CURRENT_STATE_ID.
|
||||||
* So maximum allowable snapshotID should be less than CURRENT_STATE_ID
|
* So maximum allowable snapshotID should be less than CURRENT_STATE_ID
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testValidateSnapshotIDWidth() {
|
public void testValidateSnapshotIDWidth() throws Exception {
|
||||||
FSDirectory fsdir = mock(FSDirectory.class);
|
FSDirectory fsdir = mock(FSDirectory.class);
|
||||||
SnapshotManager snapshotManager = new SnapshotManager(new Configuration(),
|
SnapshotManager snapshotManager = new SnapshotManager(new Configuration(),
|
||||||
fsdir);
|
fsdir);
|
||||||
|
Loading…
Reference in New Issue
Block a user