Browse Source

HDFS-13173. Replace ArrayList with DirectoryDiffList(SnapshotSkipList) to store DirectoryDiffs. Contributed by Shashikant Banerjee

Tsz-Wo Nicholas Sze 7 years ago
parent
commit
ba82e5c488

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

@@ -419,6 +419,15 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.namenode.snapshot.max.limit";
 
   public static final int DFS_NAMENODE_SNAPSHOT_MAX_LIMIT_DEFAULT = 65536;
+  public static final String DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL =
+      "dfs.namenode.snapshot.skiplist.interval";
+  public static final int DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL_DEFAULT =
+      10;
+  public static final String DFS_NAMENODE_SNAPSHOT_SKIPLIST_MAX_LEVELS =
+      "dfs.namenode.snapshot.skiplist.max.levels";
+  public static final int
+      DFS_NAMENODE_SNAPSHOT_SKIPLIST_MAX_SKIP_LEVELS_DEFAULT = 0;
+
   // 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;

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

@@ -138,10 +138,14 @@ abstract class AbstractINodeDiffList<N extends INode,
     return n == 0 ? null : diffs.get(n - 1);
   }
 
+  DiffList<D> newDiffs() {
+    return new DiffListByArrayList<>(
+        INodeDirectory.DEFAULT_FILES_PER_DIRECTORY);
+  }
+
   private void createDiffsIfNeeded() {
     if (diffs == null) {
-      diffs =
-          new DiffListByArrayList<>(INodeDirectory.DEFAULT_FILES_PER_DIRECTORY);
+      diffs = newDiffs();
     }
   }
 

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryDiffList.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DiffListBySkipList.java

@@ -68,9 +68,9 @@ import java.util.concurrent.ThreadLocalRandom;
  * <p>
  * Once a snapshot gets deleted, the list needs to be balanced.
  */
