Browse Source

HDFS-4447. Refactor INodeDirectoryWithSnapshot for supporting general INode diff lists.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1440216 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 years ago
parent
commit
6f4fe09687
12 changed files with 406 additions and 300 deletions
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt
  2. 0 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  3. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  4. 117 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
  5. 150 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
  6. 10 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectorySnapshottable.java
  7. 84 217
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java
  8. 12 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java
  9. 2 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/diff/Diff.java
  10. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java
  11. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotRename.java
  12. 19 23
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/diff/TestDiff.java

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt

@@ -133,3 +133,6 @@ Branch-2802 Snapshot (Unreleased)
 
   HDFS-4131. Add capability to namenode to get snapshot diff. (Jing Zhao via
   suresh)
+
+  HDFS-4447. Refactor INodeDirectoryWithSnapshot for support general INode diff
+  lists.  (szetszwo)

+ 0 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -42,7 +42,6 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileUnderConstructio
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.diff.Diff;
-import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.util.StringUtils;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -58,20 +57,6 @@ import com.google.common.primitives.SignedBytes;
 public abstract class INode implements Diff.Element<byte[]> {
   public static final Log LOG = LogFactory.getLog(INode.class);
 
-  static final ReadOnlyList<INode> EMPTY_READ_ONLY_LIST
-      = ReadOnlyList.Util.emptyList();
-  
-  /**
-   * Assert that the snapshot parameter must be null since this class only take
-   * care current state. Subclasses should override the methods for handling the
-   * snapshot states.
-   */
-  static void assertNull(Snapshot snapshot) {
-    if (snapshot != null) {
-      throw new AssertionError("snapshot is not null: " + snapshot);
-    }
-  }
-
   /** A pair of objects. */
   public static class Pair<L, R> {
     public final L left;

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

@@ -599,7 +599,7 @@ public class INodeDirectory extends INode {
    *         Note that the returned list is never null.
    */
   public ReadOnlyList<INode> getChildrenList(final Snapshot snapshot) {
-    return children == null ? EMPTY_READ_ONLY_LIST
+    return children == null ? ReadOnlyList.Util.<INode>emptyList()
         : ReadOnlyList.Util.asReadOnlyList(children);
   }
 

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

@@ -0,0 +1,117 @@
+/**
+ * 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.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * The difference of an inode between in two snapshots.
+ * {@link AbstractINodeDiff2} maintains a list of snapshot diffs,
+ * <pre>
+ *   d_1 -> d_2 -> ... -> d_n -> null,
+ * </pre>
+ * where -> denotes the {@link AbstractINodeDiff#posteriorDiff} reference. The
+ * current directory state is stored in the field of {@link INode}.
+ * The snapshot state can be obtained by applying the diffs one-by-one in
+ * reversed chronological order.  Let s_1, s_2, ..., s_n be the corresponding
+ * snapshots.  Then,
+ * <pre>
+ *   s_n                     = (current state) - d_n;
+ *   s_{n-1} = s_n - d_{n-1} = (current state) - d_n - d_{n-1};
+ *   ...
+ *   s_k     = s_{k+1} - d_k = (current state) - d_n - d_{n-1} - ... - d_k.
+ * </pre>
+ */
+abstract class AbstractINodeDiff<N extends INode,
+                                 D extends AbstractINodeDiff<N, D>>
+    implements Comparable<Snapshot> {
+  /** The snapshot will be obtained after this diff is applied. */
+  final Snapshot snapshot;
+  /** The snapshot inode data.  It is null when there is no change. */
+  N snapshotINode;
+  /**
+   * Posterior diff is the diff happened after this diff.
+   * The posterior diff should be first applied to obtain the posterior
+   * snapshot and then apply this diff in order to obtain this snapshot.
+   * If the posterior diff is null, the posterior state is the current state. 
+   */
+  private D posteriorDiff;
+
+  AbstractINodeDiff(Snapshot snapshot, N snapshotINode, D posteriorDiff) {
+    Preconditions.checkNotNull(snapshot, "snapshot is null");
+
+    this.snapshot = snapshot;
+    this.snapshotINode = snapshotINode;
+    this.posteriorDiff = posteriorDiff;
+  }
+
+  /** Compare diffs with snapshot ID. */
+  @Override
+  public final int compareTo(final Snapshot that) {
+    return Snapshot.ID_COMPARATOR.compare(this.snapshot, that);
+  }
+
+  /** @return the snapshot object of this diff. */
+  final Snapshot getSnapshot() {
+    return snapshot;
+  }
+
+  /** @return the posterior diff. */
+  final D getPosterior() {
+    return posteriorDiff;
+  }
+
+  /** @return the posterior diff. */
+  final void setPosterior(D posterior) {
+    posteriorDiff = posterior;
+  }
+
+  /** Copy the INode state to the snapshot if it is not done already. */
+  void checkAndInitINode(N snapshotCopy) {
+    if (snapshotINode == null) {
+      if (snapshotCopy == null) {
+        @SuppressWarnings("unchecked")
+        final N right = (N)getCurrentINode().createSnapshotCopy().right;
+        snapshotCopy = right;
+      }
+      snapshotINode = snapshotCopy;
+    }
+  }
+
+  /** @return the current inode. */
+  abstract N getCurrentINode();
+
+  /** @return the inode corresponding to the snapshot. */
+  N getSnapshotINode() {
+    // get from this diff, then the posterior diff and then the current inode
+    for(AbstractINodeDiff<N, D> d = this; ; d = d.posteriorDiff) {
+      if (d.snapshotINode != null) {
+        return d.snapshotINode;
+      } else if (d.posteriorDiff == null) {
+        return getCurrentINode();
+      }
+    }
+  }
+
+  /** Combine the posterior diff and collect blocks for deletion. */
+  abstract void combinePosteriorAndCollectBlocks(final D posterior,
+      final BlocksMapUpdateInfo collectedBlocks);
+}

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

@@ -0,0 +1,150 @@
+/**
+ * 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 java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
+
+/**
+ * A list of snapshot diffs for storing snapshot data.
+ *
+ * @param <N> The {@link INode} type.
+ * @param <D> The diff type, which must extend {@link AbstractINodeDiff}.
+ */
+abstract class AbstractINodeDiffList<N extends INode,
+                                     D extends AbstractINodeDiff<N, D>> 
+    implements Iterable<D> {
+  /** Diff list sorted by snapshot IDs, i.e. in chronological order. */
+  private final List<D> diffs = new ArrayList<D>();
+
+  /** @return this list as a unmodifiable {@link List}. */
+  final List<D> asList() {
+    return Collections.unmodifiableList(diffs);
+  }
+
+  /** @return the current inode. */
+  abstract N getCurrentINode();
+  
+  /** Add a {@link AbstractINodeDiff} for the given snapshot and inode. */
+  abstract D addSnapshotDiff(Snapshot snapshot, N inode, boolean isSnapshotCreation); 
+
+  /**
+   * Delete the snapshot with the given name. The synchronization of the diff
+   * list will be done outside.
+   * 
+   * If the diff to remove is not the first one in the diff list, we need to 
+   * combine the diff with its previous one:
+   * 
+   * @param snapshot The snapshot to be deleted
+   * @param collectedBlocks Used to collect information for blocksMap update
+   * @return The SnapshotDiff containing the deleted snapshot. 
+   *         Null if the snapshot with the given name does not exist. 
+   */
+  final AbstractINodeDiff<N, D> deleteSnapshotDiff(Snapshot snapshot,
+      final BlocksMapUpdateInfo collectedBlocks) {
+    int snapshotIndex = Collections.binarySearch(diffs, snapshot);
+    if (snapshotIndex < 0) {
+      return null;
+    } else {
+      final D removed = diffs.remove(snapshotIndex);
+      if (snapshotIndex > 0) {
+        // combine the to-be-removed diff with its previous diff
+        final AbstractINodeDiff<N, D> previous = diffs.get(snapshotIndex - 1);
+        previous.combinePosteriorAndCollectBlocks(removed, collectedBlocks);
+        previous.setPosterior(removed.getPosterior());
+      }
+      removed.setPosterior(null);
+      return removed;
+    }
+  }
+
+  /** Append the diff at the end of the list. */
+  final D append(D diff) {
+    final AbstractINodeDiff<N, D> last = getLast();
+    diffs.add(diff);
+    if (last != null) {
+      last.setPosterior(diff);
+    }
+    return diff;
+  }
+  
+  /** Insert the diff to the beginning of the list. */
+  final void insert(D diff) {
+    diffs.add(0, diff);
+  }
+  
+  /** @return the last diff. */
+  final D getLast() {
+    final int n = diffs.size();
+    return n == 0? null: diffs.get(n - 1);
+  }
+
+  /** @return the last snapshot. */
+  final Snapshot getLastSnapshot() {
+    final AbstractINodeDiff<N, D> last = getLast();
+    return last == null? null: last.getSnapshot();
+  }
+
+  /**
+   * @return the diff corresponding to the given snapshot.
+   *         When the diff is null, it means that the current state and
+   *         the corresponding snapshot state are the same. 
+   */
+  final D getDiff(Snapshot snapshot) {
+    if (snapshot == null) {
+      // snapshot == null means the current state, therefore, return null.
+      return null;
+    }
+    final int i = Collections.binarySearch(diffs, snapshot);
+    if (i >= 0) {
+      // exact match
+      return diffs.get(i);
+    } else {
+      // Exact match not found means that there were no changes between
+      // given snapshot and the next state so that the diff for the given
+      // snapshot was not recorded.  Thus, return the next state.
+      final int j = -i - 1;
+      return j < diffs.size()? diffs.get(j): null;
+    }
+  }
+  
+  /**
+   * Check if the latest snapshot diff exists.  If not, add it.
+   * @return the latest snapshot diff, which is never null.
+   */
+  final D checkAndAddLatestSnapshotDiff(Snapshot latest) {
+    final D last = getLast();
+    return last != null && last.snapshot.equals(latest)? last
+        : addSnapshotDiff(latest, getCurrentINode(), false);
+  }
+  
+  @Override
+  public Iterator<D> iterator() {
+    return diffs.iterator();
+  }
+
+  @Override
+  public String toString() {
+    return "diffs=" + diffs;
+  }
+}

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

@@ -150,7 +150,7 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
 
   public INodeDirectorySnapshottable(INodeDirectory dir) {
     super(dir, true, dir instanceof INodeDirectoryWithSnapshot ? 
-        ((INodeDirectoryWithSnapshot) dir).getSnapshotDiffs() : null);
+        ((INodeDirectoryWithSnapshot) dir).getDiffs(): null);
   }
   
   /** @return the number of existing snapshots. */
@@ -252,7 +252,7 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
           + "snapshot with the same name \"" + name + "\".");
     }
 
