Skip to content

Commit b2decd6

Browse files
Shashikant Banerjeemukul1987
authored andcommitted
HDFS-15497. Make snapshot limit on global as well per snapshot root directory configurable (apache#2175).
Change-Id: I286b6453c45471ee723fe81ef5e477c57f1b6a56 (cherry picked from commit 63e89cb)
1 parent 6bb2648 commit b2decd6

File tree

4 files changed

+82
-16
lines changed

4 files changed

+82
-16
lines changed

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -443,6 +443,12 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
443443
public static final String DFS_NAMENODE_SNAPSHOT_MAX_LIMIT =
444444
"dfs.namenode.snapshot.max.limit";
445445
public static final int DFS_NAMENODE_SNAPSHOT_MAX_LIMIT_DEFAULT = 65536;
446+
public static final String
447+
DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT =
448+
"dfs.namenode.snapshot.filesystem.limit";
449+
// default value is same as snapshot quota set for a snapshottable directory
450+
public static final int
451+
DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT_DEFAULT = 65536;
446452

447453
public static final String DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL =
448454
"dfs.namenode.snapshot.skiplist.interval";

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java

Lines changed: 27 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -122,12 +122,14 @@ public class SnapshotManager implements SnapshotStatsMXBean {
122122
private final boolean snapshotDeletionOrdered;
123123
private int snapshotCounter = 0;
124124
private final int maxSnapshotLimit;
125+
private final int maxSnapshotFSLimit;
125126

126127
/** All snapshottable directories in the namesystem. */
127128
private final Map<Long, INodeDirectory> snapshottables =
128129
new ConcurrentHashMap<>();
129130

130-
public SnapshotManager(final Configuration conf, final FSDirectory fsdir) {
131+
public SnapshotManager(final Configuration conf, final FSDirectory fsdir)
132+
throws SnapshotException {
131133
this.fsdir = fsdir;
132134
this.captureOpenFiles = conf.getBoolean(
133135
DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES,
@@ -140,13 +142,20 @@ public SnapshotManager(final Configuration conf, final FSDirectory fsdir) {
140142
DFSConfigKeys.
141143
DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT_DEFAULT);
142144
this.maxSnapshotLimit = conf.getInt(
143-
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_MAX_LIMIT,
144-
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_MAX_LIMIT_DEFAULT);
145+
DFSConfigKeys.
146+
DFS_NAMENODE_SNAPSHOT_MAX_LIMIT,
147+
DFSConfigKeys.
148+
DFS_NAMENODE_SNAPSHOT_MAX_LIMIT_DEFAULT);
149+
this.maxSnapshotFSLimit = conf.getInt(
150+
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT,
151+
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT_DEFAULT);
145152
LOG.info("Loaded config captureOpenFiles: " + captureOpenFiles
146153
+ ", skipCaptureAccessTimeOnlyChange: "
147154
+ skipCaptureAccessTimeOnlyChange
148155
+ ", snapshotDiffAllowSnapRootDescendant: "
149156
+ snapshotDiffAllowSnapRootDescendant
157+
+ ", maxSnapshotFSLimit: "
158+
+ maxSnapshotFSLimit
150159
+ ", maxSnapshotLimit: "
151160
+ maxSnapshotLimit);
152161

@@ -162,6 +171,13 @@ public SnapshotManager(final Configuration conf, final FSDirectory fsdir) {
162171
final int skipInterval = conf.getInt(
163172
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL,
164173
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL_DEFAULT);
174+
if (maxSnapshotLimit > maxSnapshotFSLimit) {
175+
final String errMsg = DFSConfigKeys.
176+
DFS_NAMENODE_SNAPSHOT_MAX_LIMIT
177+
+ " cannot be greater than " +
178+
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT;
179+
throw new SnapshotException(errMsg);
180+
}
165181
DirectoryDiffListFactory.init(skipInterval, maxLevels, LOG);
166182
}
167183

@@ -405,6 +421,14 @@ public String createSnapshot(final LeaseManager leaseManager,
405421
"Failed to create the snapshot. The FileSystem has run out of " +
406422
"snapshot IDs and ID rollover is not supported.");
407423
}
424+
int n = numSnapshots.get();
425+
if (n >= maxSnapshotFSLimit) {
426+
// We have reached the maximum snapshot limit
427+
throw new SnapshotException(
428+
"Failed to create snapshot: there are already " + (n + 1)
429+
+ " snapshot(s) and the max snapshot limit is "
430+
+ maxSnapshotFSLimit);
431+
}
408432

409433
srcRoot.addSnapshot(snapshotCounter, snapshotName, leaseManager,
410434
this.captureOpenFiles, maxSnapshotLimit);

hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -4647,6 +4647,15 @@
46474647
for maximum no of snapshots allowed is 65536.
46484648
</description>
46494649
</property>
4650+
<property>
4651+
<name>dfs.namenode.snapshot.filesystem.limit</name>
4652+
<value>65536</value>
4653+
<description>
4654+
Limits the maximum number of snapshots allowed on the entire filesystem.
4655+
If the configuration is not set, the default limit
4656+
for maximum no of snapshots allowed is 65536.
4657+
</description>
4658+
</property>
46504659

46514660
<property>
46524661
<name>dfs.namenode.snapshot.skiplist.max.levels</name>

hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotManager.java

Lines changed: 40 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
import static org.mockito.Mockito.spy;
2525

2626
import org.apache.hadoop.conf.Configuration;
27+
import org.apache.hadoop.hdfs.DFSConfigKeys;
2728
import org.apache.hadoop.hdfs.protocol.SnapshotException;
2829
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
2930
import org.apache.hadoop.hdfs.server.namenode.INode;
@@ -34,32 +35,54 @@
3435
import org.junit.Assert;
3536
import org.junit.Test;
3637

38+
import java.io.IOException;
39+
3740

3841
/**
3942
* Testing snapshot manager functionality.
4043
*/
4144
public class TestSnapshotManager {
42-
private static final int testMaxSnapshotLimit = 7;
45+
private static final int testMaxSnapshotIDLimit = 7;
4346

4447
/**
45-
* Test that the global limit on snapshots is honored.
48+
* Test that the global limit on snapshot Ids is honored.
4649
*/
4750
@Test (timeout=10000)
48-
public void testSnapshotLimits() throws Exception {
49-
// Setup mock objects for SnapshotManager.createSnapshot.
50-
//
51+
public void testSnapshotIDLimits() throws Exception {
52+
testMaxSnapshotLimit(testMaxSnapshotIDLimit, "rollover",
53+
new Configuration(), testMaxSnapshotIDLimit);
54+
}
55+
56+
/**
57+
* Tests that the global limit on snapshots is honored.
58+
*/
59+
@Test (timeout=10000)
60+
public void testMaxSnapshotLimit() throws Exception {
61+
Configuration conf = new Configuration();
62+
conf.setInt(DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT,
63+
testMaxSnapshotIDLimit);
64+
conf.setInt(DFSConfigKeys.
65+
DFS_NAMENODE_SNAPSHOT_MAX_LIMIT,
66+
testMaxSnapshotIDLimit);
67+
testMaxSnapshotLimit(testMaxSnapshotIDLimit,"max snapshot limit" ,
68+
conf, testMaxSnapshotIDLimit * 2);
69+
}
70+
71+
private void testMaxSnapshotLimit(int maxSnapshotLimit, String errMsg,
72+
Configuration conf, int maxSnapID)
73+
throws IOException {
5174
LeaseManager leaseManager = mock(LeaseManager.class);
5275
INodeDirectory ids = mock(INodeDirectory.class);
5376
FSDirectory fsdir = mock(FSDirectory.class);
5477
INodesInPath iip = mock(INodesInPath.class);
5578

56-
SnapshotManager sm = spy(new SnapshotManager(new Configuration(), fsdir));
79+
SnapshotManager sm = spy(new SnapshotManager(conf, fsdir));
5780
doReturn(ids).when(sm).getSnapshottableRoot((INodesInPath) anyObject());
58-
doReturn(testMaxSnapshotLimit).when(sm).getMaxSnapshotID();
81+
doReturn(maxSnapID).when(sm).getMaxSnapshotID();
5982

6083
// Create testMaxSnapshotLimit snapshots. These should all succeed.
6184
//
62-
for (Integer i = 0; i < testMaxSnapshotLimit; ++i) {
85+
for (Integer i = 0; i < maxSnapshotLimit; ++i) {
6386
sm.createSnapshot(leaseManager, iip, "dummy", i.toString());
6487
}
6588

@@ -71,7 +94,7 @@ public void testSnapshotLimits() throws Exception {
7194
Assert.fail("Expected SnapshotException not thrown");
7295
} catch (SnapshotException se) {
7396
Assert.assertTrue(
74-
StringUtils.toLowerCase(se.getMessage()).contains("rollover"));
97+
StringUtils.toLowerCase(se.getMessage()).contains(errMsg));
7598
}
7699

77100
// Delete a snapshot to free up a slot.
@@ -81,21 +104,25 @@ public void testSnapshotLimits() throws Exception {
81104
// Attempt to create a snapshot again. It should still fail due
82105
// to snapshot ID rollover.
83106
//
107+
84108
try {
85109
sm.createSnapshot(leaseManager, iip, "dummy", "shouldFailSnapshot2");
86-
Assert.fail("Expected SnapshotException not thrown");
110+
// in case the snapshot ID limit is hit, further creation of snapshots
111+
// even post deletions of snapshots won't succeed
112+
if (maxSnapID < maxSnapshotLimit) {
113+
Assert.fail("CreateSnapshot should succeed");
114+
}
87115
} catch (SnapshotException se) {
88116
Assert.assertTrue(
89-
StringUtils.toLowerCase(se.getMessage()).contains("rollover"));
117+
StringUtils.toLowerCase(se.getMessage()).contains(errMsg));
90118
}
91119
}
92-
93120
/**
94121
* Snapshot is identified by INODE CURRENT_STATE_ID.
95122
* So maximum allowable snapshotID should be less than CURRENT_STATE_ID
96123
*/
97124
@Test
98-
public void testValidateSnapshotIDWidth() {
125+
public void testValidateSnapshotIDWidth() throws Exception {
99126
FSDirectory fsdir = mock(FSDirectory.class);
100127
SnapshotManager snapshotManager = new SnapshotManager(new Configuration(),
101128
fsdir);

0 commit comments

Comments
 (0)