瀏覽代碼

HDFS-13171. Handle Deletion of nodes in SnasphotSkipList. Contributed by Shashikant Banerjee

Tsz-Wo Nicholas Sze 7 年之前
父節點
當前提交
60080fbfcf

+ 67 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryDiffList.java

@@ -22,6 +22,8 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.
     DirectoryWithSnapshotFeature.DirectoryDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.
     DirectoryWithSnapshotFeature.ChildrenDiff;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.List;
 import java.util.ArrayList;
@@ -67,6 +69,8 @@ import java.util.concurrent.ThreadLocalRandom;
  * Once a snapshot gets deleted, the list needs to be balanced.
  */
 public class DirectoryDiffList implements DiffList<DirectoryDiff> {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(DirectoryDiffList.class);
 
   private static class SkipDiff {
     /**
@@ -100,7 +104,7 @@ public class DirectoryDiffList implements DiffList<DirectoryDiff> {
 
     @Override
     public String toString() {
-      return "->" + skipTo;
+      return "->" + skipTo + (diff == null? " (diff==null)": "");
     }
   }
 
@@ -138,6 +142,13 @@ public class DirectoryDiffList implements DiffList<DirectoryDiff> {
       return skipDiffList.size() - 1;
     }
 
+    void trim() {
+      for (int level = level();
+           level > 0 && getSkipNode(level) == null; level--) {
+        skipDiffList.remove(level);
+      }
+    }
+
     public DirectoryDiff getDiff() {
       return diff;
     }
@@ -281,24 +292,33 @@ public class DirectoryDiffList implements DiffList<DirectoryDiff> {
     skipNodeList.add(0, newNode);
   }
 
-  /**
-   * Adds the specified data element to the end of the SkipList,
-   * if the element is not already present.
-   * @param diff the element to be inserted
-   */
-  @Override
-  public boolean addLast(DirectoryDiff diff) {
-    final int nodeLevel = randomLevel(skipInterval, maxSkipLevels);
+  private SkipListNode[] findPreviousNodes(SkipListNode node, int nodeLevel) {
     final SkipListNode[] nodePath = new SkipListNode[nodeLevel + 1];
     SkipListNode cur = head;
     final int headLevel = head.level();
     for (int level = headLevel < nodeLevel ? headLevel : nodeLevel;
          level >= 0; level--) {
-      while (cur.getSkipNode(level) != null) {
+      while (cur.getSkipNode(level) != node) {
         cur = cur.getSkipNode(level);
       }
       nodePath[level] = cur;
     }
+    for (int level = headLevel + 1; level <= nodeLevel; level++) {
+      nodePath[level] = head;
+    }
+    return nodePath;
+  }
+
+  /**
+   * Adds the specified data element to the end of the SkipList,
+   * if the element is not already present.
+   * @param diff the element to be inserted
+   */
+  @Override
+  public boolean addLast(DirectoryDiff diff) {
+    final int nodeLevel = randomLevel(skipInterval, maxSkipLevels);
+    final int headLevel = head.level();
+    final SkipListNode[] nodePath = findPreviousNodes(null, nodeLevel);
     for (int level = headLevel + 1; level <= nodeLevel; level++) {
       head.skipDiffList.add(new SkipDiff(null));
       nodePath[level] = head;
@@ -327,6 +347,7 @@ public class DirectoryDiffList implements DiffList<DirectoryDiff> {
         }
       }
       nodePath[level].setSkipTo(current, level);
+      current.setSkipTo(null, level);
     }
     return skipNodeList.add(current);
   }
@@ -366,11 +387,44 @@ public class DirectoryDiffList implements DiffList<DirectoryDiff> {
    * Removes the element at the specified position in this list.
    *
    * @param index the index of the element to be removed
-   * @throws UnsupportedOperationException {@inheritDoc}
+   * @return the removed DirectoryDiff
    */
   @Override
   public DirectoryDiff remove(int index) {
-    throw new UnsupportedOperationException();
+    SkipListNode node = getNode(index);
+    int headLevel = head.level();
+    int nodeLevel = node.level();
+    final SkipListNode[] nodePath = findPreviousNodes(node, nodeLevel);
+    for (int level = 0; level <= nodeLevel; level++) {
+      if (nodePath[level] != head && level > 0) {
+        // if the last snapshot is deleted, for all the skip level nodes
+        // pointing to the last one, the combined children diff at each level
+        // > 0 should be made null and skip pointers will be updated to null.
+        // if the snapshot being deleted is not the last one, we have to merge
+        // the diff of deleted node at each level to the previous skip level
+        // node at that level and the skip pointers will be updated to point to
+        // the skip nodes of the deleted node.
+        if (index == size() - 1) {
+          nodePath[level].setSkipDiff(null, level);
+        } else {
+          /* Ideally at level 0, the deleted diff will be combined with
+           * the previous diff , and deleted inodes will be cleaned up
+           * by passing a deleted processor here while combining the diffs.
+           * Level 0 merge with previous diff will be handled inside the
+           * {@link AbstractINodeDiffList#deleteSnapshotDiff} function.
+           */
+          if (node.getChildrenDiff(level) != null) {
+            nodePath[level].getChildrenDiff(level)
+                .combinePosterior(node.getChildrenDiff(level), null);
+          }
+        }
+      }
+      nodePath[level].setSkipTo(node.getSkipNode(level), level);
+    }
+    if (nodeLevel == headLevel) {
+      head.trim();
+    }
+    return skipNodeList.remove(index).getDiff();
   }
 
   /**
@@ -482,7 +536,7 @@ public class DirectoryDiffList implements DiffList<DirectoryDiff> {
   @Override
   public String toString() {
     final StringBuilder b = new StringBuilder(getClass().getSimpleName());
-    b.append("\nhead: ").append(head).append(head.skipDiffList);
+    b.append(" head: ").append(head).append(head.skipDiffList);
     for (SkipListNode n : skipNodeList) {
       b.append("\n  ").append(n).append(n.skipDiffList);
     }

+ 82 - 20
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestDirectoryDiffList.java

@@ -35,11 +35,14 @@ import org.junit.Test;
 
 import java.util.Collections;
 import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.IntFunction;
 
 /**
  * This class tests the DirectoryDiffList API's.
  */
 public class TestDirectoryDiffList{
+  static final int NUM_SNAPSHOTS = 100;
   static {
     SnapshotTestHelper.disableLogs();
   }
@@ -78,15 +81,15 @@ public class TestDirectoryDiffList{
   static void verifyChildrenList(DirectoryDiffList skip, INodeDirectory dir) {
     final int n = skip.size();
     for (int i = 0; i < skip.size(); i++) {
-      final List<INode> expected =
-          ReadOnlyList.Util.asList(dir.getChildrenList(i));
+      final List<INode> expected = ReadOnlyList.Util.asList(
+          dir.getChildrenList(dir.getDiffs().asList().get(i).getSnapshotId()));
       final List<INode> computed = getChildrenList(skip, i, n, dir);
       try {
         assertList(expected, computed);
       } catch (AssertionError ae) {
         throw new AssertionError(
             "i = " + i + "\ncomputed = " + computed + "\nexpected = "
-                + expected, ae);
+                + expected + "\n" + skip, ae);
       }
     }
   }
@@ -136,28 +139,16 @@ public class TestDirectoryDiffList{
 
   @Test
   public void testAddLast() throws Exception {
-    testAddLast(7);
+    testAddLast(NUM_SNAPSHOTS);
   }
 
   static void testAddLast(int n) throws Exception {
     final Path root = new Path("/testAddLast" + n);
-    hdfs.mkdirs(root);
-    SnapshotTestHelper.createSnapshot(hdfs, root, "s0");
-    for (int i = 1; i < n; i++) {
-      final Path child = getChildPath(root, i);
-      hdfs.mkdirs(child);
-      SnapshotTestHelper.createSnapshot(hdfs, root, "s" + i);
-    }
-    INodeDirectory dir = fsdir.getINode(root.toString()).asDirectory();
-    DiffList<DirectoryDiff> diffs = dir.getDiffs().asList();
+    DirectoryDiffList.LOG.info("run " + root);
 
     final DirectoryDiffList skipList = new DirectoryDiffList(0, 3, 5);
     final DiffList<DirectoryDiff> arrayList = new DiffListByArrayList<>(0);
-    for (DirectoryDiff d : diffs) {
-      skipList.addLast(d);
-      arrayList.addLast(d);
-    }
-
+    INodeDirectory dir = addDiff(n, skipList, arrayList, root);
     // verify that the both the children list obtained from hdfs and
     // DirectoryDiffList are same
     verifyChildrenList(skipList, dir);
@@ -167,11 +158,13 @@ public class TestDirectoryDiffList{
 
   @Test
   public void testAddFirst() throws Exception {
-    testAddFirst(7);
+    testAddFirst(NUM_SNAPSHOTS);
   }
 
   static void testAddFirst(int n) throws Exception {
     final Path root = new Path("/testAddFirst" + n);
+    DirectoryDiffList.LOG.info("run " + root);
+
     hdfs.mkdirs(root);
     for (int i = 1; i < n; i++) {
       final Path child = getChildPath(root, i);
@@ -182,7 +175,7 @@ public class TestDirectoryDiffList{
     for (int i = 1; i < n; i++) {
       final Path child = getChildPath(root, n - i);
       hdfs.delete(child, false);
-      SnapshotTestHelper.createSnapshot(hdfs, root, "s" + i);
+      hdfs.createSnapshot(root, "s" + i);
     }
     DiffList<DirectoryDiff> diffs = dir.getDiffs().asList();
     List<INode> childrenList = ReadOnlyList.Util.asList(dir.getChildrenList(
@@ -199,4 +192,73 @@ public class TestDirectoryDiffList{
     verifyChildrenList(skipList, dir);
     verifyChildrenList(arrayList, skipList, dir, childrenList);
   }
+
+  static INodeDirectory addDiff(int n, DiffList skipList, DiffList arrayList,
+      final Path root) throws Exception {
+    hdfs.mkdirs(root);
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s0");
+    for (int i = 1; i < n; i++) {
+      final Path child = getChildPath(root, i);
+      hdfs.mkdirs(child);
+      hdfs.createSnapshot(root, "s" + i);
+    }
+    INodeDirectory dir = fsdir.getINode(root.toString()).asDirectory();
+    DiffList<DirectoryDiff> diffs = dir.getDiffs().asList();
+    for (DirectoryDiff d : diffs) {
+      skipList.addLast(d);
+      arrayList.addLast(d);
+    }
+    return dir;
+  }
+
+  @Test
+  public void testRemoveFromTail() throws Exception {
+    final int n = NUM_SNAPSHOTS;
+    testRemove("FromTail", n, i -> n - 1 - i);
+  }
+
+  @Test
+  public void testReomveFromHead() throws Exception {
+    testRemove("FromHead", NUM_SNAPSHOTS, i -> 0);
+  }
+
+  @Test
+  public void testRemoveRandom() throws Exception {
+    final int n = NUM_SNAPSHOTS;
+    testRemove("Random", n, i -> ThreadLocalRandom.current().nextInt(n - i));
+  }
+
+  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);
+
+    final DirectoryDiffList skipList = new DirectoryDiffList(0, 3, 5);
+    final DiffList<DirectoryDiff> arrayList = new DiffListByArrayList<>(0);
+    final INodeDirectory dir = addDiff(n, skipList, arrayList, root);
+    Assert.assertEquals(n, arrayList.size());
+    Assert.assertEquals(n, skipList.size());
+
+    for(int i = 0; i < n; i++) {
+      final int index = indexFunction.apply(i);
+      final DirectoryDiff diff = remove(index, skipList, arrayList);
+      hdfs.deleteSnapshot(root, "s" + diff.getSnapshotId());
+      verifyChildrenList(skipList, dir);
+      verifyChildrenList(arrayList, skipList, dir, Collections.emptyList());
+    }
+  }
+
+  static DirectoryDiff remove(int i, DirectoryDiffList skip,
+      DiffList<DirectoryDiff> array) {
+    DirectoryDiffList.LOG.info("remove " + i);
+    final DirectoryDiff expected = array.remove(i);
+    final DirectoryDiff computed = skip.remove(i);
+    assertDirectoryDiff(expected, computed);
+    return expected;
+  }
+
+  static void assertDirectoryDiff(DirectoryDiff expected,
+      DirectoryDiff computed) {
+    Assert.assertEquals(expected.getSnapshotId(), computed.getSnapshotId());
+  }
 }