You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ae...@apache.org on 2018/02/15 23:51:08 UTC

[39/50] [abbrv] hadoop git commit: HDFS-13142. Define and Implement a DiifList Interface to store and manage SnapshotDiffs. Contributed by Shashikant Banerjee

HDFS-13142. Define and Implement a DiifList Interface to store and manage SnapshotDiffs.  Contributed by Shashikant Banerjee


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6ea7d78c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6ea7d78c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6ea7d78c

Branch: refs/heads/HDFS-7240
Commit: 6ea7d78ccb0d1c4af9bcac02a4cff89bdffff252
Parents: 8f66aff
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Thu Feb 15 19:33:44 2018 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Thu Feb 15 19:33:44 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hdfs/server/namenode/INodeFile.java  |   3 +-
 .../snapshot/AbstractINodeDiffList.java         |  27 ++--
 .../hdfs/server/namenode/snapshot/DiffList.java | 140 +++++++++++++++++++
 .../namenode/snapshot/DiffListByArrayList.java  |  80 +++++++++++
 .../snapshot/DirectoryWithSnapshotFeature.java  |  10 +-
 .../snapshot/FSImageFormatPBSnapshot.java       |   4 +-
 .../server/namenode/snapshot/FileDiffList.java  |  11 +-
 .../snapshot/FileWithSnapshotFeature.java       |   2 +-
 .../snapshot/SnapshotFSImageFormat.java         |   4 +-
 .../namenode/TestFSImageWithSnapshot.java       |   3 +-
 .../snapshot/TestRenameWithSnapshots.java       |  40 +++---
 .../snapshot/TestSetQuotaWithSnapshot.java      |   3 +-
 .../namenode/snapshot/TestSnapshotRename.java   |   3 +-
 13 files changed, 276 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ea7d78c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 90659f3..6693297 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DiffList;
 import org.apache.hadoop.hdfs.util.LongBitFormat;
 import org.apache.hadoop.util.StringUtils;
 import static org.apache.hadoop.io.erasurecode.ErasureCodeConstants.REPLICATION_POLICY_ID;