-public class DirectoryDiffList implements DiffList<DirectoryDiff> {
+public class DiffListBySkipList implements DiffList<DirectoryDiff> {
   public static final Logger LOG =
-      LoggerFactory.getLogger(DirectoryDiffList.class);
+      LoggerFactory.getLogger(DiffListBySkipList.class);
 
   private static class SkipDiff {
     /**
@@ -240,7 +240,7 @@ public class DirectoryDiffList implements DiffList<DirectoryDiff> {
   /**
    * Constructs a new, empty instance of SkipList.
    */
-  public DirectoryDiffList(int capacity, int interval, int skipLevel) {
+  public DiffListBySkipList(int capacity, int interval, int skipLevel) {
     skipNodeList = new ArrayList<>(capacity);
     head = new SkipListNode(null, 0);
     this.maxSkipLevels = skipLevel;

+ 46 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryDiffListFactory.java

@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.snapshot;
+
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
+
+import java.util.function.IntFunction;
+
+/** For creating {@link DiffList} for {@link DirectoryDiff}. */
+public abstract class DirectoryDiffListFactory {
+  public static DiffList<DirectoryDiff> createDiffList(int capacity) {
+    return constructor.apply(capacity);
+  }
+
+  public static void init(int skipInterval, int maxLevels, Log log) {
+    if (maxLevels > 0) {
+      constructor = c -> new DiffListBySkipList(c, skipInterval, maxLevels);
+      log.info("SkipList is enabled with skipInterval=" + skipInterval
+          + ", maxLevels=" + maxLevels);
+    } else {
+      constructor = c -> new DiffListByArrayList<>(c);
+      log.info("SkipList is disabled");
+    }
+  }
+
+  private static volatile IntFunction<DiffList<DirectoryDiff>> constructor
+      = c -> new DiffListByArrayList<>(c);
+
+  private DirectoryDiffListFactory() {}
+}

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java

@@ -334,6 +334,12 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
         : new INodeDirectoryAttributes.SnapshotCopy(currentDir);
     }
 
+    @Override
+    DiffList<DirectoryDiff> newDiffs() {
+      return DirectoryDiffListFactory
+          .createDiffList(INodeDirectory.DEFAULT_FILES_PER_DIRECTORY);
+    }
+
     /** Replace the given child in the created/deleted list, if there is any. */
     public boolean replaceChild(final ListType type, final INode oldChild,
         final INode newChild) {

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

@@ -127,6 +127,14 @@ public class SnapshotManager implements SnapshotStatsMXBean {
         + snapshotDiffAllowSnapRootDescendant
         + ", maxSnapshotLimit: "
         + maxSnapshotLimit);
+
+    final int maxLevels = conf.getInt(
+        DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_SKIPLIST_MAX_LEVELS,
+        DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_SKIPLIST_MAX_SKIP_LEVELS_DEFAULT);
+    final int skipInterval = conf.getInt(
+        DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL,
+        DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL_DEFAULT);
+    DirectoryDiffListFactory.init(skipInterval, maxLevels, LOG);
   }
 
   @VisibleForTesting

+ 18 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -4362,7 +4362,6 @@
     across to the client within one rpc call.
   </description>
 </property>
-
 <property>
   <name>dfs.namenode.snapshot.max.limit</name>
   <value>65536</value>
@@ -4373,6 +4372,24 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.namenode.snapshot.skiplist.max.levels</name>
+  <value>0</value>
+  <description>
+    Maximum no of the skip levels to be maintained in the skip list for
+    storing directory snapshot diffs. By default, it is set to 0 and a linear
+    list will be used to store the directory snapshot diffs.
+  </description>
+</property>
+<property>
+  <name>dfs.namenode.snapshot.skiplist.interval</name>
+  <value>10</value>
+  <description>
+    The interval after which the skip levels will be formed in the skip list
+    for storing directory snapshot diffs. By default, value is set to 10.
+  </description>
+</property>
+
 <property>
   <name>dfs.pipeline.ecn</name>
   <value>false</value>

+ 13 - 13
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestDirectoryDiffList.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestDiffListBySkipList.java

@@ -41,7 +41,7 @@ import java.util.function.IntFunction;
 /**
  * This class tests the DirectoryDiffList API's.
  */
-public class TestDirectoryDiffList{
+public class TestDiffListBySkipList {
   static final int NUM_SNAPSHOTS = 100;
   static {
     SnapshotTestHelper.disableLogs();
@@ -78,7 +78,7 @@ public class TestDirectoryDiffList{
     }
   }
 
-  static void verifyChildrenList(DirectoryDiffList skip, INodeDirectory dir) {
+  static void verifyChildrenList(DiffListBySkipList skip, INodeDirectory dir) {
     final int n = skip.size();
     for (int i = 0; i < skip.size(); i++) {
       final List<INode> expected = ReadOnlyList.Util.asList(
@@ -95,7 +95,7 @@ public class TestDirectoryDiffList{
   }
 
   static void verifyChildrenList(
-      DiffList<DirectoryDiff> array, DirectoryDiffList skip,
+      DiffList<DirectoryDiff> array, DiffListBySkipList skip,
       INodeDirectory dir, List<INode> childrenList) {
     final int n = array.size();
     Assert.assertEquals(n, skip.size());
@@ -144,13 +144,13 @@ public class TestDirectoryDiffList{
 
   static void testAddLast(int n) throws Exception {
     final Path root = new Path("/testAddLast" + n);
-    DirectoryDiffList.LOG.info("run " + root);
+    DiffListBySkipList.LOG.info("run " + root);
 
-    final DirectoryDiffList skipList = new DirectoryDiffList(0, 3, 5);
+    final DiffListBySkipList skipList = new DiffListBySkipList(0, 3, 5);
     final DiffList<DirectoryDiff> arrayList = new DiffListByArrayList<>(0);
     INodeDirectory dir = addDiff(n, skipList, arrayList, root);
     // verify that the both the children list obtained from hdfs and
-    // DirectoryDiffList are same
+    // DiffListBySkipList are same
     verifyChildrenList(skipList, dir);
     verifyChildrenList(arrayList, skipList, dir, Collections.emptyList());
   }
@@ -163,7 +163,7 @@ public class TestDirectoryDiffList{
 
   static void testAddFirst(int n) throws Exception {
     final Path root = new Path("/testAddFirst" + n);
-    DirectoryDiffList.LOG.info("run " + root);
+    DiffListBySkipList.LOG.info("run " + root);
 
     hdfs.mkdirs(root);
     for (int i = 1; i < n; i++) {
@@ -180,7 +180,7 @@ public class TestDirectoryDiffList{
     DiffList<DirectoryDiff> diffs = dir.getDiffs().asList();
     List<INode> childrenList = ReadOnlyList.Util.asList(dir.getChildrenList(
         diffs.get(0).getSnapshotId()));
-    final DirectoryDiffList skipList = new DirectoryDiffList(0, 3, 5);
+    final DiffListBySkipList skipList = new DiffListBySkipList(0, 3, 5);
     final DiffList<DirectoryDiff> arrayList = new DiffListByArrayList<>(0);
     for (int i = diffs.size() - 1; i >= 0; i--) {
       final DirectoryDiff d = diffs.get(i);
@@ -188,7 +188,7 @@ public class TestDirectoryDiffList{
       arrayList.addFirst(d);
     }
     // verify that the both the children list obtained from hdfs and
-    // DirectoryDiffList are same
+    // DiffListBySkipList are same
     verifyChildrenList(skipList, dir);
     verifyChildrenList(arrayList, skipList, dir, childrenList);
   }
@@ -231,9 +231,9 @@ public class TestDirectoryDiffList{
   static void testRemove(String name, int n, IntFunction<Integer> indexFunction)
       throws Exception {
     final Path root = new Path("/testRemove" + name + n);
-    DirectoryDiffList.LOG.info("run " + root);
+    DiffListBySkipList.LOG.info("run " + root);
 
-    final DirectoryDiffList skipList = new DirectoryDiffList(0, 3, 5);
+    final DiffListBySkipList skipList = new DiffListBySkipList(0, 3, 5);
     final DiffList<DirectoryDiff> arrayList = new DiffListByArrayList<>(0);
     final INodeDirectory dir = addDiff(n, skipList, arrayList, root);
     Assert.assertEquals(n, arrayList.size());
@@ -248,9 +248,9 @@ public class TestDirectoryDiffList{
     }
   }
 
-  static DirectoryDiff remove(int i, DirectoryDiffList skip,
+  static DirectoryDiff remove(int i, DiffListBySkipList skip,
       DiffList<DirectoryDiff> array) {
-    DirectoryDiffList.LOG.info("remove " + i);
+    DiffListBySkipList.LOG.info("remove " + i);
     final DirectoryDiff expected = array.remove(i);
     final DirectoryDiff computed = skip.remove(i);
     assertDirectoryDiff(expected, computed);