Browse Source

HDFS-12947. Limit the number of Snapshots allowed to be created for a Snapshottable Directory. Contributed by Shashikant Banerjee

Tsz-Wo Nicholas Sze 7 years ago
parent
commit
4d1bf33d0f

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

@@ -414,6 +414,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.namenode.snapshotdiff.listing.limit";
   public static final int
       DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT_DEFAULT = 1000;
+
+  public static final String DFS_NAMENODE_SNAPSHOT_MAX_LIMIT =
+      "dfs.namenode.snapshot.max.limit";
+
+  public static final int DFS_NAMENODE_SNAPSHOT_MAX_LIMIT_DEFAULT = 65536;
   // Whether to enable datanode's stale state detection and usage for reads
   public static final String DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY = "dfs.namenode.avoid.read.stale.datanode";
   public static final boolean DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_DEFAULT = false;

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -280,10 +280,11 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
 
   public Snapshot addSnapshot(int id, String name,
-      final LeaseManager leaseManager, final boolean captureOpenFiles)
+      final LeaseManager leaseManager, final boolean captureOpenFiles,
+      int maxSnapshotLimit)
       throws SnapshotException, QuotaExceededException {
     return getDirectorySnapshottableFeature().addSnapshot(this, id, name,
-        leaseManager, captureOpenFiles);
+        leaseManager, captureOpenFiles, maxSnapshotLimit);
   }
 
   public Snapshot removeSnapshot(

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

@@ -61,7 +61,7 @@ import com.google.common.collect.Lists;
 @InterfaceAudience.Private
 public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature {
   /** Limit the number of snapshot per snapshottable directory. */
-  static final int SNAPSHOT_LIMIT = 1 << 16;
+  static final int SNAPSHOT_QUOTA_DEFAULT = 1 << 16;
 
   /**
    * Snapshots of this directory in ascending order of snapshot names.
@@ -70,7 +70,7 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
    */
   private final List<Snapshot> snapshotsByNames = new ArrayList<Snapshot>();
   /** Number of snapshots allowed. */
-  private int snapshotQuota = SNAPSHOT_LIMIT;
+  private int snapshotQuota = SNAPSHOT_QUOTA_DEFAULT;
 
   public DirectorySnapshottableFeature(DirectoryWithSnapshotFeature feature) {
     super(feature == null ? null : feature.getDiffs());
@@ -170,7 +170,8 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
 
   /** Add a snapshot. */
   public Snapshot addSnapshot(INodeDirectory snapshotRoot, int id, String name,
-      final LeaseManager leaseManager, final boolean captureOpenFiles)
+      final LeaseManager leaseManager, final boolean captureOpenFiles,
+      int maxSnapshotLimit)
       throws SnapshotException, QuotaExceededException {
     //check snapshot quota
     final int n = getNumSnapshots();
@@ -178,6 +179,11 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
       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);
     }
     final Snapshot s = new Snapshot(id, name, snapshotRoot);
     final byte[] nameBytes = s.getRoot().getLocalNameBytes();

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java

@@ -155,7 +155,8 @@ public class FSImageFormatPBSnapshot {
           dir.addSnapshottableFeature();
         } else {
           // dir is root, and admin set root to snapshottable before
-          dir.setSnapshotQuota(DirectorySnapshottableFeature.SNAPSHOT_LIMIT);
+          dir.setSnapshotQuota(
+              DirectorySnapshottableFeature.SNAPSHOT_QUOTA_DEFAULT);
         }
         sm.addSnapshottable(dir);
       }

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

@@ -99,6 +99,7 @@ public class SnapshotManager implements SnapshotStatsMXBean {
 
   private boolean allowNestedSnapshots = false;
   private int snapshotCounter = 0;
+  private final int maxSnapshotLimit;
   
   /** All snapshottable directories in the namesystem. */
   private final Map<Long, INodeDirectory> snapshottables =
@@ -116,11 +117,16 @@ public class SnapshotManager implements SnapshotStatsMXBean {
         DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT,
         DFSConfigKeys.
             DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT_DEFAULT);
+    this.maxSnapshotLimit = conf.getInt(
+        DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_MAX_LIMIT,
+        DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_MAX_LIMIT_DEFAULT);
     LOG.info("Loaded config captureOpenFiles: " + captureOpenFiles
         + ", skipCaptureAccessTimeOnlyChange: "
         + skipCaptureAccessTimeOnlyChange
         + ", snapshotDiffAllowSnapRootDescendant: "
-        + snapshotDiffAllowSnapRootDescendant);
+        + snapshotDiffAllowSnapRootDescendant
+        + ", maxSnapshotLimit: "
+        + maxSnapshotLimit);
   }
 
   @VisibleForTesting
