Skip to content

Commit e072d33

Browse files
authored
HDFS-15497. Make snapshot limit on global as well per snapshot root directory configurable (#2175)
1 parent ab2b3df commit e072d33

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
@@ -501,6 +501,12 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
501501
public static final String DFS_NAMENODE_SNAPSHOT_MAX_LIMIT =
502502
"dfs.namenode.snapshot.max.limit";
503503
public static final int DFS_NAMENODE_SNAPSHOT_MAX_LIMIT_DEFAULT = 65536;
504+
public static final String
505+
DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT =
506+
"dfs.namenode.snapshot.filesystem.limit";
507+
// default value is same as snapshot quota set for a snapshottable directory
508+
public static final int
509+
DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT_DEFAULT = 65536;
504510

505511
public static final String DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL =
506512
"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
@@ -120,12 +120,14 @@ public class SnapshotManager implements SnapshotStatsMXBean {
120120
private final boolean snapshotDeletionOrdered;
121121
private int snapshotCounter = 0;
122122
private final int maxSnapshotLimit;
123+
private final int maxSnapshotFSLimit;
123124

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

128-
public SnapshotManager(final Configuration conf, final FSDirectory fsdir) {
129+
public SnapshotManager(final Configuration conf, final FSDirectory fsdir)
130+
throws SnapshotException {
129131
this.fsdir = fsdir;
130132
this.captureOpenFiles = conf.getBoolean(
131133
DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES,
@@ -138,13 +140,20 @@ public SnapshotManager(final Configuration conf, final FSDirectory fsdir) {
138140
DFSConfigKeys.
139141
DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT_DEFAULT);
140142
this.maxSnapshotLimit = conf.getInt(
141-
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_MAX_LIMIT,
142-
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_MAX_LIMIT_DEFAULT);
143+
DFSConfigKeys.
144+
DFS_NAMENODE_SNAPSHOT_MAX_LIMIT,
145+
DFSConfigKeys.
146+
DFS_NAMENODE_SNAPSHOT_MAX_LIMIT_DEFAULT);
147+
this.maxSnapshotFSLimit = conf.getInt(
148+
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT,
149+
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT_DEFAULT);
143150
LOG.info("Loaded config captureOpenFiles: " + captureOpenFiles
144151
+ ", skipCaptureAccessTimeOnlyChange: "
145152
+ skipCaptureAccessTimeOnlyChange
146153
+ ", snapshotDiffAllowSnapRootDescendant: "
147154
+ snapshotDiffAllowSnapRootDescendant
155+
+ ", maxSnapshotFSLimit: "
156+
+ maxSnapshotFSLimit
148157
+ ", maxSnapshotLimit: "
149158
+ maxSnapshotLimit);
150159

@@ -160,6 +169,13 @@ public SnapshotManager(final Configuration conf, final FSDirectory fsdir) {
160169
final int skipInterval = conf.getInt(
161170
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL,
162171
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL_DEFAULT);
172+
if (maxSnapshotLimit > maxSnapshotFSLimit) {
173+
final String errMsg = DFSConfigKeys.
174+
DFS_NAMENODE_SNAPSHOT_MAX_LIMIT
175+
+ " cannot be greater than " +
176+
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_FILESYSTEM_LIMIT;
177+
throw new SnapshotException(errMsg);
178+
}
163179
DirectoryDiffListFactory.init(skipInterval, maxLevels, LOG);
164180
}
165181

@@ -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, mtime);

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
@@ -5101,6 +5101,15 @@
51015101
for maximum no of snapshots allowed is 65536.
51025102
</description>
51035103
</property>
5104+
<property>
5105+
<name>dfs.namenode.snapshot.filesystem.limit</name>
5106+
<value>65536</value>
5107+
<description>
5108+
Limits the maximum number of snapshots allowed on the entire filesystem.
5109+
If the configuration is not set, the default limit
5110+
for maximum no of snapshots allowed is 65536.
5111+
</description>
5112+
</property>
51045113

51055114
<property>
51065115
<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;
@@ -35,32 +36,54 @@
3536
import org.junit.Assert;
3637
import org.junit.Test;
3738

39+
import java.io.IOException;
40+
3841

3942
/**
4043
* Testing snapshot manager functionality.
4144
*/
4245
public class TestSnapshotManager {
43-
private static final int testMaxSnapshotLimit = 7;
46+
private static final int testMaxSnapshotIDLimit = 7;
4447

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

57-
SnapshotManager sm = spy(new SnapshotManager(new Configuration(), fsdir));
80+
SnapshotManager sm = spy(new SnapshotManager(conf, fsdir));
5881
doReturn(ids).when(sm).getSnapshottableRoot(any());
59-
doReturn(testMaxSnapshotLimit).when(sm).getMaxSnapshotID();
82+
doReturn(maxSnapID).when(sm).getMaxSnapshotID();
6083

6184
// Create testMaxSnapshotLimit snapshots. These should all succeed.
6285
//
63-
for (Integer i = 0; i < testMaxSnapshotLimit; ++i) {
86+
for (Integer i = 0; i < maxSnapshotLimit; ++i) {
6487
sm.createSnapshot(leaseManager, iip, "dummy", i.toString(), Time.now());
6588
}
6689

@@ -73,7 +96,7 @@ public void testSnapshotLimits() throws Exception {
7396
Assert.fail("Expected SnapshotException not thrown");
7497
} catch (SnapshotException se) {
7598
Assert.assertTrue(
76-
StringUtils.toLowerCase(se.getMessage()).contains("rollover"));
99+
StringUtils.toLowerCase(se.getMessage()).contains(errMsg));
77100
}
78101

79102
// Delete a snapshot to free up a slot.
@@ -83,22 +106,26 @@ public void testSnapshotLimits() throws Exception {
83106
// Attempt to create a snapshot again. It should still fail due
84107
// to snapshot ID rollover.
85108
//
109+
86110
try {
87111
sm.createSnapshot(leaseManager, iip, "dummy", "shouldFailSnapshot2",
88112
Time.now());
89-
Assert.fail("Expected SnapshotException not thrown");
113+
// in case the snapshot ID limit is hit, further creation of snapshots
114+
// even post deletions of snapshots won't succeed
115+
if (maxSnapID < maxSnapshotLimit) {
116+
Assert.fail("CreateSnapshot should succeed");
117+
}
90118
} catch (SnapshotException se) {
91119
Assert.assertTrue(
92-
StringUtils.toLowerCase(se.getMessage()).contains("rollover"));
120+
StringUtils.toLowerCase(se.getMessage()).contains(errMsg));
93121
}
94122
}
95-
96123
/**
97124
* Snapshot is identified by INODE CURRENT_STATE_ID.
98125
* So maximum allowable snapshotID should be less than CURRENT_STATE_ID
99126
*/
100127
@Test
101-
public void testValidateSnapshotIDWidth() {
128+
public void testValidateSnapshotIDWidth() throws Exception {
102129
FSDirectory fsdir = mock(FSDirectory.class);
103130
SnapshotManager snapshotManager = new SnapshotManager(new Configuration(),
104131
fsdir);

0 commit comments

Comments
 (0)