-    addSnapshotDiff(s, this, true);
+    getDiffs().addSnapshotDiff(s, this, true);
     snapshotsByNames.add(-i - 1, s);
 
     //set modification time
@@ -264,7 +264,7 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
   
   /**
    * Remove the snapshot with the given name from {@link #snapshotsByNames},
-   * and delete all the corresponding SnapshotDiff.
+   * and delete all the corresponding DirectoryDiff.
    * 
    * @param snapshotName The name of the snapshot to be removed
    * @param collectedBlocks Used to collect information to update blocksMap
@@ -286,14 +286,14 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
   }
   
   /**
-   * Recursively delete SnapshotDiff associated with the given snapshot under a
+   * Recursively delete DirectoryDiff associated with the given snapshot under a
    * directory
    */
   private void deleteDiffsForSnapshot(Snapshot snapshot, INodeDirectory dir,
       BlocksMapUpdateInfo collectedBlocks) {
     if (dir instanceof INodeDirectoryWithSnapshot) {
       INodeDirectoryWithSnapshot sdir = (INodeDirectoryWithSnapshot) dir;
-      sdir.deleteSnapshotDiff(snapshot, collectedBlocks);
+      sdir.getDiffs().deleteSnapshotDiff(snapshot, collectedBlocks);
     }
     ReadOnlyList<INode> children = dir.getChildrenList(null);
     for (INode child : children) {
@@ -401,7 +401,7 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
       out.println();
       out.print(prefix);
       int n = 0;
-      for(SnapshotDiff diff : getSnapshotDiffs()) {
+      for(DirectoryDiff diff : getDiffs()) {
         if (diff.isSnapshotRoot()) {
           n++;
         }
@@ -415,12 +415,12 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
         @Override
         public Iterator<Pair<? extends INode, Snapshot>> iterator() {
           return new Iterator<Pair<? extends INode, Snapshot>>() {
-            final Iterator<SnapshotDiff> i = getSnapshotDiffs().iterator();
-            private SnapshotDiff next = findNext();
+            final Iterator<DirectoryDiff> i = getDiffs().iterator();
+            private DirectoryDiff next = findNext();
   
-            private SnapshotDiff findNext() {
+            private DirectoryDiff findNext() {
               for(; i.hasNext(); ) {
-                final SnapshotDiff diff = i.next();
+                final DirectoryDiff diff = i.next();
                 if (diff.isSnapshotRoot()) {
                   return diff;
                 }

+ 84 - 217
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java

@@ -50,7 +50,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
    * The difference between the current state and a previous snapshot
    * of the children list of an INodeDirectory.
    */
-  public static class ChildrenDiff extends Diff<byte[], INode> {
+  static class ChildrenDiff extends Diff<byte[], INode> {
     ChildrenDiff() {}
     
     private ChildrenDiff(final List<INode> created, final List<INode> deleted) {
@@ -173,100 +173,56 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
   }
   
   /**
-   * The difference between two snapshots. {@link INodeDirectoryWithSnapshot}
-   * maintains a list of snapshot diffs,
-   * <pre>
-   *   d_1 -> d_2 -> ... -> d_n -> null,
-   * </pre>
-   * where -> denotes the {@link SnapshotDiff#posteriorDiff} reference. The
-   * current directory state is stored in the field of {@link INodeDirectory}.
-   * The snapshot state can be obtained by applying the diffs one-by-one in
-   * reversed chronological order.  Let s_1, s_2, ..., s_n be the corresponding
-   * snapshots.  Then,
-   * <pre>
-   *   s_n                     = (current state) - d_n;
-   *   s_{n-1} = s_n - d_{n-1} = (current state) - d_n - d_{n-1};
-   *   ...
-   *   s_k     = s_{k+1} - d_k = (current state) - d_n - d_{n-1} - ... - d_k.
-   * </pre>
+   * The difference of an {@link INodeDirectory} between two snapshots.
    */
-  public class SnapshotDiff implements Comparable<Snapshot> {
-    /** The snapshot will be obtained after this diff is applied. */
-    final Snapshot snapshot;
+  class DirectoryDiff extends AbstractINodeDiff<INodeDirectory, DirectoryDiff> {
     /** The size of the children list at snapshot creation time. */
-    final int childrenSize;
-    /**
-     * Posterior diff is the diff happened after this diff.
-     * The posterior diff should be first applied to obtain the posterior
-     * snapshot and then apply this diff in order to obtain this snapshot.
-     * If the posterior diff is null, the posterior state is the current state. 
-     */
-    private SnapshotDiff posteriorDiff;
+    private final int childrenSize;
     /** The children list diff. */
     private final ChildrenDiff diff;
-    /** The snapshot inode data.  It is null when there is no change. */
-    private INodeDirectory snapshotINode = null;
 
-    private SnapshotDiff(Snapshot snapshot, INodeDirectory dir) {
-      Preconditions.checkNotNull(snapshot, "snapshot is null");
+    private DirectoryDiff(Snapshot snapshot, INodeDirectory dir) {
+      super(snapshot, null, null);
 
-      this.snapshot = snapshot;
       this.childrenSize = dir.getChildrenList(null).size();
       this.diff = new ChildrenDiff();
     }
 
     /** Constructor used by FSImage loading */
-    SnapshotDiff(Snapshot snapshot,
-        int childrenSize, INodeDirectory snapshotINode,
-        SnapshotDiff posteriorDiff, List<INode> createdList,
-        List<INode> deletedList) {
-      this.snapshot = snapshot;
+    DirectoryDiff(Snapshot snapshot, INodeDirectory snapshotINode,
+        DirectoryDiff posteriorDiff, int childrenSize,
+        List<INode> createdList, List<INode> deletedList) {
+      super(snapshot, snapshotINode, posteriorDiff);
       this.childrenSize = childrenSize;
-      this.snapshotINode = snapshotINode;
-      this.posteriorDiff = posteriorDiff;
       this.diff = new ChildrenDiff(createdList, deletedList);
     }
     
-    ChildrenDiff getDiff() {
+    ChildrenDiff getChildrenDiff() {
       return diff;
     }
-
-    /** Compare diffs with snapshot ID. */
-    @Override
-    public int compareTo(final Snapshot that) {
-      return Snapshot.ID_COMPARATOR.compare(this.snapshot, that);
-    }
     
     /** Is the inode the root of the snapshot? */
     boolean isSnapshotRoot() {
       return snapshotINode == snapshot.getRoot();
     }
 
-    /** Copy the INode state to the snapshot if it is not done already. */
-    private void checkAndInitINode(INodeDirectory snapshotCopy) {
-      if (snapshotINode == null) {
-        if (snapshotCopy == null) {
-          snapshotCopy = new INodeDirectory(INodeDirectoryWithSnapshot.this,
-              false);
-        }
-        snapshotINode = snapshotCopy;
-      }
-    }
-
-    /** @return the snapshot object of this diff. */
-    Snapshot getSnapshot() {
-      return snapshot;
+    @Override
+    INodeDirectory getCurrentINode() {
+      return INodeDirectoryWithSnapshot.this;
     }
 
-    private INodeDirectory getSnapshotINode() {
-      // get from this diff, then the posterior diff and then the current inode
-      for(SnapshotDiff d = this; ; d = d.posteriorDiff) {
-        if (d.snapshotINode != null) {
-          return d.snapshotINode;
-        } else if (d.posteriorDiff == null) {
-          return INodeDirectoryWithSnapshot.this;
+    @Override
+    void combinePosteriorAndCollectBlocks(final DirectoryDiff posterior,
+        final BlocksMapUpdateInfo collectedBlocks) {
+      diff.combinePosterior(posterior.diff, new Diff.Processor<INode>() {
+        /** Collect blocks for deleted files. */
+        @Override
+        public void process(INode inode) {
+          if (inode != null && inode instanceof INodeFile) {
+            ((INodeFile)inode).collectSubtreeBlocksAndClear(collectedBlocks);
+          }
         }
-      }
+      });
     }
 
     /**
@@ -281,11 +237,11 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
         private List<INode> initChildren() {
           if (children == null) {
             final ChildrenDiff combined = new ChildrenDiff();
-            for(SnapshotDiff d = SnapshotDiff.this; d != null; d = d.posteriorDiff) {
+            for(DirectoryDiff d = DirectoryDiff.this; d != null; d = d.getPosterior()) {
               combined.combinePosterior(d.diff, null);
             }
             children = combined.apply2Current(ReadOnlyList.Util.asList(
-                INodeDirectoryWithSnapshot.this.getChildrenList(null)));
+                getCurrentINode().getChildrenList(null)));
           }
           return children;
         }
@@ -314,7 +270,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
 
     /** @return the child with the given name. */
     INode getChild(byte[] name, boolean checkPosterior) {
-      for(SnapshotDiff d = this; ; d = d.posteriorDiff) {
+      for(DirectoryDiff d = this; ; d = d.getPosterior()) {
         final Container<INode> returned = d.diff.accessPrevious(name);
         if (returned != null) {
           // the diff is able to determine the inode
@@ -322,17 +278,18 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
         } else if (!checkPosterior) {
           // Since checkPosterior is false, return null, i.e. not found.   
           return null;
-        } else if (d.posteriorDiff == null) {
+        } else if (d.getPosterior() == null) {
           // no more posterior diff, get from current inode.
-          return INodeDirectoryWithSnapshot.this.getChild(name, null);
+          return getCurrentINode().getChild(name, null);
         }
       }
     }
     
     @Override
     public String toString() {
+      final DirectoryDiff posterior = getPosterior();
       return "\n  " + snapshot + " (-> "
-          + (posteriorDiff == null? null: posteriorDiff.snapshot)
+          + (posterior == null? null: posterior.snapshot)
           + ") childrenSize=" + childrenSize + ", " + diff;
     }
     
@@ -361,9 +318,24 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
       // Write diff. Node need to write poseriorDiff, since diffs is a list.
       diff.write(out);
     }
-    
-    private List<INodeDirectory> getSnapshotDirectory() {
-      return diff.getDirsInDeleted();
+  }
+
+  /** A list of directory diffs. */
+  class DirectoryDiffList extends AbstractINodeDiffList<INodeDirectory, DirectoryDiff> {
+    @Override
+    INodeDirectoryWithSnapshot getCurrentINode() {
+      return INodeDirectoryWithSnapshot.this;
+    }
+
+    @Override
+    DirectoryDiff addSnapshotDiff(Snapshot snapshot, INodeDirectory dir,
+        boolean isSnapshotCreation) {
+      final DirectoryDiff d = new DirectoryDiff(snapshot, dir); 
+      if (isSnapshotCreation) {
+        //for snapshot creation, snapshotINode is the same as the snapshot root
+        d.snapshotINode = snapshot.getRoot();
+      }
+      return append(d);
     }
   }
 
@@ -374,7 +346,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
         = new INodeDirectoryWithSnapshot(dir, true, null);
     if (latest != null) {
       // add a diff for the latest snapshot
-      withSnapshot.addSnapshotDiff(latest, dir, false);
+      withSnapshot.diffs.addSnapshotDiff(latest, dir, false);
     }
     return withSnapshot;
   }
@@ -402,15 +374,17 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
       laterSnapshot = fromSnapshot;
     }
     
-    int earlierDiffIndex = Collections.binarySearch(diffs, earlierSnapshot);
-    if (earlierDiffIndex < 0 && (-earlierDiffIndex - 1) == diffs.size()) {
+    final List<DirectoryDiff> difflist = diffs.asList();
+    final int size = difflist.size();
+    int earlierDiffIndex = Collections.binarySearch(difflist, earlierSnapshot);
+    if (earlierDiffIndex < 0 && (-earlierDiffIndex - 1) == size) {
       // if the earlierSnapshot is after the latest SnapshotDiff stored in diffs,
       // no modification happened after the earlierSnapshot
       return false;
     }
-    int laterDiffIndex = diffs.size();
+    int laterDiffIndex = size;
     if (laterSnapshot != null) {
-      laterDiffIndex = Collections.binarySearch(diffs, laterSnapshot);
+      laterDiffIndex = Collections.binarySearch(difflist, laterSnapshot);
       if (laterDiffIndex == -1 || laterDiffIndex == 0) {
         // if the endSnapshot is the earliest SnapshotDiff stored in
         // diffs, or before it, no modification happened before the endSnapshot
@@ -425,7 +399,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     boolean dirMetadataChanged = false;
     INodeDirectory dirCopy = null;
     for (int i = earlierDiffIndex; i < laterDiffIndex; i++) {
-      SnapshotDiff sdiff = diffs.get(i);
+      DirectoryDiff sdiff = difflist.get(i);
       diff.combinePosterior(sdiff.diff, null);
       if (dirMetadataChanged == false && sdiff.snapshotINode != null) {
         if (dirCopy == null) {
@@ -441,9 +415,8 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     if (!diff.isEmpty() || dirMetadataChanged) {
       return true;
     } else if (dirCopy != null) {
-      for (int i = laterDiffIndex; i < diffs.size(); i++) {
-        if (diffs.get(i).snapshotINode != null
-            && !dirCopy.metadataEquals(diffs.get(i).snapshotINode)) {
+      for (int i = laterDiffIndex; i < size; i++) {
+        if (!dirCopy.metadataEquals(difflist.get(i).snapshotINode)) {
           return true;
         }
       }
@@ -453,134 +426,28 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
   }
 
   /** Diff list sorted by snapshot IDs, i.e. in chronological order. */
-  private final List<SnapshotDiff> diffs;
+  private final DirectoryDiffList diffs;
 
   INodeDirectoryWithSnapshot(INodeDirectory that, boolean adopt,
-      List<SnapshotDiff> diffs) {
+      DirectoryDiffList diffs) {
     super(that, adopt, that.getNsQuota(), that.getDsQuota());
-    this.diffs = diffs != null? diffs: new ArrayList<SnapshotDiff>();
-  }
-  
-  /**
-   * Delete the snapshot with the given name. The synchronization of the diff
-   * list will be done outside.
-   * 
-   * If the diff to remove is not the first one in the diff list, we need to 
-   * combine the diff with its previous one:
-   * 
-   * @param snapshot The snapshot to be deleted
-   * @param collectedBlocks Used to collect information for blocksMap update
-   * @return The SnapshotDiff containing the deleted snapshot. 
-   *         Null if the snapshot with the given name does not exist. 
-   */
-  SnapshotDiff deleteSnapshotDiff(Snapshot snapshot,
-      final BlocksMapUpdateInfo collectedBlocks) {
-    int snapshotIndex = Collections.binarySearch(diffs, snapshot);
-    if (snapshotIndex == -1) {
-      return null;
-    } else {
-      SnapshotDiff diffToRemove = null;
-      diffToRemove = diffs.remove(snapshotIndex);
-      if (snapshotIndex > 0) {
-        // combine the to-be-removed diff with its previous diff
-        SnapshotDiff previousDiff = diffs.get(snapshotIndex - 1);
-        previousDiff.diff.combinePosterior(diffToRemove.diff,
-            new Diff.Processor<INode>() {
-          /** Collect blocks for deleted files. */
-          @Override
-          public void process(INode inode) {
-            if (inode != null && inode instanceof INodeFile) {
-              ((INodeFile)inode).collectSubtreeBlocksAndClear(collectedBlocks);
-            }
-          }
-        });
-
-        previousDiff.posteriorDiff = diffToRemove.posteriorDiff;
-        diffToRemove.posteriorDiff = null;
-      }
-      return diffToRemove;
-    }
-  }
-  
-  /** Insert a SnapshotDiff to the head of diffs */
-  public void insertDiff(SnapshotDiff diff) {
-    diffs.add(0, diff);
-  }
-  
-  /** Add a {@link SnapshotDiff} for the given snapshot and directory. */
-  SnapshotDiff addSnapshotDiff(Snapshot snapshot, INodeDirectory dir,
-      boolean isSnapshotCreation) {
-    final SnapshotDiff last = getLastSnapshotDiff();
-    final SnapshotDiff d = new SnapshotDiff(snapshot, dir); 
-
-    if (isSnapshotCreation) {
-      //for snapshot creation, snapshotINode is the same as the snapshot root
-      d.snapshotINode = snapshot.getRoot();
-    }
-    diffs.add(d);
-    if (last != null) {
-      last.posteriorDiff = d;
-    }
-    return d;
-  }
-  
-  SnapshotDiff getLastSnapshotDiff() {
-    final int n = diffs.size();
-    return n == 0? null: diffs.get(n - 1);
+    this.diffs = diffs != null? diffs: new DirectoryDiffList();
   }
 
   /** @return the last snapshot. */
   public Snapshot getLastSnapshot() {
-    final SnapshotDiff last = getLastSnapshotDiff();
-    return last == null? null: last.getSnapshot();
+    return diffs.getLastSnapshot();
   }
 
-  /**
-   * Check if the latest snapshot diff exists.  If not, add it.
-   * @return the latest snapshot diff, which is never null.
-   */
-  private SnapshotDiff checkAndAddLatestSnapshotDiff(Snapshot latest) {
-    final SnapshotDiff last = getLastSnapshotDiff();
-    return last != null && last.snapshot.equals(latest)? last
-        : addSnapshotDiff(latest, this, false);
-  }
-  
-  /**
-   * Check if the latest {@link ChildrenDiff} exists.  If not, add it.
-   * @return the latest {@link ChildrenDiff}, which is never null.
-   */
-  ChildrenDiff checkAndAddLatestDiff(Snapshot latest) {
-    return checkAndAddLatestSnapshotDiff(latest).diff;
-  }
-
-  /**
-   * @return {@link #snapshots}
-   */
-  List<SnapshotDiff> getSnapshotDiffs() {
+  /** @return the snapshot diff list. */
+  DirectoryDiffList getDiffs() {
     return diffs;
   }
 
-  /**
-   * @return the diff corresponding to the given snapshot.
-   *         When the diff is null, it means that the current state and
-   *         the corresponding snapshot state are the same. 
-   */
-  SnapshotDiff getSnapshotDiff(Snapshot snapshot) {
-    if (snapshot == null) {
-      // snapshot == null means the current state, therefore, return null.
-      return null;
-    }
-    final int i = Collections.binarySearch(diffs, snapshot);
-    if (i >= 0) {
-      // exact match
-      return diffs.get(i);
-    } else {
-      // Exact match not found means that there were no changes between
-      // given snapshot and the next state so that the diff for the given
-      // snapshot was not recorded.  Thus, return the next state.
-      final int j = -i - 1;
-      return j < diffs.size()? diffs.get(j): null;
-    }
+  @Override
+  public Pair<INodeDirectoryWithSnapshot, INodeDirectory> createSnapshotCopy() {
+    return new Pair<INodeDirectoryWithSnapshot, INodeDirectory>(this,
+        new INodeDirectory(this, false));
   }
 
   @Override
@@ -592,7 +459,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
   /** Save the snapshot copy to the latest snapshot. */
   public void saveSelf2Snapshot(Snapshot latest, INodeDirectory snapshotCopy) {
     if (latest != null) {
-      checkAndAddLatestSnapshotDiff(latest).checkAndInitINode(snapshotCopy);
+      diffs.checkAndAddLatestSnapshotDiff(latest).checkAndInitINode(snapshotCopy);
     }
   }
 
@@ -604,7 +471,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
       return child;
     }
 
-    final SnapshotDiff diff = checkAndAddLatestSnapshotDiff(latest);
+    final DirectoryDiff diff = diffs.checkAndAddLatestSnapshotDiff(latest);
     if (diff.getChild(child.getLocalNameBytes(), false) != null) {
       // it was already saved in the latest snapshot earlier.  
       return child;
@@ -629,7 +496,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     ChildrenDiff diff = null;
     Integer undoInfo = null;
     if (latest != null) {
-      diff = checkAndAddLatestDiff(latest);
+      diff = diffs.checkAndAddLatestSnapshotDiff(latest).diff;
       undoInfo = diff.create(inode);
     }
     final boolean added = super.addChild(inode, setModTime, null);
@@ -644,7 +511,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     ChildrenDiff diff = null;
     UndoInfo<INode> undoInfo = null;
     if (latest != null) {
-      diff = checkAndAddLatestDiff(latest);
+      diff = diffs.checkAndAddLatestSnapshotDiff(latest).diff;
       undoInfo = diff.delete(child);
     }
     final INode removed = super.removeChild(child, null);
@@ -665,54 +532,54 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
 
   @Override
   public ReadOnlyList<INode> getChildrenList(Snapshot snapshot) {
-    final SnapshotDiff diff = getSnapshotDiff(snapshot);
+    final DirectoryDiff diff = diffs.getDiff(snapshot);
     return diff != null? diff.getChildrenList(): super.getChildrenList(null);
   }
 
   @Override
   public INode getChild(byte[] name, Snapshot snapshot) {
-    final SnapshotDiff diff = getSnapshotDiff(snapshot);
+    final DirectoryDiff diff = diffs.getDiff(snapshot);
     return diff != null? diff.getChild(name, true): super.getChild(name, null);
   }
 
   @Override
   public String getUserName(Snapshot snapshot) {
-    final SnapshotDiff diff = getSnapshotDiff(snapshot);
+    final DirectoryDiff diff = diffs.getDiff(snapshot);
     return diff != null? diff.getSnapshotINode().getUserName()
         : super.getUserName(null);
   }
 
   @Override
   public String getGroupName(Snapshot snapshot) {
-    final SnapshotDiff diff = getSnapshotDiff(snapshot);
+    final DirectoryDiff diff = diffs.getDiff(snapshot);
     return diff != null? diff.getSnapshotINode().getGroupName()
         : super.getGroupName(null);
   }
 
   @Override
   public FsPermission getFsPermission(Snapshot snapshot) {
-    final SnapshotDiff diff = getSnapshotDiff(snapshot);
+    final DirectoryDiff diff = diffs.getDiff(snapshot);
     return diff != null? diff.getSnapshotINode().getFsPermission()
         : super.getFsPermission(null);
   }
 
   @Override
   public long getAccessTime(Snapshot snapshot) {
-    final SnapshotDiff diff = getSnapshotDiff(snapshot);
+    final DirectoryDiff diff = diffs.getDiff(snapshot);
     return diff != null? diff.getSnapshotINode().getAccessTime()
         : super.getAccessTime(null);
   }
 
   @Override
   public long getModificationTime(Snapshot snapshot) {
-    final SnapshotDiff diff = getSnapshotDiff(snapshot);
+    final DirectoryDiff diff = diffs.getDiff(snapshot);
     return diff != null? diff.getSnapshotINode().getModificationTime()
         : super.getModificationTime(null);
   }
   
   @Override
   public String toString() {
-    return super.toString() + ", diffs=" + getSnapshotDiffs();
+    return super.toString() + ", " + diffs;
   }
   
   /**
@@ -726,8 +593,8 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
   public int getSnapshotDirectory(
       Map<Snapshot, List<INodeDirectory>> snapshotDirMap) {
     int dirNum = 0;
-    for (SnapshotDiff sdiff : diffs) {
-      List<INodeDirectory> list = sdiff.getSnapshotDirectory();
+    for (DirectoryDiff sdiff : diffs) {
+      List<INodeDirectory> list = sdiff.getChildrenDiff().getDirsInDeleted();
       if (list.size() > 0) {
         snapshotDirMap.put(sdiff.snapshot, list);
         dirNum += list.size();

+ 12 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java

@@ -33,7 +33,7 @@ import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.ChildrenDiff;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.SnapshotDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.Root;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
@@ -70,13 +70,13 @@ public class SnapshotFSImageFormat {
   public static void saveSnapshotDiffs(INodeDirectoryWithSnapshot sNode,
       DataOutputStream out) throws IOException {
     // # of SnapshotDiff
-    List<SnapshotDiff> diffs = sNode.getSnapshotDiffs();
+    List<DirectoryDiff> diffs = sNode.getDiffs().asList();
     // Record the SnapshotDiff in reversed order, so that we can find the
     // correct reference for INodes in the created list when loading the
     // FSImage
     out.writeInt(diffs.size());
     for (int i = diffs.size() - 1; i >= 0; i--) {
-      SnapshotDiff sdiff = diffs.get(i);
+      DirectoryDiff sdiff = diffs.get(i);
       sdiff.write(out);
     }
   }
@@ -91,8 +91,8 @@ public class SnapshotFSImageFormat {
       INodeDirectoryWithSnapshot parent) throws IOException {
     // the INode in the created list should be a reference to another INode
     // in posterior SnapshotDiffs or one of the current children
-    for (SnapshotDiff postDiff : parent.getSnapshotDiffs()) {
-      INode created = findCreated(createdNodeName, postDiff.getDiff());
+    for (DirectoryDiff postDiff : parent.getDiffs()) {
+      INode created = findCreated(createdNodeName, postDiff.getChildrenDiff());
       if (created != null) {
         return created;
       } // else go to the next SnapshotDiff
@@ -266,8 +266,8 @@ public class SnapshotFSImageFormat {
       DataInputStream in, FSImageFormat.Loader loader)
       throws IOException {
     for (int i = 0; i < numSnapshotDiffs; i++) {
-      SnapshotDiff diff = loadSnapshotDiff(parentWithSnapshot, in, loader);
-      parentWithSnapshot.insertDiff(diff);
+      DirectoryDiff diff = loadSnapshotDiff(parentWithSnapshot, in, loader);
+      parentWithSnapshot.getDiffs().insert(diff);
     }
   }
   
@@ -321,7 +321,7 @@ public class SnapshotFSImageFormat {
    *               using.
    * @return A {@link SnapshotDiff}.
    */
-  private static SnapshotDiff loadSnapshotDiff(
+  private static DirectoryDiff loadSnapshotDiff(
       INodeDirectoryWithSnapshot parent, DataInputStream in,
       FSImageFormat.Loader loader) throws IOException {
     // 1. Load SnapshotDiff#childrenSize
@@ -342,9 +342,10 @@ public class SnapshotFSImageFormat {
     List<INode> deletedList = loadDeletedList(parent, createdList, in, loader);
     
     // 6. Compose the SnapshotDiff
-    SnapshotDiff sdiff = parent.new SnapshotDiff(snapshot, childrenSize,
-        snapshotINode, parent.getSnapshotDiffs().isEmpty() ? null : parent
-            .getSnapshotDiffs().get(0), createdList, deletedList);
+    List<DirectoryDiff> diffs = parent.getDiffs().asList();
+    DirectoryDiff sdiff = parent.new DirectoryDiff(snapshot, snapshotINode,
+        diffs.isEmpty() ? null : diffs.get(0),
+        childrenSize, createdList, deletedList);
     return sdiff;
   }
   

+ 2 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/diff/Diff.java

@@ -451,23 +451,10 @@ public class Diff<K, E extends Diff.Element<K>> {
     }
   }
 
-  /** Convert the element list to a compact string. */
-  static <E> String toString(List<E> elements) {
-    if (elements == null || elements.isEmpty()) {
-      return "<empty>";
-    }
-    final StringBuilder b = new StringBuilder("[")
-        .append(elements.get(0));
-    for(int i = 1; i < elements.size(); i++) {
-      b.append(", ").append(elements.get(i));
-    }
-    return b.append("]").toString();
-  }
-
   @Override
   public String toString() {
     return getClass().getSimpleName()
-        + "{created=" + toString(created)
-        + ", deleted=" + toString(deleted) + "}";
+        +  "{created=" + getCreatedList()
+        + ", deleted=" + getDeletedList() + "}";
   }
 }

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java

@@ -108,7 +108,7 @@ public class TestINodeFileUnderConstructionWithSnapshot {
     INodeFile fileNode = (INodeFile) fsdir.getINode(file.toString());
     INodeDirectorySnapshottable dirNode = (INodeDirectorySnapshottable) fsdir
         .getINode(dir.toString());
-    ChildrenDiff diff = dirNode.getLastSnapshotDiff().getDiff();
+    ChildrenDiff diff = dirNode.getDiffs().getLast().getChildrenDiff();
     INode nodeInCreated = diff.searchCreated(fileNode.getLocalNameBytes());
     assertTrue(fileNode == nodeInCreated);
     INode nodeInDeleted = diff.searchDeleted(fileNode.getLocalNameBytes());
@@ -120,7 +120,7 @@ public class TestINodeFileUnderConstructionWithSnapshot {
     DFSTestUtil.appendFile(hdfs, file, BLOCKSIZE);
     
     // check the circular list and corresponding inodes
-    diff = dirNode.getLastSnapshotDiff().getDiff();
+    diff = dirNode.getDiffs().getLast().getChildrenDiff();
     fileNode = (INodeFile) fsdir.getINode(file.toString());
     nodeInCreated = diff.searchCreated(fileNode.getLocalNameBytes());
     assertTrue(fileNode == nodeInCreated);
@@ -140,7 +140,7 @@ public class TestINodeFileUnderConstructionWithSnapshot {
     DFSTestUtil.appendFile(hdfs, file, BLOCKSIZE);
     
     // check the circular list and corresponding inodes
-    diff = dirNode.getLastSnapshotDiff().getDiff();
+    diff = dirNode.getDiffs().getLast().getChildrenDiff();
     fileNode = (INodeFile) fsdir.getINode(file.toString());
     nodeInCreated = diff.searchCreated(fileNode.getLocalNameBytes());
     assertTrue(fileNode == nodeInCreated);
@@ -187,7 +187,7 @@ public class TestINodeFileUnderConstructionWithSnapshot {
     assertEquals(BLOCKSIZE * 2, ((INodeFile) fileNode).computeFileSize(true));
     INodeDirectorySnapshottable dirNode = (INodeDirectorySnapshottable) fsdir
         .getINode(dir.toString());
-    ChildrenDiff diff = dirNode.getLastSnapshotDiff().getDiff();
+    ChildrenDiff diff = dirNode.getDiffs().getLast().getChildrenDiff();
     INode nodeInDeleted_S0 = diff.searchDeleted(fileNode.getLocalNameBytes());
     assertTrue(nodeInDeleted_S0 instanceof INodeFileUnderConstructionSnapshot);
     assertEquals(BLOCKSIZE * 2,
@@ -199,7 +199,7 @@ public class TestINodeFileUnderConstructionWithSnapshot {
     
     // re-check nodeInDeleted_S0
     dirNode = (INodeDirectorySnapshottable) fsdir.getINode(dir.toString());
-    diff = dirNode.getLastSnapshotDiff().getDiff();
+    diff = dirNode.getDiffs().getLast().getChildrenDiff();
     nodeInDeleted_S0 = diff.searchDeleted(fileNode.getLocalNameBytes());
     assertTrue(nodeInDeleted_S0 instanceof INodeFileUnderConstructionSnapshot);
     assertEquals(BLOCKSIZE * 2,
@@ -213,7 +213,7 @@ public class TestINodeFileUnderConstructionWithSnapshot {
     // have been stored in s1's deleted list
     fileNode = (INodeFile) fsdir.getINode(file.toString());
     dirNode = (INodeDirectorySnapshottable) fsdir.getINode(dir.toString());
-    diff = dirNode.getLastSnapshotDiff().getDiff();
+    diff = dirNode.getDiffs().getLast().getChildrenDiff();
     INode nodeInCreated_S1 = diff.searchCreated(fileNode.getLocalNameBytes());
     assertTrue(fileNode == nodeInCreated_S1);
     assertTrue(fileNode instanceof INodeFileWithSnapshot);

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

@@ -31,7 +31,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.SnapshotDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.ipc.RemoteException;
 import org.junit.After;
@@ -91,7 +91,7 @@ public class TestSnapshotRename {
     for (int i = 0; i < listByName.size(); i++) {
       assertEquals(sortedNames[i], listByName.get(i).getRoot().getLocalName());
     }
-    List<SnapshotDiff> listByTime = srcRoot.getSnapshotDiffs();
+    List<DirectoryDiff> listByTime = srcRoot.getDiffs().asList();
     assertEquals(names.length, listByTime.size());
     for (int i = 0; i < listByTime.size(); i++) {
       assertEquals(names[i], listByTime.get(i).getSnapshot().getRoot().getLocalName());

+ 19 - 23
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/diff/TestDiff.java

@@ -124,10 +124,10 @@ public class TestDiff {
         c = diffs.get(i).apply2Previous(c);
       }
       if (!hasIdenticalElements(current, c)) {
-        System.out.println("previous = " + Diff.toString(previous));
+        System.out.println("previous = " + previous);
         System.out.println();
-        System.out.println("current  = " + Diff.toString(current));
-        System.out.println("c        = " + Diff.toString(c));
+        System.out.println("current  = " + current);
+        System.out.println("c        = " + c);
         throw new AssertionError("current and c are not identical.");
       }
 
@@ -137,10 +137,10 @@ public class TestDiff {
         p = diffs.get(i).apply2Current(p);
       }
       if (!hasIdenticalElements(previous, p)) {
-        System.out.println("previous = " + Diff.toString(previous));
-        System.out.println("p        = " + Diff.toString(p));
+        System.out.println("previous = " + previous);
+        System.out.println("p        = " + p);
         System.out.println();
-        System.out.println("current  = " + Diff.toString(current));
+        System.out.println("current  = " + current);
         throw new AssertionError("previous and p are not identical.");
       }
     }
@@ -155,20 +155,20 @@ public class TestDiff {
       // check if current == previous + combined
       final List<INode> c = combined.apply2Previous(previous);
       if (!hasIdenticalElements(current, c)) {
-        System.out.println("previous = " + Diff.toString(previous));
+        System.out.println("previous = " + previous);
         System.out.println();
-        System.out.println("current  = " + Diff.toString(current));
-        System.out.println("c        = " + Diff.toString(c));
+        System.out.println("current  = " + current);
+        System.out.println("c        = " + c);
         throw new AssertionError("current and c are not identical.");
       }
 
       // check if previous == current - combined
       final List<INode> p = combined.apply2Current(current);
       if (!hasIdenticalElements(previous, p)) {
-        System.out.println("previous = " + Diff.toString(previous));
-        System.out.println("p        = " + Diff.toString(p));
+        System.out.println("previous = " + previous);
+        System.out.println("p        = " + p);
         System.out.println();
-        System.out.println("current  = " + Diff.toString(current));
+        System.out.println("current  = " + current);
         throw new AssertionError("previous and p are not identical.");
       }
     }
@@ -228,10 +228,6 @@ public class TestDiff {
     return true;
   }
 
-  static String toString(Diff<byte[], INode> diff) {
-    return diff.toString();
-  }
-
   static String toString(INode inode) {
     return inode == null? null
         : inode.getLocalName() + ":" + inode.getModificationTime();
@@ -256,13 +252,13 @@ public class TestDiff {
       final boolean testUndo = RANDOM.nextInt(UNDO_TEST_P) == 0;
       String before = null;
       if (testUndo) {
-        before = toString(diff);
+        before = diff.toString();
       }
 
       final int undoInfo = diff.create(inode);
 
       if (testUndo) {
-        final String after = toString(diff);
+        final String after = diff.toString();
         //undo
         diff.undoCreate(inode, undoInfo);
         assertDiff(before, diff);
@@ -281,13 +277,13 @@ public class TestDiff {
       final boolean testUndo = RANDOM.nextInt(UNDO_TEST_P) == 0;
       String before = null;
       if (testUndo) {
-        before = toString(diff);
+        before = diff.toString();
       }
 
       final UndoInfo<INode> undoInfo = diff.delete(inode);
 
       if (testUndo) {
-        final String after = toString(diff);
+        final String after = diff.toString();
         //undo
         diff.undoDelete(inode, undoInfo);
         assertDiff(before, diff);
@@ -311,13 +307,13 @@ public class TestDiff {
       final boolean testUndo = RANDOM.nextInt(UNDO_TEST_P) == 0;
       String before = null;
       if (testUndo) {
-        before = toString(diff);
+        before = diff.toString();
       }
 
       final UndoInfo<INode> undoInfo = diff.modify(oldinode, newinode);
 
       if (testUndo) {
-        final String after = toString(diff);
+        final String after = diff.toString();
         //undo
         diff.undoModify(oldinode, newinode, undoInfo);
         assertDiff(before, diff);
@@ -329,6 +325,6 @@ public class TestDiff {
   }
   
   static void assertDiff(String s, Diff<byte[], INode> diff) {
-    Assert.assertEquals(s, toString(diff));
+    Assert.assertEquals(s, diff.toString());
   }
 }