Browse Source

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

bshashikant 4 years ago
parent
commit
425f48799c

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -508,6 +508,14 @@ public class FSDirectory implements Closeable {
     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.

+ 4 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -283,12 +283,11 @@ public class INodeDirectory extends INodeWithAdditionalFields
    * @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);
   }
 
   /**

+ 7 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java

@@ -172,28 +172,25 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
   /**
    * 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 class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
     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);

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

@@ -368,6 +368,17 @@ public class SnapshotManager implements SnapshotStatsMXBean {
     }
   }
 
+  /**
+   * 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 class SnapshotManager implements SnapshotStatsMXBean {
           "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

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

@@ -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();
+      }
+    }
+  }
+
 }