@@ -176,7 +182,7 @@ public class SnapshotManager implements SnapshotStatsMXBean {
 
     if (d.isSnapshottable()) {
       //The directory is already a snapshottable directory.
-      d.setSnapshotQuota(DirectorySnapshottableFeature.SNAPSHOT_LIMIT);
+      d.setSnapshotQuota(DirectorySnapshottableFeature.SNAPSHOT_QUOTA_DEFAULT);
     } else {
       d.addSnapshottableFeature();
     }
@@ -301,7 +307,7 @@ public class SnapshotManager implements SnapshotStatsMXBean {
     }
 
     srcRoot.addSnapshot(snapshotCounter, snapshotName, leaseManager,
-        this.captureOpenFiles);
+        this.captureOpenFiles, maxSnapshotLimit);
       
     //create success, update id
     snapshotCounter++;

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -4363,6 +4363,16 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.namenode.snapshot.max.limit</name>
+  <value>65536</value>
+  <description>
+    Limits the maximum number of snapshots allowed per snapshottable
+    directory.If the configuration is not set, the default limit
+    for maximum no of snapshots allowed is 65536.
+  </description>
+</property>
+
 <property>
   <name>dfs.pipeline.ecn</name>
   <value>false</value>

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSnapshotCommands.java

@@ -45,6 +45,7 @@ public class TestSnapshotCommands {
   @BeforeClass
   public static void clusterSetUp() throws IOException {
     conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_MAX_LIMIT, 3);
     cluster = new MiniDFSCluster.Builder(conf).build();
     cluster.waitActive();
     fs = cluster.getFileSystem();
@@ -114,6 +115,23 @@ public class TestSnapshotCommands {
     DFSTestUtil.FsShellRun("-ls /sub1/.snapshot/sn1", 0, "/sub1/.snapshot/sn1/sub1sub3", conf);
   }
 
+  @Test
+  public void testMaxSnapshotLimit() throws Exception {
+    DFSTestUtil.FsShellRun("-mkdir /sub3", conf);
+    DFSTestUtil.DFSAdminRun("-allowSnapshot /sub3", 0,
+        "Allowing snapshot " + "on /sub3 succeeded", conf);
+    // test createSnapshot
+    DFSTestUtil.FsShellRun("-createSnapshot /sub3 sn0", 0,
+        "Created snapshot /sub3/.snapshot/sn0", conf);
+    DFSTestUtil.FsShellRun("-createSnapshot /sub3 sn1", 0,
+        "Created snapshot /sub3/.snapshot/sn1", conf);
+    DFSTestUtil.FsShellRun("-createSnapshot /sub3 sn2", 0,
+        "Created snapshot /sub3/.snapshot/sn2", conf);
+    DFSTestUtil.FsShellRun("-createSnapshot /sub3 sn3", 1,
+        "Failed to add snapshot: there are already 3 snapshot(s) and "
+            + "the max snapshot limit is 3", conf);
+  }
+
   @Test
   public void testMkdirUsingReservedName() throws Exception {
     // test can not create dir with reserved name: .snapshot

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestNestedSnapshots.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
-import static org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature.SNAPSHOT_LIMIT;
+import static org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature.SNAPSHOT_QUOTA_DEFAULT;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
@@ -208,7 +208,7 @@ public class TestNestedSnapshots {
     hdfs.allowSnapshot(dir);
 
     int s = 0;
-    for(; s < SNAPSHOT_LIMIT; s++) {
+    for(; s < SNAPSHOT_QUOTA_DEFAULT; s++) {
       final String snapshotName = "s" + s;
       hdfs.createSnapshot(dir, snapshotName);
 
@@ -226,10 +226,10 @@ public class TestNestedSnapshots {
       SnapshotTestHelper.LOG.info("The exception is expected.", ioe);
     }
 
-    for(int f = 0; f < SNAPSHOT_LIMIT; f += step) {
+    for(int f = 0; f < SNAPSHOT_QUOTA_DEFAULT; f += step) {
       final String file = "f" + f;
       s = RANDOM.nextInt(step);
-      for(; s < SNAPSHOT_LIMIT; s += RANDOM.nextInt(step)) {
+      for(; s < SNAPSHOT_QUOTA_DEFAULT; s += RANDOM.nextInt(step)) {
         final Path p = SnapshotTestHelper.getSnapshotPath(dir, "s" + s, file);
         //the file #f exists in snapshot #s iff s > f.
         Assert.assertEquals(s > f, hdfs.exists(p));

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java

@@ -435,13 +435,13 @@ public class TestSnapshot {
     hdfs.allowSnapshot(root);
     rootNode = fsdir.getINode4Write(root.toString()).asDirectory();
     assertTrue(rootNode.isSnapshottable());
-    assertEquals(DirectorySnapshottableFeature.SNAPSHOT_LIMIT,
+    assertEquals(DirectorySnapshottableFeature.SNAPSHOT_QUOTA_DEFAULT,
         rootNode.getDirectorySnapshottableFeature().getSnapshotQuota());
     // call allowSnapshot again
     hdfs.allowSnapshot(root);
     rootNode = fsdir.getINode4Write(root.toString()).asDirectory();
     assertTrue(rootNode.isSnapshottable());
-    assertEquals(DirectorySnapshottableFeature.SNAPSHOT_LIMIT,
+    assertEquals(DirectorySnapshottableFeature.SNAPSHOT_QUOTA_DEFAULT,
         rootNode.getDirectorySnapshottableFeature().getSnapshotQuota());
     
     // disallowSnapshot on dir