You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by sz...@apache.org on 2013/04/24 04:11:19 UTC

svn commit: r1471228 - in /hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/client/ src/main/java/org/apache/hadoop/hdfs/protocol/ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/main/j...

Author: szetszwo
Date: Wed Apr 24 02:11:18 2013
New Revision: 1471228

URL: http://svn.apache.org/r1471228
Log:
HDFS-4738. Changes AbstractINodeDiff to implement Comparable<Integer>, and fix javadoc and other warnings.

Modified:
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Content.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/Diff.java

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt?rev=1471228&r1=1471227&r2=1471228&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt Wed Apr 24 02:11:18 2013
@@ -1,3 +1,21 @@
+/**
+ * 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.
+ */
+
 Branch-2802 Snapshot (Unreleased)
 
   HDFS-4076. Support snapshot of single files.  (szetszwo)
@@ -272,3 +290,6 @@ Branch-2802 Snapshot (Unreleased)
 
   HDFS-4735. DisallowSnapshot throws IllegalStateException for nested
   snapshottable directories.  (Jing Zhao via szetszwo)
+
+  HDFS-4738. Changes AbstractINodeDiff to implement Comparable<Integer>, and
+  fix javadoc and other warnings.  (szetszwo)

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java?rev=1471228&r1=1471227&r2=1471228&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java Wed Apr 24 02:11:18 2013
@@ -108,7 +108,7 @@ public class HdfsAdmin {
   
   /**
    * Allow snapshot on a directory.
-   * @param the path of the directory where snapshots will be taken
+   * @param path The path of the directory where snapshots will be taken.
    */
   public void allowSnapshot(Path path) throws IOException {
     dfs.allowSnapshot(path);
@@ -116,7 +116,7 @@ public class HdfsAdmin {
   
   /**
    * Disallow snapshot on a directory.
-   * @param path of the snapshottable directory.
+   * @param path The path of the snapshottable directory.
    */
   public void disallowSnapshot(Path path) throws IOException {
     dfs.disallowSnapshot(path);

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java?rev=1471228&r1=1471227&r2=1471228&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java Wed Apr 24 02:11:18 2013
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.protocol;
 
 import java.io.PrintStream;
 import java.text.SimpleDateFormat;
+import java.util.Comparator;
 import java.util.Date;
 
 import org.apache.hadoop.fs.Path;
@@ -28,8 +29,20 @@ import org.apache.hadoop.hdfs.DFSUtil;
 /**
  * Metadata about a snapshottable directory
  */
-public class SnapshottableDirectoryStatus
-    implements Comparable<SnapshottableDirectoryStatus> {
+public class SnapshottableDirectoryStatus {
+  /** Compare the statuses by full paths. */
+  public static final Comparator<SnapshottableDirectoryStatus> COMPARATOR
+      = new Comparator<SnapshottableDirectoryStatus>() {
+    @Override
+    public int compare(SnapshottableDirectoryStatus left,
+                       SnapshottableDirectoryStatus right) {
+      int d = DFSUtil.compareBytes(left.parentFullPath, right.parentFullPath);
+      return d != 0? d
+          : DFSUtil.compareBytes(left.dirStatus.getLocalNameInBytes(),
+              right.dirStatus.getLocalNameInBytes());
+    }
+  };
+
   /** Basic information of the snapshottable directory */
   private HdfsFileStatus dirStatus;
   
@@ -145,12 +158,4 @@ public class SnapshottableDirectoryStatu
   private static int maxLength(int n, Object value) {
     return Math.max(n, String.valueOf(value).length());
   }
-
-  @Override
-  public int compareTo(SnapshottableDirectoryStatus that) {
-    int d = DFSUtil.compareBytes(this.parentFullPath, that.parentFullPath);
-    return d != 0? d
-        : DFSUtil.compareBytes(this.dirStatus.getLocalNameInBytes(),
-            that.dirStatus.getLocalNameInBytes());
-  }
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Content.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Content.java?rev=1471228&r1=1471227&r2=1471228&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Content.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Content.java Wed Apr 24 02:11:18 2013
@@ -20,8 +20,7 @@ package org.apache.hadoop.hdfs.server.na
 import org.apache.hadoop.hdfs.util.EnumCounters;
 
 /**
- * The content types such as file, directory and symlink to be computed
- * in {@link INode#computeContentSummary(CountsMap)}.
+ * The content types such as file, directory and symlink to be computed.
  */
 public enum Content {
   /** The number of files. */

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java?rev=1471228&r1=1471227&r2=1471228&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java Wed Apr 24 02:11:18 2013
@@ -474,10 +474,7 @@ public class INodeDirectory extends INod
     clearChildren();
   }
 
-  /**
-   * Call {@link INode#cleanSubtree(SnapshotDeletionInfo, BlocksMapUpdateInfo)}
-   * recursively down the subtree.
-   */
+  /** Call cleanSubtree(..) recursively down the subtree. */
   public Quota.Counts cleanSubtreeRecursively(final Snapshot snapshot,
       Snapshot prior, final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) throws QuotaExceededException {

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java?rev=1471228&r1=1471227&r2=1471228&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java Wed Apr 24 02:11:18 2013
@@ -48,7 +48,7 @@ import com.google.common.base.Preconditi
  */
 abstract class AbstractINodeDiff<N extends INode,
                                  D extends AbstractINodeDiff<N, D>>
-    implements Comparable<Snapshot> {
+    implements Comparable<Integer> {
 
   /** The snapshot will be obtained after this diff is applied. */
   Snapshot snapshot;
@@ -72,8 +72,8 @@ abstract class AbstractINodeDiff<N exten
 
   /** Compare diffs with snapshot ID. */
   @Override
-  public final int compareTo(final Snapshot that) {
-    return Snapshot.ID_COMPARATOR.compare(this.snapshot, that);
+  public final int compareTo(final Integer that) {
+    return Snapshot.ID_INTEGER_COMPARATOR.compare(this.snapshot.getId(), that);
   }
 
   /** @return the snapshot object of this diff. */

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java?rev=1471228&r1=1471227&r2=1471228&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java Wed Apr 24 02:11:18 2013
@@ -69,7 +69,7 @@ abstract class AbstractINodeDiffList<N e
   final Quota.Counts deleteSnapshotDiff(final Snapshot snapshot,
       Snapshot prior, final N currentINode,
       final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
-    int snapshotIndex = Collections.binarySearch(diffs, snapshot);
+    int snapshotIndex = Collections.binarySearch(diffs, snapshot.getId());
     
     Quota.Counts counts = Quota.Counts.newInstance();
     D removed = null;
@@ -151,7 +151,7 @@ abstract class AbstractINodeDiffList<N e
     if (anchor == null) {
       return getLastSnapshot();
     }
-    final int i = Collections.binarySearch(diffs, anchor);
+    final int i = Collections.binarySearch(diffs, anchor.getId());
     if (i == -1 || i == 0) {
       return null;
     } else {
@@ -182,7 +182,7 @@ abstract class AbstractINodeDiffList<N e
       // snapshot == null means the current state, therefore, return null.
       return null;
     }
-    final int i = Collections.binarySearch(diffs, snapshot);
+    final int i = Collections.binarySearch(diffs, snapshot.getId());
     if (i >= 0) {
       // exact match
       return diffs.get(i);
@@ -197,23 +197,22 @@ abstract class AbstractINodeDiffList<N e
   
   /**
    * Check if changes have happened between two snapshots.
-   * @param earlierSnapshot The snapshot taken earlier
-   * @param laterSnapshot The snapshot taken later
+   * @param earlier The snapshot taken earlier
+   * @param later The snapshot taken later
    * @return Whether or not modifications (including diretory/file metadata
    *         change, file creation/deletion under the directory) have happened
    *         between snapshots.
    */
-  final boolean changedBetweenSnapshots(Snapshot earlierSnapshot,
-      Snapshot laterSnapshot) {
+  final boolean changedBetweenSnapshots(Snapshot earlier, Snapshot later) {
     final int size = diffs.size();
-    int earlierDiffIndex = Collections.binarySearch(diffs, earlierSnapshot);
+    int earlierDiffIndex = Collections.binarySearch(diffs, earlier.getId());
     if (-earlierDiffIndex - 1 == size) {
       // if the earlierSnapshot is after the latest SnapshotDiff stored in
       // diffs, no modification happened after the earlierSnapshot
       return false;
     }
-    if (laterSnapshot != null) {
-      int laterDiffIndex = Collections.binarySearch(diffs, laterSnapshot);
+    if (later != null) {
+      int laterDiffIndex = Collections.binarySearch(diffs, later.getId());
       if (laterDiffIndex == -1 || laterDiffIndex == 0) {
         // if the laterSnapshot is the earliest SnapshotDiff stored in diffs, or
         // before it, no modification happened before the laterSnapshot

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java?rev=1471228&r1=1471227&r2=1471228&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java Wed Apr 24 02:11:18 2013
@@ -476,24 +476,24 @@ public class INodeDirectoryWithSnapshot 
    */
   boolean computeDiffBetweenSnapshots(Snapshot fromSnapshot,
       Snapshot toSnapshot, ChildrenDiff diff) {
-    Snapshot earlierSnapshot = fromSnapshot;
-    Snapshot laterSnapshot = toSnapshot;
+    Snapshot earlier = fromSnapshot;
+    Snapshot later = toSnapshot;
     if (Snapshot.ID_COMPARATOR.compare(fromSnapshot, toSnapshot) > 0) {
-      earlierSnapshot = toSnapshot;
-      laterSnapshot = fromSnapshot;
+      earlier = toSnapshot;
+      later = fromSnapshot;
     }
     
-    boolean modified = diffs.changedBetweenSnapshots(earlierSnapshot,
-        laterSnapshot);
+    boolean modified = diffs.changedBetweenSnapshots(earlier,
+        later);
     if (!modified) {
       return false;
     }
     
     final List<DirectoryDiff> difflist = diffs.asList();
     final int size = difflist.size();
-    int earlierDiffIndex = Collections.binarySearch(difflist, earlierSnapshot);
-    int laterDiffIndex = laterSnapshot == null ? size : Collections
-        .binarySearch(difflist, laterSnapshot);
+    int earlierDiffIndex = Collections.binarySearch(difflist, earlier.getId());
+    int laterDiffIndex = later == null ? size : Collections
+        .binarySearch(difflist, later.getId());
     earlierDiffIndex = earlierDiffIndex < 0 ? (-earlierDiffIndex - 1)
         : earlierDiffIndex;
     laterDiffIndex = laterDiffIndex < 0 ? (-laterDiffIndex - 1)
@@ -507,10 +507,8 @@ public class INodeDirectoryWithSnapshot 
       if (dirMetadataChanged == false && sdiff.snapshotINode != null) {
         if (dirCopy == null) {
           dirCopy = sdiff.snapshotINode;
-        } else {
-          if (!dirCopy.metadataEquals(sdiff.snapshotINode)) {
-            dirMetadataChanged = true;
-          }
+        } else if (!dirCopy.metadataEquals(sdiff.snapshotINode)) {
+          dirMetadataChanged = true;
         }
       }
     }
@@ -524,8 +522,9 @@ public class INodeDirectoryWithSnapshot 
         }
       }
       return !dirCopy.metadataEquals(this);
+    } else {
+      return false;
     }
-    return false;
   }
 
   /** Diff list sorted by snapshot IDs, i.e. in chronological order. */

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java?rev=1471228&r1=1471227&r2=1471228&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java Wed Apr 24 02:11:18 2013
@@ -71,18 +71,32 @@ public class Snapshot implements Compara
   }
 
   /**
-   * Compare snapshot IDs. Null indicates the current status thus is greater
-   * than non-null snapshots.
+   * Compare snapshot with IDs, where null indicates the current status thus
+   * is greater than any non-null snapshot.
    */
   public static final Comparator<Snapshot> ID_COMPARATOR
       = new Comparator<Snapshot>() {
     @Override
     public int compare(Snapshot left, Snapshot right) {
+      return ID_INTEGER_COMPARATOR.compare(
+          left == null? null: left.getId(),
+          right == null? null: right.getId());
+    }
+  };
+
+  /**
+   * Compare snapshot with IDs, where null indicates the current status thus
+   * is greater than any non-null ID.
+   */
+  public static final Comparator<Integer> ID_INTEGER_COMPARATOR
+      = new Comparator<Integer>() {
+    @Override
+    public int compare(Integer left, Integer right) {
       // null means the current state, thus should be the largest
       if (left == null) {
         return right == null? 0: 1;
       } else {
-        return right == null? -1: left.id - right.id; 
+        return right == null? -1: left - right; 
       }
     }
   };

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java?rev=1471228&r1=1471227&r2=1471228&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java Wed Apr 24 02:11:18 2013
@@ -229,8 +229,6 @@ public class SnapshotFSImageFormat {
    * Load the {@link SnapshotDiff} list for the INodeDirectoryWithSnapshot
    * directory.
    * @param dir The snapshottable directory for loading.
-   * @param numSnapshotDiffs The number of {@link SnapshotDiff} that the 
-   *                         directory has.
    * @param in The {@link DataInput} instance to read.
    * @param loader The {@link Loader} instance that this loading procedure is 
    *               using.

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java?rev=1471228&r1=1471227&r2=1471228&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java Wed Apr 24 02:11:18 2013
@@ -214,7 +214,7 @@ public class SnapshotManager implements 
   
   /**
    * Write {@link #snapshotCounter}, {@link #numSnapshots},
-   * {@link #numSnapshottableDirs} and all snapshots to the DataOutput.
+   * and all snapshots to the DataOutput.
    */
   public void write(DataOutput out) throws IOException {
     out.writeInt(snapshotCounter);
@@ -230,7 +230,7 @@ public class SnapshotManager implements 
   
   /**
    * Read values of {@link #snapshotCounter}, {@link #numSnapshots}, and
-   * {@link #numSnapshottableDirs} and all snapshots from the DataInput
+   * all snapshots from the DataInput
    */
   public Map<Integer, Snapshot> read(DataInput in, FSImageFormat.Loader loader
       ) throws IOException {
@@ -273,7 +273,7 @@ public class SnapshotManager implements 
         statusList.add(status);
       }
     }
-    Collections.sort(statusList);
+    Collections.sort(statusList, SnapshottableDirectoryStatus.COMPARATOR);
     return statusList.toArray(
         new SnapshottableDirectoryStatus[statusList.size()]);
   }
@@ -302,5 +302,4 @@ public class SnapshotManager implements 
     
     return snapshotRoot.computeDiff(from, to);
   }
- 
 }
\ No newline at end of file

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/Diff.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/Diff.java?rev=1471228&r1=1471227&r2=1471228&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/Diff.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/Diff.java Wed Apr 24 02:11:18 2013
@@ -104,7 +104,7 @@ public class Diff<K, E extends Diff.Elem
   }
   
   /** 
-   * Undo information for some operations such as {@link Diff#delete(E)}
+   * Undo information for some operations such as delete(E)
    * and {@link Diff#modify(Element, Element)}.
    */
   public static class UndoInfo<E> {
@@ -215,8 +215,8 @@ public class Diff<K, E extends Diff.Elem
   }
 
   /**
-   * Undo the previous {@link #create(E)} operation. Note that the behavior is
-   * undefined if the previous operation is not {@link #create(E)}.
+   * Undo the previous create(E) operation. Note that the behavior is
+   * undefined if the previous operation is not create(E).
    */
   public void undoCreate(final E element, final int insertionPoint) {
     remove(created, insertionPoint, element);
@@ -242,8 +242,8 @@ public class Diff<K, E extends Diff.Elem
   }
   
   /**
-   * Undo the previous {@link #delete(E)} operation. Note that the behavior is
-   * undefined if the previous operation is not {@link #delete(E)}.
+   * Undo the previous delete(E) operation. Note that the behavior is
+   * undefined if the previous operation is not delete(E).
    */
   public void undoDelete(final E element, final UndoInfo<E> undoInfo) {
     final int c = undoInfo.createdInsertionPoint;
@@ -285,8 +285,8 @@ public class Diff<K, E extends Diff.Elem
   }
 
   /**
-   * Undo the previous {@link #modify(E, E)} operation. Note that the behavior
-   * is undefined if the previous operation is not {@link #modify(E, E)}.
+   * Undo the previous modify(E, E) operation. Note that the behavior
+   * is undefined if the previous operation is not modify(E, E).
    */
   public void undoModify(final E oldElement, final E newElement,
       final UndoInfo<E> undoInfo) {
@@ -383,24 +383,24 @@ public class Diff<K, E extends Diff.Elem
    * 1.2 (0, d')  in this diff: put in c-list --> (c, d')
    * 1.3 (c', d') in this diff: impossible
    * 1.4 (0, 0)   in this diff: put in c-list --> (c, 0)
-   * This is the same logic as {@link #create(E)}.
+   * This is the same logic as create(E).
    * 
    * 2. For (0, d) in the posterior diff,
    * 2.1 (c', 0)  in this diff: remove from c-list --> (0, 0)
    * 2.2 (0, d')  in this diff: impossible
    * 2.3 (c', d') in this diff: remove from c-list --> (0, d')
    * 2.4 (0, 0)   in this diff: put in d-list --> (0, d)
-   * This is the same logic as {@link #delete(E)}.
+   * This is the same logic as delete(E).
    * 
    * 3. For (c, d) in the posterior diff,
    * 3.1 (c', 0)  in this diff: replace the element in c-list --> (c, 0)
    * 3.2 (0, d')  in this diff: impossible
    * 3.3 (c', d') in this diff: replace the element in c-list --> (c, d')
    * 3.4 (0, 0)   in this diff: put in c-list and d-list --> (c, d)
-   * This is the same logic as {@link #modify(E, E)}.
+   * This is the same logic as modify(E, E).
    * </pre>
    * 
-   * @param the posterior diff to combine with.
+   * @param posterior The posterior diff to combine with.
    * @param deletedProcesser
    *     process the deleted/overwritten elements in case 2.1, 2.3, 3.1 and 3.3.
    */