HDFS-15568. namenode start failed to start when dfs.namenode.max.snapshot.limit set. (#2296)

This commit is contained in:
bshashikant 2020-09-17 14:50:08 +05:30 committed by GitHub
parent e4cb0d3514
commit 425f48799c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 122 additions and 26 deletions

View File

@ -508,6 +508,14 @@ FSNamesystem getFSNamesystem() {
return namesystem;
}
/**
* Indicates whether the image loading is complete or not.
* @return true if image loading is complete, false otherwise
*/
public boolean isImageLoaded() {
return namesystem.isImageLoaded();
}
/**
* Parse configuration setting dfs.namenode.protected.directories to
* retrieve the set of protected directories.

View File

@ -283,12 +283,11 @@ public void setSnapshotQuota(int snapshotQuota) {
* @param name Name of the snapshot.
* @param mtime The snapshot creation time set by Time.now().
*/
public Snapshot addSnapshot(int id, String name,
final LeaseManager leaseManager, final boolean captureOpenFiles,
int maxSnapshotLimit, long mtime)
public Snapshot addSnapshot(SnapshotManager snapshotManager, String name,
final LeaseManager leaseManager, long mtime)
throws SnapshotException {
return getDirectorySnapshottableFeature().addSnapshot(this, id, name,
leaseManager, captureOpenFiles, maxSnapshotLimit, mtime);
return getDirectorySnapshottableFeature().addSnapshot(this,
snapshotManager, name, leaseManager, mtime);
}
/**

View File

@ -172,28 +172,25 @@ void addSnapshot(Snapshot snapshot) {
/**
* Add a snapshot.
* @param snapshotRoot Root of the snapshot.
* @param snapshotManager SnapshotManager Instance.
* @param name Name of the snapshot.
* @param leaseManager
* @param captureOpenFiles
* @throws SnapshotException Throw SnapshotException when there is a snapshot
* with the same name already exists or snapshot quota exceeds
*/
public Snapshot addSnapshot(INodeDirectory snapshotRoot, int id, String name,
final LeaseManager leaseManager, final boolean captureOpenFiles,
int maxSnapshotLimit, long now)
public Snapshot addSnapshot(INodeDirectory snapshotRoot,
SnapshotManager snapshotManager, String name,
final LeaseManager leaseManager, long now)
throws SnapshotException {
int id = snapshotManager.getSnapshotCounter();
//check snapshot quota
final int n = getNumSnapshots();
if (n + 1 > snapshotQuota) {
throw new SnapshotException("Failed to add snapshot: there are already "
+ n + " snapshot(s) and the snapshot quota is "
+ snapshotQuota);
} else if (n + 1 > maxSnapshotLimit) {
throw new SnapshotException(
"Failed to add snapshot: there are already " + n
+ " snapshot(s) and the max snapshot limit is "
+ maxSnapshotLimit);
}
snapshotManager.checkPerDirectorySnapshotLimit(n);
final Snapshot s = new Snapshot(id, name, snapshotRoot);
final byte[] nameBytes = s.getRoot().getLocalNameBytes();
final int i = searchSnapshot(nameBytes);
@ -210,7 +207,7 @@ public Snapshot addSnapshot(INodeDirectory snapshotRoot, int id, String name,
snapshotRoot.updateModificationTime(now, Snapshot.CURRENT_STATE_ID);
s.getRoot().setModificationTime(now, Snapshot.CURRENT_STATE_ID);
if (captureOpenFiles) {
if (snapshotManager.captureOpenFiles()) {
try {
Set<INodesInPath> openFilesIIP =
leaseManager.getINodeWithLeases(snapshotRoot);

View File

@ -368,6 +368,17 @@ void assertFirstSnapshot(INodeDirectory dir,
}
}
/**
* Return CaptureOpenFiles config value.
*/
boolean captureOpenFiles() {
return captureOpenFiles;
}
@VisibleForTesting
int getMaxSnapshotLimit() {
return maxSnapshotLimit;
}
/**
* Get the snapshot root directory for the given directory. The given
* directory must either be a snapshot root or a descendant of any
@ -448,22 +459,39 @@ public String createSnapshot(final LeaseManager leaseManager,
"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,
this.captureOpenFiles, maxSnapshotLimit, mtime);
checkFileSystemSnapshotLimit(n);
srcRoot.addSnapshot(this, snapshotName, leaseManager, mtime);
//create success, update id
snapshotCounter++;
numSnapshots.getAndIncrement();
return Snapshot.getSnapshotPath(snapshotRoot, snapshotName);
}
void checkFileSystemSnapshotLimit(int n) throws SnapshotException {
checkSnapshotLimit(maxSnapshotFSLimit, n, "file system");
}
void checkPerDirectorySnapshotLimit(int n) throws SnapshotException {
checkSnapshotLimit(maxSnapshotLimit, n, "per directory");
}
void checkSnapshotLimit(int limit, int snapshotCount, String type)
throws SnapshotException {
if (snapshotCount >= limit) {
String msg = "there are already " + (snapshotCount + 1)
+ " snapshot(s) and the " + type + " snapshot limit is "
+ limit;
if (fsdir.isImageLoaded()) {
// We have reached the maximum snapshot limit
throw new SnapshotException(
"Failed to create snapshot: " + msg);
} else {
// image is getting loaded. LOG an error msg and continue
LOG.error(msg);
}
}
}
/**
* Delete a snapshot for a snapshottable directory

View File

@ -24,13 +24,17 @@
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 void testMaxSnapshotLimit() throws Exception {
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 @@ private void testMaxSnapshotLimit(int maxSnapshotLimit, String errMsg,
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 void testValidateSnapshotIDWidth() throws Exception {
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();
}
}
}
}