@@ -988,7 +989,7 @@ public class INodeFile extends INodeWithAdditionalFields
     } else {
       // Collect all distinct blocks
       Set<BlockInfo> allBlocks = new HashSet<>(Arrays.asList(getBlocks()));
-      List<FileDiff> diffs = sf.getDiffs().asList();
+      DiffList<FileDiff> diffs = sf.getDiffs().asList();
       for(FileDiff diff : diffs) {
         BlockInfo[] diffBlocks = diff.getBlocks();
         if (diffBlocks != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ea7d78c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
index 98d8c53..8f2465a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -36,14 +35,15 @@ abstract class AbstractINodeDiffList<N extends INode,
                                      A extends INodeAttributes,
                                      D extends AbstractINodeDiff<N, A, D>> 
     implements Iterable<D> {
+
   /** Diff list sorted by snapshot IDs, i.e. in chronological order.
     * Created lazily to avoid wasting memory by empty lists. */
-  private List<D> diffs;
+  private DiffList<D> diffs;
 
   /** @return this list as a unmodifiable {@link List}. */
-  public final List<D> asList() {
+  public final DiffList<D> asList() {
     return diffs != null ?
-        Collections.unmodifiableList(diffs) : Collections.emptyList();
+        DiffList.unmodifiableList(diffs) : DiffList.emptyList();
   }
   
   /** Clear the list. */
@@ -72,7 +72,7 @@ abstract class AbstractINodeDiffList<N extends INode,
     if (diffs == null) {
       return;
     }
-    int snapshotIndex = Collections.binarySearch(diffs, snapshot);
+    int snapshotIndex = diffs.binarySearch(snapshot);
 
     D removed;
     if (snapshotIndex == 0) {
@@ -114,7 +114,7 @@ abstract class AbstractINodeDiffList<N extends INode,
   private D addLast(D diff) {
     createDiffsIfNeeded();
     final D last = getLast();
-    diffs.add(diff);
+    diffs.addLast(diff);
     if (last != null) {
       last.setPosterior(diff);
     }
@@ -125,7 +125,7 @@ abstract class AbstractINodeDiffList<N extends INode,
   final void addFirst(D diff) {
     createDiffsIfNeeded();
     final D first = diffs.isEmpty()? null : diffs.get(0);
-    diffs.add(0, diff);
+    diffs.addFirst(diff);
     diff.setPosterior(first);
   }
 
@@ -140,7 +140,8 @@ abstract class AbstractINodeDiffList<N extends INode,
 
   private void createDiffsIfNeeded() {
     if (diffs == null) {
-      diffs = new ArrayList<>(INodeDirectory.DEFAULT_FILES_PER_DIRECTORY);
+      diffs =
+          new DiffListByArrayList<>(INodeDirectory.DEFAULT_FILES_PER_DIRECTORY);
     }
   }
 
@@ -169,7 +170,7 @@ abstract class AbstractINodeDiffList<N extends INode,
       }
       return last;
     }
-    final int i = Collections.binarySearch(diffs, anchorId);
+    final int i = diffs.binarySearch(anchorId);
     if (exclusive) { // must be the one before
       if (i == -1 || i == 0) {
         return Snapshot.NO_SNAPSHOT_ID;
@@ -208,7 +209,7 @@ abstract class AbstractINodeDiffList<N extends INode,
     if (snapshotId == Snapshot.CURRENT_STATE_ID || diffs == null) {
       return null;
     }
-    final int i = Collections.binarySearch(diffs, snapshotId);
+    final int i = diffs.binarySearch(snapshotId);
     if (i >= 0) {
       // exact match
       return diffs.get(i);
@@ -242,9 +243,9 @@ abstract class AbstractINodeDiffList<N extends INode,
     }
 
     final int size = diffs.size();
-    int earlierDiffIndex = Collections.binarySearch(diffs, earlier.getId());
-    int laterDiffIndex = later == null ? size : Collections
-        .binarySearch(diffs, later.getId());
+    int earlierDiffIndex = diffs.binarySearch(earlier.getId());
+    int laterDiffIndex = later == null ? size
+        : diffs.binarySearch(later.getId());
     if (-earlierDiffIndex - 1 == size) {
       // if the earlierSnapshot is after the latest SnapshotDiff stored in
       // diffs, no modification happened after the earlierSnapshot

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ea7d78c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DiffList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DiffList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DiffList.java
new file mode 100644
index 0000000..82fd3f9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DiffList.java
@@ -0,0 +1,140 @@
+/**
+ * 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.Collections;
+import java.util.Iterator;
+
+/**
+ * This interface defines the methods used to store and manage InodeDiffs.
+ * @param <T> Type of the object in this list.
+ */
+public interface DiffList<T extends Comparable<Integer>> extends Iterable<T> {
+  DiffList EMPTY_LIST = new DiffListByArrayList(Collections.emptyList());
+
+  /**
+   * Returns an empty DiffList.
+   */
+  static <T extends Comparable<Integer>> DiffList<T> emptyList() {
+    return EMPTY_LIST;
+  }
+
+  /**
+   * Returns an unmodifiable diffList.
+   * @param diffs DiffList
+   * @param <T> Type of the object in the the diffList
+   * @return Unmodifiable diffList
+   */
+  static <T extends Comparable<Integer>> DiffList<T> unmodifiableList(
+      DiffList<T> diffs) {
+    return new DiffList<T>() {
+      @Override
+      public T get(int i) {
+        return diffs.get(i);
+      }
+
+      @Override
+      public boolean isEmpty() {
+        return diffs.isEmpty();
+      }
+
+      @Override
+      public int size() {
+        return diffs.size();
+      }
+
+      @Override
+      public T remove(int i) {
+        throw new UnsupportedOperationException("This list is unmodifiable.");
+      }
+
+      @Override
+      public boolean addLast(T t) {
+        throw new UnsupportedOperationException("This list is unmodifiable.");
+      }
+
+      @Override
+      public void addFirst(T t) {
+        throw new UnsupportedOperationException("This list is unmodifiable.");
+      }
+
+      @Override
+      public int binarySearch(int i) {
+        return diffs.binarySearch(i);
+      }
+
+      @Override
+      public Iterator<T> iterator() {
+        return diffs.iterator();
+      }
+    };
+  }
+
+  /**
+   * Returns the element at the specified position in this list.
+   *
+   * @param index index of the element to return
+   * @return the element at the specified position in this list
+   * @throws IndexOutOfBoundsException if the index is out of range
+   *         (<tt>index &lt; 0 || index &gt;= size()</tt>)
+   */
+  T get(int index);
+
+  /**
+   * Returns true if this list contains no elements.
+   *
+   * @return true if this list contains no elements
+   */
+  boolean isEmpty();
+
+  /**
+   * Returns the number of elements in this list.
+   * @return the number of elements in this list.
+   */
+  int size();
+
+  /**
+   * Removes the element at the specified position in this list.
+   * @param index the index of the element to be removed
+   * @return the element previously at the specified position
+   */
+  T remove(int index);
+
+  /**
+   * Adds an element at the end of the list.
+   * @param t element to be appended to this list
+   * @return true, if insertion is successful
+   */
+  boolean addLast(T t);
+
+  /**
+   * Adds an element at the beginning of the list.
+   * @param t element to be added to this list
+   */
+  void addFirst(T t);
+
+  /**
+   * Searches the list for the specified object using the binary
+   * search algorithm.
+   * @param key key to be searched for
+   * @return the index of the search key, if it is contained in the list
+   *         otherwise, (-insertion point - 1).
+   */
+  int binarySearch(int key);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ea7d78c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DiffListByArrayList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DiffListByArrayList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DiffListByArrayList.java
new file mode 100644
index 0000000..03aa5c2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DiffListByArrayList.java
@@ -0,0 +1,80 @@
+/**
+ * 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;
+
+/**
+ * Resizable-array implementation of the DiffList interface.
+ * @param <T> Type of the object in the list
+ */
+public class DiffListByArrayList<T extends Comparable<Integer>>
+    implements DiffList<T> {
+  private final List<T> list;
+
+  DiffListByArrayList(List<T> list) {
+    this.list = list;
+  }
+
+  public DiffListByArrayList(int initialCapacity) {
+    this(new ArrayList<>(initialCapacity));
+  }
+
+  @Override
+  public T get(int i) {
+    return list.get(i);
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return list.isEmpty();
+  }
+
+  @Override
+  public int size() {
+    return list.size();
+  }
+
+  @Override
+  public T remove(int i) {
+    return list.remove(i);
+  }
+
+  @Override
+  public boolean addLast(T t) {
+    return list.add(t);
+  }
+
+  @Override
+  public void addFirst(T t) {
+    list.add(0, t);
+  }
+
+  @Override
+  public int binarySearch(int i) {
+    return Collections.binarySearch(list, i);
+  }
+
+  @Override
+  public Iterator<T> iterator() {
+    return list.iterator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ea7d78c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
index 7535879..8ed9c7a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
@@ -225,7 +225,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
         private List<INode> initChildren() {
           if (children == null) {
             final ChildrenDiff combined = new ChildrenDiff();
-            for (DirectoryDiff d = DirectoryDiff.this; d != null; 
+            for (DirectoryDiff d = DirectoryDiff.this; d != null;
                 d = d.getPosterior()) {
               combined.combinePosterior(d.diff, null);
             }
@@ -334,7 +334,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     /** 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) {
-      final List<DirectoryDiff> diffList = asList();
+      final DiffList<DirectoryDiff> diffList = asList();
       for(int i = diffList.size() - 1; i >= 0; i--) {
         final ChildrenDiff diff = diffList.get(i).diff;
         if (diff.replace(type, oldChild, newChild)) {
@@ -346,7 +346,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
 
     /** Remove the given child in the created/deleted list, if there is any. */
     public boolean removeChild(final ListType type, final INode child) {
-      final List<DirectoryDiff> diffList = asList();
+      final DiffList<DirectoryDiff> diffList = asList();
       for(int i = diffList.size() - 1; i >= 0; i--) {
         final ChildrenDiff diff = diffList.get(i).diff;
         if (diff.removeChild(type, child)) {
@@ -363,7 +363,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
      * given inode is not in any of the snapshot.
      */
     public int findSnapshotDeleted(final INode child) {
-      final List<DirectoryDiff> diffList = asList();
+      final DiffList<DirectoryDiff> diffList = asList();
       for(int i = diffList.size() - 1; i >= 0; i--) {
         final ChildrenDiff diff = diffList.get(i).diff;
         final int d = diff.searchIndex(ListType.DELETED,
@@ -669,7 +669,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
 
     boolean dirMetadataChanged = false;
     INodeDirectoryAttributes dirCopy = null;
-    List<DirectoryDiff> difflist = diffs.asList();
+    DiffList<DirectoryDiff> difflist = diffs.asList();
     for (int i = earlierDiffIndex; i < laterDiffIndex; i++) {
       DirectoryDiff sdiff = difflist.get(i);
       diff.combinePosterior(sdiff.diff, null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ea7d78c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
index f31743a..4b619a4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
@@ -523,7 +523,7 @@ public class FSImageFormatPBSnapshot {
         throws IOException {
       FileWithSnapshotFeature sf = file.getFileWithSnapshotFeature();
       if (sf != null) {
-        List<FileDiff> diffList = sf.getDiffs().asList();
+        DiffList<FileDiff> diffList = sf.getDiffs().asList();
         SnapshotDiffSection.DiffEntry entry = SnapshotDiffSection.DiffEntry
             .newBuilder().setInodeId(file.getId()).setType(Type.FILEDIFF)
             .setNumOfDiff(diffList.size()).build();
@@ -563,7 +563,7 @@ public class FSImageFormatPBSnapshot {
         throws IOException {
       DirectoryWithSnapshotFeature sf = dir.getDirectoryWithSnapshotFeature();
       if (sf != null) {
-        List<DirectoryDiff> diffList = sf.getDiffs().asList();
+        DiffList<DirectoryDiff> diffList = sf.getDiffs().asList();
         SnapshotDiffSection.DiffEntry entry = SnapshotDiffSection.DiffEntry
             .newBuilder().setInodeId(dir.getId()).setType(Type.DIRECTORYDIFF)
             .setNumOfDiff(diffList.size()).build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ea7d78c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
index 9dcd4d8..2c04a49 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
-import java.util.Collections;
-import java.util.List;
-
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
@@ -66,8 +63,8 @@ public class FileDiffList extends
     if (snapshotId == Snapshot.CURRENT_STATE_ID) {
       return null;
     }
-    List<FileDiff> diffs = this.asList();
-    int i = Collections.binarySearch(diffs, snapshotId);
+    DiffList<FileDiff> diffs = this.asList();
+    int i = diffs.binarySearch(snapshotId);
     BlockInfo[] blocks = null;
     for(i = i >= 0 ? i : -i-2; i >= 0; i--) {
       blocks = diffs.get(i).getBlocks();
@@ -83,8 +80,8 @@ public class FileDiffList extends
     if (snapshotId == Snapshot.CURRENT_STATE_ID) {
       return null;
     }
-    List<FileDiff> diffs = this.asList();
-    int i = Collections.binarySearch(diffs, snapshotId);
+    DiffList<FileDiff> diffs = this.asList();
+    int i = diffs.binarySearch(snapshotId);
     BlockInfo[] blocks = null;
     for (i = i >= 0 ? i+1 : -i-1; i < diffs.size(); i++) {
       blocks = diffs.get(i).getBlocks();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ea7d78c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
index b52e8d6..80061c3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
@@ -86,7 +86,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
     int earlierDiffIndex = diffIndexPair[0];
     int laterDiffIndex = diffIndexPair[1];
 
-    final List<FileDiff> diffList = diffs.asList();
+    final DiffList<FileDiff> diffList = diffs.asList();
     final long earlierLength = diffList.get(earlierDiffIndex).getFileSize();
     final long laterLength = laterDiffIndex == diffList.size() ? file
         .computeFileSize(true, false) : diffList.get(laterDiffIndex)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ea7d78c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java
index fcab53a..d1ae293 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java
@@ -82,7 +82,7 @@ public class SnapshotFSImageFormat {
     if (diffs == null) {
       out.writeInt(-1); // no diffs
     } else {
-      final List<D> list = diffs.asList();
+      final DiffList<D> list = diffs.asList();
       final int size = list.size();
       out.writeInt(size);
       for (int i = size - 1; i >= 0; i--) {
@@ -306,7 +306,7 @@ public class SnapshotFSImageFormat {
     List<INode> deletedList = loadDeletedList(parent, createdList, in, loader);
     
     // 6. Compose the SnapshotDiff
-    List<DirectoryDiff> diffs = parent.getDiffs().asList();
+    DiffList<DirectoryDiff> diffs = parent.getDiffs().asList();
     DirectoryDiff sdiff = new DirectoryDiff(snapshot.getId(), snapshotINode,
         diffs.isEmpty() ? null : diffs.get(0), childrenSize, createdList,
         deletedList, snapshotINode == snapshot.getRoot());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ea7d78c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
index 82f5cfb..58ecc8a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
@@ -199,7 +200,7 @@ public class TestFSImageWithSnapshot {
     assertTrue("The children list of root should be empty", 
         rootNode.getChildrenList(Snapshot.CURRENT_STATE_ID).isEmpty());
     // one snapshot on root: s1
-    List<DirectoryDiff> diffList = rootNode.getDiffs().asList();
+    DiffList<DirectoryDiff> diffList = rootNode.getDiffs().asList();
     assertEquals(1, diffList.size());
     Snapshot s1 = rootNode.getSnapshot(DFSUtil.string2Bytes("s1"));
     assertEquals(s1.getId(), diffList.get(0).getSnapshotId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ea7d78c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
index 91eec78..770651e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
@@ -983,7 +983,7 @@ public class TestRenameWithSnapshots {
     // 5 references: s1, s22, s333, s2222, current tree of sdir1
     assertEquals(5, fooWithCount.getReferenceCount());
     INodeDirectory foo = fooWithCount.asDirectory();
-    List<DirectoryDiff> fooDiffs = foo.getDiffs().asList();
+    DiffList<DirectoryDiff> fooDiffs = foo.getDiffs().asList();
     assertEquals(4, fooDiffs.size());
     
     Snapshot s2222 = sdir2Node.getSnapshot(DFSUtil.string2Bytes("s2222"));
@@ -996,7 +996,7 @@ public class TestRenameWithSnapshots {
     assertEquals(s22.getId(), fooDiffs.get(1).getSnapshotId());
     assertEquals(s1.getId(), fooDiffs.get(0).getSnapshotId());
     INodeFile bar1 = fsdir.getINode4Write(bar1_dir1.toString()).asFile();
-    List<FileDiff> bar1Diffs = bar1.getDiffs().asList();
+    DiffList<FileDiff> bar1Diffs = bar1.getDiffs().asList();
     assertEquals(3, bar1Diffs.size());
     assertEquals(s333.getId(), bar1Diffs.get(2).getSnapshotId());
     assertEquals(s22.getId(), bar1Diffs.get(1).getSnapshotId());
@@ -1008,7 +1008,7 @@ public class TestRenameWithSnapshots {
     // 5 references: s1, s22, s333, s2222, current tree of sdir1
     assertEquals(5, barWithCount.getReferenceCount());
     INodeFile bar = barWithCount.asFile();
-    List<FileDiff> barDiffs = bar.getDiffs().asList();
+    DiffList<FileDiff> barDiffs = bar.getDiffs().asList();
     assertEquals(4, barDiffs.size());
     assertEquals(s2222.getId(), barDiffs.get(3).getSnapshotId());
     assertEquals(s333.getId(), barDiffs.get(2).getSnapshotId());
@@ -1188,7 +1188,7 @@ public class TestRenameWithSnapshots {
     INodeReference.WithCount fooWC = (WithCount) fooRef.getReferredINode();
     assertEquals(1, fooWC.getReferenceCount());
     INodeDirectory fooDir = fooWC.getReferredINode().asDirectory();
-    List<DirectoryDiff> diffs = fooDir.getDiffs().asList();
+    DiffList<DirectoryDiff> diffs = fooDir.getDiffs().asList();
     assertEquals(1, diffs.size());
     assertEquals(s2.getId(), diffs.get(0).getSnapshotId());
     
@@ -1294,7 +1294,7 @@ public class TestRenameWithSnapshots {
         .getChildrenList(Snapshot.CURRENT_STATE_ID);
     assertEquals(1, dir1Children.size());
     assertEquals(foo.getName(), dir1Children.get(0).getLocalName());
-    List<DirectoryDiff> dir1Diffs = dir1Node.getDiffs().asList();
+    DiffList<DirectoryDiff> dir1Diffs = dir1Node.getDiffs().asList();
     assertEquals(1, dir1Diffs.size());
     assertEquals(s1.getId(), dir1Diffs.get(0).getSnapshotId());
     
@@ -1306,7 +1306,8 @@ public class TestRenameWithSnapshots {
     
     INode fooNode = fsdir.getINode4Write(foo.toString());
     assertTrue(fooNode.isDirectory() && fooNode.asDirectory().isWithSnapshot());
-    List<DirectoryDiff> fooDiffs = fooNode.asDirectory().getDiffs().asList();
+    DiffList<DirectoryDiff> fooDiffs =
+        fooNode.asDirectory().getDiffs().asList();
     assertEquals(1, fooDiffs.size());
     assertEquals(s1.getId(), fooDiffs.get(0).getSnapshotId());
     
@@ -1364,7 +1365,7 @@ public class TestRenameWithSnapshots {
         .getChildrenList(Snapshot.CURRENT_STATE_ID);
     assertEquals(1, dir1Children.size());
     assertEquals(foo.getName(), dir1Children.get(0).getLocalName());
-    List<DirectoryDiff> dir1Diffs = dir1Node.getDiffs().asList();
+    DiffList<DirectoryDiff> dir1Diffs = dir1Node.getDiffs().asList();
     assertEquals(1, dir1Diffs.size());
     assertEquals(s1.getId(), dir1Diffs.get(0).getSnapshotId());
     
@@ -1433,7 +1434,7 @@ public class TestRenameWithSnapshots {
     ReadOnlyList<INode> dir2Children = dir2Node
         .getChildrenList(Snapshot.CURRENT_STATE_ID);
     assertEquals(1, dir2Children.size());
-    List<DirectoryDiff> dir2Diffs = dir2Node.getDiffs().asList();
+    DiffList<DirectoryDiff> dir2Diffs = dir2Node.getDiffs().asList();
     assertEquals(1, dir2Diffs.size());
     assertEquals(s2.getId(), dir2Diffs.get(0).getSnapshotId());
     ChildrenDiff childrenDiff = dir2Diffs.get(0).getChildrenDiff();
@@ -1445,7 +1446,8 @@ public class TestRenameWithSnapshots {
     INode fooNode = fsdir.getINode4Write(foo_dir2.toString());
     assertTrue(childrenDiff.getList(ListType.CREATED).get(0) == fooNode);
     assertTrue(fooNode instanceof INodeReference.DstReference);
-    List<DirectoryDiff> fooDiffs = fooNode.asDirectory().getDiffs().asList();
+    DiffList<DirectoryDiff> fooDiffs =
+        fooNode.asDirectory().getDiffs().asList();
     assertEquals(1, fooDiffs.size());
     assertEquals(s1.getId(), fooDiffs.get(0).getSnapshotId());
     
@@ -1594,7 +1596,7 @@ public class TestRenameWithSnapshots {
     INode barNode = fsdir2.getINode4Write(bar.toString());
     assertTrue(barNode.getClass() == INodeFile.class);
     assertSame(fooNode, barNode.getParent());
-    List<DirectoryDiff> diffList = dir1Node
+    DiffList<DirectoryDiff> diffList = dir1Node
         .getDiffs().asList();
     assertEquals(1, diffList.size());
     DirectoryDiff diff = diffList.get(0);
@@ -1668,7 +1670,7 @@ public class TestRenameWithSnapshots {
     INode fooNode = childrenList.get(0);
     assertTrue(fooNode.asDirectory().isWithSnapshot());
     assertSame(dir1Node, fooNode.getParent());
-    List<DirectoryDiff> diffList = dir1Node
+    DiffList<DirectoryDiff> diffList = dir1Node
         .getDiffs().asList();
     assertEquals(1, diffList.size());
     DirectoryDiff diff = diffList.get(0);
@@ -1728,7 +1730,7 @@ public class TestRenameWithSnapshots {
     ReadOnlyList<INode> children = fooNode
         .getChildrenList(Snapshot.CURRENT_STATE_ID);
     assertEquals(1, children.size());
-    List<DirectoryDiff> diffList = fooNode.getDiffs().asList();
+    DiffList<DirectoryDiff> diffList = fooNode.getDiffs().asList();
     assertEquals(1, diffList.size());
     DirectoryDiff diff = diffList.get(0);
     // this diff is generated while renaming
@@ -1742,7 +1744,7 @@ public class TestRenameWithSnapshots {
     INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
     assertSame(barNode, children.get(0));
     assertSame(fooNode, barNode.getParent());
-    List<FileDiff> barDiffList = barNode.getDiffs().asList();
+    DiffList<FileDiff> barDiffList = barNode.getDiffs().asList();
     assertEquals(1, barDiffList.size());
     FileDiff barDiff = barDiffList.get(0);
     assertEquals(s1.getId(), barDiff.getSnapshotId());
@@ -1982,7 +1984,7 @@ public class TestRenameWithSnapshots {
         .getChildrenList(Snapshot.CURRENT_STATE_ID);
     assertEquals(1, children.size());
     assertEquals(bar.getName(), children.get(0).getLocalName());
-    List<DirectoryDiff> diffList = fooNode.getDiffs().asList();
+    DiffList<DirectoryDiff> diffList = fooNode.getDiffs().asList();
     assertEquals(1, diffList.size());
     Snapshot s1 = dir1Node.getSnapshot(DFSUtil.string2Bytes("s1"));
     assertEquals(s1.getId(), diffList.get(0).getSnapshotId());
@@ -2054,7 +2056,7 @@ public class TestRenameWithSnapshots {
     assertEquals(bar.getName(), children.get(0).getLocalName());
     assertEquals(bar2.getName(), children.get(1).getLocalName());
     assertEquals(bar3.getName(), children.get(2).getLocalName());
-    List<DirectoryDiff> diffList = fooNode.getDiffs().asList();
+    DiffList<DirectoryDiff> diffList = fooNode.getDiffs().asList();
     assertEquals(1, diffList.size());
     Snapshot s1 = dir1Node.getSnapshot(DFSUtil.string2Bytes("s1"));
     assertEquals(s1.getId(), diffList.get(0).getSnapshotId());
@@ -2231,7 +2233,7 @@ public class TestRenameWithSnapshots {
     // check dir1: foo should be in the created list of s0
     INodeDirectory dir1Node = fsdir.getINode4Write(dir1.toString())
         .asDirectory();
-    List<DirectoryDiff> dir1DiffList = dir1Node.getDiffs().asList();
+    DiffList<DirectoryDiff> dir1DiffList = dir1Node.getDiffs().asList();
     assertEquals(1, dir1DiffList.size());
     List<INode> dList = dir1DiffList.get(0).getChildrenDiff()
         .getList(ListType.DELETED);
@@ -2249,7 +2251,7 @@ public class TestRenameWithSnapshots {
         .asDirectory();
     assertSame(fooNode.asDirectory(), barNode.getParent());
     // bar should only have a snapshot diff for s0
-    List<DirectoryDiff> barDiffList = barNode.getDiffs().asList();
+    DiffList<DirectoryDiff> barDiffList = barNode.getDiffs().asList();
     assertEquals(1, barDiffList.size());
     DirectoryDiff diff = barDiffList.get(0);
     INodeDirectory testNode = fsdir.getINode4Write(test.toString())
@@ -2264,7 +2266,7 @@ public class TestRenameWithSnapshots {
     // of the snapshot diff for s2
     INodeDirectory dir2Node = fsdir.getINode4Write(dir2.toString())
         .asDirectory();
-    List<DirectoryDiff> dir2DiffList = dir2Node.getDiffs().asList();
+    DiffList<DirectoryDiff> dir2DiffList = dir2Node.getDiffs().asList();
     // dir2Node should contain 1 snapshot diffs for s2
     assertEquals(1, dir2DiffList.size());
     dList = dir2DiffList.get(0).getChildrenDiff().getList(ListType.DELETED);
@@ -2318,7 +2320,7 @@ public class TestRenameWithSnapshots {
         "foo/bar");
     INodeDirectory barNode = fsdir.getINode(barInS0.toString()).asDirectory();
     assertEquals(0, barNode.getChildrenList(Snapshot.CURRENT_STATE_ID).size());
-    List<DirectoryDiff> diffList = barNode.getDiffs().asList();
+    DiffList<DirectoryDiff> diffList = barNode.getDiffs().asList();
     assertEquals(1, diffList.size());
     DirectoryDiff diff = diffList.get(0);
     assertEquals(0, diff.getChildrenDiff().getList(ListType.DELETED).size());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ea7d78c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSetQuotaWithSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSetQuotaWithSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSetQuotaWithSnapshot.java
index c5ac26e..2fecbb1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSetQuotaWithSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSetQuotaWithSnapshot.java
@@ -149,7 +149,8 @@ public class TestSetQuotaWithSnapshot {
     hdfs.setQuota(dir, HdfsConstants.QUOTA_RESET, HdfsConstants.QUOTA_RESET);
     INode subNode = fsdir.getINode4Write(subDir.toString());
     assertTrue(subNode.asDirectory().isWithSnapshot());
-    List<DirectoryDiff> diffList = subNode.asDirectory().getDiffs().asList();
+    DiffList<DirectoryDiff> diffList =
+        subNode.asDirectory().getDiffs().asList();
     assertEquals(1, diffList.size());
     Snapshot s2 = dirNode.getSnapshot(DFSUtil.string2Bytes("s2"));
     assertEquals(s2.getId(), diffList.get(0).getSnapshotId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ea7d78c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotRename.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotRename.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotRename.java
index 8c8fca7..01157e8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotRename.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotRename.java
@@ -24,7 +24,6 @@ import static org.junit.Assert.fail;
 
 import java.io.ByteArrayOutputStream;
 import java.io.PrintStream;
-import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -101,7 +100,7 @@ public class TestSnapshotRename {
     for (int i = 0; i < listByName.size(); i++) {
       assertEquals(sortedNames[i], listByName.get(i).getRoot().getLocalName());
     }
-    List<DirectoryDiff> listByTime = srcRoot.getDiffs().asList();
+    DiffList<DirectoryDiff> listByTime = srcRoot.getDiffs().asList();
     assertEquals(names.length, listByTime.size());
     for (int i = 0; i < listByTime.size(); i++) {
       Snapshot s = srcRoot.getDirectorySnapshottableFeature().getSnapshotById(


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org