Преглед изворни кода

Merge r1480838 and r1480840 from trunk for merging HDFS Snapshot feature.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1488089 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze пре 12 година
родитељ
комит
82f91de0b6
100 измењених фајлова са 19081 додато и 1542 уклоњено
  1. 45 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
  2. 18 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
  3. 1 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java
  4. 173 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/SnapshotCommands.java
  5. 2 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
  6. 342 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  7. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
  8. 144 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  9. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  10. 62 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  11. 53 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  12. 16 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
  13. 107 19
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  14. 23 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/FSLimitException.java
  15. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
  16. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
  17. 13 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/NSQuotaExceededException.java
  18. 193 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java
  19. 82 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotInfo.java
  20. 168 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
  21. 124 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  22. 109 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  23. 153 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  24. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java
  25. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
  26. 72 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Content.java
  27. 475 170
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  28. 60 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  29. 65 35
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  30. 309 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  31. 6 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java
  32. 55 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  33. 8 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageCompression.java
  34. 444 138
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  35. 173 61
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  36. 385 72
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  37. 61 47
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
  38. 469 318
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  39. 450 284
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  40. 100 38
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java
  41. 280 63
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  42. 58 42
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
  43. 131 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
  44. 681 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
  45. 58 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
  46. 257 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java
  47. 417 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
  48. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
  49. 62 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  50. 13 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
  51. 64 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Quota.java
  52. 43 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java
  53. 144 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
  54. 318 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
  55. 57 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
  56. 213 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshot.java
  57. 529 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectorySnapshottable.java
  58. 895 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java
  59. 126 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileUnderConstructionWithSnapshot.java
  60. 113 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithSnapshot.java
  61. 215 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java
  62. 33 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotAccessControlException.java
  63. 33 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotException.java
  64. 361 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java
  65. 370 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
  66. 36 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotStats.java
  67. 62 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
  68. 262 42
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
  69. 32 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.java
  70. 58 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/snapshot/LsSnapshottableDir.java
  71. 90 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/snapshot/SnapshotDiff.java
  72. 485 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/Diff.java
  73. 170 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/EnumCounters.java
  74. 240 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ReadOnlyList.java
  75. 1 18
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  76. 69 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  77. 55 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
  78. 262 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/xdoc/HdfsSnapshots.xml
  79. 19 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
  80. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
  81. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
  82. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
  83. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
  84. 9 25
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.java
  85. 453 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
  86. 11 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java
  87. 38 66
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
  88. 438 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
  89. 480 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
  90. 154 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestDisallowModifyROSnapshot.java
  91. 186 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java
  92. 368 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestNestedSnapshots.java
  93. 1918 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
  94. 161 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSetQuotaWithSnapshot.java
  95. 940 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java
  96. 208 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java
  97. 887 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
  98. 305 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java
  99. 120 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotListing.java
  100. 80 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotManager.java

+ 45 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java

@@ -2266,6 +2266,51 @@ public abstract class FileSystem extends Configured implements Closeable {
       ) throws IOException {
       ) throws IOException {
   }
   }
 
 
+  /**
+   * Create a snapshot with a default name.
+   * @param path The directory where snapshots will be taken.
+   * @return the snapshot path.
+   */
+  public final Path createSnapshot(Path path) throws IOException {
+    return createSnapshot(path, null);
+  }
+
+  /**
+   * Create a snapshot
+   * @param path The directory where snapshots will be taken.
+   * @param snapshotName The name of the snapshot
+   * @return the snapshot path.
+   */
+  public Path createSnapshot(Path path, String snapshotName)
+      throws IOException {
+    throw new UnsupportedOperationException(getClass().getSimpleName()
+        + " doesn't support createSnapshot");
+  }
+  
+  /**
+   * Rename a snapshot
+   * @param path The directory path where the snapshot was taken
+   * @param snapshotOldName Old name of the snapshot
+   * @param snapshotNewName New name of the snapshot
+   * @throws IOException
+   */
+  public void renameSnapshot(Path path, String snapshotOldName,
+      String snapshotNewName) throws IOException {
+    throw new UnsupportedOperationException(getClass().getSimpleName()
+        + " doesn't support renameSnapshot");
+  }
+  
+  /**
+   * Delete a snapshot of a directory
+   * @param path  The directory that the to-be-deleted snapshot belongs to
+   * @param snapshotName The name of the snapshot
+   */
+  public void deleteSnapshot(Path path, String snapshotName)
+      throws IOException {
+    throw new UnsupportedOperationException(getClass().getSimpleName()
+        + " doesn't support deleteSnapshot");
+  }
+  
   // making it volatile to be able to do a double checked locking
   // making it volatile to be able to do a double checked locking
   private volatile static boolean FILE_SYSTEMS_LOADED = false;
   private volatile static boolean FILE_SYSTEMS_LOADED = false;
 
 

+ 18 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java

@@ -462,4 +462,22 @@ public class FilterFileSystem extends FileSystem {
   public FileSystem[] getChildFileSystems() {
   public FileSystem[] getChildFileSystems() {
     return new FileSystem[]{fs};
     return new FileSystem[]{fs};
   }
   }
+
+  @Override // FileSystem
+  public Path createSnapshot(Path path, String snapshotName)
+      throws IOException {
+    return fs.createSnapshot(path, snapshotName);
+  }
+  
+  @Override // FileSystem
+  public void renameSnapshot(Path path, String snapshotOldName,
+      String snapshotNewName) throws IOException {
+    fs.renameSnapshot(path, snapshotOldName, snapshotNewName);
+  }
+  
+  @Override // FileSystem
+  public void deleteSnapshot(Path path, String snapshotName)
+      throws IOException {
+    fs.deleteSnapshot(path, snapshotName);
+  }
 }
 }

+ 1 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java

@@ -57,6 +57,7 @@ abstract public class FsCommand extends Command {
     factory.registerCommands(Tail.class);
     factory.registerCommands(Tail.class);
     factory.registerCommands(Test.class);
     factory.registerCommands(Test.class);
     factory.registerCommands(Touch.class);
     factory.registerCommands(Touch.class);
+    factory.registerCommands(SnapshotCommands.class);
   }
   }
 
 
   protected FsCommand() {}
   protected FsCommand() {}

+ 173 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/SnapshotCommands.java

@@ -0,0 +1,173 @@
+/**
+ * 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.fs.shell;
+
+import java.io.IOException;
+import java.util.LinkedList;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIsNotDirectoryException;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Snapshot related operations
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+
+class SnapshotCommands extends FsCommand {
+  private final static String CREATE_SNAPSHOT = "createSnapshot";
+  private final static String DELETE_SNAPSHOT = "deleteSnapshot";
+  private final static String RENAME_SNAPSHOT = "renameSnapshot";
+  
+  public static void registerCommands(CommandFactory factory) {
+    factory.addClass(CreateSnapshot.class, "-" + CREATE_SNAPSHOT);
+    factory.addClass(DeleteSnapshot.class, "-" + DELETE_SNAPSHOT);
+    factory.addClass(RenameSnapshot.class, "-" + RENAME_SNAPSHOT);
+  }
+  
+  /**
+   *  Create a snapshot
+   */
+  public static class CreateSnapshot extends FsCommand {
+    public static final String NAME = CREATE_SNAPSHOT;
+    public static final String USAGE = "<snapshotDir> [<snapshotName>]";
+    public static final String DESCRIPTION = "Create a snapshot on a directory";
+
+    private String snapshotName = null;
+
+    @Override
+    protected void processPath(PathData item) throws IOException {
+      if (!item.stat.isDirectory()) {
+        throw new PathIsNotDirectoryException(item.toString());
+      }
+    }
+    
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      if (args.size() == 0) {
+        throw new IllegalArgumentException("<snapshotDir> is missing.");
+      } 
+      if (args.size() > 2) {
+        throw new IllegalArgumentException("Too many arguements.");
+      }
+      if (args.size() == 2) {
+        snapshotName = args.removeLast();
+      }
+    }
+
+    @Override
+    protected void processArguments(LinkedList<PathData> items)
+    throws IOException {
+      super.processArguments(items);
+      if (exitCode != 0) { // check for error collecting paths
+        return;
+      }
+      assert(items.size() == 1);
+      PathData sroot = items.getFirst();
+      Path snapshotPath = sroot.fs.createSnapshot(sroot.path, snapshotName);
+      out.println("Created snapshot " + snapshotPath);
+    }    
+  }
+
+  /**
+   * Delete a snapshot
+   */
+  public static class DeleteSnapshot extends FsCommand {
+    public static final String NAME = DELETE_SNAPSHOT;
+    public static final String USAGE = "<snapshotDir> <snapshotName>";
+    public static final String DESCRIPTION = 
+        "Delete a snapshot from a directory";
+
+    private String snapshotName;
+
+    @Override
+    protected void processPath(PathData item) throws IOException {
+      if (!item.stat.isDirectory()) {
+        throw new PathIsNotDirectoryException(item.toString());
+      }
+    }
+
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      if (args.size() != 2) {
+        throw new IOException("args number not 2: " + args.size());
+      }
+      snapshotName = args.removeLast();
+    }
+
+    @Override
+    protected void processArguments(LinkedList<PathData> items)
+        throws IOException {
+      super.processArguments(items);
+      if (exitCode != 0) { // check for error collecting paths
+        return;
+      }
+      assert (items.size() == 1);
+      PathData sroot = items.getFirst();
+      sroot.fs.deleteSnapshot(sroot.path, snapshotName);
+    }
+  }
+  
+  /**
+   * Rename a snapshot
+   */
+  public static class RenameSnapshot extends FsCommand {
+    public static final String NAME = RENAME_SNAPSHOT;
+    public static final String USAGE = "<snapshotDir> <oldName> <newName>";
+    public static final String DESCRIPTION = 
+        "Rename a snapshot from oldName to newName";
+    
+    private String oldName;
+    private String newName;
+    
+    @Override
+    protected void processPath(PathData item) throws IOException {
+      if (!item.stat.isDirectory()) {
+        throw new PathIsNotDirectoryException(item.toString());
+      }
+    }
+
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      if (args.size() != 3) {
+        throw new IOException("args number not 3: " + args.size());
+      }
+      newName = args.removeLast();
+      oldName = args.removeLast();
+    }
+
+    @Override
+    protected void processArguments(LinkedList<PathData> items)
+        throws IOException {
+      super.processArguments(items);
+      if (exitCode != 0) { // check for error collecting paths
+        return;
+      }
+      Preconditions.checkArgument(items.size() == 1);
+      PathData sroot = items.getFirst();
+      sroot.fs.renameSnapshot(sroot.path, oldName, newName);
+    }
+    
+  }
+}
+

+ 2 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java

@@ -218,6 +218,8 @@ public class TestFilterFileSystem {
         continue;
         continue;
       if (Modifier.isPrivate(m.getModifiers()))
       if (Modifier.isPrivate(m.getModifiers()))
         continue;
         continue;
+      if (Modifier.isFinal(m.getModifiers()))
+        continue;
       
       
       try {
       try {
         DontCheck.class.getMethod(m.getName(), m.getParameterTypes());
         DontCheck.class.getMethod(m.getName(), m.getParameterTypes());

+ 342 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -9,6 +9,9 @@ Release 2.0.5-beta - UNRELEASED
 
 
     HDFS-4434. Provide a mapping from INodeId to INode. (suresh)
     HDFS-4434. Provide a mapping from INodeId to INode. (suresh)
 
 
+    HDFS-2802. Add HDFS Snapshot feature.  (See breakdown of tasks below for
+    subtasks and contributors)
+
   NEW FEATURES
   NEW FEATURES
 
 
     HDFS-1804. Add a new block-volume device choosing policy that looks at
     HDFS-1804. Add a new block-volume device choosing policy that looks at
@@ -423,6 +426,345 @@ Release 2.0.5-beta - UNRELEASED
     HDFS-4610. Use common utils FileUtil#setReadable/Writable/Executable and 
     HDFS-4610. Use common utils FileUtil#setReadable/Writable/Executable and 
     FileUtil#canRead/Write/Execute. (Ivan Mitic via suresh)
     FileUtil#canRead/Write/Execute. (Ivan Mitic via suresh)
 
 
+  BREAKDOWN OF HDFS-2802 HDFS SNAPSHOT SUBTASKS
+
+    HDFS-4076. Support snapshot of single files.  (szetszwo)
+
+    HDFS-4082. Add editlog opcodes for snapshot create and delete operations.
+    (suresh via szetszwo)
+
+    HDFS-4086. Add editlog opcodes to allow and disallow snapshots on a
+    directory. (Brandon Li via suresh)
+
+    HDFS-4083. Protocol changes for snapshots. (suresh)
+
+    HDFS-4077. Add support for Snapshottable Directory. (szetszwo via suresh)
+
+    HDFS-4087. Protocol changes for listSnapshots functionality.
+    (Brandon Li via suresh)
+
+    HDFS-4079. Add SnapshotManager which maintains a list for all the
+    snapshottable directories and supports snapshot methods such as setting a
+    directory to snapshottable and creating a snapshot.  (szetszwo)
+
+    HDFS-4078. Handle replication in snapshots.  (szetszwo)
+
+    HDFS-4084. Provide CLI support to allow and disallow snapshot
+    on a directory. (Brondon Li via suresh)
+
+    HDFS-4091. Add snapshot quota to limit the number of snapshots allowed.
+    (szetszwo)
+
+    HDFS-4097. Provide CLI support for createSnapshot. (Brandon Li via suresh)
+
+    HDFS-4092. Update file deletion logic for snapshot so that the current inode
+    is removed from the circular linked list; and if some blocks at the end of
+    the block list no longer belong to any other inode, collect them and update
+    the block list.  (szetszwo)
+
+    HDFS-4111. Support snapshot of subtrees. (szetszwo via suresh)
+
+    HDFS-4119. Complete the allowSnapshot code and add a test for it. (szetszwo)
+
+    HDFS-4133. Add testcases for testing basic snapshot functionalities.
+    (Jing Zhao via suresh)
+
+    HDFS-4116. Add auditlog for some snapshot operations. (Jing Zhao via suresh)
+
+    HDFS-4095. Add some snapshot related metrics. (Jing Zhao via suresh)
+
+    HDFS-4141. Support directory diff - the difference between the current state
+    and a previous snapshot of an INodeDirectory. (szetszwo)
+
+    HDFS-4146. Use getter and setter in INodeFileWithLink to access blocks and
+    initialize root directory as snapshottable. (szetszwo)
+
+    HDFS-4149. Implement the disallowSnapshot(..) in FSNamesystem and add
+    resetSnapshottable(..) to SnapshotManager. (szetszwo)
+
+    HDFS-4147. When there is a snapshot in a subtree, deletion of the subtree
+    should fail. (Jing Zhao via szetszwo)
+
+    HDFS-4150.  Update the inode in the block map when a snapshotted file or a
+    snapshot file is deleted. (Jing Zhao via szetszwo)
+
+    HDFS-4159. Rename should fail when the destination directory is
+    snapshottable and has snapshots. (Jing Zhao via szetszwo)
+
+    HDFS-4170. Add snapshot information to INodesInPath.  (szetszwo)
+
+    HDFS-4177. Add a snapshot parameter to INodeDirectory.getChildrenList() for
+    selecting particular snapshot children list views.  (szetszwo)
+
+    HDFS-4148. Disallow write/modify operations on files and directories in a
+    snapshot. (Brandon Li via suresh)
+
+    HDFS-4188. Add Snapshot.ID_COMPARATOR for comparing IDs and fix a bug in
+    ReadOnlyList.Util.binarySearch(..).  (szetszwo)
+
+    HDFS-4187. Add tests for replication handling in snapshots. (Jing Zhao via
+    szetszwo)
+
+    HDFS-4196. Support renaming of snapshots. (Jing Zhao via szetszwo)
+
+    HDFS-4175. Additional snapshot tests for more complicated directory
+    structure and modifications. (Jing Zhao via suresh)
+
+    HDFS-4293. Fix TestSnapshot failure. (Jing Zhao via suresh)
+
+    HDFS-4317. Change INode and its subclasses to support HDFS-4103. (szetszwo)
+
+    HDFS-4103. Support O(1) snapshot creation. (szetszwo)
+
+    HDFS-4330. Support snapshots up to the snapshot limit. (szetszwo)
+
+    HDFS-4357. Fix a bug that if an inode is replaced, further INode operations
+    should apply to the new inode. (Jing Zhao via szetszwo)
+
+    HDFS-4230. Support listing of all the snapshottable directories.  (Jing Zhao
+    via szetszwo)
+
+    HDFS-4244. Support snapshot deletion.  (Jing Zhao via szetszwo)
+
+    HDFS-4245. Include snapshot related operations in TestOfflineEditsViewer.
+    (Jing Zhao via szetszwo)
+
+    HDFS-4395. In INodeDirectorySnapshottable's constructor, the passed-in dir
+    could be an INodeDirectoryWithSnapshot.  (Jing Zhao via szetszwo)
+
+    HDFS-4397. Fix a bug in INodeDirectoryWithSnapshot.Diff.combinePostDiff(..)
+    that it may put the wrong node into the deleted list.  (szetszwo)
+
+    HDFS-4407. Change INodeDirectoryWithSnapshot.Diff.combinePostDiff(..) to
+    merge-sort like and keep the postDiff parameter unmodified.  (szetszwo)
+
+    HDFS-4098. Add FileWithSnapshot, INodeFileUnderConstructionWithSnapshot and
+    INodeFileUnderConstructionSnapshot for supporting append to snapshotted
+    files.  (szetszwo)
+
+    HDFS-4126. Add reading/writing snapshot information to FSImage.
+    (Jing Zhao via suresh)
+
+    HDFS-4436. Change INode.recordModification(..) to return only the current
+    inode and remove the updateCircularList parameter from some methods in
+    INodeDirectoryWithSnapshot.Diff.  (szetszwo)
+
+    HDFS-4429. When the latest snapshot exists, INodeFileUnderConstruction
+    should be replaced with INodeFileWithSnapshot but not INodeFile.
+    (Jing Zhao via szetszwo)
+
+    HDFS-4441. Move INodeDirectoryWithSnapshot.Diff and the related classes to a
+    package.  (szetszwo)
+
+    HDFS-4432. Support INodeFileUnderConstructionWithSnapshot in FSImage
+    saving/loading. (Jing Zhao via suresh)
+
+    HDFS-4131. Add capability to namenode to get snapshot diff. (Jing Zhao via
+    suresh)
+
+    HDFS-4447. Refactor INodeDirectoryWithSnapshot for supporting general INode
+    diff lists.  (szetszwo)
+
+    HDFS-4189. Renames the getMutableXxx methods to getXxx4Write and fix a bug
+    that some getExistingPathINodes calls should be getINodesInPath4Write.
+    (szetszwo)
+
+    HDFS-4361. When listing snapshottable directories, only return those
+    where the user has permission to take snapshots.  (Jing Zhao via szetszwo)
+
+    HDFS-4464. Combine collectSubtreeBlocksAndClear with deleteDiffsForSnapshot
+    and rename it to destroySubtreeAndCollectBlocks.  (szetszwo)
+
+    HDFS-4414. Add support for getting snapshot diff from DistributedFileSystem.
+    (Jing Zhao via suresh)
+
+    HDFS-4446. Support file snapshots with diff lists.  (szetszwo)
+
+    HDFS-4480. Eliminate the file snapshot circular linked list.  (szetszwo)
+
+    HDFS-4481. Change fsimage to support snapshot file diffs.  (szetszwo)
+
+    HDFS-4500. Refactor snapshot INode methods.  (szetszwo)
+
+    HDFS-4487. Fix snapshot diff report for HDFS-4446.  (Jing Zhao via szetszwo)
+
+    HDFS-4431. Support snapshot in OfflineImageViewer.  (Jing Zhao via szetszwo)
+
+    HDFS-4503. Update computeContentSummary(..), spaceConsumedInTree(..) and
+    diskspaceConsumed(..) in INode for snapshot.  (szetszwo)
+
+    HDFS-4499. Fix file/directory/snapshot deletion for file diff.  (Jing Zhao
+    via szetszwo)
+
+    HDFS-4524. Update SnapshotManager#snapshottables when loading fsimage.
+    (Jing Zhao via szetszwo)
+
+    HDFS-4520. Support listing snapshots under a snapshottable directory using
+    ls.  (Jing Zhao via szetszwo)
+
+    HDFS-4514. Add CLI for supporting snapshot rename, diff report, and
+    snapshottable directory listing.  (Jing Zhao via szetszwo)
+
+    HDFS-4523. Fix INodeFile replacement, TestQuota and javac errors from trunk
+    merge.  (szetszwo)
+
+    HDFS-4507. Update quota verification for snapshots.  (szetszwo)
+
+    HDFS-4545. With snapshots, FSDirectory.unprotectedSetReplication(..) always
+    changes file replication but it may or may not changes block replication.
+    (szetszwo)
+
+    HDFS-4557. Fix FSDirectory#delete when INode#cleanSubtree returns 0.
+    (Jing Zhao via szetszwo)
+
+    HDFS-4579. Annotate snapshot tests. (Arpit Agarwal via suresh)
+
+    HDFS-4574. Move Diff to the util package.  (szetszwo)
+
+    HDFS-4563. Update namespace/diskspace usage after deleting snapshots.
+    (Jing Zhao via szetszwo)
+
+    HDFS-4144. Create test for all snapshot-related metrics.
+    (Jing Zhao via suresh)
+
+    HDFS-4556. Add snapshotdiff and LsSnapshottableDir tools to hdfs script.
+    (Arpit Agarwal via szetszwo)
+
+    HDFS-4534. Add INodeReference in order to support rename with snapshots.
+    (szetszwo)
+
+    HDFS-4616. Update the FilesDeleted metric while deleting file/dir in the
+    current tree.  (Jing Zhao via szetszwo)
+
+    HDFS-4627. Fix FSImageFormat#Loader NPE and synchronization issues.
+    (Jing Zhao via suresh)
+
+    HDFS-4612. Not to use INode.getParent() when generating snapshot diff
+    report.  (Jing Zhao via szetszwo)
+
+    HDFS-4636. Update quota usage when deleting files/dirs that were created
+    after taking the latest snapshot. (Jing Zhao via szetszwo)
+
+    HDFS-4648. For snapshot deletion, when merging the diff from to-delete
+    snapshot to the prior snapshot, make sure files/directories created after
+    the prior snapshot get deleted. (Jing Zhao via szetszwo)
+
+    HDFS-4637. INodeDirectory#replaceSelf4Quota may incorrectly convert a newly
+    created directory to an INodeDirectoryWithSnapshot. (Jing Zhao via szetszwo)
+
+    HDFS-4611. Update FSImage for INodeReference.  (szetszwo)
+
+    HDFS-4647. Rename should call setLocalName after an inode is removed from
+    snapshots.  (Arpit Agarwal via szetszwo)
+
+    HDFS-4684. Use INode id for image serialization when writing INodeReference.
+    (szetszwo)
+
+    HDFS-4675. Fix rename across snapshottable directories.  (Jing Zhao via
+    szetszwo)
+
+    HDFS-4692. Use timestamp as default snapshot names.  (szetszwo)
+
+    HDFS-4666. Define ".snapshot" as a reserved inode name so that users cannot
+    create a file/directory with ".snapshot" as the name.  If ".snapshot" is
+    used in a previous version of HDFS, it must be renamed before upgrade;
+    otherwise, upgrade will fail.  (szetszwo)
+
+    HDFS-4700. Fix the undo section of rename with snapshots.  (Jing Zhao via
+    szetszwo)
+
+    HDFS-4529. Disallow concat when one of the src files is in some snapshot.
+    (szetszwo)
+
+    HDFS-4550. Refactor INodeDirectory.INodesInPath to a standalone class.
+    (szetszwo)
+
+    HDFS-4707. Add snapshot methods to FilterFileSystem and fix findbugs
+    warnings.  (szetszwo)
+
+    HDFS-4706. Do not replace root inode for disallowSnapshot.  (szetszwo)
+
+    HDFS-4717. Change the path parameter type of the snapshot methods in
+    HdfsAdmin from String to Path.  (szetszwo)
+
+    HDFS-4708. Add snapshot user documentation.  (szetszwo)
+
+    HDFS-4726. Fix test failures after merging the INodeId-INode mapping
+    from trunk.  (Jing Zhao via szetszwo)
+
+    HDFS-4727. Update inodeMap after deleting files/directories/snapshots.
+    (Jing Zhao via szetszwo)
+
+    HDFS-4719. Remove AbstractINodeDiff.Factory and move its methods to
+    AbstractINodeDiffList.  (Arpit Agarwal via szetszwo)
+
+    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)
+
+    HDFS-4686. Update quota computation for rename and INodeReference.
+    (Jing Zhao via szetszwo)
+
+    HDFS-4729. Fix OfflineImageViewer and permission checking for snapshot
+    operations.  (Jing Zhao via szetszwo)
+
+    HDFS-4749. Use INodeId to identify the corresponding directory node in
+    FSImage saving/loading.  (Jing Zhao via szetszwo)
+
+    HDFS-4742. Fix appending to a renamed file with snapshot.  (Jing Zhao via
+    szetszwo)
+
+    HDFS-4755. Fix AccessControlException message and moves "implements
+    LinkedElement" from INode to INodeWithAdditionalFields.  (szetszwo)
+
+    HDFS-4650. Fix a bug in FSDirectory and add more unit tests for rename with
+    existence of snapshottable directories and snapshots.  (Jing Zhao via
+    szetszwo)
+
+    HDFS-4650. When passing two non-existing snapshot names to snapshotDiff, it
+    returns success if the names are the same.  (Jing Zhao via szetszwo)
+
+    HDFS-4767. If a directory is snapshottable, do not replace it when clearing
+    quota.  (Jing Zhao via szetszwo)
+
+    HDFS-4578.  Restrict snapshot IDs to 24-bit wide.  (Arpit Agarwal via
+    szetszwo)
+
+    HDFS-4773. Fix bugs in quota usage computation and OfflineImageViewer.
+    (Jing Zhao via szetszwo)
+
+    HDFS-4760. Update inodeMap after node replacement.  (Jing Zhao via szetszwo)
+
+    HDFS-4758. Disallow nested snapshottable directories and unwrap
+    RemoteException.  (szetszwo)
+
+    HDFS-4781. Fix a NullPointerException when listing .snapshot under
+    a non-existing directory.  (szetszwo)
+
+    HDFS-4791. Update and fix deletion of reference inode.  (Jing Zhao via
+    szetszwo)
+
+    HDFS-4798. Update computeContentSummary() for the reference nodes in
+    snapshots.  (szetszwo)
+
+    HDFS-4800. Fix INodeDirectoryWithSnapshot#cleanDeletedINode.  (Jing Zhao via
+    szetszwo)
+
+    HDFS-4801. lsSnapshottableDir throws IllegalArgumentException when root is
+    snapshottable.  (Jing Zhao via szetszwo)
+
+    HDFS-4802. Disallowing snapshot on / twice should throw SnapshotException
+    but not IllegalStateException.  (Jing Zhao via szetszwo)
+
+    HDFS-4806. In INodeDirectoryWithSnapshot, use isInLatestSnapshot() to
+    determine if an added/removed child should be recorded in the snapshot diff.
+    (Jing Zhao via szetszwo)
+
+    HDFS-4809. When a QuotaExceededException is thrown during rename, the quota
+    usage should be subtracted back.  (Jing Zhao via szetszwo)
+
 Release 2.0.4-alpha - 2013-04-25
 Release 2.0.4-alpha - 2013-04-25
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs

@@ -53,6 +53,9 @@ function print_usage(){
   echo "  fetchdt              fetch a delegation token from the NameNode"
   echo "  fetchdt              fetch a delegation token from the NameNode"
   echo "  getconf              get config values from configuration"
   echo "  getconf              get config values from configuration"
   echo "  groups               get the groups which users belong to"
   echo "  groups               get the groups which users belong to"
+  echo "  snapshotDiff         diff two snapshots of a directory or diff the"
+  echo "                       current directory contents with a snapshot"
+  echo "  lsSnapshottableDir   list all snapshottable dirs owned by the current user"
   echo "						Use -help to see options"
   echo "						Use -help to see options"
   echo ""
   echo ""
   echo "Most commands print help when invoked w/o parameters."
   echo "Most commands print help when invoked w/o parameters."
@@ -142,6 +145,10 @@ elif [ "$COMMAND" = "getconf" ] ; then
   CLASS=org.apache.hadoop.hdfs.tools.GetConf
   CLASS=org.apache.hadoop.hdfs.tools.GetConf
 elif [ "$COMMAND" = "groups" ] ; then
 elif [ "$COMMAND" = "groups" ] ; then
   CLASS=org.apache.hadoop.hdfs.tools.GetGroups
   CLASS=org.apache.hadoop.hdfs.tools.GetGroups
+elif [ "$COMMAND" = "snapshotDiff" ] ; then
+  CLASS=org.apache.hadoop.hdfs.tools.snapshot.SnapshotDiff
+elif [ "$COMMAND" = "lsSnapshottableDir" ] ; then
+  CLASS=org.apache.hadoop.hdfs.tools.snapshot.LsSnapshottableDir
 else
 else
   CLASS="$COMMAND"
   CLASS="$COMMAND"
 fi
 fi

+ 144 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -41,18 +41,16 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPAC
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT;
 
 
 import java.io.BufferedOutputStream;
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
 import java.io.DataInputStream;
@@ -121,6 +119,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
@@ -137,6 +137,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotAccessControlException;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
@@ -1381,7 +1382,8 @@ public class DFSClient implements java.io.Closeable {
                                      ParentNotDirectoryException.class,
                                      ParentNotDirectoryException.class,
                                      NSQuotaExceededException.class, 
                                      NSQuotaExceededException.class, 
                                      DSQuotaExceededException.class,
                                      DSQuotaExceededException.class,
-                                     UnresolvedPathException.class);
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
     }
     }
   }
   }
 
 
@@ -1412,7 +1414,8 @@ public class DFSClient implements java.io.Closeable {
                                      SafeModeException.class,
                                      SafeModeException.class,
                                      DSQuotaExceededException.class,
                                      DSQuotaExceededException.class,
                                      UnsupportedOperationException.class,
                                      UnsupportedOperationException.class,
-                                     UnresolvedPathException.class);
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
     }
     }
     return DFSOutputStream.newStreamForAppend(this, src, buffersize, progress,
     return DFSOutputStream.newStreamForAppend(this, src, buffersize, progress,
         lastBlock, stat, dfsClientConf.createChecksum());
         lastBlock, stat, dfsClientConf.createChecksum());
@@ -1465,7 +1468,8 @@ public class DFSClient implements java.io.Closeable {
                                      FileNotFoundException.class,
                                      FileNotFoundException.class,
                                      SafeModeException.class,
                                      SafeModeException.class,
                                      DSQuotaExceededException.class,
                                      DSQuotaExceededException.class,
-                                     UnresolvedPathException.class);
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
     }
     }
   }
   }
 
 
@@ -1483,7 +1487,8 @@ public class DFSClient implements java.io.Closeable {
       throw re.unwrapRemoteException(AccessControlException.class,
       throw re.unwrapRemoteException(AccessControlException.class,
                                      NSQuotaExceededException.class,
                                      NSQuotaExceededException.class,
                                      DSQuotaExceededException.class,
                                      DSQuotaExceededException.class,
-                                     UnresolvedPathException.class);
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
     }
     }
   }
   }
 
 
@@ -1497,7 +1502,8 @@ public class DFSClient implements java.io.Closeable {
       namenode.concat(trg, srcs);
       namenode.concat(trg, srcs);
     } catch(RemoteException re) {
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
       throw re.unwrapRemoteException(AccessControlException.class,
-                                     UnresolvedPathException.class);
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
     }
     }
   }
   }
   /**
   /**
@@ -1517,7 +1523,8 @@ public class DFSClient implements java.io.Closeable {
                                      ParentNotDirectoryException.class,
                                      ParentNotDirectoryException.class,
                                      SafeModeException.class,
                                      SafeModeException.class,
                                      NSQuotaExceededException.class,
                                      NSQuotaExceededException.class,
-                                     UnresolvedPathException.class);
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
     }
     }
   }
   }
   /**
   /**
@@ -1545,7 +1552,8 @@ public class DFSClient implements java.io.Closeable {
       throw re.unwrapRemoteException(AccessControlException.class,
       throw re.unwrapRemoteException(AccessControlException.class,
                                      FileNotFoundException.class,
                                      FileNotFoundException.class,
                                      SafeModeException.class,
                                      SafeModeException.class,
-                                     UnresolvedPathException.class);
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
     }
     }
   }
   }
   
   
@@ -1968,7 +1976,8 @@ public class DFSClient implements java.io.Closeable {
       throw re.unwrapRemoteException(AccessControlException.class,
       throw re.unwrapRemoteException(AccessControlException.class,
                                      FileNotFoundException.class,
                                      FileNotFoundException.class,
                                      SafeModeException.class,
                                      SafeModeException.class,
-                                     UnresolvedPathException.class);
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
     }
     }
   }
   }
 
 
@@ -1989,7 +1998,8 @@ public class DFSClient implements java.io.Closeable {
       throw re.unwrapRemoteException(AccessControlException.class,
       throw re.unwrapRemoteException(AccessControlException.class,
                                      FileNotFoundException.class,
                                      FileNotFoundException.class,
                                      SafeModeException.class,
                                      SafeModeException.class,
-                                     UnresolvedPathException.class);                                   
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);                                   
     }
     }
   }
   }
 
 
@@ -2064,7 +2074,121 @@ public class DFSClient implements java.io.Closeable {
   public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException{
   public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException{
     return namenode.setSafeMode(action, isChecked);    
     return namenode.setSafeMode(action, isChecked);    
   }
   }
+ 
+  /**
+   * Create one snapshot.
+   * 
+   * @param snapshotRoot The directory where the snapshot is to be taken
+   * @param snapshotName Name of the snapshot
+   * @return the snapshot path.
+   * @see ClientProtocol#createSnapshot(String, String)
+   */
+  public String createSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException {
+    checkOpen();
+    try {
+      return namenode.createSnapshot(snapshotRoot, snapshotName);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
+  }
+  
+  /**
+   * Delete a snapshot of a snapshottable directory.
+   * 
+   * @param snapshotRoot The snapshottable directory that the 
+   *                    to-be-deleted snapshot belongs to
+   * @param snapshotName The name of the to-be-deleted snapshot
+   * @throws IOException
+   * @see ClientProtocol#deleteSnapshot(String, String)
+   */
+  public void deleteSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException {
+    try {
+      namenode.deleteSnapshot(snapshotRoot, snapshotName);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
+  }
+  
+  /**
+   * Rename a snapshot.
+   * @param snapshotDir The directory path where the snapshot was taken
+   * @param snapshotOldName Old name of the snapshot
+   * @param snapshotNewName New name of the snapshot
+   * @throws IOException
+   * @see ClientProtocol#renameSnapshot(String, String, String)
+   */
+  public void renameSnapshot(String snapshotDir, String snapshotOldName,
+      String snapshotNewName) throws IOException {
+    checkOpen();
+    try {
+      namenode.renameSnapshot(snapshotDir, snapshotOldName, snapshotNewName);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
+  }
+  
+  /**
+   * Get all the current snapshottable directories.
+   * @return All the current snapshottable directories
+   * @throws IOException
+   * @see ClientProtocol#getSnapshottableDirListing()
+   */
+  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+      throws IOException {
+    checkOpen();
+    try {
+      return namenode.getSnapshottableDirListing();
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
+  }
 
 
+  /**
+   * Allow snapshot on a directory.
+   * 
+   * @see ClientProtocol#allowSnapshot(String snapshotRoot)
+   */
+  public void allowSnapshot(String snapshotRoot) throws IOException {
+    checkOpen();
+    try {
+      namenode.allowSnapshot(snapshotRoot);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
+  }
+  
+  /**
+   * Disallow snapshot on a directory.
+   * 
+   * @see ClientProtocol#disallowSnapshot(String snapshotRoot)
+   */
+  public void disallowSnapshot(String snapshotRoot) throws IOException {
+    checkOpen();
+    try {
+      namenode.disallowSnapshot(snapshotRoot);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
+  }
+  
+  /**
+   * Get the difference between two snapshots, or between a snapshot and the
+   * current tree of a directory.
+   * @see ClientProtocol#getSnapshotDiffReport(String, String, String)
+   */
+  public SnapshotDiffReport getSnapshotDiffReport(Path snapshotDir,
+      String fromSnapshot, String toSnapshot) throws IOException {
+    checkOpen();
+    try {
+      return namenode.getSnapshotDiffReport(snapshotDir.toString(),
+          fromSnapshot, toSnapshot);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
+  }
+  
   /**
   /**
    * Save namespace image.
    * Save namespace image.
    * 
    * 
@@ -2206,7 +2330,8 @@ public class DFSClient implements java.io.Closeable {
                                      SafeModeException.class,
                                      SafeModeException.class,
                                      NSQuotaExceededException.class,
                                      NSQuotaExceededException.class,
                                      DSQuotaExceededException.class,
                                      DSQuotaExceededException.class,
-                                     UnresolvedPathException.class);
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
     }
     }
   }
   }
   
   
@@ -2249,7 +2374,8 @@ public class DFSClient implements java.io.Closeable {
                                      FileNotFoundException.class,
                                      FileNotFoundException.class,
                                      NSQuotaExceededException.class,
                                      NSQuotaExceededException.class,
                                      DSQuotaExceededException.class,
                                      DSQuotaExceededException.class,
-                                     UnresolvedPathException.class);
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
     }
     }
   }
   }
 
 
@@ -2265,7 +2391,8 @@ public class DFSClient implements java.io.Closeable {
     } catch(RemoteException re) {
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
       throw re.unwrapRemoteException(AccessControlException.class,
                                      FileNotFoundException.class,
                                      FileNotFoundException.class,
-                                     UnresolvedPathException.class);
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
     }
     }
   }
   }
 
 

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -71,6 +71,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotAccessControlException;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
@@ -1386,7 +1387,8 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
                                      ParentNotDirectoryException.class,
                                      ParentNotDirectoryException.class,
                                      NSQuotaExceededException.class,
                                      NSQuotaExceededException.class,
                                      SafeModeException.class,
                                      SafeModeException.class,
-                                     UnresolvedPathException.class);
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
     }
     }
     final DFSOutputStream out = new DFSOutputStream(dfsClient, src, stat,
     final DFSOutputStream out = new DFSOutputStream(dfsClient, src, stat,
         flag, progress, checksum, favoredNodes);
         flag, progress, checksum, favoredNodes);

+ 62 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -91,12 +91,26 @@ import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Maps;
+import com.google.common.primitives.SignedBytes;
 import com.google.protobuf.BlockingService;
 import com.google.protobuf.BlockingService;
 
 
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public class DFSUtil {
 public class DFSUtil {
   public static final Log LOG = LogFactory.getLog(DFSUtil.class.getName());
   public static final Log LOG = LogFactory.getLog(DFSUtil.class.getName());
   
   
+  public static final byte[] EMPTY_BYTES = {};
+
+  /** Compare two byte arrays by lexicographical order. */
+  public static int compareBytes(byte[] left, byte[] right) {
+    if (left == null) {
+      left = EMPTY_BYTES;
+    }
+    if (right == null) {
+      right = EMPTY_BYTES;
+    }
+    return SignedBytes.lexicographicalComparator().compare(left, right);
+  }
+
   private DFSUtil() { /* Hidden constructor */ }
   private DFSUtil() { /* Hidden constructor */ }
   private static final ThreadLocal<Random> RANDOM = new ThreadLocal<Random>() {
   private static final ThreadLocal<Random> RANDOM = new ThreadLocal<Random>() {
     @Override
     @Override
@@ -217,8 +231,21 @@ public class DFSUtil {
    * Converts a byte array to a string using UTF8 encoding.
    * Converts a byte array to a string using UTF8 encoding.
    */
    */
   public static String bytes2String(byte[] bytes) {
   public static String bytes2String(byte[] bytes) {
+    return bytes2String(bytes, 0, bytes.length);
+  }
+  
+  /**
+   * Decode a specific range of bytes of the given byte array to a string
+   * using UTF8.
+   * 
+   * @param bytes The bytes to be decoded into characters
+   * @param offset The index of the first byte to decode
+   * @param length The number of bytes to decode
+   * @return The decoded string
+   */
+  public static String bytes2String(byte[] bytes, int offset, int length) {
     try {
     try {
-      return new String(bytes, "UTF8");
+      return new String(bytes, offset, length, "UTF8");
     } catch(UnsupportedEncodingException e) {
     } catch(UnsupportedEncodingException e) {
       assert false : "UTF8 encoding is not supported ";
       assert false : "UTF8 encoding is not supported ";
     }
     }
@@ -236,9 +263,10 @@ public class DFSUtil {
    * Given a list of path components returns a path as a UTF8 String
    * Given a list of path components returns a path as a UTF8 String
    */
    */
   public static String byteArray2PathString(byte[][] pathComponents) {
   public static String byteArray2PathString(byte[][] pathComponents) {
-    if (pathComponents.length == 0)
+    if (pathComponents.length == 0) {
       return "";
       return "";
-    if (pathComponents.length == 1 && pathComponents[0].length == 0) {
+    } else if (pathComponents.length == 1
+        && (pathComponents[0] == null || pathComponents[0].length == 0)) {
       return Path.SEPARATOR;
       return Path.SEPARATOR;
     }
     }
     StringBuilder result = new StringBuilder();
     StringBuilder result = new StringBuilder();
@@ -250,6 +278,37 @@ public class DFSUtil {
     }
     }
     return result.toString();
     return result.toString();
   }
   }
+  
+  /**
+   * Given a list of path components returns a byte array
+   */
+  public static byte[] byteArray2bytes(byte[][] pathComponents) {
+    if (pathComponents.length == 0) {
+      return EMPTY_BYTES;
+    } else if (pathComponents.length == 1
+        && (pathComponents[0] == null || pathComponents[0].length == 0)) {
+      return new byte[]{(byte) Path.SEPARATOR_CHAR};
+    }
+    int length = 0;
+    for (int i = 0; i < pathComponents.length; i++) {
+      length += pathComponents[i].length;
+      if (i < pathComponents.length - 1) {
+        length++; // for SEPARATOR
+      }
+    }
+    byte[] path = new byte[length];
+    int index = 0;
+    for (int i = 0; i < pathComponents.length; i++) {
+      System.arraycopy(pathComponents[i], 0, path, index,
+          pathComponents[i].length);
+      index += pathComponents[i].length;
+      if (i < pathComponents.length - 1) {
+        path[index] = (byte) Path.SEPARATOR_CHAR;
+        index++;
+      }
+    }
+    return path;
+  }
 
 
   /** Convert an object representing a path to a string. */
   /** Convert an object representing a path to a string. */
   public static String path2String(final Object path) {
   public static String path2String(final Object path) {

+ 53 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -30,10 +30,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.BlockStorageLocation;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.BlockStorageLocation;
-import org.apache.hadoop.fs.VolumeId;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
@@ -46,18 +45,22 @@ import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.VolumeId;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -982,6 +985,54 @@ public class DistributedFileSystem extends FileSystem {
   public boolean isInSafeMode() throws IOException {
   public boolean isInSafeMode() throws IOException {
     return setSafeMode(SafeModeAction.SAFEMODE_GET, true);
     return setSafeMode(SafeModeAction.SAFEMODE_GET, true);
   }
   }
+
+  /** @see HdfsAdmin#allowSnapshot(Path) */
+  public void allowSnapshot(Path path) throws IOException {
+    dfs.allowSnapshot(getPathName(path));
+  }
+  
+  /** @see HdfsAdmin#disallowSnapshot(Path) */
+  public void disallowSnapshot(Path path) throws IOException {
+    dfs.disallowSnapshot(getPathName(path));
+  }
+  
+  @Override
+  public Path createSnapshot(Path path, String snapshotName) 
+      throws IOException {
+    return new Path(dfs.createSnapshot(getPathName(path), snapshotName));
+  }
+  
+  @Override
+  public void renameSnapshot(Path path, String snapshotOldName,
+      String snapshotNewName) throws IOException {
+    dfs.renameSnapshot(getPathName(path), snapshotOldName, snapshotNewName);
+  }
+  
+  /**
+   * @return All the snapshottable directories
+   * @throws IOException
+   */
+  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+      throws IOException {
+    return dfs.getSnapshottableDirListing();
+  }
+  
+  @Override
+  public void deleteSnapshot(Path snapshotDir, String snapshotName)
+      throws IOException {
+    dfs.deleteSnapshot(getPathName(snapshotDir), snapshotName);
+  }
+
+  /**
+   * Get the difference between two snapshots, or between a snapshot and the
+   * current tree of a directory.
+   * 
+   * @see DFSClient#getSnapshotDiffReport(Path, String, String)
+   */
+  public SnapshotDiffReport getSnapshotDiffReport(Path snapshotDir,
+      String fromSnapshot, String toSnapshot) throws IOException {
+    return dfs.getSnapshotDiffReport(snapshotDir, fromSnapshot, toSnapshot);
+  }
  
  
   /**
   /**
    * Get the close status of a file
    * Get the close status of a file

+ 16 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java

@@ -105,4 +105,20 @@ public class HdfsAdmin {
   public void clearSpaceQuota(Path src) throws IOException {
   public void clearSpaceQuota(Path src) throws IOException {
     dfs.setQuota(src, HdfsConstants.QUOTA_DONT_SET, HdfsConstants.QUOTA_RESET);
     dfs.setQuota(src, HdfsConstants.QUOTA_DONT_SET, HdfsConstants.QUOTA_RESET);
   }
   }
+  
+  /**
+   * Allow snapshot on a directory.
+   * @param path The path of the directory where snapshots will be taken.
+   */
+  public void allowSnapshot(Path path) throws IOException {
+    dfs.allowSnapshot(path);
+  }
+  
+  /**
+   * Disallow snapshot on a directory.
+   * @param path The path of the snapshottable directory.
+   */
+  public void disallowSnapshot(Path path) throws IOException {
+    dfs.disallowSnapshot(path);
+  }
 }
 }

+ 107 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -24,17 +24,21 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotAccessControlException;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.retry.Idempotent;
 import org.apache.hadoop.io.retry.Idempotent;
@@ -42,9 +46,6 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenInfo;
 import org.apache.hadoop.security.token.TokenInfo;
-import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 
 
 /**********************************************************************
 /**********************************************************************
  * ClientProtocol is used by user code via 
  * ClientProtocol is used by user code via 
@@ -165,6 +166,7 @@ public interface ClientProtocol {
    *           quota restriction
    *           quota restriction
    * @throws SafeModeException create not allowed in safemode
    * @throws SafeModeException create not allowed in safemode
    * @throws UnresolvedLinkException If <code>src</code> contains a symlink
    * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
    * @throws IOException If an I/O error occurred
    * @throws IOException If an I/O error occurred
    *
    *
    * RuntimeExceptions:
    * RuntimeExceptions:
@@ -177,7 +179,7 @@ public interface ClientProtocol {
       DSQuotaExceededException, FileAlreadyExistsException,
       DSQuotaExceededException, FileAlreadyExistsException,
       FileNotFoundException, NSQuotaExceededException,
       FileNotFoundException, NSQuotaExceededException,
       ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
       ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException;
+      SnapshotAccessControlException, IOException;
 
 
   /**
   /**
    * Append to the end of the file. 
    * Append to the end of the file. 
@@ -197,6 +199,7 @@ public interface ClientProtocol {
    *           restriction
    *           restriction
    * @throws SafeModeException append not allowed in safemode
    * @throws SafeModeException append not allowed in safemode
    * @throws UnresolvedLinkException If <code>src</code> contains a symlink
    * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
    * @throws IOException If an I/O error occurred.
    * @throws IOException If an I/O error occurred.
    *
    *
    * RuntimeExceptions:
    * RuntimeExceptions:
@@ -205,7 +208,7 @@ public interface ClientProtocol {
   public LocatedBlock append(String src, String clientName)
   public LocatedBlock append(String src, String clientName)
       throws AccessControlException, DSQuotaExceededException,
       throws AccessControlException, DSQuotaExceededException,
       FileNotFoundException, SafeModeException, UnresolvedLinkException,
       FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException;
+      SnapshotAccessControlException, IOException;
 
 
   /**
   /**
    * Set replication for an existing file.
    * Set replication for an existing file.
@@ -227,13 +230,14 @@ public interface ClientProtocol {
    * @throws FileNotFoundException If file <code>src</code> is not found
    * @throws FileNotFoundException If file <code>src</code> is not found
    * @throws SafeModeException not allowed in safemode
    * @throws SafeModeException not allowed in safemode
    * @throws UnresolvedLinkException if <code>src</code> contains a symlink
    * @throws UnresolvedLinkException if <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
    * @throws IOException If an I/O error occurred
    * @throws IOException If an I/O error occurred
    */
    */
   @Idempotent
   @Idempotent
   public boolean setReplication(String src, short replication)
   public boolean setReplication(String src, short replication)
       throws AccessControlException, DSQuotaExceededException,
       throws AccessControlException, DSQuotaExceededException,
       FileNotFoundException, SafeModeException, UnresolvedLinkException,
       FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException;
+      SnapshotAccessControlException, IOException;
 
 
   /**
   /**
    * Set permissions for an existing file/directory.
    * Set permissions for an existing file/directory.
@@ -242,12 +246,13 @@ public interface ClientProtocol {
    * @throws FileNotFoundException If file <code>src</code> is not found
    * @throws FileNotFoundException If file <code>src</code> is not found
    * @throws SafeModeException not allowed in safemode
    * @throws SafeModeException not allowed in safemode
    * @throws UnresolvedLinkException If <code>src</code> contains a symlink
    * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
    * @throws IOException If an I/O error occurred
    * @throws IOException If an I/O error occurred
    */
    */
   @Idempotent
   @Idempotent
   public void setPermission(String src, FsPermission permission)
   public void setPermission(String src, FsPermission permission)
       throws AccessControlException, FileNotFoundException, SafeModeException,
       throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException;
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
 
 
   /**
   /**
    * Set Owner of a path (i.e. a file or a directory).
    * Set Owner of a path (i.e. a file or a directory).
@@ -260,12 +265,13 @@ public interface ClientProtocol {
    * @throws FileNotFoundException If file <code>src</code> is not found
    * @throws FileNotFoundException If file <code>src</code> is not found
    * @throws SafeModeException not allowed in safemode
    * @throws SafeModeException not allowed in safemode
    * @throws UnresolvedLinkException If <code>src</code> contains a symlink
    * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
    * @throws IOException If an I/O error occurred
    * @throws IOException If an I/O error occurred
    */
    */
   @Idempotent
   @Idempotent
   public void setOwner(String src, String username, String groupname)
   public void setOwner(String src, String username, String groupname)
       throws AccessControlException, FileNotFoundException, SafeModeException,
       throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException;
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
 
 
   /**
   /**
    * The client can give up on a block by calling abandonBlock().
    * The client can give up on a block by calling abandonBlock().
@@ -392,10 +398,11 @@ public interface ClientProtocol {
    * @return true if successful, or false if the old name does not exist
    * @return true if successful, or false if the old name does not exist
    * or if the new name already belongs to the namespace.
    * or if the new name already belongs to the namespace.
    * 
    * 
+   * @throws SnapshotAccessControlException if path is in RO snapshot
    * @throws IOException an I/O error occurred 
    * @throws IOException an I/O error occurred 
    */
    */
   public boolean rename(String src, String dst) 
   public boolean rename(String src, String dst) 
-      throws UnresolvedLinkException, IOException;
+      throws UnresolvedLinkException, SnapshotAccessControlException, IOException;
 
 
   /**
   /**
    * Moves blocks from srcs to trg and delete srcs
    * Moves blocks from srcs to trg and delete srcs
@@ -405,9 +412,10 @@ public interface ClientProtocol {
    * @throws IOException if some arguments are invalid
    * @throws IOException if some arguments are invalid
    * @throws UnresolvedLinkException if <code>trg</code> or <code>srcs</code>
    * @throws UnresolvedLinkException if <code>trg</code> or <code>srcs</code>
    *           contains a symlink
    *           contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
    */
    */
   public void concat(String trg, String[] srcs) 
   public void concat(String trg, String[] srcs) 
-      throws IOException, UnresolvedLinkException;
+      throws IOException, UnresolvedLinkException, SnapshotAccessControlException;
 
 
   /**
   /**
    * Rename src to dst.
    * Rename src to dst.
@@ -441,13 +449,14 @@ public interface ClientProtocol {
    * @throws SafeModeException rename not allowed in safemode
    * @throws SafeModeException rename not allowed in safemode
    * @throws UnresolvedLinkException If <code>src</code> or
    * @throws UnresolvedLinkException If <code>src</code> or
    *           <code>dst</code> contains a symlink
    *           <code>dst</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
    * @throws IOException If an I/O error occurred
    * @throws IOException If an I/O error occurred
    */
    */
   public void rename2(String src, String dst, Options.Rename... options)
   public void rename2(String src, String dst, Options.Rename... options)
       throws AccessControlException, DSQuotaExceededException,
       throws AccessControlException, DSQuotaExceededException,
       FileAlreadyExistsException, FileNotFoundException,
       FileAlreadyExistsException, FileNotFoundException,
       NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
       NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, IOException;
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
   
   
   /**
   /**
    * Delete the given file or directory from the file system.
    * Delete the given file or directory from the file system.
@@ -464,11 +473,12 @@ public interface ClientProtocol {
    * @throws FileNotFoundException If file <code>src</code> is not found
    * @throws FileNotFoundException If file <code>src</code> is not found
    * @throws SafeModeException create not allowed in safemode
    * @throws SafeModeException create not allowed in safemode
    * @throws UnresolvedLinkException If <code>src</code> contains a symlink
    * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
    * @throws IOException If an I/O error occurred
    * @throws IOException If an I/O error occurred
    */
    */
   public boolean delete(String src, boolean recursive)
   public boolean delete(String src, boolean recursive)
       throws AccessControlException, FileNotFoundException, SafeModeException,
       throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException;
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
   
   
   /**
   /**
    * Create a directory (or hierarchy of directories) with the given
    * Create a directory (or hierarchy of directories) with the given
@@ -489,6 +499,7 @@ public interface ClientProtocol {
    *           is not a directory
    *           is not a directory
    * @throws SafeModeException create not allowed in safemode
    * @throws SafeModeException create not allowed in safemode
    * @throws UnresolvedLinkException If <code>src</code> contains a symlink
    * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
    * @throws IOException If an I/O error occurred.
    * @throws IOException If an I/O error occurred.
    *
    *
    * RunTimeExceptions:
    * RunTimeExceptions:
@@ -499,7 +510,7 @@ public interface ClientProtocol {
       throws AccessControlException, FileAlreadyExistsException,
       throws AccessControlException, FileAlreadyExistsException,
       FileNotFoundException, NSQuotaExceededException,
       FileNotFoundException, NSQuotaExceededException,
       ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
       ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException;
+      SnapshotAccessControlException, IOException;
 
 
   /**
   /**
    * Get a partial listing of the indicated directory
    * Get a partial listing of the indicated directory
@@ -521,6 +532,16 @@ public interface ClientProtocol {
                                      boolean needLocation)
                                      boolean needLocation)
       throws AccessControlException, FileNotFoundException,
       throws AccessControlException, FileNotFoundException,
       UnresolvedLinkException, IOException;
       UnresolvedLinkException, IOException;
+  
+  /**
+   * Get listing of all the snapshottable directories
+   * 
+   * @return Information about all the current snapshottable directory
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+      throws IOException;
 
 
   ///////////////////////////////////////
   ///////////////////////////////////////
   // System issues and management
   // System issues and management
@@ -824,12 +845,13 @@ public interface ClientProtocol {
    * @throws QuotaExceededException if the directory size 
    * @throws QuotaExceededException if the directory size 
    *           is greater than the given quota
    *           is greater than the given quota
    * @throws UnresolvedLinkException if the <code>path</code> contains a symlink. 
    * @throws UnresolvedLinkException if the <code>path</code> contains a symlink. 
+   * @throws SnapshotAccessControlException if path is in RO snapshot
    * @throws IOException If an I/O error occurred
    * @throws IOException If an I/O error occurred
    */
    */
   @Idempotent
   @Idempotent
   public void setQuota(String path, long namespaceQuota, long diskspaceQuota)
   public void setQuota(String path, long namespaceQuota, long diskspaceQuota)
       throws AccessControlException, FileNotFoundException,
       throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
 
 
   /**
   /**
    * Write all metadata for this file into persistent storage.
    * Write all metadata for this file into persistent storage.
@@ -861,12 +883,13 @@ public interface ClientProtocol {
    * @throws AccessControlException permission denied
    * @throws AccessControlException permission denied
    * @throws FileNotFoundException file <code>src</code> is not found
    * @throws FileNotFoundException file <code>src</code> is not found
    * @throws UnresolvedLinkException if <code>src</code> contains a symlink. 
    * @throws UnresolvedLinkException if <code>src</code> contains a symlink. 
+   * @throws SnapshotAccessControlException if path is in RO snapshot
    * @throws IOException If an I/O error occurred
    * @throws IOException If an I/O error occurred
    */
    */
   @Idempotent
   @Idempotent
   public void setTimes(String src, long mtime, long atime)
   public void setTimes(String src, long mtime, long atime)
       throws AccessControlException, FileNotFoundException, 
       throws AccessControlException, FileNotFoundException, 
-      UnresolvedLinkException, IOException;
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
 
 
   /**
   /**
    * Create symlink to a file or directory.
    * Create symlink to a file or directory.
@@ -884,13 +907,14 @@ public interface ClientProtocol {
    * @throws ParentNotDirectoryException If parent of <code>link</code> is not a
    * @throws ParentNotDirectoryException If parent of <code>link</code> is not a
    *           directory.
    *           directory.
    * @throws UnresolvedLinkException if <code>link</target> contains a symlink. 
    * @throws UnresolvedLinkException if <code>link</target> contains a symlink. 
+   * @throws SnapshotAccessControlException if path is in RO snapshot
    * @throws IOException If an I/O error occurred
    * @throws IOException If an I/O error occurred
    */
    */
   public void createSymlink(String target, String link, FsPermission dirPerm,
   public void createSymlink(String target, String link, FsPermission dirPerm,
       boolean createParent) throws AccessControlException,
       boolean createParent) throws AccessControlException,
       FileAlreadyExistsException, FileNotFoundException,
       FileAlreadyExistsException, FileNotFoundException,
       ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
       ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException;
+      SnapshotAccessControlException, IOException;
 
 
   /**
   /**
    * Return the target of the given symlink. If there is an intermediate
    * Return the target of the given symlink. If there is an intermediate
@@ -974,4 +998,68 @@ public interface ClientProtocol {
    * @throws IOException
    * @throws IOException
    */
    */
   public DataEncryptionKey getDataEncryptionKey() throws IOException;
   public DataEncryptionKey getDataEncryptionKey() throws IOException;
+  
+  /**
+   * Create a snapshot
+   * @param snapshotRoot the path that is being snapshotted
+   * @param snapshotName name of the snapshot created
+   * @return the snapshot path.
+   * @throws IOException
+   */
+  public String createSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException;
+
+  /**
+   * Delete a specific snapshot of a snapshottable directory
+   * @param snapshotRoot  The snapshottable directory
+   * @param snapshotName Name of the snapshot for the snapshottable directory
+   * @throws IOException
+   */
+  public void deleteSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException;
+  
+  /**
+   * Rename a snapshot
+   * @param snapshotRoot the directory path where the snapshot was taken 
+   * @param snapshotOldName old name of the snapshot
+   * @param snapshotNewName new name of the snapshot
+   * @throws IOException
+   */
+  public void renameSnapshot(String snapshotRoot, String snapshotOldName,
+      String snapshotNewName) throws IOException;
+  
+  /**
+   * Allow snapshot on a directory.
+   * @param snapshotRoot the directory to be snapped
+   * @throws IOException on error
+   */
+  public void allowSnapshot(String snapshotRoot)
+      throws IOException;
+    
+  /**
+   * Disallow snapshot on a directory.
+   * @param snapshotRoot the directory to disallow snapshot
+   * @throws IOException on error
+   */
+  public void disallowSnapshot(String snapshotRoot)
+      throws IOException;
+  
+  /**
+   * Get the difference between two snapshots, or between a snapshot and the
+   * current tree of a directory.
+   * 
+   * @param snapshotRoot
+   *          full path of the directory where snapshots are taken
+   * @param fromSnapshot
+   *          snapshot name of the from point. Null indicates the current
+   *          tree
+   * @param toSnapshot
+   *          snapshot name of the to point. Null indicates the current
+   *          tree.
+   * @return The difference report represented as a {@link SnapshotDiffReport}.
+   * @throws IOException on error
+   */
+  public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
+      String fromSnapshot, String toSnapshot) throws IOException;
 }
 }
+

+ 23 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/FSLimitException.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.protocol;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.Path;
 
 
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
@@ -48,21 +47,29 @@ public abstract class FSLimitException extends QuotaExceededException {
   class PathComponentTooLongException extends FSLimitException {
   class PathComponentTooLongException extends FSLimitException {
     protected static final long serialVersionUID = 1L;
     protected static final long serialVersionUID = 1L;
 
 
+    private String childName;
+
     protected PathComponentTooLongException() {}
     protected PathComponentTooLongException() {}
 
 
     protected PathComponentTooLongException(String msg) {
     protected PathComponentTooLongException(String msg) {
       super(msg);
       super(msg);
     }
     }
     
     
-    public PathComponentTooLongException(long quota, long count) {
+    public PathComponentTooLongException(long quota, long count,
+        String parentPath, String childName) {
       super(quota, count);
       super(quota, count);
+      setPathName(parentPath);
+      this.childName = childName;
+    }
+
+    String getParentPath() {
+      return pathName;
     }
     }
 
 
     @Override
     @Override
     public String getMessage() {
     public String getMessage() {
-      Path violator = new Path(pathName);
-      return "The maximum path component name limit of " + violator.getName() +
-      " in directory " + violator.getParent() +
+      return "The maximum path component name limit of " + childName +
+      " in directory " + getParentPath() +
       " is exceeded: limit=" + quota + " length=" + count; 
       " is exceeded: limit=" + quota + " length=" + count; 
     }
     }
   }
   }
@@ -90,4 +97,15 @@ public abstract class FSLimitException extends QuotaExceededException {
       " is exceeded: limit=" + quota + " items=" + count; 
       " is exceeded: limit=" + quota + " items=" + count; 
     }
     }
   }
   }
+
+  /** The given name is illegal. */
+  public static final class IllegalNameException extends FSLimitException {
+    public static final long serialVersionUID = 1L;
+    
+    public IllegalNameException() {}
+
+    public IllegalNameException(String msg) {
+      super(msg);
+    }
+  }
 }
 }

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java

@@ -18,7 +18,9 @@
 package org.apache.hadoop.hdfs.protocol;
 package org.apache.hadoop.hdfs.protocol;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 
 
 /************************************
 /************************************
@@ -102,4 +104,15 @@ public class HdfsConstants {
    */
    */
   public static final int LAYOUT_VERSION = LayoutVersion
   public static final int LAYOUT_VERSION = LayoutVersion
       .getCurrentLayoutVersion();
       .getCurrentLayoutVersion();
+  
+  /**
+   * A special path component contained in the path for a snapshot file/dir
+   */
+  public static final String DOT_SNAPSHOT_DIR = ".snapshot";
+
+  public static final byte[] DOT_SNAPSHOT_DIR_BYTES
+      = DFSUtil.string2Bytes(DOT_SNAPSHOT_DIR);
+  
+  public static final String SEPARATOR_DOT_SNAPSHOT_DIR
+      = Path.SEPARATOR + DOT_SNAPSHOT_DIR; 
 }
 }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java

@@ -97,7 +97,8 @@ public class LayoutVersion {
         "Serialize block lists with delta-encoded variable length ints, " +
         "Serialize block lists with delta-encoded variable length ints, " +
         "add OP_UPDATE_BLOCKS"),
         "add OP_UPDATE_BLOCKS"),
     RESERVED_REL1_2_0(-41, -32, "Reserved for release 1.2.0", true, CONCAT),
     RESERVED_REL1_2_0(-41, -32, "Reserved for release 1.2.0", true, CONCAT),
-    ADD_INODE_ID(-42, -40, "Assign a unique inode id for each inode", false);
+    ADD_INODE_ID(-42, -40, "Assign a unique inode id for each inode", false),
+    SNAPSHOT(-43, "Support for snapshot feature");
     
     
     final int lv;
     final int lv;
     final int ancestorLV;
     final int ancestorLV;

+ 13 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/NSQuotaExceededException.java

@@ -26,6 +26,8 @@ import org.apache.hadoop.classification.InterfaceStability;
 public final class NSQuotaExceededException extends QuotaExceededException {
 public final class NSQuotaExceededException extends QuotaExceededException {
   protected static final long serialVersionUID = 1L;
   protected static final long serialVersionUID = 1L;
   
   
+  private String prefix;
+  
   public NSQuotaExceededException() {}
   public NSQuotaExceededException() {}
 
 
   public NSQuotaExceededException(String msg) {
   public NSQuotaExceededException(String msg) {
@@ -40,11 +42,19 @@ public final class NSQuotaExceededException extends QuotaExceededException {
   public String getMessage() {
   public String getMessage() {
     String msg = super.getMessage();
     String msg = super.getMessage();
     if (msg == null) {
     if (msg == null) {
-      return "The NameSpace quota (directories and files)" + 
+      msg = "The NameSpace quota (directories and files)" + 
       (pathName==null?"":(" of directory " + pathName)) + 
       (pathName==null?"":(" of directory " + pathName)) + 
           " is exceeded: quota=" + quota + " file count=" + count; 
           " is exceeded: quota=" + quota + " file count=" + count; 
-    } else {
-      return msg;
+
+      if (prefix != null) {
+        msg = prefix + ": " + msg;
+      }
     }
     }
+    return msg;
+  }
+
+  /** Set a prefix for the error message. */
+  public void setMessagePrefix(final String prefix) {
+    this.prefix = prefix;
   }
   }
 }
 }

+ 193 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java

@@ -0,0 +1,193 @@
+/**
+ * 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.protocol;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable.SnapshotDiffInfo;
+
+/**
+ * This class represents to end users the difference between two snapshots of 
+ * the same directory, or the difference between a snapshot of the directory and
+ * its current state. Instead of capturing all the details of the diff, which 
+ * is stored in {@link SnapshotDiffInfo}, this class only lists where the 
+ * changes happened and their types.
+ */
+public class SnapshotDiffReport {
+  private final static String LINE_SEPARATOR = System.getProperty(
+      "line.separator", "\n");
+
+  /**
+   * Types of the difference, which include CREATE, MODIFY, DELETE, and RENAME.
+   * Each type has a label for representation: +/M/-/R represent CREATE, MODIFY,
+   * DELETE, and RENAME respectively.
+   */
+  public enum DiffType {
+    CREATE("+"),     
+    MODIFY("M"),    
+    DELETE("-"), 
+    RENAME("R");
+    
+    private String label;
+    
+    private DiffType(String label) {
+      this.label = label;
+    }
+    
+    public String getLabel() {
+      return label;
+    }
+    
+    public static DiffType getTypeFromLabel(String label) {
+      if (label.equals(CREATE.getLabel())) {
+        return CREATE;
+      } else if (label.equals(MODIFY.getLabel())) {
+        return MODIFY;
+      } else if (label.equals(DELETE.getLabel())) {
+        return DELETE;
+      } else if (label.equals(RENAME.getLabel())) {
+        return RENAME;
+      }
+      return null;
+    }
+  };
+  
+  /**
+   * Representing the full path and diff type of a file/directory where changes
+   * have happened.
+   */
+  public static class DiffReportEntry {
+    /** The type of the difference. */
+    private final DiffType type;
+    /**
+     * The relative path (related to the snapshot root) of the file/directory
+     * where changes have happened
+     */
+    private final byte[] relativePath;
+
+    public DiffReportEntry(DiffType type, byte[] path) {
+      this.type = type;
+      this.relativePath = path;
+    }
+    
+    public DiffReportEntry(DiffType type, byte[][] pathComponents) {
+      this.type = type;
+      this.relativePath = DFSUtil.byteArray2bytes(pathComponents);
+    }
+    
+    @Override
+    public String toString() {
+      return type.getLabel() + "\t" + getRelativePathString();
+    }
+    
+    public DiffType getType() {
+      return type;
+    }
+
+    public String getRelativePathString() {
+      String path = DFSUtil.bytes2String(relativePath);
+      if (path.isEmpty()) {
+        return Path.CUR_DIR;
+      } else {
+        return Path.CUR_DIR + Path.SEPARATOR + path;
+      }
+    }
+
+    public byte[] getRelativePath() {
+      return relativePath;
+    }
+    
+    @Override
+    public boolean equals(Object other) {
+      if (this == other) {
+        return true;
+      } 
+      if (other != null && other instanceof DiffReportEntry) {
+        DiffReportEntry entry = (DiffReportEntry) other;
+        return type.equals(entry.getType())
+            && Arrays.equals(relativePath, entry.getRelativePath());
+      }
+      return false;
+    }
+    
+    @Override
+    public int hashCode() {
+      return Arrays.hashCode(relativePath);
+    }
+  }
+  
+  /** snapshot root full path */
+  private final String snapshotRoot;
+
+  /** start point of the diff */
+  private final String fromSnapshot;
+  
+  /** end point of the diff */
+  private final String toSnapshot;
+  
+  /** list of diff */
+  private final List<DiffReportEntry> diffList;
+  
+  public SnapshotDiffReport(String snapshotRoot, String fromSnapshot,
+      String toSnapshot, List<DiffReportEntry> entryList) {
+    this.snapshotRoot = snapshotRoot;
+    this.fromSnapshot = fromSnapshot;
+    this.toSnapshot = toSnapshot;
+    this.diffList = entryList != null ? entryList : Collections
+        .<DiffReportEntry> emptyList();
+  }
+  
+  /** @return {@link #snapshotRoot}*/
+  public String getSnapshotRoot() {
+    return snapshotRoot;
+  }
+
+  /** @return {@link #fromSnapshot} */
+  public String getFromSnapshot() {
+    return fromSnapshot;
+  }
+
+  /** @return {@link #toSnapshot} */
+  public String getLaterSnapshotName() {
+    return toSnapshot;
+  }
+  
+  /** @return {@link #diffList} */
+  public List<DiffReportEntry> getDiffList() {
+    return diffList;
+  }
+  
+  @Override
+  public String toString() {
+    StringBuilder str = new StringBuilder();
+    String from = fromSnapshot == null || fromSnapshot.isEmpty() ? 
+        "current directory" : "snapshot " + fromSnapshot;
+    String to = toSnapshot == null || toSnapshot.isEmpty() ? "current directory"
+        : "snapshot " + toSnapshot;
+    str.append("Difference between " + from + " and " + to
+        + " under directory " + snapshotRoot + ":" + LINE_SEPARATOR);
+    for (DiffReportEntry entry : diffList) {
+      str.append(entry.toString() + LINE_SEPARATOR);
+    }
+    return str.toString();
+  }
+}

+ 82 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotInfo.java

@@ -0,0 +1,82 @@
+/**
+ * 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.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto;
+
+/**
+ * SnapshotInfo maintains information for a snapshot
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class SnapshotInfo {
+  private final String snapshotName;
+  private final String snapshotRoot;
+  private final String createTime;
+  private final FsPermissionProto permission;
+  private final String owner;
+  private final String group;
+
+  public SnapshotInfo(String sname, String sroot, String ctime,
+      FsPermissionProto permission, String owner, String group) {
+    this.snapshotName = sname;
+    this.snapshotRoot = sroot;
+    this.createTime = ctime;
+    this.permission = permission;
+    this.owner = owner;
+    this.group = group;
+  }
+
+  final public String getSnapshotName() {
+    return snapshotName;
+  }
+
+  final public String getSnapshotRoot() {
+    return snapshotRoot;
+  }
+
+  final public String getCreateTime() {
+    return createTime;
+  }
+  
+  final public FsPermissionProto getPermission() {
+    return permission;
+  }
+  
+  final public String getOwner() {
+    return owner;
+  }
+  
+  final public String getGroup() {
+    return group;
+  }
+  
+  @Override
+  public String toString() {
+    return getClass().getSimpleName()
+        + "{snapshotName=" + snapshotName
+        + "; snapshotRoot=" + snapshotRoot
+        + "; createTime=" + createTime
+        + "; permission=" + permission
+        + "; owner=" + owner
+        + "; group=" + group
+        + "}";
+  }
+}

+ 168 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java

@@ -0,0 +1,168 @@
+/**
+ * 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.protocol;
+
+import java.io.PrintStream;
+import java.text.SimpleDateFormat;
+import java.util.Comparator;
+import java.util.Date;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSUtil;
+
+/**
+ * Metadata about a snapshottable directory
+ */
+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;
+  
+  /** Number of snapshots that have been taken*/
+  private int snapshotNumber;
+  
+  /** Number of snapshots allowed. */
+  private int snapshotQuota;
+  
+  /** Full path of the parent. */
+  private byte[] parentFullPath;
+  
+  public SnapshottableDirectoryStatus(long modification_time, long access_time,
+      FsPermission permission, String owner, String group, byte[] localName,
+      long inodeId,
+      int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
+    this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
+        access_time, permission, owner, group, null, localName, inodeId);
+    this.snapshotNumber = snapshotNumber;
+    this.snapshotQuota = snapshotQuota;
+    this.parentFullPath = parentFullPath;
+  }
+
+  /**
+   * @return Number of snapshots that have been taken for the directory
+   */
+  public int getSnapshotNumber() {
+    return snapshotNumber;
+  }
+
+  /**
+   * @return Number of snapshots allowed for the directory
+   */
+  public int getSnapshotQuota() {
+    return snapshotQuota;
+  }
+  
+  /**
+   * @return Full path of the parent
+   */
+  public byte[] getParentFullPath() {
+    return parentFullPath;
+  }
+
+  /**
+   * @return The basic information of the directory
+   */
+  public HdfsFileStatus getDirStatus() {
+    return dirStatus;
+  }
+  
+  /**
+   * @return Full path of the file
+   */
+  public Path getFullPath() {
+    String parentFullPathStr = 
+        (parentFullPath == null || parentFullPath.length == 0) ? 
+            null : DFSUtil.bytes2String(parentFullPath);
+    if (parentFullPathStr == null
+        && dirStatus.getLocalNameInBytes().length == 0) {
+      // root
+      return new Path("/");
+    } else {
+      return parentFullPathStr == null ? new Path(dirStatus.getLocalName())
+          : new Path(parentFullPathStr, dirStatus.getLocalName());
+    }
+  }
+  
+  /**
+   * Print a list of {@link SnapshottableDirectoryStatus} out to a given stream.
+   * @param stats The list of {@link SnapshottableDirectoryStatus}
+   * @param out The given stream for printing.
+   */
+  public static void print(SnapshottableDirectoryStatus[] stats, 
+      PrintStream out) {
+    if (stats == null || stats.length == 0) {
+      out.println();
+      return;
+    }
+    int maxRepl = 0, maxLen = 0, maxOwner = 0, maxGroup = 0;
+    int maxSnapshotNum = 0, maxSnapshotQuota = 0;
+    for (SnapshottableDirectoryStatus status : stats) {
+      maxRepl = maxLength(maxRepl, status.dirStatus.getReplication());
+      maxLen = maxLength(maxLen, status.dirStatus.getLen());
+      maxOwner = maxLength(maxOwner, status.dirStatus.getOwner());
+      maxGroup = maxLength(maxGroup, status.dirStatus.getGroup());
+      maxSnapshotNum = maxLength(maxSnapshotNum, status.snapshotNumber);
+      maxSnapshotQuota = maxLength(maxSnapshotQuota, status.snapshotQuota);
+    }
+    
+    StringBuilder fmt = new StringBuilder();
+    fmt.append("%s%s "); // permission string
+    fmt.append("%"  + maxRepl  + "s ");
+    fmt.append((maxOwner > 0) ? "%-" + maxOwner + "s " : "%s");
+    fmt.append((maxGroup > 0) ? "%-" + maxGroup + "s " : "%s");
+    fmt.append("%"  + maxLen   + "s ");
+    fmt.append("%s "); // mod time
+    fmt.append("%"  + maxSnapshotNum  + "s ");
+    fmt.append("%"  + maxSnapshotQuota  + "s ");
+    fmt.append("%s"); // path
+    
+    String lineFormat = fmt.toString();
+    SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm");
+         
+    for (SnapshottableDirectoryStatus status : stats) {
+      String line = String.format(lineFormat, "d", 
+          status.dirStatus.getPermission(),
+          status.dirStatus.getReplication(),
+          status.dirStatus.getOwner(),
+          status.dirStatus.getGroup(),
+          String.valueOf(status.dirStatus.getLen()),
+          dateFormat.format(new Date(status.dirStatus.getModificationTime())),
+          status.snapshotNumber, status.snapshotQuota, 
+          status.getFullPath().toString()
+      );
+      out.println(line);
+    }
+  }
+
+  private static int maxLength(int n, Object value) {
+    return Math.max(n, String.valueOf(value).length());
+  }
+}

+ 124 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -31,10 +31,14 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
@@ -43,23 +47,29 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Concat
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSnapshotRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSnapshotResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteSnapshotRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteSnapshotResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DisallowSnapshotRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DisallowSnapshotResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FinalizeUpgradeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FinalizeUpgradeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FinalizeUpgradeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FinalizeUpgradeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto.Builder;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto.Builder;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
@@ -76,6 +86,10 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPre
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
@@ -92,6 +106,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2ResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2ResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ReportBadBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ReportBadBlocksRequestProto;
@@ -149,6 +165,17 @@ import com.google.protobuf.ServiceException;
 public class ClientNamenodeProtocolServerSideTranslatorPB implements
 public class ClientNamenodeProtocolServerSideTranslatorPB implements
     ClientNamenodeProtocolPB {
     ClientNamenodeProtocolPB {
   final private ClientProtocol server;
   final private ClientProtocol server;
+  static final DeleteSnapshotResponseProto VOID_DELETE_SNAPSHOT_RESPONSE =
+      DeleteSnapshotResponseProto.newBuilder().build();
+  static final RenameSnapshotResponseProto VOID_RENAME_SNAPSHOT_RESPONSE =
+      RenameSnapshotResponseProto.newBuilder().build();
+  static final AllowSnapshotResponseProto VOID_ALLOW_SNAPSHOT_RESPONSE = 
+      AllowSnapshotResponseProto.newBuilder().build();
+  static final DisallowSnapshotResponseProto VOID_DISALLOW_SNAPSHOT_RESPONSE =
+      DisallowSnapshotResponseProto.newBuilder().build();
+  static final GetSnapshottableDirListingResponseProto 
+      NULL_GET_SNAPSHOTTABLE_DIR_LISTING_RESPONSE = 
+      GetSnapshottableDirListingResponseProto.newBuilder().build();
 
 
   private static final CreateResponseProto VOID_CREATE_RESPONSE = 
   private static final CreateResponseProto VOID_CREATE_RESPONSE = 
   CreateResponseProto.newBuilder().build();
   CreateResponseProto.newBuilder().build();
@@ -870,6 +897,101 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     }
     }
   }
   }
 
 
+  @Override
+  public CreateSnapshotResponseProto createSnapshot(RpcController controller,
+      CreateSnapshotRequestProto req) throws ServiceException {
+    try {
+      final CreateSnapshotResponseProto.Builder builder
+          = CreateSnapshotResponseProto.newBuilder();
+      final String snapshotPath = server.createSnapshot(req.getSnapshotRoot(),
+          req.hasSnapshotName()? req.getSnapshotName(): null);
+      if (snapshotPath != null) {
+        builder.setSnapshotPath(snapshotPath);
+      }
+      return builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public DeleteSnapshotResponseProto deleteSnapshot(RpcController controller,
+      DeleteSnapshotRequestProto req) throws ServiceException {
+    try {
+      server.deleteSnapshot(req.getSnapshotRoot(), req.getSnapshotName());
+      return VOID_DELETE_SNAPSHOT_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+  
+  @Override
+  public AllowSnapshotResponseProto allowSnapshot(RpcController controller,
+      AllowSnapshotRequestProto req) throws ServiceException {
+    try {
+      server.allowSnapshot(req.getSnapshotRoot());
+      return VOID_ALLOW_SNAPSHOT_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public DisallowSnapshotResponseProto disallowSnapshot(RpcController controller,
+      DisallowSnapshotRequestProto req) throws ServiceException {
+    try {
+      server.disallowSnapshot(req.getSnapshotRoot());
+      return VOID_DISALLOW_SNAPSHOT_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public RenameSnapshotResponseProto renameSnapshot(RpcController controller,
+      RenameSnapshotRequestProto request) throws ServiceException {
+    try {
+      server.renameSnapshot(request.getSnapshotRoot(),
+          request.getSnapshotOldName(), request.getSnapshotNewName());
+      return VOID_RENAME_SNAPSHOT_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public GetSnapshottableDirListingResponseProto getSnapshottableDirListing(
+      RpcController controller, GetSnapshottableDirListingRequestProto request)
+      throws ServiceException {
+    try {
+      SnapshottableDirectoryStatus[] result = server
+          .getSnapshottableDirListing();
+      if (result != null) {
+        return GetSnapshottableDirListingResponseProto.newBuilder().
+            setSnapshottableDirList(PBHelper.convert(result)).build();
+      } else {
+        return NULL_GET_SNAPSHOTTABLE_DIR_LISTING_RESPONSE;
+      }
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public GetSnapshotDiffReportResponseProto getSnapshotDiffReport(
+      RpcController controller, GetSnapshotDiffReportRequestProto request)
+      throws ServiceException {
+    try {
+      SnapshotDiffReport report = server.getSnapshotDiffReport(
+          request.getSnapshotRoot(), request.getFromSnapshot(),
+          request.getToSnapshot());
+      return GetSnapshotDiffReportResponseProto.newBuilder()
+          .setDiffReport(PBHelper.convert(report)).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
   @Override
   @Override
   public IsFileClosedResponseProto isFileClosed(
   public IsFileClosedResponseProto isFileClosed(
       RpcController controller, IsFileClosedRequestProto request) 
       RpcController controller, IsFileClosedRequestProto request) 

+ 109 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -42,28 +42,33 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.ipc.ProtocolTranslator;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSnapshotRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteSnapshotRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DisallowSnapshotRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FinalizeUpgradeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FinalizeUpgradeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
@@ -77,6 +82,10 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLis
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
@@ -85,6 +94,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Recove
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ReportBadBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ReportBadBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageRequestProto;
@@ -102,13 +112,13 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Update
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
+import org.apache.hadoop.ipc.ProtocolTranslator;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RpcClientUtil;
 import org.apache.hadoop.ipc.RpcClientUtil;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
@@ -873,5 +883,101 @@ public class ClientNamenodeProtocolTranslatorPB implements
   public Object getUnderlyingProxyObject() {
   public Object getUnderlyingProxyObject() {
     return rpcProxy;
     return rpcProxy;
   }
   }
+
+  @Override
+  public String createSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException {
+    final CreateSnapshotRequestProto.Builder builder
+        = CreateSnapshotRequestProto.newBuilder().setSnapshotRoot(snapshotRoot);
+    if (snapshotName != null) {
+      builder.setSnapshotName(snapshotName);
+    }
+    final CreateSnapshotRequestProto req = builder.build();
+    try {
+      return rpcProxy.createSnapshot(null, req).getSnapshotPath();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+  @Override
+  public void deleteSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException {
+    DeleteSnapshotRequestProto req = DeleteSnapshotRequestProto.newBuilder()
+        .setSnapshotRoot(snapshotRoot).setSnapshotName(snapshotName).build();
+    try {
+      rpcProxy.deleteSnapshot(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
   
   
+  @Override
+  public void allowSnapshot(String snapshotRoot) throws IOException {
+    AllowSnapshotRequestProto req = AllowSnapshotRequestProto.newBuilder()
+        .setSnapshotRoot(snapshotRoot).build();
+    try {
+      rpcProxy.allowSnapshot(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void disallowSnapshot(String snapshotRoot) throws IOException {
+    DisallowSnapshotRequestProto req = DisallowSnapshotRequestProto
+        .newBuilder().setSnapshotRoot(snapshotRoot).build();
+    try {
+      rpcProxy.disallowSnapshot(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void renameSnapshot(String snapshotRoot, String snapshotOldName,
+      String snapshotNewName) throws IOException {
+    RenameSnapshotRequestProto req = RenameSnapshotRequestProto.newBuilder()
+        .setSnapshotRoot(snapshotRoot).setSnapshotOldName(snapshotOldName)
+        .setSnapshotNewName(snapshotNewName).build();
+    try {
+      rpcProxy.renameSnapshot(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+      throws IOException {
+    GetSnapshottableDirListingRequestProto req = 
+        GetSnapshottableDirListingRequestProto.newBuilder().build();
+    try {
+      GetSnapshottableDirListingResponseProto result = rpcProxy
+          .getSnapshottableDirListing(null, req);
+      
+      if (result.hasSnapshottableDirList()) {
+        return PBHelper.convert(result.getSnapshottableDirList());
+      }
+      return null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
+      String fromSnapshot, String toSnapshot) throws IOException {
+    GetSnapshotDiffReportRequestProto req = GetSnapshotDiffReportRequestProto
+        .newBuilder().setSnapshotRoot(snapshotRoot)
+        .setFromSnapshot(fromSnapshot).setToSnapshot(toSnapshot).build();
+    try {
+      GetSnapshotDiffReportResponseProto result = 
+          rpcProxy.getSnapshotDiffReport(null, req);
+    
+      return PBHelper.convert(result.getDiffReport());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
 }
 }

+ 153 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -30,23 +30,26 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
-import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
@@ -64,7 +67,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.NNHAStatusHe
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReportProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DataEncryptionKeyProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
@@ -73,6 +76,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ContentSummaryProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ContentSummaryProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CorruptFileBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CorruptFileBlocksProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DataEncryptionKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState;
@@ -89,17 +93,21 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builde
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.NamenodeRoleProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamespaceInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamespaceInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.NamenodeRoleProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
-import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.BlockKey;
 import org.apache.hadoop.hdfs.security.token.block.BlockKey;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
@@ -122,15 +130,16 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
 import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
 import org.apache.hadoop.hdfs.server.protocol.JournalInfo;
 import org.apache.hadoop.hdfs.server.protocol.JournalInfo;
 import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
 import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
+import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
-import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
 import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
 import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.util.ExactSizeInputStream;
 import org.apache.hadoop.hdfs.util.ExactSizeInputStream;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
@@ -291,8 +300,8 @@ public class PBHelper {
 
 
   public static BlockKeyProto convert(BlockKey key) {
   public static BlockKeyProto convert(BlockKey key) {
     byte[] encodedKey = key.getEncodedKey();
     byte[] encodedKey = key.getEncodedKey();
-    ByteString keyBytes = ByteString.copyFrom(encodedKey == null ? new byte[0]
-        : encodedKey);
+    ByteString keyBytes = ByteString.copyFrom(encodedKey == null ? 
+        DFSUtil.EMPTY_BYTES : encodedKey);
     return BlockKeyProto.newBuilder().setKeyId(key.getKeyId())
     return BlockKeyProto.newBuilder().setKeyId(key.getKeyId())
         .setKeyBytes(keyBytes).setExpiryDate(key.getExpiryDate()).build();
         .setKeyBytes(keyBytes).setExpiryDate(key.getExpiryDate()).build();
   }
   }
@@ -1034,7 +1043,6 @@ public class PBHelper {
     return new EnumSetWritable<CreateFlag>(result);
     return new EnumSetWritable<CreateFlag>(result);
   }
   }
   
   
-  
   public static HdfsFileStatus convert(HdfsFileStatusProto fs) {
   public static HdfsFileStatus convert(HdfsFileStatusProto fs) {
     if (fs == null)
     if (fs == null)
       return null;
       return null;
@@ -1050,6 +1058,25 @@ public class PBHelper {
         fs.hasLocations() ? PBHelper.convert(fs.getLocations()) : null);
         fs.hasLocations() ? PBHelper.convert(fs.getLocations()) : null);
   }
   }
 
 
+  public static SnapshottableDirectoryStatus convert(
+      SnapshottableDirectoryStatusProto sdirStatusProto) {
+    if (sdirStatusProto == null) {
+      return null;
+    }
+    final HdfsFileStatusProto status = sdirStatusProto.getDirStatus();
+    return new SnapshottableDirectoryStatus(
+        status.getModificationTime(),
+        status.getAccessTime(),
+        PBHelper.convert(status.getPermission()),
+        status.getOwner(),
+        status.getGroup(),
+        status.getPath().toByteArray(),
+        status.getFileId(),
+        sdirStatusProto.getSnapshotNumber(),
+        sdirStatusProto.getSnapshotQuota(),
+        sdirStatusProto.getParentFullpath().toByteArray());
+  }
+  
   public static HdfsFileStatusProto convert(HdfsFileStatus fs) {
   public static HdfsFileStatusProto convert(HdfsFileStatus fs) {
     if (fs == null)
     if (fs == null)
       return null;
       return null;
@@ -1085,6 +1112,25 @@ public class PBHelper {
     return builder.build();
     return builder.build();
   }
   }
   
   
+  public static SnapshottableDirectoryStatusProto convert(
+      SnapshottableDirectoryStatus status) {
+    if (status == null) {
+      return null;
+    }
+    int snapshotNumber = status.getSnapshotNumber();
+    int snapshotQuota = status.getSnapshotQuota();
+    byte[] parentFullPath = status.getParentFullPath();
+    ByteString parentFullPathBytes = ByteString.copyFrom(
+        parentFullPath == null ? DFSUtil.EMPTY_BYTES : parentFullPath);
+    HdfsFileStatusProto fs = convert(status.getDirStatus());
+    SnapshottableDirectoryStatusProto.Builder builder = 
+        SnapshottableDirectoryStatusProto
+        .newBuilder().setSnapshotNumber(snapshotNumber)
+        .setSnapshotQuota(snapshotQuota).setParentFullpath(parentFullPathBytes)
+        .setDirStatus(fs);
+    return builder.build();
+  }
+  
   public static HdfsFileStatusProto[] convert(HdfsFileStatus[] fs) {
   public static HdfsFileStatusProto[] convert(HdfsFileStatus[] fs) {
     if (fs == null) return null;
     if (fs == null) return null;
     final int len = fs.length;
     final int len = fs.length;
@@ -1326,6 +1372,104 @@ public class PBHelper {
     return JournalInfoProto.newBuilder().setClusterID(j.getClusterId())
     return JournalInfoProto.newBuilder().setClusterID(j.getClusterId())
         .setLayoutVersion(j.getLayoutVersion())
         .setLayoutVersion(j.getLayoutVersion())
         .setNamespaceID(j.getNamespaceId()).build();
         .setNamespaceID(j.getNamespaceId()).build();
+  } 
+  
+  public static SnapshottableDirectoryStatus[] convert(
+      SnapshottableDirectoryListingProto sdlp) {
+    if (sdlp == null)
+      return null;
+    List<SnapshottableDirectoryStatusProto> list = sdlp
+        .getSnapshottableDirListingList();
+    if (list.isEmpty()) {
+      return new SnapshottableDirectoryStatus[0];
+    } else {
+      SnapshottableDirectoryStatus[] result = 
+          new SnapshottableDirectoryStatus[list.size()];
+      for (int i = 0; i < list.size(); i++) {
+        result[i] = PBHelper.convert(list.get(i));
+      }
+      return result;
+    }
+  }
+  
+  public static SnapshottableDirectoryListingProto convert(
+      SnapshottableDirectoryStatus[] status) {
+    if (status == null)
+      return null;
+    SnapshottableDirectoryStatusProto[] protos = 
+        new SnapshottableDirectoryStatusProto[status.length];
+    for (int i = 0; i < status.length; i++) {
+      protos[i] = PBHelper.convert(status[i]);
+    }
+    List<SnapshottableDirectoryStatusProto> protoList = Arrays.asList(protos);
+    return SnapshottableDirectoryListingProto.newBuilder()
+        .addAllSnapshottableDirListing(protoList).build();
+  }
+  
+  public static DiffReportEntry convert(SnapshotDiffReportEntryProto entry) {
+    if (entry == null) {
+      return null;
+    }
+    DiffType type = DiffType.getTypeFromLabel(entry
+        .getModificationLabel());
+    return type == null ? null : 
+      new DiffReportEntry(type, entry.getFullpath().toByteArray());
+  }
+  
+  public static SnapshotDiffReportEntryProto convert(DiffReportEntry entry) {
+    if (entry == null) {
+      return null;
+    }
+    byte[] fullPath = entry.getRelativePath();
+    ByteString fullPathString = ByteString
+        .copyFrom(fullPath == null ? DFSUtil.EMPTY_BYTES : fullPath);
+    
+    String modification = entry.getType().getLabel();
+    
+    SnapshotDiffReportEntryProto entryProto = SnapshotDiffReportEntryProto
+        .newBuilder().setFullpath(fullPathString)
+        .setModificationLabel(modification).build();
+    return entryProto;
+  }
+  
+  public static SnapshotDiffReport convert(SnapshotDiffReportProto reportProto) {
+    if (reportProto == null) {
+      return null;
+    }
+    String snapshotDir = reportProto.getSnapshotRoot();
+    String fromSnapshot = reportProto.getFromSnapshot();
+    String toSnapshot = reportProto.getToSnapshot();
+    List<SnapshotDiffReportEntryProto> list = reportProto
+        .getDiffReportEntriesList();
+    List<DiffReportEntry> entries = new ArrayList<DiffReportEntry>();
+    for (SnapshotDiffReportEntryProto entryProto : list) {
+      DiffReportEntry entry = convert(entryProto);
+      if (entry != null)
+        entries.add(entry);
+    }
+    return new SnapshotDiffReport(snapshotDir, fromSnapshot, toSnapshot,
+        entries);
+  }
+  
+  public static SnapshotDiffReportProto convert(SnapshotDiffReport report) {
+    if (report == null) {
+      return null;
+    }
+    List<DiffReportEntry> entries = report.getDiffList();
+    List<SnapshotDiffReportEntryProto> entryProtos = 
+        new ArrayList<SnapshotDiffReportEntryProto>();
+    for (DiffReportEntry entry : entries) {
+      SnapshotDiffReportEntryProto entryProto = convert(entry);
+      if (entryProto != null)
+        entryProtos.add(entryProto);
+    }
+    
+    SnapshotDiffReportProto reportProto = SnapshotDiffReportProto.newBuilder()
+        .setSnapshotRoot(report.getSnapshotRoot())
+        .setFromSnapshot(report.getFromSnapshot())
+        .setToSnapshot(report.getLaterSnapshotName())
+        .addAllDiffReportEntries(entryProtos).build();
+    return reportProto;
   }
   }
 
 
   public static DataChecksum.Type convert(HdfsProtos.ChecksumTypeProto type) {
   public static DataChecksum.Type convert(HdfsProtos.ChecksumTypeProto type) {

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java

@@ -18,7 +18,7 @@
 
 
 package org.apache.hadoop.hdfs.security.token.delegation;
 package org.apache.hadoop.hdfs.security.token.delegation;
 
 
-import java.io.DataInputStream;
+import java.io.DataInput;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.io.InterruptedIOException;
@@ -110,7 +110,7 @@ public class DelegationTokenSecretManager
    * @param in input stream to read fsimage
    * @param in input stream to read fsimage
    * @throws IOException
    * @throws IOException
    */
    */
-  public synchronized void loadSecretManagerState(DataInputStream in)
+  public synchronized void loadSecretManagerState(DataInput in)
       throws IOException {
       throws IOException {
     if (running) {
     if (running) {
       // a safety check
       // a safety check
@@ -266,7 +266,7 @@ public class DelegationTokenSecretManager
   /**
   /**
    * Private helper methods to load Delegation tokens from fsimage
    * Private helper methods to load Delegation tokens from fsimage
    */
    */
-  private synchronized void loadCurrentTokens(DataInputStream in)
+  private synchronized void loadCurrentTokens(DataInput in)
       throws IOException {
       throws IOException {
     int numberOfTokens = in.readInt();
     int numberOfTokens = in.readInt();
     for (int i = 0; i < numberOfTokens; i++) {
     for (int i = 0; i < numberOfTokens; i++) {
@@ -282,7 +282,7 @@ public class DelegationTokenSecretManager
    * @param in
    * @param in
    * @throws IOException
    * @throws IOException
    */
    */
-  private synchronized void loadAllKeys(DataInputStream in) throws IOException {
+  private synchronized void loadAllKeys(DataInput in) throws IOException {
     int numberOfKeys = in.readInt();
     int numberOfKeys = in.readInt();
     for (int i = 0; i < numberOfKeys; i++) {
     for (int i = 0; i < numberOfKeys; i++) {
       DelegationKey value = new DelegationKey();
       DelegationKey value = new DelegationKey();

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

@@ -230,7 +230,7 @@ public class BackupImage extends FSImage {
       }
       }
       lastAppliedTxId = logLoader.getLastAppliedTxId();
       lastAppliedTxId = logLoader.getLastAppliedTxId();
 
 
-      namesystem.dir.updateCountForINodeWithQuota(); // inefficient!
+      FSImage.updateCountForQuota(namesystem.dir.rootDir); // inefficient!
     } finally {
     } finally {
       backupInputStream.clear();
       backupInputStream.clear();
     }
     }

+ 72 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Content.java

@@ -0,0 +1,72 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hdfs.util.EnumCounters;
+
+/**
+ * The content types such as file, directory and symlink to be computed.
+ */
+public enum Content {
+  /** The number of files. */
+  FILE,
+  /** The number of directories. */
+  DIRECTORY,
+  /** The number of symlinks. */
+  SYMLINK,
+
+  /** The total of file length in bytes. */
+  LENGTH,
+  /** The total of disk space usage in bytes including replication. */
+  DISKSPACE,
+
+  /** The number of snapshots. */
+  SNAPSHOT,
+  /** The number of snapshottable directories. */
+  SNAPSHOTTABLE_DIRECTORY;
+
+  /** Content counts. */
+  public static class Counts extends EnumCounters<Content> {
+    public static Counts newInstance() {
+      return new Counts();
+    }
+
+    private Counts() {
+      super(Content.values());
+    }
+  }
+
+  private static final EnumCounters.Factory<Content, Counts> FACTORY
+      = new EnumCounters.Factory<Content, Counts>() {
+    @Override
+    public Counts newInstance() {
+      return Counts.newInstance();
+    }
+  };
+
+  /** A map of counters for the current state and the snapshots. */
+  public static class CountsMap
+      extends EnumCounters.Map<CountsMap.Key, Content, Counts> {
+    /** The key type of the map. */
+    public static enum Key { CURRENT, SNAPSHOT }
+
+    CountsMap() {
+      super(FACTORY);
+    }
+  }
+}

Разлика између датотеке није приказан због своје велике величине
+ 475 - 170
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java


+ 60 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -42,7 +42,33 @@ import static org.apache.hadoop.util.ExitUtil.terminate;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
 import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.*;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CloseOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ConcatDeleteOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CreateSnapshotOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteSnapshotOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DisallowSnapshotOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.GetDelegationTokenOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.LogSegmentOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.OpInstanceCache;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameSnapshotOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenewDelegationTokenOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetOwnerOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetPermissionsOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetReplicationOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
 import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
 import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
@@ -642,7 +668,7 @@ public class FSEditLog implements LogsPurgeable {
     AddOp op = AddOp.getInstance(cache.get())
     AddOp op = AddOp.getInstance(cache.get())
       .setInodeId(newNode.getId())
       .setInodeId(newNode.getId())
       .setPath(path)
       .setPath(path)
-      .setReplication(newNode.getBlockReplication())
+      .setReplication(newNode.getFileReplication())
       .setModificationTime(newNode.getModificationTime())
       .setModificationTime(newNode.getModificationTime())
       .setAccessTime(newNode.getAccessTime())
       .setAccessTime(newNode.getAccessTime())
       .setBlockSize(newNode.getPreferredBlockSize())
       .setBlockSize(newNode.getPreferredBlockSize())
@@ -660,7 +686,7 @@ public class FSEditLog implements LogsPurgeable {
   public void logCloseFile(String path, INodeFile newNode) {
   public void logCloseFile(String path, INodeFile newNode) {
     CloseOp op = CloseOp.getInstance(cache.get())
     CloseOp op = CloseOp.getInstance(cache.get())
       .setPath(path)
       .setPath(path)
-      .setReplication(newNode.getBlockReplication())
+      .setReplication(newNode.getFileReplication())
       .setModificationTime(newNode.getModificationTime())
       .setModificationTime(newNode.getModificationTime())
       .setAccessTime(newNode.getAccessTime())
       .setAccessTime(newNode.getAccessTime())
       .setBlockSize(newNode.getPreferredBlockSize())
       .setBlockSize(newNode.getPreferredBlockSize())
@@ -850,6 +876,37 @@ public class FSEditLog implements LogsPurgeable {
     logEdit(op);
     logEdit(op);
   }
   }
   
   
+  void logCreateSnapshot(String snapRoot, String snapName) {
+    CreateSnapshotOp op = CreateSnapshotOp.getInstance(cache.get())
+        .setSnapshotRoot(snapRoot).setSnapshotName(snapName);
+    logEdit(op);
+  }
+  
+  void logDeleteSnapshot(String snapRoot, String snapName) {
+    DeleteSnapshotOp op = DeleteSnapshotOp.getInstance(cache.get())
+        .setSnapshotRoot(snapRoot).setSnapshotName(snapName);
+    logEdit(op);
+  }
+  
+  void logRenameSnapshot(String path, String snapOldName, String snapNewName) {
+    RenameSnapshotOp op = RenameSnapshotOp.getInstance(cache.get())
+        .setSnapshotRoot(path).setSnapshotOldName(snapOldName)
+        .setSnapshotNewName(snapNewName);
+    logEdit(op);
+  }
+  
+  void logAllowSnapshot(String path) {
+    AllowSnapshotOp op = AllowSnapshotOp.getInstance(cache.get())
+        .setSnapshotRoot(path);
+    logEdit(op);
+  }
+
+  void logDisallowSnapshot(String path) {
+    DisallowSnapshotOp op = DisallowSnapshotOp.getInstance(cache.get())
+        .setSnapshotRoot(path);
+    logEdit(op);
+  }
+  
   /**
   /**
    * Get all the journals this edit log is currently operating on.
    * Get all the journals this edit log is currently operating on.
    */
    */

+ 65 - 35
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -23,8 +23,10 @@ import java.io.File;
 import java.io.FilterInputStream;
 import java.io.FilterInputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.EnumMap;
 import java.util.EnumMap;
+import java.util.List;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -39,16 +41,21 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream.LogHeaderCorruptException;
 import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream.LogHeaderCorruptException;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.BlockListUpdatingOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.BlockListUpdatingOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ClearNSQuotaOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ClearNSQuotaOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ConcatDeleteOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ConcatDeleteOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CreateSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteSnapshotOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DisallowSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.GetDelegationTokenOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.GetDelegationTokenOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenewDelegationTokenOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenewDelegationTokenOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetNSQuotaOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetNSQuotaOp;
@@ -60,6 +67,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
+import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.util.Holder;
 import org.apache.hadoop.hdfs.util.Holder;
 
 
@@ -270,7 +278,9 @@ public class FSEditLogLoader {
       // 3. OP_ADD to open file for append
       // 3. OP_ADD to open file for append
 
 
       // See if the file already exists (persistBlocks call)
       // See if the file already exists (persistBlocks call)
-      INodeFile oldFile = getINodeFile(fsDir, addCloseOp.path);
+      final INodesInPath iip = fsDir.getLastINodeInPath(addCloseOp.path);
+      final INodeFile oldFile = INodeFile.valueOf(
+          iip.getINode(0), addCloseOp.path, true);
       INodeFile newFile = oldFile;
       INodeFile newFile = oldFile;
       if (oldFile == null) { // this is OP_ADD on a new file (case 1)
       if (oldFile == null) { // this is OP_ADD on a new file (case 1)
         // versions > 0 support per file replication
         // versions > 0 support per file replication
@@ -282,7 +292,7 @@ public class FSEditLogLoader {
         // add to the file tree
         // add to the file tree
         inodeId = getAndUpdateLastInodeId(addCloseOp.inodeId, logVersion,
         inodeId = getAndUpdateLastInodeId(addCloseOp.inodeId, logVersion,
             lastInodeId);
             lastInodeId);
-        newFile = (INodeFile) fsDir.unprotectedAddFile(inodeId,
+        newFile = fsDir.unprotectedAddFile(inodeId,
             addCloseOp.path, addCloseOp.permissions, replication,
             addCloseOp.path, addCloseOp.permissions, replication,
             addCloseOp.mtime, addCloseOp.atime, addCloseOp.blockSize, true,
             addCloseOp.mtime, addCloseOp.atime, addCloseOp.blockSize, true,
             addCloseOp.clientName, addCloseOp.clientMachine);
             addCloseOp.clientName, addCloseOp.clientMachine);
@@ -297,8 +307,9 @@ public class FSEditLogLoader {
           }
           }
           fsNamesys.prepareFileForWrite(addCloseOp.path, oldFile,
           fsNamesys.prepareFileForWrite(addCloseOp.path, oldFile,
               addCloseOp.clientName, addCloseOp.clientMachine, null,
               addCloseOp.clientName, addCloseOp.clientMachine, null,
-              false);
-          newFile = getINodeFile(fsDir, addCloseOp.path);
+              false, iip.getLatestSnapshot());
+          newFile = INodeFile.valueOf(fsDir.getINode(addCloseOp.path),
+              addCloseOp.path, true);
         }
         }
       }
       }
       // Fall-through for case 2.
       // Fall-through for case 2.
@@ -306,8 +317,8 @@ public class FSEditLogLoader {
       // update the block list.
       // update the block list.
       
       
       // Update the salient file attributes.
       // Update the salient file attributes.
-      newFile.setAccessTime(addCloseOp.atime);
-      newFile.setModificationTimeForce(addCloseOp.mtime);
+      newFile.setAccessTime(addCloseOp.atime, null, fsDir.getINodeMap());
+      newFile.setModificationTime(addCloseOp.mtime, null, fsDir.getINodeMap());
       updateBlocks(fsDir, addCloseOp, newFile);
       updateBlocks(fsDir, addCloseOp, newFile);
       break;
       break;
     }
     }
@@ -321,15 +332,12 @@ public class FSEditLogLoader {
             " clientMachine " + addCloseOp.clientMachine);
             " clientMachine " + addCloseOp.clientMachine);
       }
       }
 
 
-      INodeFile oldFile = getINodeFile(fsDir, addCloseOp.path);
-      if (oldFile == null) {
-        throw new IOException("Operation trying to close non-existent file " +
-            addCloseOp.path);
-      }
-      
+      final INodesInPath iip = fsDir.getLastINodeInPath(addCloseOp.path);
+      final INodeFile oldFile = INodeFile.valueOf(iip.getINode(0), addCloseOp.path);
+
       // Update the salient file attributes.
       // Update the salient file attributes.
-      oldFile.setAccessTime(addCloseOp.atime);
-      oldFile.setModificationTimeForce(addCloseOp.mtime);
+      oldFile.setAccessTime(addCloseOp.atime, null, fsDir.getINodeMap());
+      oldFile.setModificationTime(addCloseOp.mtime, null, fsDir.getINodeMap());
       updateBlocks(fsDir, addCloseOp, oldFile);
       updateBlocks(fsDir, addCloseOp, oldFile);
 
 
       // Now close the file
       // Now close the file
@@ -346,8 +354,8 @@ public class FSEditLogLoader {
       if (oldFile.isUnderConstruction()) {
       if (oldFile.isUnderConstruction()) {
         INodeFileUnderConstruction ucFile = (INodeFileUnderConstruction) oldFile;
         INodeFileUnderConstruction ucFile = (INodeFileUnderConstruction) oldFile;
         fsNamesys.leaseManager.removeLeaseWithPrefixPath(addCloseOp.path);
         fsNamesys.leaseManager.removeLeaseWithPrefixPath(addCloseOp.path);
-        INodeFile newFile = ucFile.convertToInodeFile();
-        fsDir.unprotectedReplaceNode(addCloseOp.path, ucFile, newFile);
+        INodeFile newFile = ucFile.toINodeFile(ucFile.getModificationTime());
+        fsDir.unprotectedReplaceINodeFile(addCloseOp.path, ucFile, newFile);
       }
       }
       break;
       break;
     }
     }
@@ -357,13 +365,8 @@ public class FSEditLogLoader {
         FSNamesystem.LOG.debug(op.opCode + ": " + updateOp.path +
         FSNamesystem.LOG.debug(op.opCode + ": " + updateOp.path +
             " numblocks : " + updateOp.blocks.length);
             " numblocks : " + updateOp.blocks.length);
       }
       }
-      INodeFile oldFile = getINodeFile(fsDir, updateOp.path);
-      if (oldFile == null) {
-        throw new IOException(
-            "Operation trying to update blocks in non-existent file " +
-            updateOp.path);
-      }
-      
+      INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(updateOp.path),
+          updateOp.path);
       // Update in-memory data structures
       // Update in-memory data structures
       updateBlocks(fsDir, updateOp, oldFile);
       updateBlocks(fsDir, updateOp, oldFile);
       break;
       break;
@@ -512,6 +515,44 @@ public class FSEditLogLoader {
       // no data in here currently.
       // no data in here currently.
       break;
       break;
     }
     }
+    case OP_CREATE_SNAPSHOT: {
+      CreateSnapshotOp createSnapshotOp = (CreateSnapshotOp) op;
+      fsNamesys.getSnapshotManager().createSnapshot(
+          createSnapshotOp.snapshotRoot, createSnapshotOp.snapshotName);
+      break;
+    }
+    case OP_DELETE_SNAPSHOT: {
+      DeleteSnapshotOp deleteSnapshotOp = (DeleteSnapshotOp) op;
+      BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
+      List<INode> removedINodes = new ArrayList<INode>();
+      fsNamesys.getSnapshotManager().deleteSnapshot(
+          deleteSnapshotOp.snapshotRoot, deleteSnapshotOp.snapshotName,
+          collectedBlocks, removedINodes);
+      fsNamesys.removeBlocks(collectedBlocks);
+      collectedBlocks.clear();
+      fsNamesys.dir.removeFromInodeMap(removedINodes);
+      removedINodes.clear();
+      break;
+    }
+    case OP_RENAME_SNAPSHOT: {
+      RenameSnapshotOp renameSnapshotOp = (RenameSnapshotOp) op;
+      fsNamesys.getSnapshotManager().renameSnapshot(
+          renameSnapshotOp.snapshotRoot, renameSnapshotOp.snapshotOldName,
+          renameSnapshotOp.snapshotNewName);
+      break;
+    }
+    case OP_ALLOW_SNAPSHOT: {
+      AllowSnapshotOp allowSnapshotOp = (AllowSnapshotOp) op;
+      fsNamesys.getSnapshotManager().setSnapshottable(
+          allowSnapshotOp.snapshotRoot, false);
+      break;
+    }
+    case OP_DISALLOW_SNAPSHOT: {
+      DisallowSnapshotOp disallowSnapshotOp = (DisallowSnapshotOp) op;
+      fsNamesys.getSnapshotManager().resetSnapshottable(
+          disallowSnapshotOp.snapshotRoot);
+      break;
+    }
     default:
     default:
       throw new IOException("Invalid operation read " + op.opCode);
       throw new IOException("Invalid operation read " + op.opCode);
     }
     }
@@ -534,18 +575,7 @@ public class FSEditLogLoader {
     }
     }
     return sb.toString();
     return sb.toString();
   }
   }
-  
-  private static INodeFile getINodeFile(FSDirectory fsDir, String path)
-      throws IOException {
-    INode inode = fsDir.getINode(path);
-    if (inode != null) {
-      if (!(inode instanceof INodeFile)) {
-        throw new IOException("Operation trying to get non-file " + path);
-      }
-    }
-    return (INodeFile)inode;
-  }
-  
+
   /**
   /**
    * Update in-memory data structures with new block information.
    * Update in-memory data structures with new block information.
    * @throws IOException
    * @throws IOException

+ 309 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -110,6 +110,12 @@ public abstract class FSEditLogOp {
       inst.put(OP_END_LOG_SEGMENT,
       inst.put(OP_END_LOG_SEGMENT,
                     new LogSegmentOp(OP_END_LOG_SEGMENT));
                     new LogSegmentOp(OP_END_LOG_SEGMENT));
       inst.put(OP_UPDATE_BLOCKS, new UpdateBlocksOp());
       inst.put(OP_UPDATE_BLOCKS, new UpdateBlocksOp());
+
+      inst.put(OP_ALLOW_SNAPSHOT, new AllowSnapshotOp());
+      inst.put(OP_DISALLOW_SNAPSHOT, new DisallowSnapshotOp());
+      inst.put(OP_CREATE_SNAPSHOT, new CreateSnapshotOp());
+      inst.put(OP_DELETE_SNAPSHOT, new DeleteSnapshotOp());
+      inst.put(OP_RENAME_SNAPSHOT, new RenameSnapshotOp());
     }
     }
     
     
     public FSEditLogOp get(FSEditLogOpCodes opcode) {
     public FSEditLogOp get(FSEditLogOpCodes opcode) {
@@ -2210,6 +2216,309 @@ public abstract class FSEditLogOp {
     }
     }
   }
   }
 
 
+  /**
+   * Operation corresponding to creating a snapshot
+   */
+  static class CreateSnapshotOp extends FSEditLogOp {
+    String snapshotRoot;
+    String snapshotName;
+    
+    public CreateSnapshotOp() {
+      super(OP_CREATE_SNAPSHOT);
+    }
+    
+    static CreateSnapshotOp getInstance(OpInstanceCache cache) {
+      return (CreateSnapshotOp)cache.get(OP_CREATE_SNAPSHOT);
+    }
+    
+    CreateSnapshotOp setSnapshotName(String snapName) {
+      this.snapshotName = snapName;
+      return this;
+    }
+
+    public CreateSnapshotOp setSnapshotRoot(String snapRoot) {
+      snapshotRoot = snapRoot;
+      return this;
+    }
+    
+    @Override
+    void readFields(DataInputStream in, int logVersion) throws IOException {
+      snapshotRoot = FSImageSerialization.readString(in);
+      snapshotName = FSImageSerialization.readString(in);
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeString(snapshotRoot, out);
+      FSImageSerialization.writeString(snapshotName, out);
+    }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "SNAPSHOTROOT", snapshotRoot);
+      XMLUtils.addSaxString(contentHandler, "SNAPSHOTNAME", snapshotName);
+    }
+
+    @Override
+    void fromXml(Stanza st) throws InvalidXmlException {
+      snapshotRoot = st.getValue("SNAPSHOTROOT");
+      snapshotName = st.getValue("SNAPSHOTNAME");
+    }
+    
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("CreateSnapshotOp [snapshotRoot=");
+      builder.append(snapshotRoot);
+      builder.append(", snapshotName=");
+      builder.append(snapshotName);
+      builder.append("]");
+      return builder.toString();
+    }
+  }
+  
+  /**
+   * Operation corresponding to delete a snapshot
+   */
+  static class DeleteSnapshotOp extends FSEditLogOp {
+    String snapshotRoot;
+    String snapshotName;
+    
+    DeleteSnapshotOp() {
+      super(OP_DELETE_SNAPSHOT);
+    }
+    
+    static DeleteSnapshotOp getInstance(OpInstanceCache cache) {
+      return (DeleteSnapshotOp)cache.get(OP_DELETE_SNAPSHOT);
+    }
+    
+    DeleteSnapshotOp setSnapshotName(String snapName) {
+      this.snapshotName = snapName;
+      return this;
+    }
+
+    DeleteSnapshotOp setSnapshotRoot(String snapRoot) {
+      snapshotRoot = snapRoot;
+      return this;
+    }
+    
+    @Override
+    void readFields(DataInputStream in, int logVersion) throws IOException {
+      snapshotRoot = FSImageSerialization.readString(in);
+      snapshotName = FSImageSerialization.readString(in);
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeString(snapshotRoot, out);
+      FSImageSerialization.writeString(snapshotName, out);
+    }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "SNAPSHOTROOT", snapshotRoot);
+      XMLUtils.addSaxString(contentHandler, "SNAPSHOTNAME", snapshotName);
+    }
+
+    @Override
+    void fromXml(Stanza st) throws InvalidXmlException {
+      snapshotRoot = st.getValue("SNAPSHOTROOT");
+      snapshotName = st.getValue("SNAPSHOTNAME");
+    }
+    
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("DeleteSnapshotOp [snapshotRoot=");
+      builder.append(snapshotRoot);
+      builder.append(", snapshotName=");
+      builder.append(snapshotName);
+      builder.append("]");
+      return builder.toString();
+    }
+  }
+  
+  /**
+   * Operation corresponding to rename a snapshot
+   */
+  static class RenameSnapshotOp extends FSEditLogOp {
+    String snapshotRoot;
+    String snapshotOldName;
+    String snapshotNewName;
+    
+    RenameSnapshotOp() {
+      super(OP_RENAME_SNAPSHOT);
+    }
+    
+    static RenameSnapshotOp getInstance(OpInstanceCache cache) {
+      return (RenameSnapshotOp) cache.get(OP_RENAME_SNAPSHOT);
+    }
+    
+    RenameSnapshotOp setSnapshotOldName(String snapshotOldName) {
+      this.snapshotOldName = snapshotOldName;
+      return this;
+    }
+
+    RenameSnapshotOp setSnapshotNewName(String snapshotNewName) {
+      this.snapshotNewName = snapshotNewName;
+      return this;
+    }
+    
+    RenameSnapshotOp setSnapshotRoot(String snapshotRoot) {
+      this.snapshotRoot = snapshotRoot;
+      return this;
+    }
+    
+    @Override
+    void readFields(DataInputStream in, int logVersion) throws IOException {
+      snapshotRoot = FSImageSerialization.readString(in);
+      snapshotOldName = FSImageSerialization.readString(in);
+      snapshotNewName = FSImageSerialization.readString(in);
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeString(snapshotRoot, out);
+      FSImageSerialization.writeString(snapshotOldName, out);
+      FSImageSerialization.writeString(snapshotNewName, out);
+    }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "SNAPSHOTROOT", snapshotRoot);
+      XMLUtils.addSaxString(contentHandler, "SNAPSHOTOLDNAME", snapshotOldName);
+      XMLUtils.addSaxString(contentHandler, "SNAPSHOTNEWNAME", snapshotNewName);
+    }
+
+    @Override
+    void fromXml(Stanza st) throws InvalidXmlException {
+      snapshotRoot = st.getValue("SNAPSHOTROOT");
+      snapshotOldName = st.getValue("SNAPSHOTOLDNAME");
+      snapshotNewName = st.getValue("SNAPSHOTNEWNAME");
+    }
+    
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("RenameSnapshotOp [snapshotRoot=");
+      builder.append(snapshotRoot);
+      builder.append(", snapshotOldName=");
+      builder.append(snapshotOldName);
+      builder.append(", snapshotNewName=");
+      builder.append(snapshotNewName);
+      builder.append("]");
+      return builder.toString();
+    }
+  }
+
+  /**
+   * Operation corresponding to allow creating snapshot on a directory
+   */
+  static class AllowSnapshotOp extends FSEditLogOp {
+    String snapshotRoot;
+
+    public AllowSnapshotOp() {
+      super(OP_ALLOW_SNAPSHOT);
+    }
+
+    public AllowSnapshotOp(String snapRoot) {
+      super(OP_ALLOW_SNAPSHOT);
+      snapshotRoot = snapRoot;
+    }
+
+    static AllowSnapshotOp getInstance(OpInstanceCache cache) {
+      return (AllowSnapshotOp) cache.get(OP_ALLOW_SNAPSHOT);
+    }
+
+    public AllowSnapshotOp setSnapshotRoot(String snapRoot) {
+      snapshotRoot = snapRoot;
+      return this;
+    }
+
+    @Override
+    void readFields(DataInputStream in, int logVersion) throws IOException {
+      snapshotRoot = FSImageSerialization.readString(in);
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeString(snapshotRoot, out);
+    }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "SNAPSHOTROOT", snapshotRoot);
+    }
+
+    @Override
+    void fromXml(Stanza st) throws InvalidXmlException {
+      snapshotRoot = st.getValue("SNAPSHOTROOT");
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("AllowSnapshotOp [snapshotRoot=");
+      builder.append(snapshotRoot);
+      builder.append("]");
+      return builder.toString();
+    }
+  }
+
+  /**
+   * Operation corresponding to disallow creating snapshot on a directory
+   */
+  static class DisallowSnapshotOp extends FSEditLogOp {
+    String snapshotRoot;
+
+    public DisallowSnapshotOp() {
+      super(OP_DISALLOW_SNAPSHOT);
+    }
+
+    public DisallowSnapshotOp(String snapRoot) {
+      super(OP_DISALLOW_SNAPSHOT);
+      snapshotRoot = snapRoot;
+    }
+
+    static DisallowSnapshotOp getInstance(OpInstanceCache cache) {
+      return (DisallowSnapshotOp) cache.get(OP_DISALLOW_SNAPSHOT);
+    }
+
+    public DisallowSnapshotOp setSnapshotRoot(String snapRoot) {
+      snapshotRoot = snapRoot;
+      return this;
+    }
+
+    @Override
+    void readFields(DataInputStream in, int logVersion) throws IOException {
+      snapshotRoot = FSImageSerialization.readString(in);
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeString(snapshotRoot, out);
+    }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "SNAPSHOTROOT", snapshotRoot);
+    }
+
+    @Override
+    void fromXml(Stanza st) throws InvalidXmlException {
+      snapshotRoot = st.getValue("SNAPSHOTROOT");
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("DisallowSnapshotOp [snapshotRoot=");
+      builder.append(snapshotRoot);
+      builder.append("]");
+      return builder.toString();
+    }
+  }
+
   static private short readShort(DataInputStream in) throws IOException {
   static private short readShort(DataInputStream in) throws IOException {
     return Short.parseShort(FSImageSerialization.readString(in));
     return Short.parseShort(FSImageSerialization.readString(in));
   }
   }

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

@@ -56,7 +56,12 @@ public enum FSEditLogOpCodes {
   OP_REASSIGN_LEASE             ((byte) 22),
   OP_REASSIGN_LEASE             ((byte) 22),
   OP_END_LOG_SEGMENT            ((byte) 23),
   OP_END_LOG_SEGMENT            ((byte) 23),
   OP_START_LOG_SEGMENT          ((byte) 24),
   OP_START_LOG_SEGMENT          ((byte) 24),
-  OP_UPDATE_BLOCKS              ((byte) 25);
+  OP_UPDATE_BLOCKS              ((byte) 25),
+  OP_CREATE_SNAPSHOT            ((byte) 26),
+  OP_DELETE_SNAPSHOT            ((byte) 27),
+  OP_RENAME_SNAPSHOT            ((byte) 28),
+  OP_ALLOW_SNAPSHOT             ((byte) 29),
+  OP_DISALLOW_SNAPSHOT          ((byte) 30);
 
 
   private byte opCode;
   private byte opCode;
 
 

+ 55 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -17,6 +17,8 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
+import static org.apache.hadoop.util.Time.now;
+
 import java.io.Closeable;
 import java.io.Closeable;
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
@@ -31,14 +33,18 @@ import java.util.Map;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
 import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -47,7 +53,6 @@ import org.apache.hadoop.hdfs.server.common.Util;
 import static org.apache.hadoop.util.Time.now;
 import static org.apache.hadoop.util.Time.now;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
-
 import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.FSImageFile;
 import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.FSImageFile;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
@@ -61,9 +66,6 @@ import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.util.IdGenerator;
 import org.apache.hadoop.util.IdGenerator;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.HAUtil;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Joiner;
@@ -736,11 +738,58 @@ public class FSImage implements Closeable {
     } finally {
     } finally {
       FSEditLog.closeAllStreams(editStreams);
       FSEditLog.closeAllStreams(editStreams);
       // update the counts
       // update the counts
-      target.dir.updateCountForINodeWithQuota();   
+      updateCountForQuota(target.dir.rootDir);   
     }
     }
     return lastAppliedTxId - prevLastAppliedTxId;
     return lastAppliedTxId - prevLastAppliedTxId;
   }
   }
 
 
+  /**
+   * Update the count of each directory with quota in the namespace.
+   * A directory's count is defined as the total number inodes in the tree
+   * rooted at the directory.
+   * 
+   * This is an update of existing state of the filesystem and does not
+   * throw QuotaExceededException.
+   */
+  static void updateCountForQuota(INodeDirectoryWithQuota root) {
+    updateCountForQuotaRecursively(root, Quota.Counts.newInstance());
+  }
+  
+  private static void updateCountForQuotaRecursively(INodeDirectory dir,
+      Quota.Counts counts) {
+    final long parentNamespace = counts.get(Quota.NAMESPACE);
+    final long parentDiskspace = counts.get(Quota.DISKSPACE);
+
+    dir.computeQuotaUsage4CurrentDirectory(counts);
+    
+    for (INode child : dir.getChildrenList(null)) {
+      if (child.isDirectory()) {
+        updateCountForQuotaRecursively(child.asDirectory(), counts);
+      } else {
+        // file or symlink: count here to reduce recursive calls.
+        child.computeQuotaUsage(counts, false);
+      }
+    }
+      
+    if (dir.isQuotaSet()) {
+      // check if quota is violated. It indicates a software bug.
+      final long namespace = counts.get(Quota.NAMESPACE) - parentNamespace;
+      if (Quota.isViolated(dir.getNsQuota(), namespace)) {
+        LOG.error("BUG: Namespace quota violation in image for "
+            + dir.getFullPathName()
+            + " quota = " + dir.getNsQuota() + " < consumed = " + namespace);
+      }
+
+      final long diskspace = counts.get(Quota.DISKSPACE) - parentDiskspace;
+      if (Quota.isViolated(dir.getDsQuota(), diskspace)) {
+        LOG.error("BUG: Diskspace quota violation in image for "
+            + dir.getFullPathName()
+            + " quota = " + dir.getDsQuota() + " < consumed = " + diskspace);
+      }
+
+      ((INodeDirectoryWithQuota)dir).setSpaceConsumed(namespace, diskspace);
+    }
+  }
 
 
   /**
   /**
    * Load the image namespace from the given image file, verifying
    * Load the image namespace from the given image file, verifying

+ 8 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageCompression.java

@@ -17,23 +17,23 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
+import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.OutputStream;
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.IOException;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
 
 
-import org.apache.hadoop.io.Text;
-
 /**
 /**
  * Simple container class that handles support for compressed fsimage files.
  * Simple container class that handles support for compressed fsimage files.
  */
  */
@@ -108,15 +108,14 @@ class FSImageCompression {
    * underlying IO fails.
    * underlying IO fails.
    */
    */
   static FSImageCompression readCompressionHeader(
   static FSImageCompression readCompressionHeader(
-    Configuration conf,
-    DataInputStream dis) throws IOException
+    Configuration conf, DataInput in) throws IOException
   {
   {
-    boolean isCompressed = dis.readBoolean();
+    boolean isCompressed = in.readBoolean();
 
 
     if (!isCompressed) {
     if (!isCompressed) {
       return createNoopCompression();
       return createNoopCompression();
     } else {
     } else {
-      String codecClassName = Text.readString(dis);
+      String codecClassName = Text.readString(in);
       return createCompression(conf, codecClassName);
       return createCompression(conf, codecClassName);
     }
     }
   }
   }

+ 444 - 138
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -19,18 +19,21 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 
 import static org.apache.hadoop.util.Time.now;
 import static org.apache.hadoop.util.Time.now;
 
 
+import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileInputStream;
+import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.security.DigestInputStream;
 import java.security.DigestInputStream;
 import java.security.DigestOutputStream;
 import java.security.DigestOutputStream;
 import java.security.MessageDigest;
 import java.security.MessageDigest;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.List;
 import java.util.List;
+import java.util.Map;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -38,14 +41,24 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIsNotDirectoryException;
+import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileUnderConstructionWithSnapshot;
+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.SnapshotFSImageFormat;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 
 
@@ -56,13 +69,14 @@ import org.apache.hadoop.io.Text;
  * In particular, the format of the FSImage looks like:
  * In particular, the format of the FSImage looks like:
  * <pre>
  * <pre>
  * FSImage {
  * FSImage {
- *   LayoutVersion: int, NamespaceID: int, NumberItemsInFSDirectoryTree: long,
- *   NamesystemGenerationStamp: long, TransactionID: long
+ *   layoutVersion: int, namespaceID: int, numberItemsInFSDirectoryTree: long,
+ *   namesystemGenerationStamp: long, transactionID: long, 
+ *   snapshotCounter: int, numberOfSnapshots: int, numOfSnapshottableDirs: int,
  *   {FSDirectoryTree, FilesUnderConstruction, SecretManagerState} (can be compressed)
  *   {FSDirectoryTree, FilesUnderConstruction, SecretManagerState} (can be compressed)
  * }
  * }
  * 
  * 
  * FSDirectoryTree (if {@link Feature#FSIMAGE_NAME_OPTIMIZATION} is supported) {
  * FSDirectoryTree (if {@link Feature#FSIMAGE_NAME_OPTIMIZATION} is supported) {
- *   INodeInfo of root, NumberOfChildren of root: int
+ *   INodeInfo of root, numberOfChildren of root: int
  *   [list of INodeInfo of root's children],
  *   [list of INodeInfo of root's children],
  *   [list of INodeDirectoryInfo of root's directory children]
  *   [list of INodeDirectoryInfo of root's directory children]
  * }
  * }
@@ -73,38 +87,83 @@ import org.apache.hadoop.io.Text;
  * 
  * 
  * INodeInfo {
  * INodeInfo {
  *   {
  *   {
- *     LocalName: short + byte[]
+ *     localName: short + byte[]
  *   } when {@link Feature#FSIMAGE_NAME_OPTIMIZATION} is supported
  *   } when {@link Feature#FSIMAGE_NAME_OPTIMIZATION} is supported
  *   or 
  *   or 
  *   {
  *   {
- *     FullPath: byte[]
+ *     fullPath: byte[]
  *   } when {@link Feature#FSIMAGE_NAME_OPTIMIZATION} is not supported
  *   } when {@link Feature#FSIMAGE_NAME_OPTIMIZATION} is not supported
- *   ReplicationFactor: short, ModificationTime: long,
- *   AccessTime: long, PreferredBlockSize: long,
- *   NumberOfBlocks: int (-1 for INodeDirectory, -2 for INodeSymLink),
+ *   replicationFactor: short, modificationTime: long,
+ *   accessTime: long, preferredBlockSize: long,
+ *   numberOfBlocks: int (-1 for INodeDirectory, -2 for INodeSymLink),
  *   { 
  *   { 
- *     NsQuota: long, DsQuota: long, FsPermission: short, PermissionStatus
+ *     nsQuota: long, dsQuota: long, 
+ *     {
+ *       isINodeSnapshottable: byte,
+ *       isINodeWithSnapshot: byte (if isINodeSnapshottable is false)
+ *     } (when {@link Feature#SNAPSHOT} is supported), 
+ *     fsPermission: short, PermissionStatus
  *   } for INodeDirectory
  *   } for INodeDirectory
  *   or 
  *   or 
  *   {
  *   {
- *     SymlinkString, FsPermission: short, PermissionStatus
+ *     symlinkString, fsPermission: short, PermissionStatus
  *   } for INodeSymlink
  *   } for INodeSymlink
  *   or
  *   or
  *   {
  *   {
- *     [list of BlockInfo], FsPermission: short, PermissionStatus
+ *     [list of BlockInfo]
+ *     [list of FileDiff]
+ *     {
+ *       isINodeFileUnderConstructionSnapshot: byte, 
+ *       {clientName: short + byte[], clientMachine: short + byte[]} (when 
+ *       isINodeFileUnderConstructionSnapshot is true),
+ *     } (when {@link Feature#SNAPSHOT} is supported and writing snapshotINode), 
+ *     fsPermission: short, PermissionStatus
  *   } for INodeFile
  *   } for INodeFile
  * }
  * }
  * 
  * 
  * INodeDirectoryInfo {
  * INodeDirectoryInfo {
- *   FullPath of the directory: short + byte[],
- *   NumberOfChildren: int, [list of INodeInfo of children INode]
- *   [list of INodeDirectoryInfo of the directory children]
+ *   fullPath of the directory: short + byte[],
+ *   numberOfChildren: int, [list of INodeInfo of children INode],
+ *   {
+ *     numberOfSnapshots: int,
+ *     [list of Snapshot] (when NumberOfSnapshots is positive),
+ *     numberOfDirectoryDiffs: int,
+ *     [list of DirectoryDiff] (NumberOfDirectoryDiffs is positive),
+ *     number of children that are directories,
+ *     [list of INodeDirectoryInfo of the directory children] (includes
+ *     snapshot copies of deleted sub-directories)
+ *   } (when {@link Feature#SNAPSHOT} is supported), 
+ * }
+ * 
+ * Snapshot {
+ *   snapshotID: int, root of Snapshot: INodeDirectoryInfo (its local name is 
+ *   the name of the snapshot)
+ * }
+ * 
+ * DirectoryDiff {
+ *   full path of the root of the associated Snapshot: short + byte[], 
+ *   childrenSize: int, 
+ *   isSnapshotRoot: byte, 
+ *   snapshotINodeIsNotNull: byte (when isSnapshotRoot is false),
+ *   snapshotINode: INodeDirectory (when SnapshotINodeIsNotNull is true), Diff 
+ * }
+ * 
+ * Diff {
+ *   createdListSize: int, [Local name of INode in created list],
+ *   deletedListSize: int, [INode in deleted list: INodeInfo]
+ * }
+ *
+ * FileDiff {
+ *   full path of the root of the associated Snapshot: short + byte[], 
+ *   fileSize: long, 
+ *   snapshotINodeIsNotNull: byte,
+ *   snapshotINode: INodeFile (when SnapshotINodeIsNotNull is true), Diff 
  * }
  * }
  * </pre>
  * </pre>
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
-class FSImageFormat {
+public class FSImageFormat {
   private static final Log LOG = FSImage.LOG;
   private static final Log LOG = FSImage.LOG;
   
   
   // Static-only class
   // Static-only class
@@ -115,7 +174,7 @@ class FSImageFormat {
    * should be called once, after which the getter methods may be used to retrieve
    * should be called once, after which the getter methods may be used to retrieve
    * information about the image that was loaded, if loading was successful.
    * information about the image that was loaded, if loading was successful.
    */
    */
-  static class Loader {
+  public static class Loader {
     private final Configuration conf;
     private final Configuration conf;
     /** which namesystem this loader is working for */
     /** which namesystem this loader is working for */
     private final FSNamesystem namesystem;
     private final FSNamesystem namesystem;
@@ -127,6 +186,9 @@ class FSImageFormat {
     private long imgTxId;
     private long imgTxId;
     /** The MD5 sum of the loaded file */
     /** The MD5 sum of the loaded file */
     private MD5Hash imgDigest;
     private MD5Hash imgDigest;
+    
+    private Map<Integer, Snapshot> snapshotMap = null;
+    private final ReferenceMap referenceMap = new ReferenceMap();
 
 
     Loader(Configuration conf, FSNamesystem namesystem) {
     Loader(Configuration conf, FSNamesystem namesystem) {
       this.conf = conf;
       this.conf = conf;
@@ -165,9 +227,7 @@ class FSImageFormat {
       }
       }
     }
     }
 
 
-    void load(File curFile)
-      throws IOException
-    {
+    void load(File curFile) throws IOException {
       checkNotLoaded();
       checkNotLoaded();
       assert curFile != null : "curFile is null";
       assert curFile != null : "curFile is null";
 
 
@@ -189,6 +249,8 @@ class FSImageFormat {
               "imgVersion " + imgVersion +
               "imgVersion " + imgVersion +
               " expected to be " + getLayoutVersion());
               " expected to be " + getLayoutVersion());
         }
         }
+        boolean supportSnapshot = LayoutVersion.supports(Feature.SNAPSHOT,
+            imgVersion);
 
 
         // read namespaceID: first appeared in version -2
         // read namespaceID: first appeared in version -2
         in.readInt();
         in.readInt();
@@ -221,6 +283,10 @@ class FSImageFormat {
           }
           }
         }
         }
         
         
+        if (supportSnapshot) {
+          snapshotMap = namesystem.getSnapshotManager().read(in, this);
+        }
+
         // read compression related info
         // read compression related info
         FSImageCompression compression;
         FSImageCompression compression;
         if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imgVersion)) {
         if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imgVersion)) {
@@ -236,12 +302,16 @@ class FSImageFormat {
         LOG.info("Number of files = " + numFiles);
         LOG.info("Number of files = " + numFiles);
         if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
         if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
             imgVersion)) {
             imgVersion)) {
-          loadLocalNameINodes(numFiles, in);
+          if (supportSnapshot) {
+            loadLocalNameINodesWithSnapshot(in);
+          } else {
+            loadLocalNameINodes(numFiles, in);
+          }
         } else {
         } else {
           loadFullNameINodes(numFiles, in);
           loadFullNameINodes(numFiles, in);
         }
         }
 
 
-        loadFilesUnderConstruction(in);
+        loadFilesUnderConstruction(in, supportSnapshot);
 
 
         loadSecretManagerState(in);
         loadSecretManagerState(in);
 
 
@@ -260,17 +330,35 @@ class FSImageFormat {
     }
     }
 
 
   /** Update the root node's attributes */
   /** Update the root node's attributes */
-  private void updateRootAttr(INode root) {                                                           
+  private void updateRootAttr(INodeWithAdditionalFields root) {                                                           
     long nsQuota = root.getNsQuota();
     long nsQuota = root.getNsQuota();
     long dsQuota = root.getDsQuota();
     long dsQuota = root.getDsQuota();
     FSDirectory fsDir = namesystem.dir;
     FSDirectory fsDir = namesystem.dir;
     if (nsQuota != -1 || dsQuota != -1) {
     if (nsQuota != -1 || dsQuota != -1) {
       fsDir.rootDir.setQuota(nsQuota, dsQuota);
       fsDir.rootDir.setQuota(nsQuota, dsQuota);
     }
     }
-    fsDir.rootDir.setModificationTime(root.getModificationTime());
+    fsDir.rootDir.cloneModificationTime(root);
     fsDir.rootDir.clonePermissionStatus(root);    
     fsDir.rootDir.clonePermissionStatus(root);    
   }
   }
-
+  
+    /**
+     * Load fsimage files when 1) only local names are stored, 
+     * and 2) snapshot is supported.
+     * 
+     * @param in Image input stream
+     */
+    private void loadLocalNameINodesWithSnapshot(DataInput in)
+        throws IOException {
+      assert LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
+          getLayoutVersion());
+      assert LayoutVersion.supports(Feature.SNAPSHOT, getLayoutVersion());
+      
+      // load root
+      loadRoot(in);
+      // load rest of the nodes recursively
+      loadDirectoryWithSnapshot(in);
+    }
+    
   /** 
   /** 
    * load fsimage files assuming only local names are stored
    * load fsimage files assuming only local names are stored
    *   
    *   
@@ -278,20 +366,16 @@ class FSImageFormat {
    * @param in image input stream
    * @param in image input stream
    * @throws IOException
    * @throws IOException
    */  
    */  
-   private void loadLocalNameINodes(long numFiles, DataInputStream in)
-        throws IOException {
+   private void loadLocalNameINodes(long numFiles, DataInput in) 
+       throws IOException {
      assert LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
      assert LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
          getLayoutVersion());
          getLayoutVersion());
      assert numFiles > 0;
      assert numFiles > 0;
 
 
      // load root
      // load root
-     if( in.readShort() != 0) {
-       throw new IOException("First node is not root");
-     }   
-     INode root = loadINode(in);
-     // update the root's attributes
-     updateRootAttr(root);
-     numFiles--;
+     loadRoot(in);
+     // have loaded the first file (the root)
+     numFiles--; 
 
 
      // load rest of the nodes directory by directory
      // load rest of the nodes directory by directory
      while (numFiles > 0) {
      while (numFiles > 0) {
@@ -302,6 +386,78 @@ class FSImageFormat {
      }
      }
    }
    }
    
    
+    /**
+     * Load information about root, and use the information to update the root
+     * directory of NameSystem.
+     * @param in The {@link DataInput} instance to read.
+     */
+    private void loadRoot(DataInput in) throws IOException {
+      // load root
+      if (in.readShort() != 0) {
+        throw new IOException("First node is not root");
+      }
+      final INodeDirectory root = loadINode(null, false, in).asDirectory();
+      // update the root's attributes
+      updateRootAttr(root);
+    }
+   
+    /** Load children nodes for the parent directory. */
+    private int loadChildren(INodeDirectory parent, DataInput in)
+        throws IOException {
+      int numChildren = in.readInt();
+      for (int i = 0; i < numChildren; i++) {
+        // load single inode
+        INode newNode = loadINodeWithLocalName(false, in);
+        addToParent(parent, newNode);
+      }
+      return numChildren;
+    }
+    
+    /**
+     * Load a directory when snapshot is supported.
+     * @param in The {@link DataInput} instance to read.
+     */
+    private void loadDirectoryWithSnapshot(DataInput in)
+        throws IOException {
+      // Step 1. Identify the parent INode
+      long inodeId = in.readLong();
+      final INodeDirectory parent = this.namesystem.dir.getInode(inodeId)
+          .asDirectory();
+      
+      // Check if the whole subtree has been saved (for reference nodes)
+      boolean toLoadSubtree = referenceMap.toProcessSubtree(parent.getId());
+      if (!toLoadSubtree) {
+        return;
+      }
+      
+      // Step 2. Load snapshots if parent is snapshottable
+      int numSnapshots = in.readInt();
+      if (numSnapshots >= 0) {
+        final INodeDirectorySnapshottable snapshottableParent
+            = INodeDirectorySnapshottable.valueOf(parent, parent.getLocalName());
+        if (snapshottableParent.getParent() != null) { // not root
+          this.namesystem.getSnapshotManager().addSnapshottable(
+              snapshottableParent);
+        }
+        // load snapshots and snapshotQuota
+        SnapshotFSImageFormat.loadSnapshotList(snapshottableParent,
+            numSnapshots, in, this);
+      }
+
+      // Step 3. Load children nodes under parent
+      loadChildren(parent, in);
+      
+      // Step 4. load Directory Diff List
+      SnapshotFSImageFormat.loadDirectoryDiffList(parent, in, this);
+      
+      // Recursively load sub-directories, including snapshot copies of deleted
+      // directories
+      int numSubTree = in.readInt();
+      for (int i = 0; i < numSubTree; i++) {
+        loadDirectoryWithSnapshot(in);
+      }
+    }
+    
    /**
    /**
     * Load all children of a directory
     * Load all children of a directory
     * 
     * 
@@ -309,24 +465,11 @@ class FSImageFormat {
     * @return number of child inodes read
     * @return number of child inodes read
     * @throws IOException
     * @throws IOException
     */
     */
-   private int loadDirectory(DataInputStream in) throws IOException {
+   private int loadDirectory(DataInput in) throws IOException {
      String parentPath = FSImageSerialization.readString(in);
      String parentPath = FSImageSerialization.readString(in);
-     FSDirectory fsDir = namesystem.dir;
      final INodeDirectory parent = INodeDirectory.valueOf(
      final INodeDirectory parent = INodeDirectory.valueOf(
-         fsDir.rootDir.getNode(parentPath, true), parentPath);
-
-     int numChildren = in.readInt();
-     for(int i=0; i<numChildren; i++) {
-       // load single inode
-       byte[] localName = new byte[in.readShort()];
-       in.readFully(localName); // read local name
-       INode newNode = loadINode(in); // read rest of inode
-
-       // add to parent
-       newNode.setLocalName(localName);
-       addToParent(parent, newNode);
-     }
-     return numChildren;
+         namesystem.dir.rootDir.getNode(parentPath, true), parentPath);
+     return loadChildren(parent, in);
    }
    }
 
 
   /**
   /**
@@ -337,38 +480,50 @@ class FSImageFormat {
    * @throws IOException if any error occurs
    * @throws IOException if any error occurs
    */
    */
   private void loadFullNameINodes(long numFiles,
   private void loadFullNameINodes(long numFiles,
-      DataInputStream in) throws IOException {
+      DataInput in) throws IOException {
     byte[][] pathComponents;
     byte[][] pathComponents;
     byte[][] parentPath = {{}};      
     byte[][] parentPath = {{}};      
     FSDirectory fsDir = namesystem.dir;
     FSDirectory fsDir = namesystem.dir;
     INodeDirectory parentINode = fsDir.rootDir;
     INodeDirectory parentINode = fsDir.rootDir;
     for (long i = 0; i < numFiles; i++) {
     for (long i = 0; i < numFiles; i++) {
       pathComponents = FSImageSerialization.readPathComponents(in);
       pathComponents = FSImageSerialization.readPathComponents(in);
-      INode newNode = loadINode(in);
+      final INode newNode = loadINode(
+          pathComponents[pathComponents.length-1], false, in);
 
 
       if (isRoot(pathComponents)) { // it is the root
       if (isRoot(pathComponents)) { // it is the root
         // update the root's attributes
         // update the root's attributes
-        updateRootAttr(newNode);
+        updateRootAttr(newNode.asDirectory());
         continue;
         continue;
       }
       }
       // check if the new inode belongs to the same parent
       // check if the new inode belongs to the same parent
       if(!isParent(pathComponents, parentPath)) {
       if(!isParent(pathComponents, parentPath)) {
-        parentINode = fsDir.rootDir.getParent(pathComponents);
+        parentINode = getParentINodeDirectory(pathComponents);
         parentPath = getParent(pathComponents);
         parentPath = getParent(pathComponents);
       }
       }
 
 
       // add new inode
       // add new inode
-      newNode.setLocalName(pathComponents[pathComponents.length-1]);
       addToParent(parentINode, newNode);
       addToParent(parentINode, newNode);
     }
     }
   }
   }
 
 
+  private INodeDirectory getParentINodeDirectory(byte[][] pathComponents
+      ) throws FileNotFoundException, PathIsNotDirectoryException,
+      UnresolvedLinkException {
+    if (pathComponents.length < 2) { // root
+      return null;
+    }
+    // Gets the parent INode
+    final INodesInPath inodes = namesystem.dir.getExistingPathINodes(
+        pathComponents);
+    return INodeDirectory.valueOf(inodes.getINode(-2), pathComponents);
+  }
+
   /**
   /**
    * Add the child node to parent and, if child is a file, update block map.
    * Add the child node to parent and, if child is a file, update block map.
    * This method is only used for image loading so that synchronization,
    * This method is only used for image loading so that synchronization,
    * modification time update and space count update are not needed.
    * modification time update and space count update are not needed.
    */
    */
-  void addToParent(INodeDirectory parent, INode child) {
+  private void addToParent(INodeDirectory parent, INode child) {
     FSDirectory fsDir = namesystem.dir;
     FSDirectory fsDir = namesystem.dir;
     if (parent == fsDir.rootDir && FSDirectory.isReservedName(child)) {
     if (parent == fsDir.rootDir && FSDirectory.isReservedName(child)) {
         throw new HadoopIllegalArgumentException("File name \""
         throw new HadoopIllegalArgumentException("File name \""
@@ -377,81 +532,166 @@ class FSImageFormat {
             + "name before upgrading to this release.");
             + "name before upgrading to this release.");
     }
     }
     // NOTE: This does not update space counts for parents
     // NOTE: This does not update space counts for parents
-    if (!parent.addChild(child, false)) {
+    if (!parent.addChild(child)) {
       return;
       return;
     }
     }
     namesystem.dir.cacheName(child);
     namesystem.dir.cacheName(child);
 
 
     if (child.isFile()) {
     if (child.isFile()) {
       // Add file->block mapping
       // Add file->block mapping
-      final INodeFile file = (INodeFile)child;
+      final INodeFile file = child.asFile();
       final BlockInfo[] blocks = file.getBlocks();
       final BlockInfo[] blocks = file.getBlocks();
-      final BlockManager bm = namesystem.getBlockManager();
-      for (int i = 0; i < blocks.length; i++) {
-        file.setBlock(i, bm.addBlockCollection(blocks[i], file));
+      if (blocks != null) {
+        final BlockManager bm = namesystem.getBlockManager();
+        for (int i = 0; i < blocks.length; i++) {
+          file.setBlock(i, bm.addBlockCollection(blocks[i], file));
+        } 
       }
       }
     }
     }
   }
   }
 
 
+    /** @return The FSDirectory of the namesystem where the fsimage is loaded */
+    public FSDirectory getFSDirectoryInLoading() {
+      return namesystem.dir;
+    }
+
+    public INode loadINodeWithLocalName(boolean isSnapshotINode,
+        DataInput in) throws IOException {
+      final byte[] localName = FSImageSerialization.readLocalName(in);
+      INode inode = loadINode(localName, isSnapshotINode, in);
+      if (LayoutVersion.supports(Feature.ADD_INODE_ID, getLayoutVersion())) {
+        namesystem.dir.addToInodeMap(inode);
+      }
+      return inode;
+    }
+  
   /**
   /**
    * load an inode from fsimage except for its name
    * load an inode from fsimage except for its name
    * 
    * 
    * @param in data input stream from which image is read
    * @param in data input stream from which image is read
    * @return an inode
    * @return an inode
    */
    */
-  private INode loadINode(DataInputStream in)
-      throws IOException {
-    long modificationTime = 0;
-    long atime = 0;
-    long blockSize = 0;
-    
-    int imgVersion = getLayoutVersion();
+  INode loadINode(final byte[] localName, boolean isSnapshotINode,
+      DataInput in) throws IOException {
+    final int imgVersion = getLayoutVersion();
+    if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
+      namesystem.getFSDirectory().verifyINodeName(localName);
+    }
+
     long inodeId = LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion) ? 
     long inodeId = LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion) ? 
            in.readLong() : namesystem.allocateNewInodeId();
            in.readLong() : namesystem.allocateNewInodeId();
     
     
-    short replication = in.readShort();
-    replication = namesystem.getBlockManager().adjustReplication(replication);
-    modificationTime = in.readLong();
+    final short replication = namesystem.getBlockManager().adjustReplication(
+        in.readShort());
+    final long modificationTime = in.readLong();
+    long atime = 0;
     if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, imgVersion)) {
     if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, imgVersion)) {
       atime = in.readLong();
       atime = in.readLong();
     }
     }
-    blockSize = in.readLong();
-    int numBlocks = in.readInt();
-    BlockInfo blocks[] = null;
+    final long blockSize = in.readLong();
+    final int numBlocks = in.readInt();
 
 
     if (numBlocks >= 0) {
     if (numBlocks >= 0) {
-      blocks = new BlockInfo[numBlocks];
-      for (int j = 0; j < numBlocks; j++) {
-        blocks[j] = new BlockInfo(replication);
-        blocks[j].readFields(in);
+      // file
+      
+      // read blocks
+      BlockInfo[] blocks = null;
+      if (numBlocks >= 0) {
+        blocks = new BlockInfo[numBlocks];
+        for (int j = 0; j < numBlocks; j++) {
+          blocks[j] = new BlockInfo(replication);
+          blocks[j].readFields(in);
+        }
+      }
+
+      String clientName = "";
+      String clientMachine = "";
+      boolean underConstruction = false;
+      FileDiffList fileDiffs = null;
+      if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
+        // read diffs
+        fileDiffs = SnapshotFSImageFormat.loadFileDiffList(in, this);
+
+        if (isSnapshotINode) {
+          underConstruction = in.readBoolean();
+          if (underConstruction) {
+            clientName = FSImageSerialization.readString(in);
+            clientMachine = FSImageSerialization.readString(in);
+          }
+        }
+      }
+
+      final PermissionStatus permissions = PermissionStatus.read(in);
+
+      // return
+      final INodeFile file = new INodeFile(inodeId, localName, permissions,
+          modificationTime, atime, blocks, replication, blockSize);
+      return fileDiffs != null? new INodeFileWithSnapshot(file, fileDiffs)
+          : underConstruction? new INodeFileUnderConstruction(
+              file, clientName, clientMachine, null)
+          : file;
+    } else if (numBlocks == -1) {
+      //directory
+      
+      //read quotas
+      final long nsQuota = in.readLong();
+      long dsQuota = -1L;
+      if (LayoutVersion.supports(Feature.DISKSPACE_QUOTA, imgVersion)) {
+        dsQuota = in.readLong();
+      }
+
+      //read snapshot info
+      boolean snapshottable = false;
+      boolean withSnapshot = false;
+      if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
+        snapshottable = in.readBoolean();
+        if (!snapshottable) {
+          withSnapshot = in.readBoolean();
+        }
       }
       }
-    }
-    
-    // get quota only when the node is a directory
-    long nsQuota = -1L;
-    if (blocks == null && numBlocks == -1) {
-      nsQuota = in.readLong();
-    }
-    long dsQuota = -1L;
-    if (LayoutVersion.supports(Feature.DISKSPACE_QUOTA, imgVersion)
-        && blocks == null && numBlocks == -1) {
-      dsQuota = in.readLong();
-    }
 
 
-    // Read the symlink only when the node is a symlink
-    String symlink = "";
-    if (numBlocks == -2) {
-      symlink = Text.readString(in);
+      final PermissionStatus permissions = PermissionStatus.read(in);
+
+      //return
+      final INodeDirectory dir = nsQuota >= 0 || dsQuota >= 0?
+          new INodeDirectoryWithQuota(inodeId, localName, permissions,
+              modificationTime, nsQuota, dsQuota)
+          : new INodeDirectory(inodeId, localName, permissions, modificationTime);
+      return snapshottable ? new INodeDirectorySnapshottable(dir)
+          : withSnapshot ? new INodeDirectoryWithSnapshot(dir)
+          : dir;
+    } else if (numBlocks == -2) {
+      //symlink
+
+      final String symlink = Text.readString(in);
+      final PermissionStatus permissions = PermissionStatus.read(in);
+      return new INodeSymlink(inodeId, localName, permissions,
+          modificationTime, atime, symlink);
+    } else if (numBlocks == -3) {
+      //reference
+      
+      final boolean isWithName = in.readBoolean();
+      // lastSnapshotId for WithName node, dstSnapshotId for DstReference node
+      int snapshotId = in.readInt();
+      
+      final INodeReference.WithCount withCount
+          = referenceMap.loadINodeReferenceWithCount(isSnapshotINode, in, this);
+
+      if (isWithName) {
+          return new INodeReference.WithName(null, withCount, localName,
+              snapshotId);
+      } else {
+        final INodeReference ref = new INodeReference.DstReference(null,
+            withCount, snapshotId);
+        return ref;
+      }
     }
     }
     
     
-    PermissionStatus permissions = PermissionStatus.read(in);
-
-    return INode.newINode(inodeId, permissions, blocks, symlink, replication,
-        modificationTime, atime, nsQuota, dsQuota, blockSize);
+    throw new IOException("Unknown inode type: numBlocks=" + numBlocks);
   }
   }
 
 
-    private void loadFilesUnderConstruction(DataInputStream in)
-    throws IOException {
+    private void loadFilesUnderConstruction(DataInput in,
+        boolean supportSnapshot) throws IOException {
       FSDirectory fsDir = namesystem.dir;
       FSDirectory fsDir = namesystem.dir;
       int size = in.readInt();
       int size = in.readInt();
 
 
@@ -463,13 +703,22 @@ class FSImageFormat {
 
 
         // verify that file exists in namespace
         // verify that file exists in namespace
         String path = cons.getLocalName();
         String path = cons.getLocalName();
-        INodeFile oldnode = INodeFile.valueOf(fsDir.getINode(path), path);
-        fsDir.replaceNode(path, oldnode, cons);
+        final INodesInPath iip = fsDir.getLastINodeInPath(path);
+        INodeFile oldnode = INodeFile.valueOf(iip.getINode(0), path);
+        cons.setLocalName(oldnode.getLocalNameBytes());
+        cons.setParent(oldnode.getParent());
+
+        if (oldnode instanceof INodeFileWithSnapshot) {
+          cons = new INodeFileUnderConstructionWithSnapshot(cons,
+              ((INodeFileWithSnapshot)oldnode).getDiffs());
+        }
+
+        fsDir.replaceINodeFile(path, oldnode, cons);
         namesystem.leaseManager.addLease(cons.getClientName(), path); 
         namesystem.leaseManager.addLease(cons.getClientName(), path); 
       }
       }
     }
     }
 
 
-    private void loadSecretManagerState(DataInputStream in)
+    private void loadSecretManagerState(DataInput in)
         throws IOException {
         throws IOException {
       int imgVersion = getLayoutVersion();
       int imgVersion = getLayoutVersion();
 
 
@@ -517,6 +766,10 @@ class FSImageFormat {
       }
       }
       return result;
       return result;
     }
     }
+    
+    public Snapshot getSnapshot(DataInput in) throws IOException {
+      return snapshotMap.get(in.readInt());
+    }
   }
   }
   
   
   /**
   /**
@@ -531,8 +784,7 @@ class FSImageFormat {
     
     
     /** The MD5 checksum of the file that was written */
     /** The MD5 checksum of the file that was written */
     private MD5Hash savedDigest;
     private MD5Hash savedDigest;
-
-    static private final byte[] PATH_SEPARATOR = DFSUtil.string2Bytes(Path.SEPARATOR);
+    private final ReferenceMap referenceMap = new ReferenceMap();
 
 
     /** @throws IllegalStateException if the instance has not yet saved an image */
     /** @throws IllegalStateException if the instance has not yet saved an image */
     private void checkSaved() {
     private void checkSaved() {
@@ -561,9 +813,7 @@ class FSImageFormat {
       return savedDigest;
       return savedDigest;
     }
     }
 
 
-    void save(File newFile,
-              FSImageCompression compression)
-      throws IOException {
+    void save(File newFile, FSImageCompression compression) throws IOException {
       checkNotSaved();
       checkNotSaved();
 
 
       final FSNamesystem sourceNamesystem = context.getSourceNamesystem();
       final FSNamesystem sourceNamesystem = context.getSourceNamesystem();
@@ -590,23 +840,22 @@ class FSImageFormat {
         out.writeLong(context.getTxId());
         out.writeLong(context.getTxId());
         out.writeLong(sourceNamesystem.getLastInodeId());
         out.writeLong(sourceNamesystem.getLastInodeId());
         
         
+        sourceNamesystem.getSnapshotManager().write(out);
+        
         // write compression info and set up compressed stream
         // write compression info and set up compressed stream
         out = compression.writeHeaderAndWrapStream(fos);
         out = compression.writeHeaderAndWrapStream(fos);
         LOG.info("Saving image file " + newFile +
         LOG.info("Saving image file " + newFile +
                  " using " + compression);
                  " using " + compression);
 
 
-
-        byte[] byteStore = new byte[4*HdfsConstants.MAX_PATH_LENGTH];
-        ByteBuffer strbuf = ByteBuffer.wrap(byteStore);
         // save the root
         // save the root
-        FSImageSerialization.saveINode2Image(fsDir.rootDir, out);
+        FSImageSerialization.saveINode2Image(fsDir.rootDir, out, false,
+            referenceMap);
         // save the rest of the nodes
         // save the rest of the nodes
-        saveImage(strbuf, fsDir.rootDir, out);
+        saveImage(fsDir.rootDir, out, true);
         // save files under construction
         // save files under construction
         sourceNamesystem.saveFilesUnderConstruction(out);
         sourceNamesystem.saveFilesUnderConstruction(out);
         context.checkCancelled();
         context.checkCancelled();
         sourceNamesystem.saveSecretManagerState(out);
         sourceNamesystem.saveSecretManagerState(out);
-        strbuf = null;
         context.checkCancelled();
         context.checkCancelled();
         out.flush();
         out.flush();
         context.checkCancelled();
         context.checkCancelled();
@@ -624,40 +873,97 @@ class FSImageFormat {
     }
     }
 
 
     /**
     /**
-     * Save file tree image starting from the given root.
-     * This is a recursive procedure, which first saves all children of
-     * a current directory and then moves inside the sub-directories.
+     * Save children INodes.
+     * @param children The list of children INodes
+     * @param out The DataOutputStream to write
+     * @return Number of children that are directory
      */
      */
-    private void saveImage(ByteBuffer currentDirName,
-                                  INodeDirectory current,
-                                  DataOutputStream out) throws IOException {
-      final List<INode> children = current.getChildrenList();
-      if (children.isEmpty())
-        return;
-      // print prefix (parent directory name)
-      int prefixLen = currentDirName.position();
-      if (prefixLen == 0) {  // root
-        out.writeShort(PATH_SEPARATOR.length);
-        out.write(PATH_SEPARATOR);
-      } else {  // non-root directories
-        out.writeShort(prefixLen);
-        out.write(currentDirName.array(), 0, prefixLen);
-      }
+    private int saveChildren(ReadOnlyList<INode> children, DataOutputStream out)
+        throws IOException {
+      // Write normal children INode. 
       out.writeInt(children.size());
       out.writeInt(children.size());
+      int dirNum = 0;
       int i = 0;
       int i = 0;
       for(INode child : children) {
       for(INode child : children) {
         // print all children first
         // print all children first
-        FSImageSerialization.saveINode2Image(child, out);
+        FSImageSerialization.saveINode2Image(child, out, false, referenceMap);
+        if (child.isDirectory()) {
+          dirNum++;
+        }
         if (i++ % 50 == 0) {
         if (i++ % 50 == 0) {
           context.checkCancelled();
           context.checkCancelled();
         }
         }
       }
       }
+      return dirNum;
+    }
+    
+    /**
+     * Save file tree image starting from the given root.
+     * This is a recursive procedure, which first saves all children and 
+     * snapshot diffs of a current directory and then moves inside the 
+     * sub-directories.
+     * 
+     * @param current The current node
+     * @param out The DataoutputStream to write the image
+     * @param snapshot The possible snapshot associated with the current node
+     * @param toSaveSubtree Whether or not to save the subtree to fsimage. For
+     *                      reference node, its subtree may already have been
+     *                      saved before.
+     */
+    private void saveImage(INodeDirectory current, DataOutputStream out,
+        boolean toSaveSubtree) throws IOException {
+      // write the inode id of the directory
+      out.writeLong(current.getId());
+      
+      if (!toSaveSubtree) {
+        return;
+      }
+      
+      final ReadOnlyList<INode> children = current.getChildrenList(null);
+      int dirNum = 0;
+      List<INodeDirectory> snapshotDirs = null;
+      if (current instanceof INodeDirectoryWithSnapshot) {
+        snapshotDirs = new ArrayList<INodeDirectory>();
+        ((INodeDirectoryWithSnapshot) current).getSnapshotDirectory(
+            snapshotDirs);
+        dirNum += snapshotDirs.size();
+      }
+      
+      // 2. Write INodeDirectorySnapshottable#snapshotsByNames to record all
+      // Snapshots
+      if (current instanceof INodeDirectorySnapshottable) {
+        INodeDirectorySnapshottable snapshottableNode = 
+            (INodeDirectorySnapshottable) current;
+        SnapshotFSImageFormat.saveSnapshots(snapshottableNode, out);
+      } else {
+        out.writeInt(-1); // # of snapshots
+      }
+
+      // 3. Write children INode 
+      dirNum += saveChildren(children, out);
+      
+      // 4. Write DirectoryDiff lists, if there is any.
+      SnapshotFSImageFormat.saveDirectoryDiffList(current, out, referenceMap);
+      
+      // Write sub-tree of sub-directories, including possible snapshots of 
+      // deleted sub-directories
+      out.writeInt(dirNum); // the number of sub-directories
       for(INode child : children) {
       for(INode child : children) {
-        if(!child.isDirectory())
+        if(!child.isDirectory()) {
           continue;
           continue;
-        currentDirName.put(PATH_SEPARATOR).put(child.getLocalNameBytes());
-        saveImage(currentDirName, (INodeDirectory)child, out);
-        currentDirName.position(prefixLen);
+        }
+        // make sure we only save the subtree under a reference node once
+        boolean toSave = child.isReference() ? 
+            referenceMap.toProcessSubtree(child.getId()) : true;
+        saveImage(child.asDirectory(), out, toSave);
+      }
+      if (snapshotDirs != null) {
+        for (INodeDirectory subDir : snapshotDirs) {
+          // make sure we only save the subtree under a reference node once
+          boolean toSave = subDir.getParentReference() != null ? 
+              referenceMap.toProcessSubtree(subDir.getId()) : true;
+          saveImage(subDir, out, toSave);
+        }
       }
       }
     }
     }
   }
   }

+ 173 - 61
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -17,7 +17,8 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
-import java.io.DataInputStream;
+import java.io.DataInput;
+import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 
 
@@ -34,11 +35,17 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.WritableUtils;
 
 
+import com.google.common.base.Preconditions;
+
 /**
 /**
  * Static utility functions for serializing various pieces of data in the correct
  * Static utility functions for serializing various pieces of data in the correct
  * format for the FSImage file.
  * format for the FSImage file.
@@ -77,11 +84,30 @@ public class FSImageSerialization {
     final FsPermission FILE_PERM = new FsPermission((short) 0);
     final FsPermission FILE_PERM = new FsPermission((short) 0);
   }
   }
 
 
+  private static void writePermissionStatus(INodeWithAdditionalFields inode,
+      DataOutput out) throws IOException {
+    final FsPermission p = TL_DATA.get().FILE_PERM;
+    p.fromShort(inode.getFsPermissionShort());
+    PermissionStatus.write(out, inode.getUserName(), inode.getGroupName(), p);
+  }
+
+  private static void writeBlocks(final Block[] blocks,
+      final DataOutput out) throws IOException {
+    if (blocks == null) {
+      out.writeInt(0);
+    } else {
+      out.writeInt(blocks.length);
+      for (Block blk : blocks) {
+        blk.write(out);
+      }
+    }
+  }
+
   // Helper function that reads in an INodeUnderConstruction
   // Helper function that reads in an INodeUnderConstruction
   // from the input stream
   // from the input stream
   //
   //
   static INodeFileUnderConstruction readINodeUnderConstruction(
   static INodeFileUnderConstruction readINodeUnderConstruction(
-      DataInputStream in, FSNamesystem fsNamesys, int imgVersion)
+      DataInput in, FSNamesystem fsNamesys, int imgVersion)
       throws IOException {
       throws IOException {
     byte[] name = readBytes(in);
     byte[] name = readBytes(in);
     long inodeId = LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion) ? in
     long inodeId = LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion) ? in
@@ -89,6 +115,7 @@ public class FSImageSerialization {
     short blockReplication = in.readShort();
     short blockReplication = in.readShort();
     long modificationTime = in.readLong();
     long modificationTime = in.readLong();
     long preferredBlockSize = in.readLong();
     long preferredBlockSize = in.readLong();
+  
     int numBlocks = in.readInt();
     int numBlocks = in.readInt();
     BlockInfo[] blocks = new BlockInfo[numBlocks];
     BlockInfo[] blocks = new BlockInfo[numBlocks];
     Block blk = new Block();
     Block blk = new Block();
@@ -133,68 +160,141 @@ public class FSImageSerialization {
                                            throws IOException {
                                            throws IOException {
     writeString(path, out);
     writeString(path, out);
     out.writeLong(cons.getId());
     out.writeLong(cons.getId());
-    out.writeShort(cons.getBlockReplication());
+    out.writeShort(cons.getFileReplication());
     out.writeLong(cons.getModificationTime());
     out.writeLong(cons.getModificationTime());
     out.writeLong(cons.getPreferredBlockSize());
     out.writeLong(cons.getPreferredBlockSize());
-    int nrBlocks = cons.getBlocks().length;
-    out.writeInt(nrBlocks);
-    for (int i = 0; i < nrBlocks; i++) {
-      cons.getBlocks()[i].write(out);
-    }
+
+    writeBlocks(cons.getBlocks(), out);
     cons.getPermissionStatus().write(out);
     cons.getPermissionStatus().write(out);
+
     writeString(cons.getClientName(), out);
     writeString(cons.getClientName(), out);
     writeString(cons.getClientMachine(), out);
     writeString(cons.getClientMachine(), out);
 
 
     out.writeInt(0); //  do not store locations of last block
     out.writeInt(0); //  do not store locations of last block
   }
   }
 
 
-  /*
-   * Save one inode's attributes to the image.
+  /**
+   * Serialize a {@link INodeFile} node
+   * @param node The node to write
+   * @param out The {@link DataOutputStream} where the fields are written
+   * @param writeBlock Whether to write block information
    */
    */
-  static void saveINode2Image(INode node,
-                              DataOutputStream out) throws IOException {
-    byte[] name = node.getLocalNameBytes();
-    out.writeShort(name.length);
-    out.write(name);
+  public static void writeINodeFile(INodeFile file, DataOutput out,
+      boolean writeUnderConstruction) throws IOException {
+    writeLocalName(file, out);
+    out.writeLong(file.getId());
+    out.writeShort(file.getFileReplication());
+    out.writeLong(file.getModificationTime());
+    out.writeLong(file.getAccessTime());
+    out.writeLong(file.getPreferredBlockSize());
+
+    writeBlocks(file.getBlocks(), out);
+    SnapshotFSImageFormat.saveFileDiffList(file, out);
+
+    if (writeUnderConstruction) {
+      if (file instanceof INodeFileUnderConstruction) {
+        out.writeBoolean(true);
+        final INodeFileUnderConstruction uc = (INodeFileUnderConstruction)file;
+        writeString(uc.getClientName(), out);
+        writeString(uc.getClientMachine(), out);
+      } else {
+        out.writeBoolean(false);
+      }
+    }
+
+    writePermissionStatus(file, out);
+  }
+
+  /**
+   * Serialize a {@link INodeDirectory}
+   * @param node The node to write
+   * @param out The {@link DataOutput} where the fields are written 
+   */
+  public static void writeINodeDirectory(INodeDirectory node, DataOutput out)
+      throws IOException {
+    writeLocalName(node, out);
     out.writeLong(node.getId());
     out.writeLong(node.getId());
-    FsPermission filePerm = TL_DATA.get().FILE_PERM;
-    if (node.isDirectory()) {
-      out.writeShort(0);  // replication
-      out.writeLong(node.getModificationTime());
-      out.writeLong(0);   // access time
-      out.writeLong(0);   // preferred block size
-      out.writeInt(-1);   // # of blocks
-      out.writeLong(node.getNsQuota());
-      out.writeLong(node.getDsQuota());
-      filePerm.fromShort(node.getFsPermissionShort());
-      PermissionStatus.write(out, node.getUserName(),
-                             node.getGroupName(),
-                             filePerm);
-    } else if (node.isSymlink()) {
-      out.writeShort(0);  // replication
-      out.writeLong(0);   // modification time
-      out.writeLong(0);   // access time
-      out.writeLong(0);   // preferred block size
-      out.writeInt(-2);   // # of blocks
-      Text.writeString(out, ((INodeSymlink)node).getSymlinkString());
-      filePerm.fromShort(node.getFsPermissionShort());
-      PermissionStatus.write(out, node.getUserName(),
-                             node.getGroupName(),
-                             filePerm);      
+    out.writeShort(0);  // replication
+    out.writeLong(node.getModificationTime());
+    out.writeLong(0);   // access time
+    out.writeLong(0);   // preferred block size
+    out.writeInt(-1);   // # of blocks
+
+    out.writeLong(node.getNsQuota());
+    out.writeLong(node.getDsQuota());
+    if (node instanceof INodeDirectorySnapshottable) {
+      out.writeBoolean(true);
     } else {
     } else {
-      INodeFile fileINode = (INodeFile)node;
-      out.writeShort(fileINode.getBlockReplication());
-      out.writeLong(fileINode.getModificationTime());
-      out.writeLong(fileINode.getAccessTime());
-      out.writeLong(fileINode.getPreferredBlockSize());
-      Block[] blocks = fileINode.getBlocks();
-      out.writeInt(blocks.length);
-      for (Block blk : blocks)
-        blk.write(out);
-      filePerm.fromShort(fileINode.getFsPermissionShort());
-      PermissionStatus.write(out, fileINode.getUserName(),
-                             fileINode.getGroupName(),
-                             filePerm);
+      out.writeBoolean(false);
+      out.writeBoolean(node instanceof INodeDirectoryWithSnapshot);
+    }
+    
+    writePermissionStatus(node, out);
+  }
+  
+  /**
+   * Serialize a {@link INodeSymlink} node
+   * @param node The node to write
+   * @param out The {@link DataOutput} where the fields are written
+   */
+  private static void writeINodeSymlink(INodeSymlink node, DataOutput out)
+      throws IOException {
+    writeLocalName(node, out);
+    out.writeLong(node.getId());
+    out.writeShort(0);  // replication
+    out.writeLong(0);   // modification time
+    out.writeLong(0);   // access time
+    out.writeLong(0);   // preferred block size
+    out.writeInt(-2);   // # of blocks
+
+    Text.writeString(out, node.getSymlinkString());
+    writePermissionStatus(node, out);
+  }
+  
+  /** Serialize a {@link INodeReference} node */
+  private static void writeINodeReference(INodeReference ref, DataOutput out,
+      boolean writeUnderConstruction, ReferenceMap referenceMap
+      ) throws IOException {
+    writeLocalName(ref, out);
+    out.writeLong(ref.getId());
+    out.writeShort(0);  // replication
+    out.writeLong(0);   // modification time
+    out.writeLong(0);   // access time
+    out.writeLong(0);   // preferred block size
+    out.writeInt(-3);   // # of blocks
+
+    final boolean isWithName = ref instanceof INodeReference.WithName;
+    out.writeBoolean(isWithName);
+    
+    if (!isWithName) {
+      Preconditions.checkState(ref instanceof INodeReference.DstReference);
+      // dst snapshot id
+      out.writeInt(((INodeReference.DstReference) ref).getDstSnapshotId());
+    } else {
+      out.writeInt(((INodeReference.WithName) ref).getLastSnapshotId());
+    }
+    
+    final INodeReference.WithCount withCount
+        = (INodeReference.WithCount)ref.getReferredINode();
+    referenceMap.writeINodeReferenceWithCount(withCount, out,
+        writeUnderConstruction);
+  }
+
+  /**
+   * Save one inode's attributes to the image.
+   */
+  public static void saveINode2Image(INode node, DataOutput out,
+      boolean writeUnderConstruction, ReferenceMap referenceMap)
+      throws IOException {
+    if (node.isReference()) {
+      writeINodeReference(node.asReference(), out, writeUnderConstruction,
+          referenceMap);
+    } else if (node.isDirectory()) {
+      writeINodeDirectory(node.asDirectory(), out);
+    } else if (node.isSymlink()) {
+      writeINodeSymlink(node.asSymlink(), out);      
+    } else if (node.isFile()) {
+      writeINodeFile(node.asFile(), out, writeUnderConstruction);
     }
     }
   }
   }
 
 
@@ -202,19 +302,19 @@ public class FSImageSerialization {
   // code is moved into this package. This method should not be called
   // code is moved into this package. This method should not be called
   // by other code.
   // by other code.
   @SuppressWarnings("deprecation")
   @SuppressWarnings("deprecation")
-  public static String readString(DataInputStream in) throws IOException {
+  public static String readString(DataInput in) throws IOException {
     DeprecatedUTF8 ustr = TL_DATA.get().U_STR;
     DeprecatedUTF8 ustr = TL_DATA.get().U_STR;
     ustr.readFields(in);
     ustr.readFields(in);
     return ustr.toStringChecked();
     return ustr.toStringChecked();
   }
   }
 
 
-  static String readString_EmptyAsNull(DataInputStream in) throws IOException {
+  static String readString_EmptyAsNull(DataInput in) throws IOException {
     final String s = readString(in);
     final String s = readString(in);
     return s.isEmpty()? null: s;
     return s.isEmpty()? null: s;
   }
   }
 
 
   @SuppressWarnings("deprecation")
   @SuppressWarnings("deprecation")
-  static void writeString(String str, DataOutputStream out) throws IOException {
+  public static void writeString(String str, DataOutput out) throws IOException {
     DeprecatedUTF8 ustr = TL_DATA.get().U_STR;
     DeprecatedUTF8 ustr = TL_DATA.get().U_STR;
     ustr.set(str);
     ustr.set(str);
     ustr.write(out);
     ustr.write(out);
@@ -222,7 +322,7 @@ public class FSImageSerialization {
 
 
   
   
   /** read the long value */
   /** read the long value */
-  static long readLong(DataInputStream in) throws IOException {
+  static long readLong(DataInput in) throws IOException {
     LongWritable ustr = TL_DATA.get().U_LONG;
     LongWritable ustr = TL_DATA.get().U_LONG;
     ustr.readFields(in);
     ustr.readFields(in);
     return ustr.get();
     return ustr.get();
@@ -236,7 +336,7 @@ public class FSImageSerialization {
   }
   }
 
 
   /** read short value */
   /** read short value */
-  static short readShort(DataInputStream in) throws IOException {
+  static short readShort(DataInput in) throws IOException {
     ShortWritable uShort = TL_DATA.get().U_SHORT;
     ShortWritable uShort = TL_DATA.get().U_SHORT;
     uShort.readFields(in);
     uShort.readFields(in);
     return uShort.get();
     return uShort.get();
@@ -251,7 +351,7 @@ public class FSImageSerialization {
   
   
   // Same comments apply for this method as for readString()
   // Same comments apply for this method as for readString()
   @SuppressWarnings("deprecation")
   @SuppressWarnings("deprecation")
-  public static byte[] readBytes(DataInputStream in) throws IOException {
+  public static byte[] readBytes(DataInput in) throws IOException {
     DeprecatedUTF8 ustr = TL_DATA.get().U_STR;
     DeprecatedUTF8 ustr = TL_DATA.get().U_STR;
     ustr.readFields(in);
     ustr.readFields(in);
     int len = ustr.getLength();
     int len = ustr.getLength();
@@ -269,7 +369,7 @@ public class FSImageSerialization {
    * @throws IOException
    * @throws IOException
    */
    */
   @SuppressWarnings("deprecation")
   @SuppressWarnings("deprecation")
-  public static byte[][] readPathComponents(DataInputStream in)
+  public static byte[][] readPathComponents(DataInput in)
       throws IOException {
       throws IOException {
     DeprecatedUTF8 ustr = TL_DATA.get().U_STR;
     DeprecatedUTF8 ustr = TL_DATA.get().U_STR;
     
     
@@ -277,7 +377,19 @@ public class FSImageSerialization {
     return DFSUtil.bytes2byteArray(ustr.getBytes(),
     return DFSUtil.bytes2byteArray(ustr.getBytes(),
       ustr.getLength(), (byte) Path.SEPARATOR_CHAR);
       ustr.getLength(), (byte) Path.SEPARATOR_CHAR);
   }
   }
+  
+  public static byte[] readLocalName(DataInput in) throws IOException {
+    byte[] createdNodeName = new byte[in.readShort()];
+    in.readFully(createdNodeName);
+    return createdNodeName;
+  }
 
 
+  private static void writeLocalName(INode inode, DataOutput out)
+      throws IOException {
+    final byte[] name = inode.getLocalNameBytes();
+    out.writeShort(name.length);
+    out.write(name);
+  }
 
 
   /**
   /**
    * Write an array of blocks as compactly as possible. This uses
    * Write an array of blocks as compactly as possible. This uses
@@ -302,7 +414,7 @@ public class FSImageSerialization {
   }
   }
   
   
   public static Block[] readCompactBlockArray(
   public static Block[] readCompactBlockArray(
-      DataInputStream in, int logVersion) throws IOException {
+      DataInput in, int logVersion) throws IOException {
     int num = WritableUtils.readVInt(in);
     int num = WritableUtils.readVInt(in);
     if (num < 0) {
     if (num < 0) {
       throw new IOException("Invalid block array length: " + num);
       throw new IOException("Invalid block array length: " + num);
@@ -320,4 +432,4 @@ public class FSImageSerialization {
     }
     }
     return ret;
     return ret;
   }
   }
-}
+}

+ 385 - 72
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -33,8 +33,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_DEF
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DEFAULT_AUDIT_LOGGER_NAME;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DEFAULT_AUDIT_LOGGER_NAME;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT;
@@ -75,6 +75,7 @@ import static org.apache.hadoop.util.Time.now;
 
 
 import java.io.BufferedWriter;
 import java.io.BufferedWriter;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayInputStream;
+import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.File;
@@ -147,6 +148,9 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
 import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
@@ -167,7 +171,6 @@ import org.apache.hadoop.hdfs.server.common.Storage.StorageDirType;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
-import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.INodesInPath;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer;
 import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer;
@@ -176,6 +179,11 @@ import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
 import org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer;
 import org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable.SnapshotDiffInfo;
+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.SnapshotManager;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -318,6 +326,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   /** The namespace tree. */
   /** The namespace tree. */
   FSDirectory dir;
   FSDirectory dir;
   private final BlockManager blockManager;
   private final BlockManager blockManager;
+  private final SnapshotManager snapshotManager;
   private final DatanodeStatistics datanodeStatistics;
   private final DatanodeStatistics datanodeStatistics;
 
 
   // Block pool ID used by this namenode
   // Block pool ID used by this namenode
@@ -607,6 +616,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
 
       this.dtSecretManager = createDelegationTokenSecretManager(conf);
       this.dtSecretManager = createDelegationTokenSecretManager(conf);
       this.dir = new FSDirectory(fsImage, this, conf);
       this.dir = new FSDirectory(fsImage, this, conf);
+      this.snapshotManager = new SnapshotManager(dir);
       this.safeMode = new SafeModeInfo(conf);
       this.safeMode = new SafeModeInfo(conf);
       this.auditLoggers = initAuditLoggers(conf);
       this.auditLoggers = initAuditLoggers(conf);
       this.isDefaultAuditLogger = auditLoggers.size() == 1 &&
       this.isDefaultAuditLogger = auditLoggers.size() == 1 &&
@@ -1366,21 +1376,25 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           doAccessTime = false;
           doAccessTime = false;
         }
         }
 
 
-        long now = now();
-        final INodeFile inode = INodeFile.valueOf(dir.getINode(src), src);
-        if (doAccessTime && isAccessTimeSupported()) {
+        final INodesInPath iip = dir.getLastINodeInPath(src);
+        final INodeFile inode = INodeFile.valueOf(iip.getLastINode(), src);
+        if (!iip.isSnapshot() //snapshots are readonly, so don't update atime.
+            && doAccessTime && isAccessTimeSupported()) {
+          final long now = now();
           if (now > inode.getAccessTime() + getAccessTimePrecision()) {
           if (now > inode.getAccessTime() + getAccessTimePrecision()) {
             // if we have to set access time but we only have the readlock, then
             // if we have to set access time but we only have the readlock, then
             // restart this entire operation with the writeLock.
             // restart this entire operation with the writeLock.
             if (isReadOp) {
             if (isReadOp) {
               continue;
               continue;
             }
             }
-            dir.setTimes(src, inode, -1, now, false);
+            dir.setTimes(src, inode, -1, now, false, iip.getLatestSnapshot());
           }
           }
         }
         }
-        return blockManager.createLocatedBlocks(inode.getBlocks(),
-            inode.computeFileSize(false), inode.isUnderConstruction(),
-            offset, length, needBlockToken);
+        final long fileSize = iip.getPathSnapshot() != null?
+            inode.computeFileSize(iip.getPathSnapshot())
+            : inode.computeFileSizeNotIncludingLastUcBlock();
+        return blockManager.createLocatedBlocks(inode.getBlocks(), fileSize,
+            inode.isUnderConstruction(), offset, length, needBlockToken);
       } finally {
       } finally {
         if (isReadOp) {
         if (isReadOp) {
           readUnlock();
           readUnlock();
@@ -1478,7 +1492,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     // replication and blocks sizes should be the same for ALL the blocks
     // replication and blocks sizes should be the same for ALL the blocks
 
 
     // check the target
     // check the target
-    final INodeFile trgInode = INodeFile.valueOf(dir.getINode(target), target);
+    final INodeFile trgInode = INodeFile.valueOf(dir.getINode4Write(target),
+        target);
     if(trgInode.isUnderConstruction()) {
     if(trgInode.isUnderConstruction()) {
       throw new HadoopIllegalArgumentException("concat: target file "
       throw new HadoopIllegalArgumentException("concat: target file "
           + target + " is under construction");
           + target + " is under construction");
@@ -1488,6 +1503,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       throw new HadoopIllegalArgumentException("concat: target file "
       throw new HadoopIllegalArgumentException("concat: target file "
           + target + " is empty");
           + target + " is empty");
     }
     }
+    if (trgInode instanceof INodeFileWithSnapshot) {
+      throw new HadoopIllegalArgumentException("concat: target file "
+          + target + " is in a snapshot");
+    }
 
 
     long blockSize = trgInode.getPreferredBlockSize();
     long blockSize = trgInode.getPreferredBlockSize();
 
 
@@ -1500,7 +1519,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
     }
 
 
     si.add(trgInode);
     si.add(trgInode);
-    short repl = trgInode.getBlockReplication();
+    final short repl = trgInode.getFileReplication();
 
 
     // now check the srcs
     // now check the srcs
     boolean endSrc = false; // final src file doesn't have to have full end block
     boolean endSrc = false; // final src file doesn't have to have full end block
@@ -1509,7 +1528,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if(i==srcs.length-1)
       if(i==srcs.length-1)
         endSrc=true;
         endSrc=true;
 
 
-      final INodeFile srcInode = INodeFile.valueOf(dir.getINode(src), src);
+      final INodeFile srcInode = INodeFile.valueOf(dir.getINode4Write(src), src);
       if(src.isEmpty() 
       if(src.isEmpty() 
           || srcInode.isUnderConstruction()
           || srcInode.isUnderConstruction()
           || srcInode.numBlocks() == 0) {
           || srcInode.numBlocks() == 0) {
@@ -1596,9 +1615,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if (isPermissionEnabled) {
       if (isPermissionEnabled) {
         checkPathAccess(pc, src, FsAction.WRITE);
         checkPathAccess(pc, src, FsAction.WRITE);
       }
       }
-      INode inode = dir.getINode(src);
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
+      final INode inode = iip.getLastINode();
       if (inode != null) {
       if (inode != null) {
-        dir.setTimes(src, inode, mtime, atime, true);
+        dir.setTimes(src, inode, mtime, atime, true, iip.getLatestSnapshot());
         resultingStat = getAuditFileInfo(src, false);
         resultingStat = getAuditFileInfo(src, false);
       } else {
       } else {
         throw new FileNotFoundException("File/Directory " + src + " does not exist.");
         throw new FileNotFoundException("File/Directory " + src + " does not exist.");
@@ -1711,11 +1731,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         checkPathAccess(pc, src, FsAction.WRITE);
         checkPathAccess(pc, src, FsAction.WRITE);
       }
       }
 
 
-      final short[] oldReplication = new short[1];
-      final Block[] blocks = dir.setReplication(src, replication, oldReplication);
+      final short[] blockRepls = new short[2]; // 0: old, 1: new
+      final Block[] blocks = dir.setReplication(src, replication, blockRepls);
       isFile = blocks != null;
       isFile = blocks != null;
       if (isFile) {
       if (isFile) {
-        blockManager.setReplication(oldReplication[0], replication, src, blocks);
+        blockManager.setReplication(blockRepls[0], blockRepls[1], src, blocks);
       }
       }
     } finally {
     } finally {
       writeUnlock();
       writeUnlock();
@@ -1864,16 +1884,18 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       ParentNotDirectoryException, IOException {
       ParentNotDirectoryException, IOException {
     assert hasWriteLock();
     assert hasWriteLock();
     // Verify that the destination does not exist as a directory already.
     // Verify that the destination does not exist as a directory already.
-    boolean pathExists = dir.exists(src);
-    if (pathExists && dir.isDir(src)) {
+    final INodesInPath iip = dir.getINodesInPath4Write(src);
+    final INode inode = iip.getLastINode();
+    if (inode != null && inode.isDirectory()) {
       throw new FileAlreadyExistsException("Cannot create file " + src
       throw new FileAlreadyExistsException("Cannot create file " + src
           + "; already exists as a directory.");
           + "; already exists as a directory.");
     }
     }
+    final INodeFile myFile = INodeFile.valueOf(inode, src, true);
 
 
     boolean overwrite = flag.contains(CreateFlag.OVERWRITE);
     boolean overwrite = flag.contains(CreateFlag.OVERWRITE);
     boolean append = flag.contains(CreateFlag.APPEND);
     boolean append = flag.contains(CreateFlag.APPEND);
     if (isPermissionEnabled) {
     if (isPermissionEnabled) {
-      if (append || (overwrite && pathExists)) {
+      if (append || (overwrite && myFile != null)) {
         checkPathAccess(pc, src, FsAction.WRITE);
         checkPathAccess(pc, src, FsAction.WRITE);
       } else {
       } else {
         checkAncestorAccess(pc, src, FsAction.WRITE);
         checkAncestorAccess(pc, src, FsAction.WRITE);
@@ -1887,7 +1909,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
     try {
       blockManager.verifyReplication(src, replication, clientMachine);
       blockManager.verifyReplication(src, replication, clientMachine);
       boolean create = flag.contains(CreateFlag.CREATE);
       boolean create = flag.contains(CreateFlag.CREATE);
-      final INode myFile = dir.getINode(src);
+      
       if (myFile == null) {
       if (myFile == null) {
         if (!create) {
         if (!create) {
           throw new FileNotFoundException("failed to overwrite or append to non-existent file "
           throw new FileNotFoundException("failed to overwrite or append to non-existent file "
@@ -1914,8 +1936,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
 
       if (append && myFile != null) {
       if (append && myFile != null) {
         final INodeFile f = INodeFile.valueOf(myFile, src); 
         final INodeFile f = INodeFile.valueOf(myFile, src); 
-        return prepareFileForWrite(
-            src, f, holder, clientMachine, clientNode, true);
+        return prepareFileForWrite(src, f, holder, clientMachine, clientNode,
+            true, iip.getLatestSnapshot());
       } else {
       } else {
        // Now we can add the name to the filesystem. This file has no
        // Now we can add the name to the filesystem. This file has no
        // blocks associated with it.
        // blocks associated with it.
@@ -1962,19 +1984,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
    */
   LocatedBlock prepareFileForWrite(String src, INodeFile file,
   LocatedBlock prepareFileForWrite(String src, INodeFile file,
       String leaseHolder, String clientMachine, DatanodeDescriptor clientNode,
       String leaseHolder, String clientMachine, DatanodeDescriptor clientNode,
-      boolean writeToEditLog) throws IOException {
-    INodeFileUnderConstruction cons = new INodeFileUnderConstruction(
-                                    file.getId(),
-                                    file.getLocalNameBytes(),
-                                    file.getBlockReplication(),
-                                    file.getModificationTime(),
-                                    file.getPreferredBlockSize(),
-                                    file.getBlocks(),
-                                    file.getPermissionStatus(),
-                                    leaseHolder,
-                                    clientMachine,
-                                    clientNode);
-    dir.replaceNode(src, file, cons);
+      boolean writeToEditLog, Snapshot latestSnapshot) throws IOException {
+    file = file.recordModification(latestSnapshot, dir.getINodeMap());
+    final INodeFileUnderConstruction cons = file.toUnderConstruction(
+        leaseHolder, clientMachine, clientNode);
+
+    dir.replaceINodeFile(src, file, cons);
     leaseManager.addLease(cons.getClientName(), src);
     leaseManager.addLease(cons.getClientName(), src);
     
     
     LocatedBlock ret = blockManager.convertLastBlockToUnderConstruction(cons);
     LocatedBlock ret = blockManager.convertLastBlockToUnderConstruction(cons);
@@ -2036,7 +2051,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return false;
     return false;
   }
   }
 
 
-  private void recoverLeaseInternal(INode fileInode, 
+  private void recoverLeaseInternal(INodeFile fileInode, 
       String src, String holder, String clientMachine, boolean force)
       String src, String holder, String clientMachine, boolean force)
       throws IOException {
       throws IOException {
     assert hasWriteLock();
     assert hasWriteLock();
@@ -2248,7 +2263,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       }
       }
       blockSize = pendingFile.getPreferredBlockSize();
       blockSize = pendingFile.getPreferredBlockSize();
       clientNode = pendingFile.getClientNode();
       clientNode = pendingFile.getClientNode();
-      replication = pendingFile.getBlockReplication();
+      replication = pendingFile.getFileReplication();
     } finally {
     } finally {
       readUnlock();
       readUnlock();
     }
     }
@@ -2288,7 +2303,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       saveAllocatedBlock(src, inodesInPath, newBlock, targets);
       saveAllocatedBlock(src, inodesInPath, newBlock, targets);
 
 
       dir.persistBlocks(src, pendingFile);
       dir.persistBlocks(src, pendingFile);
-      offset = pendingFile.computeFileSize(true);
+      offset = pendingFile.computeFileSize();
     } finally {
     } finally {
       writeUnlock();
       writeUnlock();
     }
     }
@@ -2319,10 +2334,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     checkFsObjectLimit();
     checkFsObjectLimit();
 
 
     Block previousBlock = ExtendedBlock.getLocalBlock(previous);
     Block previousBlock = ExtendedBlock.getLocalBlock(previous);
-    final INodesInPath inodesInPath = dir.rootDir.getExistingPathINodes(src, true);
-    final INode[] inodes = inodesInPath.getINodes();
+    final INodesInPath iip = dir.getINodesInPath4Write(src);
     final INodeFileUnderConstruction pendingFile
     final INodeFileUnderConstruction pendingFile
-        = checkLease(src, fileId, clientName, inodes[inodes.length - 1]);
+        = checkLease(src, fileId, clientName, iip.getLastINode());
     BlockInfo lastBlockInFile = pendingFile.getLastBlock();
     BlockInfo lastBlockInFile = pendingFile.getLastBlock();
     if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
     if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
       // The block that the client claims is the current last block
       // The block that the client claims is the current last block
@@ -2376,11 +2390,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         NameNode.stateChangeLog.info("BLOCK* allocateBlock: " +
         NameNode.stateChangeLog.info("BLOCK* allocateBlock: " +
             "caught retry for allocation of a new block in " +
             "caught retry for allocation of a new block in " +
             src + ". Returning previously allocated block " + lastBlockInFile);
             src + ". Returning previously allocated block " + lastBlockInFile);
-        long offset = pendingFile.computeFileSize(true);
+        long offset = pendingFile.computeFileSize();
         onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
         onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
             ((BlockInfoUnderConstruction)lastBlockInFile).getExpectedLocations(),
             ((BlockInfoUnderConstruction)lastBlockInFile).getExpectedLocations(),
             offset);
             offset);
-        return inodesInPath;
+        return iip;
       } else {
       } else {
         // Case 3
         // Case 3
         throw new IOException("Cannot allocate block in " + src + ": " +
         throw new IOException("Cannot allocate block in " + src + ": " +
@@ -2393,7 +2407,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     if (!checkFileProgress(pendingFile, false)) {
     if (!checkFileProgress(pendingFile, false)) {
       throw new NotReplicatedYetException("Not replicated yet: " + src);
       throw new NotReplicatedYetException("Not replicated yet: " + src);
     }
     }
-    return inodesInPath;
+    return iip;
   }
   }
 
 
   LocatedBlock makeLocatedBlock(Block blk,
   LocatedBlock makeLocatedBlock(Block blk,
@@ -2497,26 +2511,26 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return true;
     return true;
   }
   }
   
   
-  // make sure that we still have the lease on this file.
+  /** make sure that we still have the lease on this file. */
   private INodeFileUnderConstruction checkLease(String src, String holder)
   private INodeFileUnderConstruction checkLease(String src, String holder)
       throws LeaseExpiredException, UnresolvedLinkException,
       throws LeaseExpiredException, UnresolvedLinkException,
       FileNotFoundException {
       FileNotFoundException {
-    assert hasReadOrWriteLock();
     return checkLease(src, INodeId.GRANDFATHER_INODE_ID, holder,
     return checkLease(src, INodeId.GRANDFATHER_INODE_ID, holder,
         dir.getINode(src));
         dir.getINode(src));
   }
   }
   
   
   private INodeFileUnderConstruction checkLease(String src, long fileId,
   private INodeFileUnderConstruction checkLease(String src, long fileId,
-      String holder, INode file) throws LeaseExpiredException,
+      String holder, INode inode) throws LeaseExpiredException,
       FileNotFoundException {
       FileNotFoundException {
     assert hasReadOrWriteLock();
     assert hasReadOrWriteLock();
-    if (file == null || !(file instanceof INodeFile)) {
+    if (inode == null || !inode.isFile()) {
       Lease lease = leaseManager.getLease(holder);
       Lease lease = leaseManager.getLease(holder);
       throw new LeaseExpiredException(
       throw new LeaseExpiredException(
           "No lease on " + src + ": File does not exist. "
           "No lease on " + src + ": File does not exist. "
           + (lease != null ? lease.toString()
           + (lease != null ? lease.toString()
               : "Holder " + holder + " does not have any open files."));
               : "Holder " + holder + " does not have any open files."));
     }
     }
+    final INodeFile file = inode.asFile();
     if (!file.isUnderConstruction()) {
     if (!file.isUnderConstruction()) {
       Lease lease = leaseManager.getLease(holder);
       Lease lease = leaseManager.getLease(holder);
       throw new LeaseExpiredException(
       throw new LeaseExpiredException(
@@ -2571,19 +2585,23 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       String holder, Block last) throws SafeModeException,
       String holder, Block last) throws SafeModeException,
       UnresolvedLinkException, IOException {
       UnresolvedLinkException, IOException {
     assert hasWriteLock();
     assert hasWriteLock();
-    INodeFileUnderConstruction pendingFile;
+    final INodesInPath iip = dir.getLastINodeInPath(src);
+    final INodeFileUnderConstruction pendingFile;
     try {
     try {
-      pendingFile = checkLease(src, holder);
+      pendingFile = checkLease(src, INodeId.GRANDFATHER_INODE_ID,
+          holder, iip.getINode(0)); 
     } catch (LeaseExpiredException lee) {
     } catch (LeaseExpiredException lee) {
       final INode inode = dir.getINode(src);
       final INode inode = dir.getINode(src);
-      if (inode != null && inode instanceof INodeFile && !inode.isUnderConstruction()) {
+      if (inode != null
+          && inode.isFile()
+          && !inode.asFile().isUnderConstruction()) {
         // This could be a retry RPC - i.e the client tried to close
         // This could be a retry RPC - i.e the client tried to close
         // the file, but missed the RPC response. Thus, it is trying
         // the file, but missed the RPC response. Thus, it is trying
         // again to close the file. If the file still exists and
         // again to close the file. If the file still exists and
         // the client's view of the last block matches the actual
         // the client's view of the last block matches the actual
         // last block, then we'll treat it as a successful close.
         // last block, then we'll treat it as a successful close.
         // See HDFS-3031.
         // See HDFS-3031.
-        final Block realLastBlock = ((INodeFile)inode).getLastBlock();
+        final Block realLastBlock = inode.asFile().getLastBlock();
         if (Block.matchingIdAndGenStamp(last, realLastBlock)) {
         if (Block.matchingIdAndGenStamp(last, realLastBlock)) {
           NameNode.stateChangeLog.info("DIR* completeFile: " +
           NameNode.stateChangeLog.info("DIR* completeFile: " +
               "request from " + holder + " to complete " + src +
               "request from " + holder + " to complete " + src +
@@ -2601,7 +2619,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       return false;
       return false;
     }
     }
 
 
-    finalizeINodeFileUnderConstruction(src, pendingFile);
+    finalizeINodeFileUnderConstruction(src, pendingFile,
+        iip.getLatestSnapshot());
     return true;
     return true;
   }
   }
 
 
@@ -2860,6 +2879,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       throws AccessControlException, SafeModeException, UnresolvedLinkException,
       throws AccessControlException, SafeModeException, UnresolvedLinkException,
              IOException {
              IOException {
     BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
     BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
+    List<INode> removedINodes = new ArrayList<INode>();
     FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
     checkOperation(OperationCategory.WRITE);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
@@ -2878,7 +2898,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
             FsAction.ALL, false);
             FsAction.ALL, false);
       }
       }
       // Unlink the target directory from directory tree
       // Unlink the target directory from directory tree
-      if (!dir.delete(src, collectedBlocks)) {
+      if (!dir.delete(src, collectedBlocks, removedINodes)) {
         return false;
         return false;
       }
       }
     } finally {
     } finally {
@@ -2887,6 +2907,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     getEditLog().logSync(); 
     getEditLog().logSync(); 
     removeBlocks(collectedBlocks); // Incremental deletion of blocks
     removeBlocks(collectedBlocks); // Incremental deletion of blocks
     collectedBlocks.clear();
     collectedBlocks.clear();
+    dir.removeFromInodeMap(removedINodes);
+    removedINodes.clear();
     if (NameNode.stateChangeLog.isDebugEnabled()) {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* Namesystem.delete: "
       NameNode.stateChangeLog.debug("DIR* Namesystem.delete: "
         + src +" is removed");
         + src +" is removed");
@@ -2903,7 +2925,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    *          An instance of {@link BlocksMapUpdateInfo} which contains a list
    *          An instance of {@link BlocksMapUpdateInfo} which contains a list
    *          of blocks that need to be removed from blocksMap
    *          of blocks that need to be removed from blocksMap
    */
    */
-  private void removeBlocks(BlocksMapUpdateInfo blocks) {
+  void removeBlocks(BlocksMapUpdateInfo blocks) {
     int start = 0;
     int start = 0;
     int end = 0;
     int end = 0;
     List<Block> toDeleteList = blocks.getToDeleteList();
     List<Block> toDeleteList = blocks.getToDeleteList();
@@ -2923,13 +2945,21 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
   }
   
   
   /**
   /**
-   * Remove leases and blocks related to a given path
+   * Remove leases, inodes and blocks related to a given path
    * @param src The given path
    * @param src The given path
    * @param blocks Containing the list of blocks to be deleted from blocksMap
    * @param blocks Containing the list of blocks to be deleted from blocksMap
+   * @param removedINodes Containing the list of inodes to be removed from 
+   *                      inodesMap
    */
    */
-  void removePathAndBlocks(String src, BlocksMapUpdateInfo blocks) {
+  void removePathAndBlocks(String src, BlocksMapUpdateInfo blocks,
+      List<INode> removedINodes) {
     assert hasWriteLock();
     assert hasWriteLock();
     leaseManager.removeLeaseWithPrefixPath(src);
     leaseManager.removeLeaseWithPrefixPath(src);
+    // remove inodes from inodesMap
+    if (removedINodes != null) {
+      dir.removeFromInodeMap(removedINodes);
+      removedINodes.clear();
+    }
     if (blocks == null) {
     if (blocks == null) {
       return;
       return;
     }
     }
@@ -3103,7 +3133,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     if (isPermissionEnabled) {
     if (isPermissionEnabled) {
       checkTraverse(pc, src);
       checkTraverse(pc, src);
     }
     }
-    if (dir.isDir(src)) {
+    if (dir.isDirMutable(src)) {
       // all the users of mkdirs() are used to expect 'true' even if
       // all the users of mkdirs() are used to expect 'true' even if
       // a new directory is not created.
       // a new directory is not created.
       return true;
       return true;
@@ -3167,7 +3197,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
     }
     getEditLog().logSync();
     getEditLog().logSync();
   }
   }
-  
+
   /** Persist all metadata about this file.
   /** Persist all metadata about this file.
    * @param src The string representation of the path
    * @param src The string representation of the path
    * @param clientName The string representation of the client
    * @param clientName The string representation of the client
@@ -3219,8 +3249,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     assert !isInSafeMode();
     assert !isInSafeMode();
     assert hasWriteLock();
     assert hasWriteLock();
 
 
+    final INodesInPath iip = dir.getLastINodeInPath(src);
     final INodeFileUnderConstruction pendingFile
     final INodeFileUnderConstruction pendingFile
-        = INodeFileUnderConstruction.valueOf(dir.getINode(src), src);
+        = INodeFileUnderConstruction.valueOf(iip.getINode(0), src);
     int nrBlocks = pendingFile.numBlocks();
     int nrBlocks = pendingFile.numBlocks();
     BlockInfo[] blocks = pendingFile.getBlocks();
     BlockInfo[] blocks = pendingFile.getBlocks();
 
 
@@ -3237,7 +3268,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     // If there are no incomplete blocks associated with this file,
     // If there are no incomplete blocks associated with this file,
     // then reap lease immediately and close the file.
     // then reap lease immediately and close the file.
     if(nrCompleteBlocks == nrBlocks) {
     if(nrCompleteBlocks == nrBlocks) {
-      finalizeINodeFileUnderConstruction(src, pendingFile);
+      finalizeINodeFileUnderConstruction(src, pendingFile,
+          iip.getLatestSnapshot());
       NameNode.stateChangeLog.warn("BLOCK*"
       NameNode.stateChangeLog.warn("BLOCK*"
         + " internalReleaseLease: All existing blocks are COMPLETE,"
         + " internalReleaseLease: All existing blocks are COMPLETE,"
         + " lease removed, file closed.");
         + " lease removed, file closed.");
@@ -3285,7 +3317,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       // Close file if committed blocks are minimally replicated
       // Close file if committed blocks are minimally replicated
       if(penultimateBlockMinReplication &&
       if(penultimateBlockMinReplication &&
           blockManager.checkMinReplication(lastBlock)) {
           blockManager.checkMinReplication(lastBlock)) {
-        finalizeINodeFileUnderConstruction(src, pendingFile);
+        finalizeINodeFileUnderConstruction(src, pendingFile,
+            iip.getLatestSnapshot());
         NameNode.stateChangeLog.warn("BLOCK*"
         NameNode.stateChangeLog.warn("BLOCK*"
           + " internalReleaseLease: Committed blocks are minimally replicated,"
           + " internalReleaseLease: Committed blocks are minimally replicated,"
           + " lease removed, file closed.");
           + " lease removed, file closed.");
@@ -3355,7 +3388,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     if (diff > 0) {
     if (diff > 0) {
       try {
       try {
         String path = leaseManager.findPath(fileINode);
         String path = leaseManager.findPath(fileINode);
-        dir.updateSpaceConsumed(path, 0, -diff * fileINode.getBlockReplication());
+        dir.updateSpaceConsumed(path, 0, -diff*fileINode.getFileReplication());
       } catch (IOException e) {
       } catch (IOException e) {
         LOG.warn("Unexpected exception while updating disk space.", e);
         LOG.warn("Unexpected exception while updating disk space.", e);
       }
       }
@@ -3363,15 +3396,18 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
   }
 
 
   private void finalizeINodeFileUnderConstruction(String src, 
   private void finalizeINodeFileUnderConstruction(String src, 
-      INodeFileUnderConstruction pendingFile) 
+      INodeFileUnderConstruction pendingFile, Snapshot latestSnapshot) 
       throws IOException, UnresolvedLinkException {
       throws IOException, UnresolvedLinkException {
     assert hasWriteLock();
     assert hasWriteLock();
     leaseManager.removeLease(pendingFile.getClientName(), src);
     leaseManager.removeLease(pendingFile.getClientName(), src);
+    
+    pendingFile = pendingFile.recordModification(latestSnapshot,
+        dir.getINodeMap());
 
 
     // The file is no longer pending.
     // The file is no longer pending.
     // Create permanent INode, update blocks
     // Create permanent INode, update blocks
-    INodeFile newFile = pendingFile.convertToInodeFile();
-    dir.replaceNode(src, pendingFile, newFile);
+    final INodeFile newFile = pendingFile.toINodeFile(now());
+    dir.replaceINodeFile(src, pendingFile, newFile);
 
 
     // close file and persist block allocations for this file
     // close file and persist block allocations for this file
     dir.closeFile(src, newFile);
     dir.closeFile(src, newFile);
@@ -3409,7 +3445,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if (storedBlock == null) {
       if (storedBlock == null) {
         throw new IOException("Block (=" + lastblock + ") not found");
         throw new IOException("Block (=" + lastblock + ") not found");
       }
       }
-      INodeFile iFile = (INodeFile) storedBlock.getBlockCollection();
+      INodeFile iFile = ((INode)storedBlock.getBlockCollection()).asFile();
       if (!iFile.isUnderConstruction() || storedBlock.isComplete()) {
       if (!iFile.isUnderConstruction() || storedBlock.isComplete()) {
         throw new IOException("Unexpected block (=" + lastblock
         throw new IOException("Unexpected block (=" + lastblock
                               + ") since the file (=" + iFile.getLocalName()
                               + ") since the file (=" + iFile.getLocalName()
@@ -3464,7 +3500,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         commitOrCompleteLastBlock(pendingFile, storedBlock);
         commitOrCompleteLastBlock(pendingFile, storedBlock);
 
 
         //remove lease, close file
         //remove lease, close file
-        finalizeINodeFileUnderConstruction(src, pendingFile);
+        finalizeINodeFileUnderConstruction(src, pendingFile,
+            Snapshot.findLatestSnapshot(pendingFile, null));
       } else {
       } else {
         // If this commit does not want to close the file, persist blocks
         // If this commit does not want to close the file, persist blocks
         dir.persistBlocks(src, pendingFile);
         dir.persistBlocks(src, pendingFile);
@@ -3807,6 +3844,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   public int getTotalLoad() {
   public int getTotalLoad() {
     return datanodeStatistics.getXceiverCount();
     return datanodeStatistics.getXceiverCount();
   }
   }
+  
+  @Metric({ "SnapshottableDirectories", "Number of snapshottable directories" })
+  public int getNumSnapshottableDirs() {
+    return this.snapshotManager.getNumSnapshottableDirs();
+  }
+
+  @Metric({ "Snapshots", "The number of snapshots" })
+  public int getNumSnapshots() {
+    return this.snapshotManager.getNumSnapshots();
+  }
 
 
   int getNumberOfDatanodes(DatanodeReportType type) {
   int getNumberOfDatanodes(DatanodeReportType type) {
     readLock();
     readLock();
@@ -4963,7 +5010,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
     }
     
     
     // check file inode
     // check file inode
-    INodeFile file = (INodeFile) storedBlock.getBlockCollection();
+    final INodeFile file = ((INode)storedBlock.getBlockCollection()).asFile();
     if (file==null || !file.isUnderConstruction()) {
     if (file==null || !file.isUnderConstruction()) {
       throw new IOException("The file " + storedBlock + 
       throw new IOException("The file " + storedBlock + 
           " belonged to does not exist or it is not under construction.");
           " belonged to does not exist or it is not under construction.");
@@ -5243,7 +5290,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
 
       while (blkIterator.hasNext()) {
       while (blkIterator.hasNext()) {
         Block blk = blkIterator.next();
         Block blk = blkIterator.next();
-        INode inode = (INodeFile) blockManager.getBlockCollection(blk);
+        final INode inode = (INode)blockManager.getBlockCollection(blk);
         skip++;
         skip++;
         if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) {
         if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) {
           String src = FSDirectory.getFullPathName(inode);
           String src = FSDirectory.getFullPathName(inode);
@@ -5419,7 +5466,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   /**
   /**
    * @param in load the state of secret manager from input stream
    * @param in load the state of secret manager from input stream
    */
    */
-  void loadSecretManagerState(DataInputStream in) throws IOException {
+  void loadSecretManagerState(DataInput in) throws IOException {
     dtSecretManager.loadSecretManagerState(in);
     dtSecretManager.loadSecretManagerState(in);
   }
   }
 
 
@@ -5776,6 +5823,272 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return this.blockManager.getDatanodeManager()
     return this.blockManager.getDatanodeManager()
         .shouldAvoidStaleDataNodesForWrite();
         .shouldAvoidStaleDataNodesForWrite();
   }
   }
+  
+  public SnapshotManager getSnapshotManager() {
+    return snapshotManager;
+  }
+  
+  /** Allow snapshot on a directroy. */
+  void allowSnapshot(String path) throws SafeModeException, IOException {
+    writeLock();
+    try {
+      checkOperation(OperationCategory.WRITE);
+      if (isInSafeMode()) {
+        throw new SafeModeException("Cannot allow snapshot for " + path,
+            safeMode);
+      }
+      checkSuperuserPrivilege();
+
+      dir.writeLock();
+      try {
+        snapshotManager.setSnapshottable(path, true);
+      } finally {
+        dir.writeUnlock();
+      }
+      getEditLog().logAllowSnapshot(path);
+    } finally {
+      writeUnlock();
+    }
+    getEditLog().logSync();
+
+    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      logAuditEvent(true, "allowSnapshot", path, null, null);
+    }
+  }
+  
+  /** Disallow snapshot on a directory. */
+  void disallowSnapshot(String path) throws SafeModeException, IOException {
+    writeLock();
+    try {
+      checkOperation(OperationCategory.WRITE);
+      if (isInSafeMode()) {
+        throw new SafeModeException("Cannot disallow snapshot for " + path,
+            safeMode);
+      }
+      checkSuperuserPrivilege();
+
+      dir.writeLock();
+      try {
+        snapshotManager.resetSnapshottable(path);
+      } finally {
+        dir.writeUnlock();
+      }
+      getEditLog().logDisallowSnapshot(path);
+    } finally {
+      writeUnlock();
+    }
+    getEditLog().logSync();
+    
+    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      logAuditEvent(true, "disallowSnapshot", path, null, null);
+    }
+  }
+  
+  /**
+   * Create a snapshot
+   * @param snapshotRoot The directory path where the snapshot is taken
+   * @param snapshotName The name of the snapshot
+   */
+  String createSnapshot(String snapshotRoot, String snapshotName)
+      throws SafeModeException, IOException {
+    final FSPermissionChecker pc = getPermissionChecker();
+    writeLock();
+    final String snapshotPath;
+    try {
+      checkOperation(OperationCategory.WRITE);
+      if (isInSafeMode()) {
+        throw new SafeModeException("Cannot create snapshot for "
+            + snapshotRoot, safeMode);
+      }
+      if (isPermissionEnabled) {
+        checkOwner(pc, snapshotRoot);
+      }
+
+      if (snapshotName == null || snapshotName.isEmpty()) {
+        snapshotName = Snapshot.generateDefaultSnapshotName();
+      }
+      dir.verifySnapshotName(snapshotName, snapshotRoot);
+      dir.writeLock();
+      try {
+        snapshotPath = snapshotManager.createSnapshot(snapshotRoot, snapshotName);
+      } finally {
+        dir.writeUnlock();
+      }
+      getEditLog().logCreateSnapshot(snapshotRoot, snapshotName);
+    } finally {
+      writeUnlock();
+    }
+    getEditLog().logSync();
+    
+    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      logAuditEvent(true, "createSnapshot", snapshotRoot, snapshotPath, null);
+    }
+    return snapshotPath;
+  }
+  
+  /**
+   * Rename a snapshot
+   * @param path The directory path where the snapshot was taken
+   * @param snapshotOldName Old snapshot name
+   * @param snapshotNewName New snapshot name
+   * @throws SafeModeException
+   * @throws IOException 
+   */
+  void renameSnapshot(String path, String snapshotOldName,
+      String snapshotNewName) throws SafeModeException, IOException {
+    final FSPermissionChecker pc = getPermissionChecker();
+    writeLock();
+    try {
+      checkOperation(OperationCategory.WRITE);
+      if (isInSafeMode()) {
+        throw new SafeModeException("Cannot rename snapshot for " + path,
+            safeMode);
+      }
+      if (isPermissionEnabled) {
+        checkOwner(pc, path);
+      }
+      dir.verifySnapshotName(snapshotNewName, path);
+      
+      snapshotManager.renameSnapshot(path, snapshotOldName, snapshotNewName);
+      getEditLog().logRenameSnapshot(path, snapshotOldName, snapshotNewName);
+    } finally {
+      writeUnlock();
+    }
+    getEditLog().logSync();
+    
+    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      String oldSnapshotRoot = Snapshot.getSnapshotPath(path, snapshotOldName);
+      String newSnapshotRoot = Snapshot.getSnapshotPath(path, snapshotNewName);
+      logAuditEvent(true, "renameSnapshot", oldSnapshotRoot, newSnapshotRoot, null);
+    }
+  }
+  
+  /**
+   * Get the list of snapshottable directories that are owned 
+   * by the current user. Return all the snapshottable directories if the 
+   * current user is a super user.
+   * @return The list of all the current snapshottable directories
+   * @throws IOException
+   */
+  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+      throws IOException {
+    SnapshottableDirectoryStatus[] status = null;
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      FSPermissionChecker checker = new FSPermissionChecker(
+          fsOwner.getShortUserName(), supergroup);
+      final String user = checker.isSuperUser()? null : checker.getUser();
+      status = snapshotManager.getSnapshottableDirListing(user);
+    } finally {
+      readUnlock();
+    }
+    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      logAuditEvent(true, "listSnapshottableDirectory", null, null, null);
+    }
+    return status;
+  }
+  
+  /**
+   * Get the difference between two snapshots (or between a snapshot and the
+   * current status) of a snapshottable directory.
+   * 
+   * @param path The full path of the snapshottable directory.
+   * @param fromSnapshot Name of the snapshot to calculate the diff from. Null
+   *          or empty string indicates the current tree.
+   * @param toSnapshot Name of the snapshot to calculated the diff to. Null or
+   *          empty string indicates the current tree.
+   * @return A report about the difference between {@code fromSnapshot} and 
+   *         {@code toSnapshot}. Modified/deleted/created/renamed files and 
+   *         directories belonging to the snapshottable directories are listed 
+   *         and labeled as M/-/+/R respectively. 
+   * @throws IOException
+   */
+  SnapshotDiffReport getSnapshotDiffReport(String path,
+      String fromSnapshot, String toSnapshot) throws IOException {
+    SnapshotDiffInfo diffs = null;
+    final FSPermissionChecker pc = getPermissionChecker();
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      if (isPermissionEnabled) {
+        checkSubtreeReadPermission(pc, path, fromSnapshot);
+        checkSubtreeReadPermission(pc, path, toSnapshot);
+      }
+      diffs = snapshotManager.diff(path, fromSnapshot, toSnapshot);
+    } finally {
+      readUnlock();
+    }
+    
+    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      logAuditEvent(true, "computeSnapshotDiff", null, null, null);
+    }
+    return diffs != null ? diffs.generateReport() : new SnapshotDiffReport(
+        path, fromSnapshot, toSnapshot,
+        Collections.<DiffReportEntry> emptyList());
+  }
+  
+  private void checkSubtreeReadPermission(final FSPermissionChecker pc,
+      final String snapshottablePath, final String snapshot)
+          throws AccessControlException, UnresolvedLinkException {
+    final String fromPath = snapshot == null?
+        snapshottablePath: Snapshot.getSnapshotPath(snapshottablePath, snapshot);
+    checkPermission(pc, fromPath, false, null, null, FsAction.READ, FsAction.READ);
+  }
+  
+  /**
+   * Delete a snapshot of a snapshottable directory
+   * @param snapshotRoot The snapshottable directory
+   * @param snapshotName The name of the to-be-deleted snapshot
+   * @throws SafeModeException
+   * @throws IOException
+   */
+  void deleteSnapshot(String snapshotRoot, String snapshotName)
+      throws SafeModeException, IOException {
+    final FSPermissionChecker pc = getPermissionChecker();
+    writeLock();
+    try {
+      checkOperation(OperationCategory.WRITE);
+      if (isInSafeMode()) {
+        throw new SafeModeException(
+            "Cannot delete snapshot for " + snapshotRoot, safeMode);
+      }
+      checkOwner(pc, snapshotRoot);
+
+      BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
+      List<INode> removedINodes = new ArrayList<INode>();
+      dir.writeLock();
+      try {
+        snapshotManager.deleteSnapshot(snapshotRoot, snapshotName,
+            collectedBlocks, removedINodes);
+        dir.removeFromInodeMap(removedINodes);
+      } finally {
+        dir.writeUnlock();
+      }
+      removedINodes.clear();
+      this.removeBlocks(collectedBlocks);
+      collectedBlocks.clear();
+      getEditLog().logDeleteSnapshot(snapshotRoot, snapshotName);
+    } finally {
+      writeUnlock();
+    }
+    getEditLog().logSync();
+    
+    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      String rootPath = Snapshot.getSnapshotPath(snapshotRoot, snapshotName);
+      logAuditEvent(true, "deleteSnapshot", rootPath, null, null);
+    }
+  }
+
+  /**
+   * Remove a list of INodeDirectorySnapshottable from the SnapshotManager
+   * @param toRemove the list of INodeDirectorySnapshottable to be removed
+   */
+  void removeSnapshottableDirs(List<INodeDirectorySnapshottable> toRemove) {
+    if (snapshotManager != null) {
+      snapshotManager.removeSnapshottable(toRemove);
+    }
+  }
 
 
   /**
   /**
    * Default AuditLogger implementation; used when no access logger is
    * Default AuditLogger implementation; used when no access logger is

+ 61 - 47
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java

@@ -29,6 +29,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 
 
@@ -41,6 +42,15 @@ import org.apache.hadoop.security.UserGroupInformation;
  */
  */
 class FSPermissionChecker {
 class FSPermissionChecker {
   static final Log LOG = LogFactory.getLog(UserGroupInformation.class);
   static final Log LOG = LogFactory.getLog(UserGroupInformation.class);
+
+  /** @return a string for throwing {@link AccessControlException} */
+  private static String toAccessControlString(INode inode) {
+    return "\"" + inode.getFullPathName() + "\":"
+          + inode.getUserName() + ":" + inode.getGroupName()
+          + ":" + (inode.isDirectory()? "d": "-") + inode.getFsPermission();
+  }
+
+
   private final UserGroupInformation ugi;
   private final UserGroupInformation ugi;
   private final String user;  
   private final String user;  
   /** A set with group namess. Not synchronized since it is unmodifiable */
   /** A set with group namess. Not synchronized since it is unmodifiable */
@@ -134,111 +144,115 @@ class FSPermissionChecker {
           + ", resolveLink=" + resolveLink);
           + ", resolveLink=" + resolveLink);
     }
     }
     // check if (parentAccess != null) && file exists, then check sb
     // check if (parentAccess != null) && file exists, then check sb
-    // If resolveLink, the check is performed on the link target.
-    final INode[] inodes = root.getExistingPathINodes(path, resolveLink).getINodes();
-    int ancestorIndex = inodes.length - 2;
-    for(; ancestorIndex >= 0 && inodes[ancestorIndex] == null;
-        ancestorIndex--);
-    checkTraverse(inodes, ancestorIndex);
-
-    if (parentAccess != null && parentAccess.implies(FsAction.WRITE)
-        && inodes.length > 1 && inodes[inodes.length - 1] != null) {
-      checkStickyBit(inodes[inodes.length - 2], inodes[inodes.length - 1]);
-    }
-    if (ancestorAccess != null && inodes.length > 1) {
-      check(inodes, ancestorIndex, ancestorAccess);
-    }
-    if (parentAccess != null && inodes.length > 1) {
-      check(inodes, inodes.length - 2, parentAccess);
-    }
-    if (access != null) {
-      check(inodes[inodes.length - 1], access);
-    }
-    if (subAccess != null) {
-      checkSubAccess(inodes[inodes.length - 1], subAccess);
-    }
-    if (doCheckOwner) {
-      checkOwner(inodes[inodes.length - 1]);
-    }
+      // Resolve symlinks, the check is performed on the link target.
+      final INodesInPath inodesInPath = root.getINodesInPath(path, true); 
+      final Snapshot snapshot = inodesInPath.getPathSnapshot();
+      final INode[] inodes = inodesInPath.getINodes();
+      int ancestorIndex = inodes.length - 2;
+      for(; ancestorIndex >= 0 && inodes[ancestorIndex] == null;
+          ancestorIndex--);
+      checkTraverse(inodes, ancestorIndex, snapshot);
+
+      final INode last = inodes[inodes.length - 1];
+      if (parentAccess != null && parentAccess.implies(FsAction.WRITE)
+          && inodes.length > 1 && last != null) {
+        checkStickyBit(inodes[inodes.length - 2], last, snapshot);
+      }
+      if (ancestorAccess != null && inodes.length > 1) {
+        check(inodes, ancestorIndex, snapshot, ancestorAccess);
+      }
+      if (parentAccess != null && inodes.length > 1) {
+        check(inodes, inodes.length - 2, snapshot, parentAccess);
+      }
+      if (access != null) {
+        check(last, snapshot, access);
+      }
+      if (subAccess != null) {
+        checkSubAccess(last, snapshot, subAccess);
+      }
+      if (doCheckOwner) {
+        checkOwner(last, snapshot);
+      }
   }
   }
 
 
   /** Guarded by {@link FSNamesystem#readLock()} */
   /** Guarded by {@link FSNamesystem#readLock()} */
-  private void checkOwner(INode inode) throws AccessControlException {
-    if (inode != null && user.equals(inode.getUserName())) {
+  private void checkOwner(INode inode, Snapshot snapshot
+      ) throws AccessControlException {
+    if (inode != null && user.equals(inode.getUserName(snapshot))) {
       return;
       return;
     }
     }
     throw new AccessControlException("Permission denied");
     throw new AccessControlException("Permission denied");
   }
   }
 
 
   /** Guarded by {@link FSNamesystem#readLock()} */
   /** Guarded by {@link FSNamesystem#readLock()} */
-  private void checkTraverse(INode[] inodes, int last
+  private void checkTraverse(INode[] inodes, int last, Snapshot snapshot
       ) throws AccessControlException {
       ) throws AccessControlException {
     for(int j = 0; j <= last; j++) {
     for(int j = 0; j <= last; j++) {
-      check(inodes[j], FsAction.EXECUTE);
+      check(inodes[j], snapshot, FsAction.EXECUTE);
     }
     }
   }
   }
 
 
   /** Guarded by {@link FSNamesystem#readLock()} */
   /** Guarded by {@link FSNamesystem#readLock()} */
-  private void checkSubAccess(INode inode, FsAction access
+  private void checkSubAccess(INode inode, Snapshot snapshot, FsAction access
       ) throws AccessControlException {
       ) throws AccessControlException {
     if (inode == null || !inode.isDirectory()) {
     if (inode == null || !inode.isDirectory()) {
       return;
       return;
     }
     }
 
 
     Stack<INodeDirectory> directories = new Stack<INodeDirectory>();
     Stack<INodeDirectory> directories = new Stack<INodeDirectory>();
-    for(directories.push((INodeDirectory)inode); !directories.isEmpty(); ) {
+    for(directories.push(inode.asDirectory()); !directories.isEmpty(); ) {
       INodeDirectory d = directories.pop();
       INodeDirectory d = directories.pop();
-      check(d, access);
+      check(d, snapshot, access);
 
 
-      for(INode child : d.getChildrenList()) {
+      for(INode child : d.getChildrenList(snapshot)) {
         if (child.isDirectory()) {
         if (child.isDirectory()) {
-          directories.push((INodeDirectory)child);
+          directories.push(child.asDirectory());
         }
         }
       }
       }
     }
     }
   }
   }
 
 
   /** Guarded by {@link FSNamesystem#readLock()} */
   /** Guarded by {@link FSNamesystem#readLock()} */
-  private void check(INode[] inodes, int i, FsAction access
+  private void check(INode[] inodes, int i, Snapshot snapshot, FsAction access
       ) throws AccessControlException {
       ) throws AccessControlException {
-    check(i >= 0? inodes[i]: null, access);
+    check(i >= 0? inodes[i]: null, snapshot, access);
   }
   }
 
 
   /** Guarded by {@link FSNamesystem#readLock()} */
   /** Guarded by {@link FSNamesystem#readLock()} */
-  private void check(INode inode, FsAction access
+  private void check(INode inode, Snapshot snapshot, FsAction access
       ) throws AccessControlException {
       ) throws AccessControlException {
     if (inode == null) {
     if (inode == null) {
       return;
       return;
     }
     }
-    FsPermission mode = inode.getFsPermission();
+    FsPermission mode = inode.getFsPermission(snapshot);
 
 
-    if (user.equals(inode.getUserName())) { //user class
+    if (user.equals(inode.getUserName(snapshot))) { //user class
       if (mode.getUserAction().implies(access)) { return; }
       if (mode.getUserAction().implies(access)) { return; }
     }
     }
-    else if (groups.contains(inode.getGroupName())) { //group class
+    else if (groups.contains(inode.getGroupName(snapshot))) { //group class
       if (mode.getGroupAction().implies(access)) { return; }
       if (mode.getGroupAction().implies(access)) { return; }
     }
     }
     else { //other class
     else { //other class
       if (mode.getOtherAction().implies(access)) { return; }
       if (mode.getOtherAction().implies(access)) { return; }
     }
     }
     throw new AccessControlException("Permission denied: user=" + user
     throw new AccessControlException("Permission denied: user=" + user
-        + ", access=" + access + ", inode=" + inode);
+        + ", access=" + access + ", inode=" + toAccessControlString(inode));
   }
   }
 
 
   /** Guarded by {@link FSNamesystem#readLock()} */
   /** Guarded by {@link FSNamesystem#readLock()} */
-  private void checkStickyBit(INode parent, INode inode)
-      throws AccessControlException {
-    if(!parent.getFsPermission().getStickyBit()) {
+  private void checkStickyBit(INode parent, INode inode, Snapshot snapshot
+      ) throws AccessControlException {
+    if(!parent.getFsPermission(snapshot).getStickyBit()) {
       return;
       return;
     }
     }
 
 
     // If this user is the directory owner, return
     // If this user is the directory owner, return
-    if(parent.getUserName().equals(user)) {
+    if(parent.getUserName(snapshot).equals(user)) {
       return;
       return;
     }
     }
 
 
     // if this user is the file owner, return
     // if this user is the file owner, return
-    if(inode.getUserName().equals(user)) {
+    if(inode.getUserName(snapshot).equals(user)) {
       return;
       return;
     }
     }
 
 

+ 469 - 318
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -17,12 +17,14 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
+import java.io.PrintStream;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.io.StringWriter;
 import java.util.ArrayList;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 import java.util.List;
 
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
@@ -30,12 +32,17 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.util.LightWeightGSet.LinkedElement;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
+import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+import org.apache.hadoop.hdfs.util.Diff;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.primitives.SignedBytes;
+import com.google.common.base.Preconditions;
 
 
 /**
 /**
  * We keep an in-memory representation of the file/block hierarchy.
  * We keep an in-memory representation of the file/block hierarchy.
@@ -43,173 +50,202 @@ import com.google.common.primitives.SignedBytes;
  * directory inodes.
  * directory inodes.
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
-abstract class INode implements Comparable<byte[]>, LinkedElement {
-  static final List<INode> EMPTY_LIST = Collections.unmodifiableList(new ArrayList<INode>());
-
-  /** Wrapper of two counters for namespace consumed and diskspace consumed. */
-  static class DirCounts {
-    /** namespace count */
-    long nsCount = 0;
-    /** diskspace count */
-    long dsCount = 0;
-    
-    /** returns namespace count */
-    long getNsCount() {
-      return nsCount;
-    }
-    /** returns diskspace count */
-    long getDsCount() {
-      return dsCount;
-    }
-  }
-  
-  private static enum PermissionStatusFormat {
-    MODE(0, 16),
-    GROUP(MODE.OFFSET + MODE.LENGTH, 25),
-    USER(GROUP.OFFSET + GROUP.LENGTH, 23);
-
-    final int OFFSET;
-    final int LENGTH; //bit length
-    final long MASK;
-
-    PermissionStatusFormat(int offset, int length) {
-      OFFSET = offset;
-      LENGTH = length;
-      MASK = ((-1L) >>> (64 - LENGTH)) << OFFSET;
-    }
+public abstract class INode implements Diff.Element<byte[]> {
+  public static final Log LOG = LogFactory.getLog(INode.class);
 
 
-    long retrieve(long record) {
-      return (record & MASK) >>> OFFSET;
-    }
+  /** parent is either an {@link INodeDirectory} or an {@link INodeReference}.*/
+  private INode parent = null;
 
 
-    long combine(long bits, long record) {
-      return (record & ~MASK) | (bits << OFFSET);
-    }
-
-    /** Encode the {@link PermissionStatus} to a long. */
-    static long toLong(PermissionStatus ps) {
-      long permission = 0L;
-      final int user = SerialNumberManager.INSTANCE.getUserSerialNumber(
-          ps.getUserName());
-      permission = USER.combine(user, permission);
-      final int group = SerialNumberManager.INSTANCE.getGroupSerialNumber(
-          ps.getGroupName());
-      permission = GROUP.combine(group, permission);
-      final int mode = ps.getPermission().toShort();
-      permission = MODE.combine(mode, permission);
-      return permission;
-    }
+  INode(INode parent) {
+    this.parent = parent;
   }
   }
 
 
+  /** Get inode id */
+  public abstract long getId();
+
   /**
   /**
-   * The inode id
+   * Check whether this is the root inode.
    */
    */
-  final private long id;
+  final boolean isRoot() {
+    return getLocalNameBytes().length == 0;
+  }
+
+  /** Get the {@link PermissionStatus} */
+  abstract PermissionStatus getPermissionStatus(Snapshot snapshot);
+
+  /** The same as getPermissionStatus(null). */
+  final PermissionStatus getPermissionStatus() {
+    return getPermissionStatus(null);
+  }
 
 
   /**
   /**
-   *  The inode name is in java UTF8 encoding; 
-   *  The name in HdfsFileStatus should keep the same encoding as this.
-   *  if this encoding is changed, implicitly getFileInfo and listStatus in
-   *  clientProtocol are changed; The decoding at the client
-   *  side should change accordingly.
-   */
-  private byte[] name = null;
-  /** 
-   * Permission encoded using {@link PermissionStatusFormat}.
-   * Codes other than {@link #clonePermissionStatus(INode)}
-   * and {@link #updatePermissionStatus(PermissionStatusFormat, long)}
-   * should not modify it.
+   * @param snapshot
+   *          if it is not null, get the result from the given snapshot;
+   *          otherwise, get the result from the current inode.
+   * @return user name
    */
    */
-  private long permission = 0L;
-  protected INodeDirectory parent = null;
-  protected long modificationTime = 0L;
-  protected long accessTime = 0L;
-  protected LinkedElement next = null;
-
-  private INode(long id, byte[] name, long permission, INodeDirectory parent,
-      long modificationTime, long accessTime) {
-    this.id = id;
-    this.name = name;
-    this.permission = permission;
-    this.parent = parent;
-    this.modificationTime = modificationTime;
-    this.accessTime = accessTime;
+  abstract String getUserName(Snapshot snapshot);
+
+  /** The same as getUserName(null). */
+  public final String getUserName() {
+    return getUserName(null);
   }
   }
 
 
-  INode(long id, byte[] name, PermissionStatus permissions,
-      INodeDirectory parent, long modificationTime, long accessTime) {
-    this(id, name, PermissionStatusFormat.toLong(permissions), parent,
-        modificationTime, accessTime);
+  /** Set user */
+  abstract void setUser(String user);
+
+  /** Set user */
+  final INode setUser(String user, Snapshot latest, INodeMap inodeMap)
+      throws QuotaExceededException {
+    final INode nodeToUpdate = recordModification(latest, inodeMap);
+    nodeToUpdate.setUser(user);
+    return nodeToUpdate;
   }
   }
-  
-  INode(long id, PermissionStatus permissions, long mtime, long atime) {
-    this(id, null, PermissionStatusFormat.toLong(permissions), null, mtime, atime);
+  /**
+   * @param snapshot
+   *          if it is not null, get the result from the given snapshot;
+   *          otherwise, get the result from the current inode.
+   * @return group name
+   */
+  abstract String getGroupName(Snapshot snapshot);
+
+  /** The same as getGroupName(null). */
+  public final String getGroupName() {
+    return getGroupName(null);
   }
   }
-  
-  protected INode(long id, String name, PermissionStatus permissions) {
-    this(id, DFSUtil.string2Bytes(name), permissions, null, 0L, 0L);
+
+  /** Set group */
+  abstract void setGroup(String group);
+
+  /** Set group */
+  final INode setGroup(String group, Snapshot latest, INodeMap inodeMap)
+      throws QuotaExceededException {
+    final INode nodeToUpdate = recordModification(latest, inodeMap);
+    nodeToUpdate.setGroup(group);
+    return nodeToUpdate;
   }
   }
+
+  /**
+   * @param snapshot
+   *          if it is not null, get the result from the given snapshot;
+   *          otherwise, get the result from the current inode.
+   * @return permission.
+   */
+  abstract FsPermission getFsPermission(Snapshot snapshot);
   
   
-  /** @param other Other node to be copied */
-  INode(INode other) {
-    this(other.getId(), other.getLocalNameBytes(), other.permission, other
-        .getParent(), other.getModificationTime(), other.getAccessTime());
+  /** The same as getFsPermission(null). */
+  public final FsPermission getFsPermission() {
+    return getFsPermission(null);
   }
   }
 
 
-  /** Get inode id */
-  public long getId() {
-    return this.id;
+  /** Set the {@link FsPermission} of this {@link INode} */
+  abstract void setPermission(FsPermission permission);
+
+  /** Set the {@link FsPermission} of this {@link INode} */
+  INode setPermission(FsPermission permission, Snapshot latest,
+      INodeMap inodeMap) throws QuotaExceededException {
+    final INode nodeToUpdate = recordModification(latest, inodeMap);
+    nodeToUpdate.setPermission(permission);
+    return nodeToUpdate;
   }
   }
-  
+
   /**
   /**
-   * Check whether this is the root inode.
+   * @return if the given snapshot is null, return this;
+   *     otherwise return the corresponding snapshot inode.
    */
    */
-  boolean isRoot() {
-    return name.length == 0;
+  public INode getSnapshotINode(final Snapshot snapshot) {
+    return this;
   }
   }
 
 
-  /** Clone the {@link PermissionStatus}. */
-  void clonePermissionStatus(INode that) {
-    this.permission = that.permission;
-  }
-  /** Get the {@link PermissionStatus} */
-  protected PermissionStatus getPermissionStatus() {
-    return new PermissionStatus(getUserName(),getGroupName(),getFsPermission());
-  }
-  private void updatePermissionStatus(PermissionStatusFormat f, long n) {
-    permission = f.combine(n, permission);
-  }
-  /** Get user name */
-  public String getUserName() {
-    int n = (int)PermissionStatusFormat.USER.retrieve(permission);
-    return SerialNumberManager.INSTANCE.getUser(n);
-  }
-  /** Set user */
-  protected void setUser(String user) {
-    int n = SerialNumberManager.INSTANCE.getUserSerialNumber(user);
-    updatePermissionStatus(PermissionStatusFormat.USER, n);
-  }
-  /** Get group name */
-  public String getGroupName() {
-    int n = (int)PermissionStatusFormat.GROUP.retrieve(permission);
-    return SerialNumberManager.INSTANCE.getGroup(n);
+  /** Is this inode in the latest snapshot? */
+  public final boolean isInLatestSnapshot(final Snapshot latest) {
+    if (latest == null) {
+      return false;
+    }
+    // if parent is a reference node, parent must be a renamed node. We can 
+    // stop the check at the reference node.
+    if (parent != null && parent.isReference()) {
+      return true;
+    }
+    final INodeDirectory parentDir = getParent();
+    if (parentDir == null) { // root
+      return true;
+    }
+    if (!parentDir.isInLatestSnapshot(latest)) {
+      return false;
+    }
+    final INode child = parentDir.getChild(getLocalNameBytes(), latest);
+    if (this == child) {
+      return true;
+    }
+    if (child == null || !(child.isReference())) {
+      return false;
+    }
+    return this == child.asReference().getReferredINode();
   }
   }
-  /** Set group */
-  protected void setGroup(String group) {
-    int n = SerialNumberManager.INSTANCE.getGroupSerialNumber(group);
-    updatePermissionStatus(PermissionStatusFormat.GROUP, n);
+  
+  /** @return true if the given inode is an ancestor directory of this inode. */
+  public final boolean isAncestorDirectory(final INodeDirectory dir) {
+    for(INodeDirectory p = getParent(); p != null; p = p.getParent()) {
+      if (p == dir) {
+        return true;
+      }
+    }
+    return false;
   }
   }
-  /** Get the {@link FsPermission} */
-  public FsPermission getFsPermission() {
-    return new FsPermission(
-        (short)PermissionStatusFormat.MODE.retrieve(permission));
+
+  /**
+   * When {@link #recordModification} is called on a referred node,
+   * this method tells which snapshot the modification should be
+   * associated with: the snapshot that belongs to the SRC tree of the rename
+   * operation, or the snapshot belonging to the DST tree.
+   * 
+   * @param latestInDst
+   *          the latest snapshot in the DST tree above the reference node
+   * @return True: the modification should be recorded in the snapshot that
+   *         belongs to the SRC tree. False: the modification should be
+   *         recorded in the snapshot that belongs to the DST tree.
+   */
+  public final boolean shouldRecordInSrcSnapshot(final Snapshot latestInDst) {
+    Preconditions.checkState(!isReference());
+
+    if (latestInDst == null) {
+      return true;
+    }
+    INodeReference withCount = getParentReference();
+    if (withCount != null) {
+      int dstSnapshotId = withCount.getParentReference().getDstSnapshotId();
+      if (dstSnapshotId >= latestInDst.getId()) {
+        return true;
+      }
+    }
+    return false;
   }
   }
-  protected short getFsPermissionShort() {
-    return (short)PermissionStatusFormat.MODE.retrieve(permission);
+
+  /**
+   * This inode is being modified.  The previous version of the inode needs to
+   * be recorded in the latest snapshot.
+   *
+   * @param latest the latest snapshot that has been taken.
+   *        Note that it is null if no snapshots have been taken.
+   * @param inodeMap while recording modification, the inode or its parent may 
+   *                 get replaced, and the inodeMap needs to be updated.
+   * @return The current inode, which usually is the same object of this inode.
+   *         However, in some cases, this inode may be replaced with a new inode
+   *         for maintaining snapshots. The current inode is then the new inode.
+   */
+  abstract INode recordModification(final Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException;
+
+  /** Check whether it's a reference. */
+  public boolean isReference() {
+    return false;
   }
   }
-  /** Set the {@link FsPermission} of this {@link INode} */
-  void setPermission(FsPermission permission) {
-    updatePermissionStatus(PermissionStatusFormat.MODE, permission.toShort());
+
+  /** Cast this inode to an {@link INodeReference}.  */
+  public INodeReference asReference() {
+    throw new IllegalStateException("Current inode is not a reference: "
+        + this.toDetailString());
   }
   }
 
 
   /**
   /**
@@ -219,6 +255,12 @@ abstract class INode implements Comparable<byte[]>, LinkedElement {
     return false;
     return false;
   }
   }
 
 
+  /** Cast this inode to an {@link INodeFile}.  */
+  public INodeFile asFile() {
+    throw new IllegalStateException("Current inode is not a file: "
+        + this.toDetailString());
+  }
+
   /**
   /**
    * Check whether it's a directory
    * Check whether it's a directory
    */
    */
@@ -226,164 +268,346 @@ abstract class INode implements Comparable<byte[]>, LinkedElement {
     return false;
     return false;
   }
   }
 
 
+  /** Cast this inode to an {@link INodeDirectory}.  */
+  public INodeDirectory asDirectory() {
+    throw new IllegalStateException("Current inode is not a directory: "
+        + this.toDetailString());
+  }
+
+  /**
+   * Check whether it's a symlink
+   */
+  public boolean isSymlink() {
+    return false;
+  }
+
+  /** Cast this inode to an {@link INodeSymlink}.  */
+  public INodeSymlink asSymlink() {
+    throw new IllegalStateException("Current inode is not a symlink: "
+        + this.toDetailString());
+  }
+
+  /**
+   * Clean the subtree under this inode and collect the blocks from the descents
+   * for further block deletion/update. The current inode can either resides in
+   * the current tree or be stored as a snapshot copy.
+   * 
+   * <pre>
+   * In general, we have the following rules. 
+   * 1. When deleting a file/directory in the current tree, we have different 
+   * actions according to the type of the node to delete. 
+   * 
+   * 1.1 The current inode (this) is an {@link INodeFile}. 
+   * 1.1.1 If {@code prior} is null, there is no snapshot taken on ancestors 
+   * before. Thus we simply destroy (i.e., to delete completely, no need to save 
+   * snapshot copy) the current INode and collect its blocks for further 
+   * cleansing.
+   * 1.1.2 Else do nothing since the current INode will be stored as a snapshot
+   * copy.
+   * 
+   * 1.2 The current inode is an {@link INodeDirectory}.
+   * 1.2.1 If {@code prior} is null, there is no snapshot taken on ancestors 
+   * before. Similarly, we destroy the whole subtree and collect blocks.
+   * 1.2.2 Else do nothing with the current INode. Recursively clean its 
+   * children.
+   * 
+   * 1.3 The current inode is a {@link FileWithSnapshot}.
+   * Call recordModification(..) to capture the current states.
+   * Mark the INode as deleted.
+   * 
+   * 1.4 The current inode is a {@link INodeDirectoryWithSnapshot}.
+   * Call recordModification(..) to capture the current states. 
+   * Destroy files/directories created after the latest snapshot 
+   * (i.e., the inodes stored in the created list of the latest snapshot).
+   * Recursively clean remaining children. 
+   *
+   * 2. When deleting a snapshot.
+   * 2.1 To clean {@link INodeFile}: do nothing.
+   * 2.2 To clean {@link INodeDirectory}: recursively clean its children.
+   * 2.3 To clean {@link FileWithSnapshot}: delete the corresponding snapshot in
+   * its diff list.
+   * 2.4 To clean {@link INodeDirectoryWithSnapshot}: delete the corresponding 
+   * snapshot in its diff list. Recursively clean its children.
+   * </pre>
+   * 
+   * @param snapshot
+   *          The snapshot to delete. Null means to delete the current
+   *          file/directory.
+   * @param prior
+   *          The latest snapshot before the to-be-deleted snapshot. When
+   *          deleting a current inode, this parameter captures the latest
+   *          snapshot.
+   * @param collectedBlocks
+   *          blocks collected from the descents for further block
+   *          deletion/update will be added to the given map.
+   * @param removedINodes
+   *          INodes collected from the descents for further cleaning up of 
+   *          inodeMap         
+   * @return quota usage delta when deleting a snapshot
+   */
+  public abstract Quota.Counts cleanSubtree(final Snapshot snapshot,
+      Snapshot prior, BlocksMapUpdateInfo collectedBlocks,
+      List<INode> removedINodes) throws QuotaExceededException;
+  
   /**
   /**
-   * Collect all the blocks in all children of this INode. Count and return the
-   * number of files in the sub tree. Also clears references since this INode is
-   * deleted.
+   * Destroy self and clear everything! If the INode is a file, this method
+   * collects its blocks for further block deletion. If the INode is a
+   * directory, the method goes down the subtree and collects blocks from the
+   * descents, and clears its parent/children references as well. The method
+   * also clears the diff list if the INode contains snapshot diff list.
    * 
    * 
-   * @param info
-   *          Containing all the blocks collected from the children of this
-   *          INode. These blocks later should be removed from the blocksMap.
+   * @param collectedBlocks
+   *          blocks collected from the descents for further block
+   *          deletion/update will be added to this map.
+   * @param removedINodes
+   *          INodes collected from the descents for further cleaning up of
+   *          inodeMap
    */
    */
-  abstract int collectSubtreeBlocksAndClear(BlocksMapUpdateInfo info);
+  public abstract void destroyAndCollectBlocks(
+      BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes);
 
 
   /** Compute {@link ContentSummary}. */
   /** Compute {@link ContentSummary}. */
   public final ContentSummary computeContentSummary() {
   public final ContentSummary computeContentSummary() {
-    long[] a = computeContentSummary(new long[]{0,0,0,0});
-    return new ContentSummary(a[0], a[1], a[2], getNsQuota(), 
-                              a[3], getDsQuota());
+    final Content.Counts counts = computeContentSummary(
+        Content.Counts.newInstance());
+    return new ContentSummary(counts.get(Content.LENGTH),
+        counts.get(Content.FILE) + counts.get(Content.SYMLINK),
+        counts.get(Content.DIRECTORY), getNsQuota(),
+        counts.get(Content.DISKSPACE), getDsQuota());
   }
   }
+
   /**
   /**
-   * @return an array of three longs. 
-   * 0: length, 1: file count, 2: directory count 3: disk space
+   * Count subtree content summary with a {@link Content.Counts}.
+   *
+   * @param counts The subtree counts for returning.
+   * @return The same objects as the counts parameter.
    */
    */
-  abstract long[] computeContentSummary(long[] summary);
+  public abstract Content.Counts computeContentSummary(Content.Counts counts);
   
   
+  /**
+   * Check and add namespace/diskspace consumed to itself and the ancestors.
+   * @throws QuotaExceededException if quote is violated.
+   */
+  public void addSpaceConsumed(long nsDelta, long dsDelta, boolean verify,
+      int snapshotId) throws QuotaExceededException {
+    if (parent != null) {
+      parent.addSpaceConsumed(nsDelta, dsDelta, verify, snapshotId);
+    }
+  }
+
+  public void addSpaceConsumedToRenameSrc(long nsDelta, long dsDelta,
+      boolean verify, int snapshotId) throws QuotaExceededException {
+    if (parent != null) {
+      parent.addSpaceConsumedToRenameSrc(nsDelta, dsDelta, verify, snapshotId);
+    }
+  }
+
   /**
   /**
    * Get the quota set for this inode
    * Get the quota set for this inode
    * @return the quota if it is set; -1 otherwise
    * @return the quota if it is set; -1 otherwise
    */
    */
-  long getNsQuota() {
+  public long getNsQuota() {
     return -1;
     return -1;
   }
   }
 
 
-  long getDsQuota() {
+  public long getDsQuota() {
     return -1;
     return -1;
   }
   }
   
   
-  boolean isQuotaSet() {
+  public final boolean isQuotaSet() {
     return getNsQuota() >= 0 || getDsQuota() >= 0;
     return getNsQuota() >= 0 || getDsQuota() >= 0;
   }
   }
   
   
   /**
   /**
-   * Adds total number of names and total disk space taken under 
-   * this tree to counts.
-   * Returns updated counts object.
+   * Count subtree {@link Quota#NAMESPACE} and {@link Quota#DISKSPACE} usages.
    */
    */
-  abstract DirCounts spaceConsumedInTree(DirCounts counts);
+  public final Quota.Counts computeQuotaUsage() {
+    return computeQuotaUsage(new Quota.Counts(), true);
+  }
+
+  /**
+   * Count subtree {@link Quota#NAMESPACE} and {@link Quota#DISKSPACE} usages.
+   * 
+   * With the existence of {@link INodeReference}, the same inode and its
+   * subtree may be referred by multiple {@link WithName} nodes and a
+   * {@link DstReference} node. To avoid circles while quota usage computation,
+   * we have the following rules:
+   * 
+   * <pre>
+   * 1. For a {@link DstReference} node, since the node must be in the current
+   * tree (or has been deleted as the end point of a series of rename 
+   * operations), we compute the quota usage of the referred node (and its 
+   * subtree) in the regular manner, i.e., including every inode in the current
+   * tree and in snapshot copies, as well as the size of diff list.
+   * 
+   * 2. For a {@link WithName} node, since the node must be in a snapshot, we 
+   * only count the quota usage for those nodes that still existed at the 
+   * creation time of the snapshot associated with the {@link WithName} node.
+   * We do not count in the size of the diff list.  
+   * <pre>
+   * 
+   * @param counts The subtree counts for returning.
+   * @param useCache Whether to use cached quota usage. Note that 
+   *                 {@link WithName} node never uses cache for its subtree.
+   * @param lastSnapshotId {@link Snapshot#INVALID_ID} indicates the computation
+   *                       is in the current tree. Otherwise the id indicates
+   *                       the computation range for a {@link WithName} node.
+   * @return The same objects as the counts parameter.
+   */
+  public abstract Quota.Counts computeQuotaUsage(Quota.Counts counts,
+      boolean useCache, int lastSnapshotId);
+
+  public final Quota.Counts computeQuotaUsage(Quota.Counts counts,
+      boolean useCache) {
+    return computeQuotaUsage(counts, useCache, Snapshot.INVALID_ID);
+  }
   
   
   /**
   /**
    * @return null if the local name is null; otherwise, return the local name.
    * @return null if the local name is null; otherwise, return the local name.
    */
    */
-  String getLocalName() {
+  public final String getLocalName() {
+    final byte[] name = getLocalNameBytes();
     return name == null? null: DFSUtil.bytes2String(name);
     return name == null? null: DFSUtil.bytes2String(name);
   }
   }
 
 
-
-  String getLocalParentDir() {
-    INode inode = isRoot() ? this : getParent();
-    String parentDir = "";
-    if (inode != null) {
-      parentDir = inode.getFullPathName();
-    }
-    return (parentDir != null) ? parentDir : "";
-  }
-
   /**
   /**
    * @return null if the local name is null;
    * @return null if the local name is null;
    *         otherwise, return the local name byte array.
    *         otherwise, return the local name byte array.
    */
    */
-  byte[] getLocalNameBytes() {
-    return name;
-  }
+  public abstract byte[] getLocalNameBytes();
 
 
-  /**
-   * Set local file name
-   */
-  void setLocalName(String name) {
-    this.name = DFSUtil.string2Bytes(name);
+  @Override
+  public final byte[] getKey() {
+    return getLocalNameBytes();
   }
   }
 
 
   /**
   /**
    * Set local file name
    * Set local file name
    */
    */
-  void setLocalName(byte[] name) {
-    this.name = name;
-  }
+  public abstract void setLocalName(byte[] name);
 
 
   public String getFullPathName() {
   public String getFullPathName() {
     // Get the full path name of this inode.
     // Get the full path name of this inode.
     return FSDirectory.getFullPathName(this);
     return FSDirectory.getFullPathName(this);
   }
   }
-
+  
   @Override
   @Override
   public String toString() {
   public String toString() {
-    return "\"" + getFullPathName() + "\":"
-    + getUserName() + ":" + getGroupName() + ":"
-    + (isDirectory()? "d": "-") + getFsPermission();
+    return getLocalName();
   }
   }
 
 
-  /**
-   * Get parent directory 
-   * @return parent INode
-   */
-  INodeDirectory getParent() {
-    return this.parent;
+  @VisibleForTesting
+  public final String getObjectString() {
+    return getClass().getSimpleName() + "@"
+        + Integer.toHexString(super.hashCode());
   }
   }
 
 
-  /** 
-   * Get last modification time of inode.
-   * @return access time
-   */
-  public long getModificationTime() {
-    return this.modificationTime;
+  /** @return a string description of the parent. */
+  @VisibleForTesting
+  public final String getParentString() {
+    final INodeReference parentRef = getParentReference();
+    if (parentRef != null) {
+      return "parentRef=" + parentRef.getLocalName() + "->";
+    } else {
+      final INodeDirectory parentDir = getParent();
+      if (parentDir != null) {
+        return "parentDir=" + parentDir.getLocalName() + "/";
+      } else {
+        return "parent=null";
+      }
+    }
+  }
+
+  @VisibleForTesting
+  public String toDetailString() {
+    return toString() + "(" + getObjectString() + "), " + getParentString();
+  }
+
+  /** @return the parent directory */
+  public final INodeDirectory getParent() {
+    return parent == null? null
+        : parent.isReference()? getParentReference().getParent(): parent.asDirectory();
   }
   }
 
 
   /**
   /**
-   * Set last modification time of inode.
+   * @return the parent as a reference if this is a referred inode;
+   *         otherwise, return null.
    */
    */
-  void setModificationTime(long modtime) {
-    assert isDirectory();
-    if (this.modificationTime <= modtime) {
-      this.modificationTime = modtime;
-    }
+  public INodeReference getParentReference() {
+    return parent == null || !parent.isReference()? null: (INodeReference)parent;
+  }
+
+  /** Set parent directory */
+  public final void setParent(INodeDirectory parent) {
+    this.parent = parent;
+  }
+
+  /** Set container. */
+  public final void setParentReference(INodeReference parent) {
+    this.parent = parent;
+  }
+
+  /** Clear references to other objects. */
+  public void clear() {
+    setParent(null);
   }
   }
 
 
   /**
   /**
-   * Always set the last modification time of inode.
+   * @param snapshot
+   *          if it is not null, get the result from the given snapshot;
+   *          otherwise, get the result from the current inode.
+   * @return modification time.
    */
    */
-  void setModificationTimeForce(long modtime) {
-    this.modificationTime = modtime;
+  abstract long getModificationTime(Snapshot snapshot);
+
+  /** The same as getModificationTime(null). */
+  public final long getModificationTime() {
+    return getModificationTime(null);
+  }
+
+  /** Update modification time if it is larger than the current value. */
+  public abstract INode updateModificationTime(long mtime, Snapshot latest,
+      INodeMap inodeMap) throws QuotaExceededException;
+
+  /** Set the last modification time of inode. */
+  public abstract void setModificationTime(long modificationTime);
+
+  /** Set the last modification time of inode. */
+  public final INode setModificationTime(long modificationTime,
+      Snapshot latest, INodeMap inodeMap) throws QuotaExceededException {
+    final INode nodeToUpdate = recordModification(latest, inodeMap);
+    nodeToUpdate.setModificationTime(modificationTime);
+    return nodeToUpdate;
   }
   }
 
 
   /**
   /**
-   * Get access time of inode.
+   * @param snapshot
+   *          if it is not null, get the result from the given snapshot;
+   *          otherwise, get the result from the current inode.
    * @return access time
    * @return access time
    */
    */
-  public long getAccessTime() {
-    return accessTime;
+  abstract long getAccessTime(Snapshot snapshot);
+
+  /** The same as getAccessTime(null). */
+  public final long getAccessTime() {
+    return getAccessTime(null);
   }
   }
 
 
   /**
   /**
    * Set last access time of inode.
    * Set last access time of inode.
    */
    */
-  void setAccessTime(long atime) {
-    accessTime = atime;
-  }
+  public abstract void setAccessTime(long accessTime);
 
 
   /**
   /**
-   * Is this inode being constructed?
+   * Set last access time of inode.
    */
    */
-  public boolean isUnderConstruction() {
-    return false;
+  public final INode setAccessTime(long accessTime, Snapshot latest,
+      INodeMap inodeMap) throws QuotaExceededException {
+    final INode nodeToUpdate = recordModification(latest, inodeMap);
+    nodeToUpdate.setAccessTime(accessTime);
+    return nodeToUpdate;
   }
   }
 
 
-  /**
-   * Check whether it's a symlink
-   */
-  public boolean isSymlink() {
-    return false;
-  }
 
 
   /**
   /**
    * Breaks file path into components.
    * Breaks file path into components.
@@ -419,41 +643,9 @@ abstract class INode implements Comparable<byte[]>, LinkedElement {
     return StringUtils.split(path, Path.SEPARATOR_CHAR);
     return StringUtils.split(path, Path.SEPARATOR_CHAR);
   }
   }
 
 
-  /**
-   * Given some components, create a path name.
-   * @param components The path components
-   * @param start index
-   * @param end index
-   * @return concatenated path
-   */
-  static String constructPath(byte[][] components, int start, int end) {
-    StringBuilder buf = new StringBuilder();
-    for (int i = start; i < end; i++) {
-      buf.append(DFSUtil.bytes2String(components[i]));
-      if (i < end - 1) {
-        buf.append(Path.SEPARATOR);
-      }
-    }
-    return buf.toString();
-  }
-
-  boolean removeNode() {
-    if (parent == null) {
-      return false;
-    } else {
-      parent.removeChild(this);
-      parent = null;
-      return true;
-    }
-  }
-
-  private static final byte[] EMPTY_BYTES = {};
-
   @Override
   @Override
   public final int compareTo(byte[] bytes) {
   public final int compareTo(byte[] bytes) {
-    final byte[] left = name == null? EMPTY_BYTES: name;
-    final byte[] right = bytes == null? EMPTY_BYTES: bytes;
-    return SignedBytes.lexicographicalComparator().compare(left, right);
+    return DFSUtil.compareBytes(getLocalNameBytes(), bytes);
   }
   }
 
 
   @Override
   @Override
@@ -464,78 +656,47 @@ abstract class INode implements Comparable<byte[]>, LinkedElement {
     if (that == null || !(that instanceof INode)) {
     if (that == null || !(that instanceof INode)) {
       return false;
       return false;
     }
     }
-    return id == ((INode) that).id;
+    return getId() == ((INode) that).getId();
   }
   }
 
 
   @Override
   @Override
   public final int hashCode() {
   public final int hashCode() {
+    long id = getId();
     return (int)(id^(id>>>32));  
     return (int)(id^(id>>>32));  
   }
   }
   
   
-  /**
-   * Create an INode; the inode's name is not set yet
-   * 
-   * @param id preassigned inode id
-   * @param permissions permissions
-   * @param blocks blocks if a file
-   * @param symlink symblic link if a symbolic link
-   * @param replication replication factor
-   * @param modificationTime modification time
-   * @param atime access time
-   * @param nsQuota namespace quota
-   * @param dsQuota disk quota
-   * @param preferredBlockSize block size
-   * @return an inode
-   */
-  static INode newINode(long id,
-                        PermissionStatus permissions,
-                        BlockInfo[] blocks,
-                        String symlink,
-                        short replication,
-                        long modificationTime,
-                        long atime,
-                        long nsQuota,
-                        long dsQuota,
-                        long preferredBlockSize) {
-    if (symlink.length() != 0) { // check if symbolic link
-      return new INodeSymlink(id, symlink, modificationTime, atime, permissions);
-    }  else if (blocks == null) { //not sym link and blocks null? directory!
-      if (nsQuota >= 0 || dsQuota >= 0) {
-        return new INodeDirectoryWithQuota(
-             id, permissions, modificationTime, nsQuota, dsQuota);
-      } 
-      // regular directory
-      return new INodeDirectory(id, permissions, modificationTime);
-    }
-    // file
-    return new INodeFile(id, permissions, blocks, replication,
-        modificationTime, atime, preferredBlockSize);
-  }
-
   /**
   /**
    * Dump the subtree starting from this inode.
    * Dump the subtree starting from this inode.
    * @return a text representation of the tree.
    * @return a text representation of the tree.
    */
    */
   @VisibleForTesting
   @VisibleForTesting
-  public StringBuffer dumpTreeRecursively() {
+  public final StringBuffer dumpTreeRecursively() {
     final StringWriter out = new StringWriter(); 
     final StringWriter out = new StringWriter(); 
-    dumpTreeRecursively(new PrintWriter(out, true), new StringBuilder());
+    dumpTreeRecursively(new PrintWriter(out, true), new StringBuilder(), null);
     return out.getBuffer();
     return out.getBuffer();
   }
   }
 
 
+  @VisibleForTesting
+  public final void dumpTreeRecursively(PrintStream out) {
+    dumpTreeRecursively(new PrintWriter(out, true), new StringBuilder(), null);
+  }
+
   /**
   /**
    * Dump tree recursively.
    * Dump tree recursively.
    * @param prefix The prefix string that each line should print.
    * @param prefix The prefix string that each line should print.
    */
    */
   @VisibleForTesting
   @VisibleForTesting
-  public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix) {
+  public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix,
+      Snapshot snapshot) {
     out.print(prefix);
     out.print(prefix);
     out.print(" ");
     out.print(" ");
-    out.print(getLocalName());
+    final String name = getLocalName();
+    out.print(name.isEmpty()? "/": name);
     out.print("   (");
     out.print("   (");
-    final String s = super.toString();
-    out.print(s.substring(s.lastIndexOf(getClass().getSimpleName())));
-    out.println(")");
+    out.print(getObjectString());
+    out.print("), ");
+    out.print(getParentString());
+    out.print(", " + getPermissionStatus(snapshot));
   }
   }
   
   
   /**
   /**
@@ -581,14 +742,4 @@ abstract class INode implements Comparable<byte[]>, LinkedElement {
       toDeleteList.clear();
       toDeleteList.clear();
     }
     }
   }
   }
-  
-  @Override
-  public void setNext(LinkedElement next) {
-    this.next = next;
-  }
-  
-  @Override
-  public LinkedElement getNext() {
-    return next;
-  }
 }
 }

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

@@ -21,20 +21,30 @@ import java.io.FileNotFoundException;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 import java.util.List;
 
 
 import org.apache.hadoop.fs.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileUnderConstructionWithSnapshot;
+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.SnapshotAccessControlException;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 
 
 /**
 /**
  * Directory INode class.
  * Directory INode class.
  */
  */
-class INodeDirectory extends INode {
+public class INodeDirectory extends INodeWithAdditionalFields {
   /** Cast INode to INodeDirectory. */
   /** Cast INode to INodeDirectory. */
   public static INodeDirectory valueOf(INode inode, Object path
   public static INodeDirectory valueOf(INode inode, Object path
       ) throws FileNotFoundException, PathIsNotDirectoryException {
       ) throws FileNotFoundException, PathIsNotDirectoryException {
@@ -45,219 +55,318 @@ class INodeDirectory extends INode {
     if (!inode.isDirectory()) {
     if (!inode.isDirectory()) {
       throw new PathIsNotDirectoryException(DFSUtil.path2String(path));
       throw new PathIsNotDirectoryException(DFSUtil.path2String(path));
     }
     }
-    return (INodeDirectory)inode; 
+    return inode.asDirectory(); 
   }
   }
 
 
   protected static final int DEFAULT_FILES_PER_DIRECTORY = 5;
   protected static final int DEFAULT_FILES_PER_DIRECTORY = 5;
-  final static String ROOT_NAME = "";
+  final static byte[] ROOT_NAME = DFSUtil.string2Bytes("");
 
 
   private List<INode> children = null;
   private List<INode> children = null;
 
 
-  INodeDirectory(long id, String name, PermissionStatus permissions) {
-    super(id, name, permissions);
-  }
-
-  public INodeDirectory(long id, PermissionStatus permissions, long mTime) {
-    super(id, permissions, mTime, 0);
-  }
-  
   /** constructor */
   /** constructor */
-  INodeDirectory(long id, byte[] name, PermissionStatus permissions, long mtime) {
-    super(id, name, permissions, null, mtime, 0L);
+  public INodeDirectory(long id, byte[] name, PermissionStatus permissions,
+      long mtime) {
+    super(id, name, permissions, mtime, 0L);
   }
   }
   
   
-  /** copy constructor
-   * 
-   * @param other
+  /**
+   * Copy constructor
+   * @param other The INodeDirectory to be copied
+   * @param adopt Indicate whether or not need to set the parent field of child
+   *              INodes to the new node
    */
    */
-  INodeDirectory(INodeDirectory other) {
+  public INodeDirectory(INodeDirectory other, boolean adopt) {
     super(other);
     super(other);
     this.children = other.children;
     this.children = other.children;
-    if (this.children != null) {
+    if (adopt && this.children != null) {
       for (INode child : children) {
       for (INode child : children) {
-        child.parent = this;
+        child.setParent(this);
       }
       }
     }
     }
   }
   }
-  
+
   /** @return true unconditionally. */
   /** @return true unconditionally. */
   @Override
   @Override
   public final boolean isDirectory() {
   public final boolean isDirectory() {
     return true;
     return true;
   }
   }
 
 
-  private void assertChildrenNonNull() {
-    if (children == null) {
-      throw new AssertionError("children is null: " + this);
-    }
+  /** @return this object. */
+  @Override
+  public final INodeDirectory asDirectory() {
+    return this;
   }
   }
 
 
-  private int searchChildren(INode inode) {
-    return Collections.binarySearch(children, inode.getLocalNameBytes());
+  /** Is this a snapshottable directory? */
+  public boolean isSnapshottable() {
+    return false;
   }
   }
 
 
-  INode removeChild(INode node) {
-    assertChildrenNonNull();
-    final int i = searchChildren(node);
-    return i >= 0? children.remove(i): null;
+  private int searchChildren(byte[] name) {
+    return children == null? -1: Collections.binarySearch(children, name);
   }
   }
 
 
-  /** Replace a child that has the same name as newChild by newChild.
+  /**
+   * Remove the specified child from this directory.
+   * 
+   * @param child the child inode to be removed
+   * @param latest See {@link INode#recordModification(Snapshot, INodeMap)}.
+   */
+  public boolean removeChild(INode child, Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
+    if (isInLatestSnapshot(latest)) {
+      return replaceSelf4INodeDirectoryWithSnapshot(inodeMap)
+          .removeChild(child, latest, inodeMap);
+    }
+
+    return removeChild(child);
+  }
+
+  /** 
+   * Remove the specified child from this directory.
+   * The basic remove method which actually calls children.remove(..).
+   *
+   * @param child the child inode to be removed
    * 
    * 
-   * @param newChild Child node to be added
+   * @return true if the child is removed; false if the child is not found.
    */
    */
-  void replaceChild(INode newChild) {
-    assertChildrenNonNull();
+  protected final boolean removeChild(final INode child) {
+    final int i = searchChildren(child.getLocalNameBytes());
+    if (i < 0) {
+      return false;
+    }
+
+    final INode removed = children.remove(i);
+    Preconditions.checkState(removed == child);
+    return true;
+  }
 
 
-    final int low = searchChildren(newChild);
-    if (low>=0) { // an old child exists so replace by the newChild
-      children.get(low).parent = null;
-      children.set(low, newChild);
+  /**
+   * Replace itself with {@link INodeDirectoryWithQuota} or
+   * {@link INodeDirectoryWithSnapshot} depending on the latest snapshot.
+   */
+  INodeDirectoryWithQuota replaceSelf4Quota(final Snapshot latest,
+      final long nsQuota, final long dsQuota, final INodeMap inodeMap)
+      throws QuotaExceededException {
+    Preconditions.checkState(!(this instanceof INodeDirectoryWithQuota),
+        "this is already an INodeDirectoryWithQuota, this=%s", this);
+
+    if (!this.isInLatestSnapshot(latest)) {
+      final INodeDirectoryWithQuota q = new INodeDirectoryWithQuota(
+          this, true, nsQuota, dsQuota);
+      replaceSelf(q, inodeMap);
+      return q;
     } else {
     } else {
-      throw new IllegalArgumentException("No child exists to be replaced");
+      final INodeDirectoryWithSnapshot s = new INodeDirectoryWithSnapshot(this);
+      s.setQuota(nsQuota, dsQuota);
+      return replaceSelf(s, inodeMap).saveSelf2Snapshot(latest, this);
     }
     }
   }
   }
-  
-  INode getChild(String name) {
-    return getChildINode(DFSUtil.string2Bytes(name));
+  /** Replace itself with an {@link INodeDirectorySnapshottable}. */
+  public INodeDirectorySnapshottable replaceSelf4INodeDirectorySnapshottable(
+      Snapshot latest, final INodeMap inodeMap) throws QuotaExceededException {
+    Preconditions.checkState(!(this instanceof INodeDirectorySnapshottable),
+        "this is already an INodeDirectorySnapshottable, this=%s", this);
+    final INodeDirectorySnapshottable s = new INodeDirectorySnapshottable(this);
+    replaceSelf(s, inodeMap).saveSelf2Snapshot(latest, this);
+    return s;
   }
   }
 
 
-  private INode getChildINode(byte[] name) {
-    if (children == null) {
-      return null;
+  /** Replace itself with an {@link INodeDirectoryWithSnapshot}. */
+  public INodeDirectoryWithSnapshot replaceSelf4INodeDirectoryWithSnapshot(
+      final INodeMap inodeMap) {
+    return replaceSelf(new INodeDirectoryWithSnapshot(this), inodeMap);
+  }
+
+  /** Replace itself with {@link INodeDirectory}. */
+  public INodeDirectory replaceSelf4INodeDirectory(final INodeMap inodeMap) {
+    Preconditions.checkState(getClass() != INodeDirectory.class,
+        "the class is already INodeDirectory, this=%s", this);
+    return replaceSelf(new INodeDirectory(this, true), inodeMap);
+  }
+
+  /** Replace itself with the given directory. */
+  private final <N extends INodeDirectory> N replaceSelf(final N newDir,
+      final INodeMap inodeMap) {
+    final INodeReference ref = getParentReference();
+    if (ref != null) {
+      ref.setReferredINode(newDir);
+      if (inodeMap != null) {
+        inodeMap.put(newDir);
+      }
+    } else {
+      final INodeDirectory parent = getParent();
+      Preconditions.checkArgument(parent != null, "parent is null, this=%s", this);
+      parent.replaceChild(this, newDir, inodeMap);
     }
     }
-    int low = Collections.binarySearch(children, name);
-    if (low >= 0) {
-      return children.get(low);
+    clear();
+    return newDir;
+  }
+
+  /** Replace the given child with a new child. */
+  public void replaceChild(INode oldChild, final INode newChild,
+      final INodeMap inodeMap) {
+    Preconditions.checkNotNull(children);
+    final int i = searchChildren(newChild.getLocalNameBytes());
+    Preconditions.checkState(i >= 0);
+    Preconditions.checkState(oldChild == children.get(i)
+        || oldChild == children.get(i).asReference().getReferredINode()
+            .asReference().getReferredINode());
+    oldChild = children.get(i);
+    
+    if (oldChild.isReference() && !newChild.isReference()) {
+      // replace the referred inode, e.g., 
+      // INodeFileWithSnapshot -> INodeFileUnderConstructionWithSnapshot
+      final INode withCount = oldChild.asReference().getReferredINode();
+      withCount.asReference().setReferredINode(newChild);
+    } else {
+      if (oldChild.isReference()) {
+        // both are reference nodes, e.g., DstReference -> WithName
+        final INodeReference.WithCount withCount = 
+            (WithCount) oldChild.asReference().getReferredINode();
+        withCount.removeReference(oldChild.asReference());
+      }
+      children.set(i, newChild);
+    }
+    // update the inodeMap
+    if (inodeMap != null) {
+      inodeMap.put(newChild);
+    }
+  }
+
+  INodeReference.WithName replaceChild4ReferenceWithName(INode oldChild,
+      Snapshot latest) {
+    Preconditions.checkArgument(latest != null);
+    if (oldChild instanceof INodeReference.WithName) {
+      return (INodeReference.WithName)oldChild;
+    }
+
+    final INodeReference.WithCount withCount;
+    if (oldChild.isReference()) {
+      Preconditions.checkState(oldChild instanceof INodeReference.DstReference);
+      withCount = (INodeReference.WithCount) oldChild.asReference()
+          .getReferredINode();
+    } else {
+      withCount = new INodeReference.WithCount(null, oldChild);
+    }
+    final INodeReference.WithName ref = new INodeReference.WithName(this,
+        withCount, oldChild.getLocalNameBytes(), latest.getId());
+    replaceChild(oldChild, ref, null);
+    return ref;
+  }
+  
+  private void replaceChildFile(final INodeFile oldChild,
+      final INodeFile newChild, final INodeMap inodeMap) {
+    replaceChild(oldChild, newChild, inodeMap);
+    oldChild.clear();
+    newChild.updateBlockCollection();
+  }
+
+  /** Replace a child {@link INodeFile} with an {@link INodeFileWithSnapshot}. */
+  INodeFileWithSnapshot replaceChild4INodeFileWithSnapshot(
+      final INodeFile child, final INodeMap inodeMap) {
+    Preconditions.checkArgument(!(child instanceof INodeFileWithSnapshot),
+        "Child file is already an INodeFileWithSnapshot, child=" + child);
+    final INodeFileWithSnapshot newChild = new INodeFileWithSnapshot(child);
+    replaceChildFile(child, newChild, inodeMap);
+    return newChild;
+  }
+
+  /** Replace a child {@link INodeFile} with an {@link INodeFileUnderConstructionWithSnapshot}. */
+  INodeFileUnderConstructionWithSnapshot replaceChild4INodeFileUcWithSnapshot(
+      final INodeFileUnderConstruction child, final INodeMap inodeMap) {
+    Preconditions.checkArgument(!(child instanceof INodeFileUnderConstructionWithSnapshot),
+        "Child file is already an INodeFileUnderConstructionWithSnapshot, child=" + child);
+    final INodeFileUnderConstructionWithSnapshot newChild
+        = new INodeFileUnderConstructionWithSnapshot(child, null);
+    replaceChildFile(child, newChild, inodeMap);
+    return newChild;
+  }
+
+  @Override
+  public INodeDirectory recordModification(Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
+    if (isInLatestSnapshot(latest)) {
+      return replaceSelf4INodeDirectoryWithSnapshot(inodeMap)
+          .recordModification(latest, inodeMap);
+    } else {
+      return this;
     }
     }
-    return null;
   }
   }
 
 
   /**
   /**
-   * @return the INode of the last component in components, or null if the last
-   * component does not exist.
+   * Save the child to the latest snapshot.
+   * 
+   * @return the child inode, which may be replaced.
    */
    */
-  private INode getNode(byte[][] components, boolean resolveLink
-      ) throws UnresolvedLinkException {
-    INodesInPath inodesInPath = getExistingPathINodes(components, 1,
-        resolveLink);
-    return inodesInPath.inodes[0];
+  public INode saveChild2Snapshot(final INode child, final Snapshot latest,
+      final INode snapshotCopy, final INodeMap inodeMap)
+      throws QuotaExceededException {
+    if (latest == null) {
+      return child;
+    }
+    return replaceSelf4INodeDirectoryWithSnapshot(inodeMap)
+        .saveChild2Snapshot(child, latest, snapshotCopy, inodeMap);
   }
   }
 
 
   /**
   /**
-   * This is the external interface
+   * @param name the name of the child
+   * @param snapshot
+   *          if it is not null, get the result from the given snapshot;
+   *          otherwise, get the result from the current directory.
+   * @return the child inode.
    */
    */
+  public INode getChild(byte[] name, Snapshot snapshot) {
+    final ReadOnlyList<INode> c = getChildrenList(snapshot);
+    final int i = ReadOnlyList.Util.binarySearch(c, name);
+    return i < 0? null: c.get(i);
+  }
+
+  /** @return the {@link INodesInPath} containing only the last inode. */
+  INodesInPath getLastINodeInPath(String path, boolean resolveLink
+      ) throws UnresolvedLinkException {
+    return INodesInPath.resolve(this, getPathComponents(path), 1, resolveLink);
+  }
+
+  /** @return the {@link INodesInPath} containing all inodes in the path. */
+  INodesInPath getINodesInPath(String path, boolean resolveLink
+      ) throws UnresolvedLinkException {
+    final byte[][] components = getPathComponents(path);
+    return INodesInPath.resolve(this, components, components.length, resolveLink);
+  }
+
+  /** @return the last inode in the path. */
   INode getNode(String path, boolean resolveLink) 
   INode getNode(String path, boolean resolveLink) 
     throws UnresolvedLinkException {
     throws UnresolvedLinkException {
-    return getNode(getPathComponents(path), resolveLink);
+    return getLastINodeInPath(path, resolveLink).getINode(0);
   }
   }
 
 
   /**
   /**
-   * Retrieve existing INodes from a path. If existing is big enough to store
-   * all path components (existing and non-existing), then existing INodes
-   * will be stored starting from the root INode into existing[0]; if
-   * existing is not big enough to store all path components, then only the
-   * last existing and non existing INodes will be stored so that
-   * existing[existing.length-1] refers to the INode of the final component.
-   * 
-   * An UnresolvedPathException is always thrown when an intermediate path 
-   * component refers to a symbolic link. If the final path component refers 
-   * to a symbolic link then an UnresolvedPathException is only thrown if
-   * resolveLink is true.  
-   * 
-   * <p>
-   * Example: <br>
-   * Given the path /c1/c2/c3 where only /c1/c2 exists, resulting in the
-   * following path components: ["","c1","c2","c3"],
-   * 
-   * <p>
-   * <code>getExistingPathINodes(["","c1","c2"], [?])</code> should fill the
-   * array with [c2] <br>
-   * <code>getExistingPathINodes(["","c1","c2","c3"], [?])</code> should fill the
-   * array with [null]
-   * 
-   * <p>
-   * <code>getExistingPathINodes(["","c1","c2"], [?,?])</code> should fill the
-   * array with [c1,c2] <br>
-   * <code>getExistingPathINodes(["","c1","c2","c3"], [?,?])</code> should fill
-   * the array with [c2,null]
-   * 
-   * <p>
-   * <code>getExistingPathINodes(["","c1","c2"], [?,?,?,?])</code> should fill
-   * the array with [rootINode,c1,c2,null], <br>
-   * <code>getExistingPathINodes(["","c1","c2","c3"], [?,?,?,?])</code> should
-   * fill the array with [rootINode,c1,c2,null]
-   * 
-   * @param components array of path component name
-   * @param numOfINodes number of INodes to return
-   * @param resolveLink indicates whether UnresolvedLinkException should
-   *        be thrown when the path refers to a symbolic link.
-   * @return the specified number of existing INodes in the path
+   * @return the INode of the last component in src, or null if the last
+   * component does not exist.
+   * @throws UnresolvedLinkException if symlink can't be resolved
+   * @throws SnapshotAccessControlException if path is in RO snapshot
    */
    */
-  INodesInPath getExistingPathINodes(byte[][] components, int numOfINodes,
-      boolean resolveLink)
-      throws UnresolvedLinkException {
-    assert this.compareTo(components[0]) == 0 :
-        "Incorrect name " + getLocalName() + " expected "
-        + (components[0] == null? null: DFSUtil.bytes2String(components[0]));
-
-    INodesInPath existing = new INodesInPath(numOfINodes);
-    INode curNode = this;
-    int count = 0;
-    int index = numOfINodes - components.length;
-    if (index > 0) {
-      index = 0;
-    }
-    while (count < components.length && curNode != null) {
-      final boolean lastComp = (count == components.length - 1);      
-      if (index >= 0) {
-        existing.inodes[index] = curNode;
-      }
-      if (curNode.isSymlink() && (!lastComp || (lastComp && resolveLink))) {
-        final String path = constructPath(components, 0, components.length);
-        final String preceding = constructPath(components, 0, count);
-        final String remainder =
-          constructPath(components, count + 1, components.length);
-        final String link = DFSUtil.bytes2String(components[count]);
-        final String target = ((INodeSymlink)curNode).getSymlinkString();
-        if (NameNode.stateChangeLog.isDebugEnabled()) {
-          NameNode.stateChangeLog.debug("UnresolvedPathException " +
-            " path: " + path + " preceding: " + preceding +
-            " count: " + count + " link: " + link + " target: " + target +
-            " remainder: " + remainder);
-        }
-        throw new UnresolvedPathException(path, preceding, remainder, target);
-      }
-      if (lastComp || !curNode.isDirectory()) {
-        break;
-      }
-      INodeDirectory parentDir = (INodeDirectory)curNode;
-      curNode = parentDir.getChildINode(components[count + 1]);
-      count++;
-      index++;
-    }
-    return existing;
+  INode getINode4Write(String src, boolean resolveLink)
+      throws UnresolvedLinkException, SnapshotAccessControlException {
+    return getINodesInPath4Write(src, resolveLink).getLastINode();
   }
   }
 
 
   /**
   /**
-   * Retrieve the existing INodes along the given path. The first INode
-   * always exist and is this INode.
-   * 
-   * @param path the path to explore
-   * @param resolveLink indicates whether UnresolvedLinkException should 
-   *        be thrown when the path refers to a symbolic link.
-   * @return INodes array containing the existing INodes in the order they
-   *         appear when following the path from the root INode to the
-   *         deepest INodes. The array size will be the number of expected
-   *         components in the path, and non existing components will be
-   *         filled with null
-   *         
-   * @see #getExistingPathINodes(byte[][], int, boolean)
+   * @return the INodesInPath of the components in src
+   * @throws UnresolvedLinkException if symlink can't be resolved
+   * @throws SnapshotAccessControlException if path is in RO snapshot
    */
    */
-  INodesInPath getExistingPathINodes(String path, boolean resolveLink) 
-    throws UnresolvedLinkException {
-    byte[][] components = getPathComponents(path);
-    return getExistingPathINodes(components, components.length, resolveLink);
+  INodesInPath getINodesInPath4Write(String src, boolean resolveLink)
+      throws UnresolvedLinkException, SnapshotAccessControlException {
+    final byte[][] components = INode.getPathComponents(src);
+    INodesInPath inodesInPath = INodesInPath.resolve(this, components,
+        components.length, resolveLink);
+    if (inodesInPath.isSnapshot()) {
+      throw new SnapshotAccessControlException(
+          "Modification on a read-only snapshot is disallowed");
+    }
+    return inodesInPath;
   }
   }
 
 
   /**
   /**
@@ -266,11 +375,11 @@ class INodeDirectory extends INode {
    * @param name a child's name
    * @param name a child's name
    * @return the index of the next child
    * @return the index of the next child
    */
    */
-  int nextChild(byte[] name) {
+  static int nextChild(ReadOnlyList<INode> children, byte[] name) {
     if (name.length == 0) { // empty name
     if (name.length == 0) { // empty name
       return 0;
       return 0;
     }
     }
-    int nextPos = Collections.binarySearch(children, name) + 1;
+    int nextPos = ReadOnlyList.Util.binarySearch(children, name) + 1;
     if (nextPos >= 0) {
     if (nextPos >= 0) {
       return nextPos;
       return nextPos;
     }
     }
@@ -284,148 +393,171 @@ class INodeDirectory extends INode {
    * @param setModTime set modification time for the parent node
    * @param setModTime set modification time for the parent node
    *                   not needed when replaying the addition and 
    *                   not needed when replaying the addition and 
    *                   the parent already has the proper mod time
    *                   the parent already has the proper mod time
+   * @param inodeMap update the inodeMap if the directory node gets replaced
    * @return false if the child with this name already exists; 
    * @return false if the child with this name already exists; 
    *         otherwise, return true;
    *         otherwise, return true;
    */
    */
-  boolean addChild(final INode node, final boolean setModTime) {
-    if (children == null) {
-      children = new ArrayList<INode>(DEFAULT_FILES_PER_DIRECTORY);
-    }
-    final int low = searchChildren(node);
+  public boolean addChild(INode node, final boolean setModTime,
+      final Snapshot latest, final INodeMap inodeMap)
+      throws QuotaExceededException {
+    final int low = searchChildren(node.getLocalNameBytes());
     if (low >= 0) {
     if (low >= 0) {
       return false;
       return false;
     }
     }
-    node.parent = this;
-    children.add(-low - 1, node);
-    // update modification time of the parent directory
-    if (setModTime)
-      setModificationTime(node.getModificationTime());
-    if (node.getGroupName() == null) {
-      node.setGroup(getGroupName());
+
+    if (isInLatestSnapshot(latest)) {
+      INodeDirectoryWithSnapshot sdir = 
+          replaceSelf4INodeDirectoryWithSnapshot(inodeMap);
+      boolean added = sdir.addChild(node, setModTime, latest, inodeMap);
+      return added;
+    }
+    addChild(node, low);
+    if (setModTime) {
+      // update modification time of the parent directory
+      updateModificationTime(node.getModificationTime(), latest, inodeMap);
     }
     }
     return true;
     return true;
   }
   }
 
 
-  /**
-   * Add new INode to the file tree.
-   * Find the parent and insert 
-   * 
-   * @param path file path
-   * @param newNode INode to be added
-   * @return false if the node already exists; otherwise, return true;
-   * @throws FileNotFoundException if parent does not exist or 
-   * @throws UnresolvedLinkException if any path component is a symbolic link
-   * is not a directory.
-   */
-  boolean addINode(String path, INode newNode
-      ) throws FileNotFoundException, PathIsNotDirectoryException,
-      UnresolvedLinkException {
-    byte[][] pathComponents = getPathComponents(path);        
-    if (pathComponents.length < 2) { // add root
+
+  /** The same as addChild(node, false, null, false) */
+  public boolean addChild(INode node) {
+    final int low = searchChildren(node.getLocalNameBytes());
+    if (low >= 0) {
       return false;
       return false;
     }
     }
-    newNode.setLocalName(pathComponents[pathComponents.length - 1]);
-    // insert into the parent children list
-    INodeDirectory parent = getParent(pathComponents);
-    return parent.addChild(newNode, true);
+    addChild(node, low);
+    return true;
   }
   }
 
 
-  INodeDirectory getParent(byte[][] pathComponents
-      ) throws FileNotFoundException, PathIsNotDirectoryException,
-      UnresolvedLinkException {
-    if (pathComponents.length < 2)  // add root
-      return null;
-    // Gets the parent INode
-    INodesInPath inodes =  getExistingPathINodes(pathComponents, 2, false);
-    return INodeDirectory.valueOf(inodes.inodes[0], pathComponents);
-  }
+  /**
+   * Add the node to the children list at the given insertion point.
+   * The basic add method which actually calls children.add(..).
+   */
+  private void addChild(final INode node, final int insertionPoint) {
+    if (children == null) {
+      children = new ArrayList<INode>(DEFAULT_FILES_PER_DIRECTORY);
+    }
+    node.setParent(this);
+    children.add(-insertionPoint - 1, node);
 
 
-  @Override
-  DirCounts spaceConsumedInTree(DirCounts counts) {
-    counts.nsCount += 1;
-    if (children != null) {
-      for (INode child : children) {
-        child.spaceConsumedInTree(counts);
-      }
+    if (node.getGroupName() == null) {
+      node.setGroup(getGroupName());
     }
     }
-    return counts;    
   }
   }
 
 
   @Override
   @Override
-  long[] computeContentSummary(long[] summary) {
-    // Walk through the children of this node, using a new summary array
-    // for the (sub)tree rooted at this node
-    assert 4 == summary.length;
-    long[] subtreeSummary = new long[]{0,0,0,0};
+  public Quota.Counts computeQuotaUsage(Quota.Counts counts, boolean useCache,
+      int lastSnapshotId) {
     if (children != null) {
     if (children != null) {
       for (INode child : children) {
       for (INode child : children) {
-        child.computeContentSummary(subtreeSummary);
-      }
-    }
-    if (this instanceof INodeDirectoryWithQuota) {
-      // Warn if the cached and computed diskspace values differ
-      INodeDirectoryWithQuota node = (INodeDirectoryWithQuota)this;
-      long space = node.diskspaceConsumed();
-      assert -1 == node.getDsQuota() || space == subtreeSummary[3];
-      if (-1 != node.getDsQuota() && space != subtreeSummary[3]) {
-        NameNode.LOG.warn("Inconsistent diskspace for directory "
-            +getLocalName()+". Cached: "+space+" Computed: "+subtreeSummary[3]);
+        child.computeQuotaUsage(counts, useCache, lastSnapshotId);
       }
       }
     }
     }
+    return computeQuotaUsage4CurrentDirectory(counts);
+  }
+  
+  /** Add quota usage for this inode excluding children. */
+  public Quota.Counts computeQuotaUsage4CurrentDirectory(Quota.Counts counts) {
+    counts.add(Quota.NAMESPACE, 1);
+    return counts;
+  }
 
 
-    // update the passed summary array with the values for this node's subtree
-    for (int i = 0; i < summary.length; i++) {
-      summary[i] += subtreeSummary[i];
+  @Override
+  public Content.Counts computeContentSummary(final Content.Counts counts) {
+    for (INode child : getChildrenList(null)) {
+      child.computeContentSummary(counts);
     }
     }
-
-    summary[2]++;
-    return summary;
+    counts.add(Content.DIRECTORY, 1);
+    return counts;
   }
   }
 
 
   /**
   /**
-   * @return an empty list if the children list is null;
-   *         otherwise, return the children list.
-   *         The returned list should not be modified.
+   * @param snapshot
+   *          if it is not null, get the result from the given snapshot;
+   *          otherwise, get the result from the current directory.
+   * @return the current children list if the specified snapshot is null;
+   *         otherwise, return the children list corresponding to the snapshot.
+   *         Note that the returned list is never null.
    */
    */
-  public List<INode> getChildrenList() {
-    return children==null ? EMPTY_LIST : children;
+  public ReadOnlyList<INode> getChildrenList(final Snapshot snapshot) {
+    return children == null ? ReadOnlyList.Util.<INode>emptyList()
+        : ReadOnlyList.Util.asReadOnlyList(children);
+  }
+
+  /** Set the children list to null. */
+  public void clearChildren() {
+    this.children = null;
   }
   }
 
 
   @Override
   @Override
-  int collectSubtreeBlocksAndClear(BlocksMapUpdateInfo info) {
-    int total = 1;
-    if (children == null) {
-      return total;
+  public void clear() {
+    super.clear();
+    clearChildren();
+  }
+
+  /** Call cleanSubtree(..) recursively down the subtree. */
+  public Quota.Counts cleanSubtreeRecursively(final Snapshot snapshot,
+      Snapshot prior, final BlocksMapUpdateInfo collectedBlocks,
+      final List<INode> removedINodes) throws QuotaExceededException {
+    Quota.Counts counts = Quota.Counts.newInstance();
+    // in case of deletion snapshot, since this call happens after we modify
+    // the diff list, the snapshot to be deleted has been combined or renamed
+    // to its latest previous snapshot. (besides, we also need to consider nodes
+    // created after prior but before snapshot. this will be done in 
+    // INodeDirectoryWithSnapshot#cleanSubtree)
+    Snapshot s = snapshot != null && prior != null ? prior : snapshot;
+    for (INode child : getChildrenList(s)) {
+      Quota.Counts childCounts = child.cleanSubtree(snapshot, prior,
+          collectedBlocks, removedINodes);
+      counts.add(childCounts);
     }
     }
-    for (INode child : children) {
-      total += child.collectSubtreeBlocksAndClear(info);
+    return counts;
+  }
+
+  @Override
+  public void destroyAndCollectBlocks(final BlocksMapUpdateInfo collectedBlocks,
+      final List<INode> removedINodes) {
+    for (INode child : getChildrenList(null)) {
+      child.destroyAndCollectBlocks(collectedBlocks, removedINodes);
+    }
+    clear();
+    removedINodes.add(this);
+  }
+  
+  @Override
+  public Quota.Counts cleanSubtree(final Snapshot snapshot, Snapshot prior,
+      final BlocksMapUpdateInfo collectedBlocks, 
+      final List<INode> removedINodes) throws QuotaExceededException {
+    if (prior == null && snapshot == null) {
+      // destroy the whole subtree and collect blocks that should be deleted
+      Quota.Counts counts = Quota.Counts.newInstance();
+      this.computeQuotaUsage(counts, true);
+      destroyAndCollectBlocks(collectedBlocks, removedINodes);
+      return counts; 
+    } else {
+      // process recursively down the subtree
+      Quota.Counts counts = cleanSubtreeRecursively(snapshot, prior,
+          collectedBlocks, removedINodes);
+      if (isQuotaSet()) {
+        ((INodeDirectoryWithQuota) this).addSpaceConsumed2Cache(
+            -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
+      }
+      return counts;
     }
     }
-    parent = null;
-    return total;
   }
   }
   
   
   /**
   /**
-   * Used by
-   * {@link INodeDirectory#getExistingPathINodes(byte[][], int, boolean)}.
-   * Containing INodes information resolved from a given path.
+   * Compare the metadata with another INodeDirectory
    */
    */
-  static class INodesInPath {
-    private INode[] inodes;
-    
-    public INodesInPath(int number) {
-      assert (number >= 0);
-      this.inodes = new INode[number];
-    }
-    
-    INode[] getINodes() {
-      return inodes;
-    }
-    
-    void setINode(int i, INode inode) {
-      inodes[i] = inode;
-    }
+  public boolean metadataEquals(INodeDirectory other) {
+    return other != null && getNsQuota() == other.getNsQuota()
+        && getDsQuota() == other.getDsQuota()
+        && getUserName().equals(other.getUserName())
+        && getGroupName().equals(other.getGroupName())
+        && getFsPermission().equals(other.getFsPermission());
   }
   }
-
+  
   /*
   /*
    * The following code is to dump the tree recursively for testing.
    * The following code is to dump the tree recursively for testing.
    * 
    * 
@@ -441,13 +573,45 @@ class INodeDirectory extends INode {
   static final String DUMPTREE_LAST_ITEM = "\\-";
   static final String DUMPTREE_LAST_ITEM = "\\-";
   @VisibleForTesting
   @VisibleForTesting
   @Override
   @Override
-  public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix) {
-    super.dumpTreeRecursively(out, prefix);
+  public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix,
+      final Snapshot snapshot) {
+    super.dumpTreeRecursively(out, prefix, snapshot);
+    out.print(", childrenSize=" + getChildrenList(snapshot).size());
+    if (this instanceof INodeDirectoryWithQuota) {
+      out.print(((INodeDirectoryWithQuota)this).quotaString());
+    }
+    if (this instanceof Snapshot.Root) {
+      out.print(", snapshotId=" + snapshot.getId());
+    }
+    out.println();
+
     if (prefix.length() >= 2) {
     if (prefix.length() >= 2) {
       prefix.setLength(prefix.length() - 2);
       prefix.setLength(prefix.length() - 2);
       prefix.append("  ");
       prefix.append("  ");
     }
     }
-    dumpTreeRecursively(out, prefix, children);
+    dumpTreeRecursively(out, prefix, new Iterable<SnapshotAndINode>() {
+      final Iterator<INode> i = getChildrenList(snapshot).iterator();
+      
+      @Override
+      public Iterator<SnapshotAndINode> iterator() {
+        return new Iterator<SnapshotAndINode>() {
+          @Override
+          public boolean hasNext() {
+            return i.hasNext();
+          }
+
+          @Override
+          public SnapshotAndINode next() {
+            return new SnapshotAndINode(snapshot, i.next());
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+    });
   }
   }
 
 
   /**
   /**
@@ -457,27 +621,29 @@ class INodeDirectory extends INode {
    */
    */
   @VisibleForTesting
   @VisibleForTesting
   protected static void dumpTreeRecursively(PrintWriter out,
   protected static void dumpTreeRecursively(PrintWriter out,
-      StringBuilder prefix, List<? extends INode> subs) {
-    prefix.append(DUMPTREE_EXCEPT_LAST_ITEM);
-    if (subs != null && subs.size() != 0) {
-      int i = 0;
-      for(; i < subs.size() - 1; i++) {
-        subs.get(i).dumpTreeRecursively(out, prefix);
+      StringBuilder prefix, Iterable<SnapshotAndINode> subs) {
+    if (subs != null) {
+      for(final Iterator<SnapshotAndINode> i = subs.iterator(); i.hasNext();) {
+        final SnapshotAndINode pair = i.next();
+        prefix.append(i.hasNext()? DUMPTREE_EXCEPT_LAST_ITEM: DUMPTREE_LAST_ITEM);
+        pair.inode.dumpTreeRecursively(out, prefix, pair.snapshot);
         prefix.setLength(prefix.length() - 2);
         prefix.setLength(prefix.length() - 2);
-        prefix.append(DUMPTREE_EXCEPT_LAST_ITEM);
       }
       }
-
-      prefix.setLength(prefix.length() - 2);
-      prefix.append(DUMPTREE_LAST_ITEM);
-      subs.get(i).dumpTreeRecursively(out, prefix);
     }
     }
-    prefix.setLength(prefix.length() - 2);
   }
   }
-  
-  void clearChildren() {
-    if (children != null) {
-      this.children.clear();
-      this.children = null;
+
+  /** A pair of Snapshot and INode objects. */
+  protected static class SnapshotAndINode {
+    public final Snapshot snapshot;
+    public final INode inode;
+
+    public SnapshotAndINode(Snapshot snapshot, INode inode) {
+      this.snapshot = snapshot;
+      this.inode = inode;
+    }
+
+    public SnapshotAndINode(Snapshot snapshot) {
+      this(snapshot, snapshot.getRoot());
     }
     }
   }
   }
 }
 }

+ 100 - 38
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java

@@ -23,14 +23,16 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
 /**
  * Directory INode class that has a quota restriction
  * Directory INode class that has a quota restriction
  */
  */
-class INodeDirectoryWithQuota extends INodeDirectory {
+public class INodeDirectoryWithQuota extends INodeDirectory {
   /** Name space quota */
   /** Name space quota */
   private long nsQuota = Long.MAX_VALUE;
   private long nsQuota = Long.MAX_VALUE;
   /** Name space count */
   /** Name space count */
-  private long nsCount = 1L;
+  private long namespace = 1L;
   /** Disk space quota */
   /** Disk space quota */
   private long dsQuota = HdfsConstants.QUOTA_RESET;
   private long dsQuota = HdfsConstants.QUOTA_RESET;
   /** Disk space count */
   /** Disk space count */
@@ -42,35 +44,34 @@ class INodeDirectoryWithQuota extends INodeDirectory {
    * @param dsQuota Diskspace quota to be assigned to this indoe
    * @param dsQuota Diskspace quota to be assigned to this indoe
    * @param other The other inode from which all other properties are copied
    * @param other The other inode from which all other properties are copied
    */
    */
-  INodeDirectoryWithQuota(long nsQuota, long dsQuota,
-      INodeDirectory other) {
-    super(other);
-    INode.DirCounts counts = new INode.DirCounts();
-    other.spaceConsumedInTree(counts);
-    this.nsCount = counts.getNsCount();
-    this.diskspace = counts.getDsCount();
+  public INodeDirectoryWithQuota(INodeDirectory other, boolean adopt,
+      long nsQuota, long dsQuota) {
+    super(other, adopt);
+    final Quota.Counts counts = other.computeQuotaUsage();
+    this.namespace = counts.get(Quota.NAMESPACE);
+    this.diskspace = counts.get(Quota.DISKSPACE);
     this.nsQuota = nsQuota;
     this.nsQuota = nsQuota;
     this.dsQuota = dsQuota;
     this.dsQuota = dsQuota;
   }
   }
   
   
   /** constructor with no quota verification */
   /** constructor with no quota verification */
-  INodeDirectoryWithQuota(long id, PermissionStatus permissions,
+  INodeDirectoryWithQuota(long id, byte[] name, PermissionStatus permissions,
       long modificationTime, long nsQuota, long dsQuota) {
       long modificationTime, long nsQuota, long dsQuota) {
-    super(id, permissions, modificationTime);
+    super(id, name, permissions, modificationTime);
     this.nsQuota = nsQuota;
     this.nsQuota = nsQuota;
     this.dsQuota = dsQuota;
     this.dsQuota = dsQuota;
   }
   }
   
   
   /** constructor with no quota verification */
   /** constructor with no quota verification */
-  INodeDirectoryWithQuota(long id, String name, PermissionStatus permissions) {
-    super(id, name, permissions);
+  INodeDirectoryWithQuota(long id, byte[] name, PermissionStatus permissions) {
+    super(id, name, permissions, 0L);
   }
   }
   
   
   /** Get this directory's namespace quota
   /** Get this directory's namespace quota
    * @return this directory's namespace quota
    * @return this directory's namespace quota
    */
    */
   @Override
   @Override
-  long getNsQuota() {
+  public long getNsQuota() {
     return nsQuota;
     return nsQuota;
   }
   }
   
   
@@ -78,7 +79,7 @@ class INodeDirectoryWithQuota extends INodeDirectory {
    * @return this directory's diskspace quota
    * @return this directory's diskspace quota
    */
    */
   @Override
   @Override
-  long getDsQuota() {
+  public long getDsQuota() {
     return dsQuota;
     return dsQuota;
   }
   }
   
   
@@ -86,30 +87,68 @@ class INodeDirectoryWithQuota extends INodeDirectory {
    * 
    * 
    * @param nsQuota Namespace quota to be set
    * @param nsQuota Namespace quota to be set
    * @param dsQuota diskspace quota to be set
    * @param dsQuota diskspace quota to be set
-   *                                
    */
    */
-  void setQuota(long newNsQuota, long newDsQuota) {
-    nsQuota = newNsQuota;
-    dsQuota = newDsQuota;
+  public void setQuota(long nsQuota, long dsQuota) {
+    this.nsQuota = nsQuota;
+    this.dsQuota = dsQuota;
   }
   }
   
   
-  
   @Override
   @Override
-  DirCounts spaceConsumedInTree(DirCounts counts) {
-    counts.nsCount += nsCount;
-    counts.dsCount += diskspace;
+  public Quota.Counts computeQuotaUsage(Quota.Counts counts, boolean useCache,
+      int lastSnapshotId) {
+    if (useCache && isQuotaSet()) {
+      // use cache value
+      counts.add(Quota.NAMESPACE, namespace);
+      counts.add(Quota.DISKSPACE, diskspace);
+    } else {
+      super.computeQuotaUsage(counts, false, lastSnapshotId);
+    }
     return counts;
     return counts;
   }
   }
 
 
+  @Override
+  public Content.Counts computeContentSummary(
+      final Content.Counts counts) {
+    final long original = counts.get(Content.DISKSPACE);
+    super.computeContentSummary(counts);
+    checkDiskspace(counts.get(Content.DISKSPACE) - original);
+    return counts;
+  }
+  
+  private void checkDiskspace(final long computed) {
+    if (-1 != getDsQuota() && diskspace != computed) {
+      NameNode.LOG.error("BUG: Inconsistent diskspace for directory "
+          + getFullPathName() + ". Cached = " + diskspace
+          + " != Computed = " + computed);
+    }
+  }
+
   /** Get the number of names in the subtree rooted at this directory
   /** Get the number of names in the subtree rooted at this directory
    * @return the size of the subtree rooted at this directory
    * @return the size of the subtree rooted at this directory
    */
    */
   long numItemsInTree() {
   long numItemsInTree() {
-    return nsCount;
+    return namespace;
   }
   }
   
   
-  long diskspaceConsumed() {
-    return diskspace;
+  @Override
+  public final void addSpaceConsumed(final long nsDelta, final long dsDelta,
+      boolean verify, int snapshotId) throws QuotaExceededException {
+    if (isQuotaSet()) { 
+      // The following steps are important: 
+      // check quotas in this inode and all ancestors before changing counts
+      // so that no change is made if there is any quota violation.
+
+      // (1) verify quota in this inode
+      if (verify) {
+        verifyQuota(nsDelta, dsDelta);
+      }
+      // (2) verify quota and then add count in ancestors 
+      super.addSpaceConsumed(nsDelta, dsDelta, verify, snapshotId);
+      // (3) add count in this inode
+      addSpaceConsumed2Cache(nsDelta, dsDelta);
+    } else {
+      super.addSpaceConsumed(nsDelta, dsDelta, verify, snapshotId);
+    }
   }
   }
   
   
   /** Update the size of the tree
   /** Update the size of the tree
@@ -117,8 +156,9 @@ class INodeDirectoryWithQuota extends INodeDirectory {
    * @param nsDelta the change of the tree size
    * @param nsDelta the change of the tree size
    * @param dsDelta change to disk space occupied
    * @param dsDelta change to disk space occupied
    */
    */
-  void addSpaceConsumed(long nsDelta, long dsDelta) {
-    setSpaceConsumed(nsCount + nsDelta, diskspace + dsDelta);
+  protected void addSpaceConsumed2Cache(long nsDelta, long dsDelta) {
+    namespace += nsDelta;
+    diskspace += dsDelta;
   }
   }
   
   
   /** 
   /** 
@@ -130,23 +170,45 @@ class INodeDirectoryWithQuota extends INodeDirectory {
    * @param diskspace disk space take by all the nodes under this directory
    * @param diskspace disk space take by all the nodes under this directory
    */
    */
   void setSpaceConsumed(long namespace, long diskspace) {
   void setSpaceConsumed(long namespace, long diskspace) {
-    this.nsCount = namespace;
+    this.namespace = namespace;
     this.diskspace = diskspace;
     this.diskspace = diskspace;
   }
   }
   
   
+  /** Verify if the namespace quota is violated after applying delta. */
+  void verifyNamespaceQuota(long delta) throws NSQuotaExceededException {
+    if (Quota.isViolated(nsQuota, namespace, delta)) {
+      throw new NSQuotaExceededException(nsQuota, namespace + delta);
+    }
+  }
+
   /** Verify if the namespace count disk space satisfies the quota restriction 
   /** Verify if the namespace count disk space satisfies the quota restriction 
    * @throws QuotaExceededException if the given quota is less than the count
    * @throws QuotaExceededException if the given quota is less than the count
    */
    */
   void verifyQuota(long nsDelta, long dsDelta) throws QuotaExceededException {
   void verifyQuota(long nsDelta, long dsDelta) throws QuotaExceededException {
-    long newCount = nsCount + nsDelta;
-    long newDiskspace = diskspace + dsDelta;
-    if (nsDelta>0 || dsDelta>0) {
-      if (nsQuota >= 0 && nsQuota < newCount) {
-        throw new NSQuotaExceededException(nsQuota, newCount);
-      }
-      if (dsQuota >= 0 && dsQuota < newDiskspace) {
-        throw new DSQuotaExceededException(dsQuota, newDiskspace);
-      }
+    verifyNamespaceQuota(nsDelta);
+
+    if (Quota.isViolated(dsQuota, diskspace, dsDelta)) {
+      throw new DSQuotaExceededException(dsQuota, diskspace + dsDelta);
     }
     }
   }
   }
+
+  String namespaceString() {
+    return "namespace: " + (nsQuota < 0? "-": namespace + "/" + nsQuota);
+  }
+  String diskspaceString() {
+    return "diskspace: " + (dsQuota < 0? "-": diskspace + "/" + dsQuota);
+  }
+  String quotaString() {
+    return ", Quota[" + namespaceString() + ", " + diskspaceString() + "]";
+  }
+  
+  @VisibleForTesting
+  public long getNamespace() {
+    return this.namespace;
+  }
+  
+  @VisibleForTesting
+  public long getDiskspace() {
+    return this.diskspace;
+  }
 }
 }

+ 280 - 63
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -19,29 +19,51 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.List;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.Util;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 
 
 /** I-node for closed file. */
 /** I-node for closed file. */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
-class INodeFile extends INode implements BlockCollection {
-  /** Cast INode to INodeFile. */
+public class INodeFile extends INodeWithAdditionalFields implements BlockCollection {
+  /** The same as valueOf(inode, path, false). */
   public static INodeFile valueOf(INode inode, String path
   public static INodeFile valueOf(INode inode, String path
       ) throws FileNotFoundException {
       ) throws FileNotFoundException {
+    return valueOf(inode, path, false);
+  }
+
+  /** Cast INode to INodeFile. */
+  public static INodeFile valueOf(INode inode, String path, boolean acceptNull)
+      throws FileNotFoundException {
     if (inode == null) {
     if (inode == null) {
-      throw new FileNotFoundException("File does not exist: " + path);
+      if (acceptNull) {
+        return null;
+      } else {
+        throw new FileNotFoundException("File does not exist: " + path);
+      }
     }
     }
-    if (!(inode instanceof INodeFile)) {
+    if (!inode.isFile()) {
       throw new FileNotFoundException("Path is not a file: " + path);
       throw new FileNotFoundException("Path is not a file: " + path);
     }
     }
-    return (INodeFile)inode;
+    return inode.asFile();
   }
   }
 
 
   static final FsPermission UMASK = FsPermission.createImmutable((short)0111);
   static final FsPermission UMASK = FsPermission.createImmutable((short)0111);
@@ -86,15 +108,20 @@ class INodeFile extends INode implements BlockCollection {
 
 
   private BlockInfo[] blocks;
   private BlockInfo[] blocks;
 
 
-  INodeFile(long id, PermissionStatus permissions, BlockInfo[] blklist,
-      short replication, long modificationTime, long atime,
-      long preferredBlockSize) {
-    super(id, permissions, modificationTime, atime);
+  INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime, long atime,
+      BlockInfo[] blklist, short replication, long preferredBlockSize) {
+    super(id, name, permissions, mtime, atime);
     header = HeaderFormat.combineReplication(header, replication);
     header = HeaderFormat.combineReplication(header, replication);
     header = HeaderFormat.combinePreferredBlockSize(header, preferredBlockSize);
     header = HeaderFormat.combinePreferredBlockSize(header, preferredBlockSize);
     this.blocks = blklist;
     this.blocks = blklist;
   }
   }
   
   
+  public INodeFile(INodeFile that) {
+    super(that);
+    this.header = that.header;
+    this.blocks = that.blocks;
+  }
+
   /** @return true unconditionally. */
   /** @return true unconditionally. */
   @Override
   @Override
   public final boolean isFile() {
   public final boolean isFile() {
@@ -111,33 +138,114 @@ class INodeFile extends INode implements BlockCollection {
     super.setPermission(permission.applyUMask(UMASK));
     super.setPermission(permission.applyUMask(UMASK));
   }
   }
 
 
-  /** @return the replication factor of the file. */
+  /** @return this object. */
+  @Override
+  public final INodeFile asFile() {
+    return this;
+  }
+
+  /** Is this file under construction? */
+  public boolean isUnderConstruction() {
+    return false;
+  }
+
+  /** Convert this file to an {@link INodeFileUnderConstruction}. */
+  public INodeFileUnderConstruction toUnderConstruction(
+      String clientName,
+      String clientMachine,
+      DatanodeDescriptor clientNode) {
+    Preconditions.checkState(!isUnderConstruction(),
+        "file is already an INodeFileUnderConstruction");
+    return new INodeFileUnderConstruction(this,
+        clientName, clientMachine, clientNode); 
+  }
+
+  @Override
+  public INodeFile getSnapshotINode(final Snapshot snapshot) {
+    return this;
+  }
+
   @Override
   @Override
-  public short getBlockReplication() {
+  public INodeFile recordModification(final Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
+    if (isInLatestSnapshot(latest)) {
+      INodeFileWithSnapshot newFile = getParent()
+          .replaceChild4INodeFileWithSnapshot(this, inodeMap)
+          .recordModification(latest, inodeMap);
+      return newFile;
+    } else {
+      return this;
+    }
+  }
+
+  /** @return the replication factor of the file. */
+  public final short getFileReplication(Snapshot snapshot) {
+    if (snapshot != null) {
+      return getSnapshotINode(snapshot).getFileReplication();
+    }
+
     return HeaderFormat.getReplication(header);
     return HeaderFormat.getReplication(header);
   }
   }
 
 
-  void setReplication(short replication) {
+  /** The same as getFileReplication(null). */
+  public final short getFileReplication() {
+    return getFileReplication(null);
+  }
+
+  @Override
+  public final short getBlockReplication() {
+    return this instanceof FileWithSnapshot?
+        Util.getBlockReplication((FileWithSnapshot)this)
+        : getFileReplication(null);
+  }
+
+  /** Set the replication factor of this file. */
+  public final void setFileReplication(short replication) {
     header = HeaderFormat.combineReplication(header, replication);
     header = HeaderFormat.combineReplication(header, replication);
   }
   }
 
 
+  /** Set the replication factor of this file. */
+  public final INodeFile setFileReplication(short replication, Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
+    final INodeFile nodeToUpdate = recordModification(latest, inodeMap);
+    nodeToUpdate.setFileReplication(replication);
+    return nodeToUpdate;
+  }
+
   /** @return preferred block size (in bytes) of the file. */
   /** @return preferred block size (in bytes) of the file. */
   @Override
   @Override
   public long getPreferredBlockSize() {
   public long getPreferredBlockSize() {
     return HeaderFormat.getPreferredBlockSize(header);
     return HeaderFormat.getPreferredBlockSize(header);
   }
   }
 
 
+  /** @return the diskspace required for a full block. */
+  final long getBlockDiskspace() {
+    return getPreferredBlockSize() * getBlockReplication();
+  }
+
   /** @return the blocks of the file. */
   /** @return the blocks of the file. */
   @Override
   @Override
   public BlockInfo[] getBlocks() {
   public BlockInfo[] getBlocks() {
     return this.blocks;
     return this.blocks;
   }
   }
 
 
+  void updateBlockCollection() {
+    if (blocks != null) {
+      for(BlockInfo b : blocks) {
+        b.setBlockCollection(this);
+      }
+    }
+  }
+
   /**
   /**
    * append array of blocks to this.blocks
    * append array of blocks to this.blocks
    */
    */
-  void appendBlocks(INodeFile [] inodes, int totalAddedBlocks) {
+  void concatBlocks(INodeFile[] inodes) {
     int size = this.blocks.length;
     int size = this.blocks.length;
+    int totalAddedBlocks = 0;
+    for(INodeFile f : inodes) {
+      totalAddedBlocks += f.blocks.length;
+    }
     
     
     BlockInfo[] newlist = new BlockInfo[size + totalAddedBlocks];
     BlockInfo[] newlist = new BlockInfo[size + totalAddedBlocks];
     System.arraycopy(this.blocks, 0, newlist, 0, size);
     System.arraycopy(this.blocks, 0, newlist, 0, size);
@@ -146,11 +254,9 @@ class INodeFile extends INode implements BlockCollection {
       System.arraycopy(in.blocks, 0, newlist, size, in.blocks.length);
       System.arraycopy(in.blocks, 0, newlist, size, in.blocks.length);
       size += in.blocks.length;
       size += in.blocks.length;
     }
     }
-    
-    for(BlockInfo bi: newlist) {
-      bi.setBlockCollection(this);
-    }
+
     setBlocks(newlist);
     setBlocks(newlist);
+    updateBlockCollection();
   }
   }
   
   
   /**
   /**
@@ -179,16 +285,34 @@ class INodeFile extends INode implements BlockCollection {
   }
   }
 
 
   @Override
   @Override
-  int collectSubtreeBlocksAndClear(BlocksMapUpdateInfo info) {
-    parent = null;
-    if(blocks != null && info != null) {
+  public Quota.Counts cleanSubtree(final Snapshot snapshot, Snapshot prior,
+      final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
+      throws QuotaExceededException {
+    Quota.Counts counts = Quota.Counts.newInstance();
+    if (snapshot == null && prior == null) {   
+      // this only happens when deleting the current file
+      computeQuotaUsage(counts, false);
+      destroyAndCollectBlocks(collectedBlocks, removedINodes);
+    }
+    return counts;
+  }
+
+  @Override
+  public void destroyAndCollectBlocks(BlocksMapUpdateInfo collectedBlocks,
+      final List<INode> removedINodes) {
+    if (blocks != null && collectedBlocks != null) {
       for (BlockInfo blk : blocks) {
       for (BlockInfo blk : blocks) {
-        info.addDeleteBlock(blk);
+        collectedBlocks.addDeleteBlock(blk);
         blk.setBlockCollection(null);
         blk.setBlockCollection(null);
       }
       }
     }
     }
     setBlocks(null);
     setBlocks(null);
-    return 1;
+    clear();
+    removedINodes.add(this);
+    
+    if (this instanceof FileWithSnapshot) {
+      ((FileWithSnapshot) this).getDiffs().clear();
+    }
   }
   }
   
   
   @Override
   @Override
@@ -197,63 +321,144 @@ class INodeFile extends INode implements BlockCollection {
     return getFullPathName();
     return getFullPathName();
   }
   }
 
 
+  @Override
+  public final Quota.Counts computeQuotaUsage(Quota.Counts counts,
+      boolean useCache, int lastSnapshotId) {
+    long nsDelta = 1;
+    final long dsDelta;
+    if (this instanceof FileWithSnapshot) {
+      FileDiffList fileDiffList = ((FileWithSnapshot) this).getDiffs();
+      Snapshot last = fileDiffList.getLastSnapshot();
+      List<FileDiff> diffs = fileDiffList.asList();
+
+      if (lastSnapshotId == Snapshot.INVALID_ID || last == null) {
+        nsDelta += diffs.size();
+        dsDelta = diskspaceConsumed();
+      } else if (last.getId() < lastSnapshotId) {
+        dsDelta = computeFileSize(true, false) * getFileReplication();
+      } else {      
+        Snapshot s = fileDiffList.getSnapshotById(lastSnapshotId);
+        dsDelta = diskspaceConsumed(s);
+      }
+    } else {
+      dsDelta = diskspaceConsumed();
+    }
+    counts.add(Quota.NAMESPACE, nsDelta);
+    counts.add(Quota.DISKSPACE, dsDelta);
+    return counts;
+  }
 
 
   @Override
   @Override
-  long[] computeContentSummary(long[] summary) {
-    summary[0] += computeFileSize(true);
-    summary[1]++;
-    summary[3] += diskspaceConsumed();
-    return summary;
+  public final Content.Counts computeContentSummary(
+      final Content.Counts counts) {
+    computeContentSummary4Snapshot(counts);
+    computeContentSummary4Current(counts);
+    return counts;
+  }
+
+  private void computeContentSummary4Snapshot(final Content.Counts counts) {
+    // file length and diskspace only counted for the latest state of the file
+    // i.e. either the current state or the last snapshot
+    if (this instanceof FileWithSnapshot) {
+      final FileWithSnapshot withSnapshot = (FileWithSnapshot)this;
+      final FileDiffList diffs = withSnapshot.getDiffs();
+      final int n = diffs.asList().size();
+      counts.add(Content.FILE, n);
+      if (n > 0 && withSnapshot.isCurrentFileDeleted()) {
+        counts.add(Content.LENGTH, diffs.getLast().getFileSize());
+      }
+
+      if (withSnapshot.isCurrentFileDeleted()) {
+        final long lastFileSize = diffs.getLast().getFileSize();
+        counts.add(Content.DISKSPACE, lastFileSize * getBlockReplication());
+      }
+    }
+  }
+
+  private void computeContentSummary4Current(final Content.Counts counts) {
+    if (this instanceof FileWithSnapshot
+        && ((FileWithSnapshot)this).isCurrentFileDeleted()) {
+      return;
+    }
+
+    counts.add(Content.LENGTH, computeFileSize());
+    counts.add(Content.FILE, 1);
+    counts.add(Content.DISKSPACE, diskspaceConsumed());
+  }
+
+  /** The same as computeFileSize(null). */
+  public final long computeFileSize() {
+    return computeFileSize(null);
+  }
+
+  /**
+   * Compute file size of the current file if the given snapshot is null;
+   * otherwise, get the file size from the given snapshot.
+   */
+  public final long computeFileSize(Snapshot snapshot) {
+    if (snapshot != null && this instanceof FileWithSnapshot) {
+      final FileDiff d = ((FileWithSnapshot)this).getDiffs().getDiff(snapshot);
+      if (d != null) {
+        return d.getFileSize();
+      }
+    }
+
+    return computeFileSize(true, false);
+  }
+
+  /**
+   * Compute file size of the current file size
+   * but not including the last block if it is under construction.
+   */
+  public final long computeFileSizeNotIncludingLastUcBlock() {
+    return computeFileSize(false, false);
   }
   }
 
 
-  /** Compute file size.
-   * May or may not include BlockInfoUnderConstruction.
+  /**
+   * Compute file size of the current file.
+   * 
+   * @param includesLastUcBlock
+   *          If the last block is under construction, should it be included?
+   * @param usePreferredBlockSize4LastUcBlock
+   *          If the last block is under construction, should we use actual
+   *          block size or preferred block size?
+   *          Note that usePreferredBlockSize4LastUcBlock is ignored
+   *          if includesLastUcBlock == false.
+   * @return file size
    */
    */
-  long computeFileSize(boolean includesBlockInfoUnderConstruction) {
+  public final long computeFileSize(boolean includesLastUcBlock,
+      boolean usePreferredBlockSize4LastUcBlock) {
     if (blocks == null || blocks.length == 0) {
     if (blocks == null || blocks.length == 0) {
       return 0;
       return 0;
     }
     }
     final int last = blocks.length - 1;
     final int last = blocks.length - 1;
     //check if the last block is BlockInfoUnderConstruction
     //check if the last block is BlockInfoUnderConstruction
-    long bytes = blocks[last] instanceof BlockInfoUnderConstruction
-                 && !includesBlockInfoUnderConstruction?
-                     0: blocks[last].getNumBytes();
+    long size = blocks[last].getNumBytes();
+    if (blocks[last] instanceof BlockInfoUnderConstruction) {
+       if (!includesLastUcBlock) {
+         size = 0;
+       } else if (usePreferredBlockSize4LastUcBlock) {
+         size = getPreferredBlockSize();
+       }
+    }
+    //sum other blocks
     for(int i = 0; i < last; i++) {
     for(int i = 0; i < last; i++) {
-      bytes += blocks[i].getNumBytes();
+      size += blocks[i].getNumBytes();
     }
     }
-    return bytes;
+    return size;
   }
   }
-  
 
 
-  @Override
-  DirCounts spaceConsumedInTree(DirCounts counts) {
-    counts.nsCount += 1;
-    counts.dsCount += diskspaceConsumed();
-    return counts;
+  public final long diskspaceConsumed() {
+    // use preferred block size for the last block if it is under construction
+    return computeFileSize(true, true) * getBlockReplication();
   }
   }
 
 
-  long diskspaceConsumed() {
-    return diskspaceConsumed(blocks);
-  }
-  
-  private long diskspaceConsumed(Block[] blkArr) {
-    long size = 0;
-    if(blkArr == null) 
-      return 0;
-    
-    for (Block blk : blkArr) {
-      if (blk != null) {
-        size += blk.getNumBytes();
-      }
-    }
-    /* If the last block is being written to, use prefferedBlockSize
-     * rather than the actual block size.
-     */
-    if (blkArr.length > 0 && blkArr[blkArr.length-1] != null && 
-        isUnderConstruction()) {
-      size += getPreferredBlockSize() - blkArr[blkArr.length-1].getNumBytes();
+  public final long diskspaceConsumed(Snapshot lastSnapshot) {
+    if (lastSnapshot != null) {
+      return computeFileSize(lastSnapshot) * getFileReplication(lastSnapshot);
+    } else {
+      return diskspaceConsumed();
     }
     }
-    return size * getBlockReplication();
   }
   }
   
   
   /**
   /**
@@ -275,4 +480,16 @@ class INodeFile extends INode implements BlockCollection {
   public int numBlocks() {
   public int numBlocks() {
     return blocks == null ? 0 : blocks.length;
     return blocks == null ? 0 : blocks.length;
   }
   }
+
+  @VisibleForTesting
+  @Override
+  public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix,
+      final Snapshot snapshot) {
+    super.dumpTreeRecursively(out, prefix, snapshot);
+    out.print(", fileSize=" + computeFileSize(snapshot));
+    // only compare the first block
+    out.print(", blocks=");
+    out.print(blocks == null || blocks.length == 0? null: blocks[0]);
+    out.println();
+  }
 }
 }

+ 58 - 42
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java

@@ -24,17 +24,22 @@ import java.util.Arrays;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.MutableBlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.MutableBlockCollection;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileUnderConstructionWithSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+
+import com.google.common.base.Preconditions;
 
 
 /**
 /**
  * I-node for file being written.
  * I-node for file being written.
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
-class INodeFileUnderConstruction extends INodeFile implements MutableBlockCollection {
+public class INodeFileUnderConstruction extends INodeFile implements MutableBlockCollection {
   /** Cast INode to INodeFileUnderConstruction. */
   /** Cast INode to INodeFileUnderConstruction. */
   public static INodeFileUnderConstruction valueOf(INode inode, String path
   public static INodeFileUnderConstruction valueOf(INode inode, String path
       ) throws FileNotFoundException {
       ) throws FileNotFoundException {
@@ -57,11 +62,8 @@ class INodeFileUnderConstruction extends INodeFile implements MutableBlockCollec
                              String clientName,
                              String clientName,
                              String clientMachine,
                              String clientMachine,
                              DatanodeDescriptor clientNode) {
                              DatanodeDescriptor clientNode) {
-    super(id, permissions.applyUMask(UMASK), BlockInfo.EMPTY_ARRAY,
-        replication, modTime, modTime, preferredBlockSize);
-    this.clientName = clientName;
-    this.clientMachine = clientMachine;
-    this.clientNode = clientNode;
+    this(id, null, replication, modTime, preferredBlockSize, BlockInfo.EMPTY_ARRAY,
+        permissions, clientName, clientMachine, clientNode);
   }
   }
 
 
   INodeFileUnderConstruction(long id,
   INodeFileUnderConstruction(long id,
@@ -74,15 +76,24 @@ class INodeFileUnderConstruction extends INodeFile implements MutableBlockCollec
                              String clientName,
                              String clientName,
                              String clientMachine,
                              String clientMachine,
                              DatanodeDescriptor clientNode) {
                              DatanodeDescriptor clientNode) {
-    super(id, perm, blocks, blockReplication, modificationTime,
-        modificationTime, preferredBlockSize);
-    setLocalName(name);
+    super(id, name, perm, modificationTime, modificationTime,
+        blocks, blockReplication, preferredBlockSize);
+    this.clientName = clientName;
+    this.clientMachine = clientMachine;
+    this.clientNode = clientNode;
+  }
+  
+  public INodeFileUnderConstruction(final INodeFile that,
+      final String clientName,
+      final String clientMachine,
+      final DatanodeDescriptor clientNode) {
+    super(that);
     this.clientName = clientName;
     this.clientName = clientName;
     this.clientMachine = clientMachine;
     this.clientMachine = clientMachine;
     this.clientNode = clientNode;
     this.clientNode = clientNode;
   }
   }
 
 
-  String getClientName() {
+  public String getClientName() {
     return clientName;
     return clientName;
   }
   }
 
 
@@ -90,51 +101,56 @@ class INodeFileUnderConstruction extends INodeFile implements MutableBlockCollec
     this.clientName = clientName;
     this.clientName = clientName;
   }
   }
 
 
-  String getClientMachine() {
+  public String getClientMachine() {
     return clientMachine;
     return clientMachine;
   }
   }
 
 
-  DatanodeDescriptor getClientNode() {
+  public DatanodeDescriptor getClientNode() {
     return clientNode;
     return clientNode;
   }
   }
 
 
-  /**
-   * Is this inode being constructed?
-   */
+  /** @return true unconditionally. */
   @Override
   @Override
-  public boolean isUnderConstruction() {
+  public final boolean isUnderConstruction() {
     return true;
     return true;
   }
   }
 
 
-  //
-  // converts a INodeFileUnderConstruction into a INodeFile
-  // use the modification time as the access time
-  //
-  INodeFile convertToInodeFile() {
-    assert allBlocksComplete() : "Can't finalize inode " + this
-      + " since it contains non-complete blocks! Blocks are "
-      + Arrays.asList(getBlocks());
-    INodeFile obj = new INodeFile(getId(),
-                                  getPermissionStatus(),
-                                  getBlocks(),
-                                  getBlockReplication(),
-                                  getModificationTime(),
-                                  getModificationTime(),
-                                  getPreferredBlockSize());
-    return obj;
-    
-  }
-  
   /**
   /**
-   * @return true if all of the blocks in this file are marked as completed.
+   * Converts an INodeFileUnderConstruction to an INodeFile.
+   * The original modification time is used as the access time.
+   * The new modification is the specified mtime.
    */
    */
-  private boolean allBlocksComplete() {
-    for (BlockInfo b : getBlocks()) {
-      if (!b.isComplete()) {
-        return false;
-      }
+  protected INodeFile toINodeFile(long mtime) {
+    assertAllBlocksComplete();
+
+    final INodeFile f = new INodeFile(getId(), getLocalNameBytes(),
+        getPermissionStatus(), mtime, getModificationTime(),
+        getBlocks(), getFileReplication(), getPreferredBlockSize());
+    f.setParent(getParent());
+    return f;
+  }
+  
+  @Override
+  public INodeFileUnderConstruction recordModification(final Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
+    if (isInLatestSnapshot(latest)) {
+      INodeFileUnderConstructionWithSnapshot newFile = getParent()
+          .replaceChild4INodeFileUcWithSnapshot(this, inodeMap)
+          .recordModification(latest, inodeMap);
+      return newFile;
+    } else {
+      return this;
+    }
+  }
+
+  /** Assert all blocks are complete. */
+  protected void assertAllBlocksComplete() {
+    final BlockInfo[] blocks = getBlocks();
+    for (int i = 0; i < blocks.length; i++) {
+      Preconditions.checkState(blocks[i].isComplete(), "Failed to finalize"
+          + " %s %s since blocks[%s] is non-complete, where blocks=%s.",
+          getClass().getSimpleName(), this, i, Arrays.asList(getBlocks()));
     }
     }
-    return true;
   }
   }
 
 
   /**
   /**

+ 131 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java

@@ -0,0 +1,131 @@
+/**
+ * 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;
+
+import java.util.List;
+
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.namenode.Quota.Counts;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+import org.apache.hadoop.hdfs.util.GSet;
+import org.apache.hadoop.hdfs.util.LightWeightGSet;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Storing all the {@link INode}s and maintaining the mapping between INode ID
+ * and INode.  
+ */
+public class INodeMap {
+  
+  static INodeMap newInstance(INodeDirectory rootDir) {
+    // Compute the map capacity by allocating 1% of total memory
+    int capacity = LightWeightGSet.computeCapacity(1, "INodeMap");
+    GSet<INode, INodeWithAdditionalFields> map
+        = new LightWeightGSet<INode, INodeWithAdditionalFields>(capacity);
+    map.put(rootDir);
+    return new INodeMap(map);
+  }
+  
+  /** Synchronized by external lock. */
+  private GSet<INode, INodeWithAdditionalFields> map;
+  
+  private INodeMap(GSet<INode, INodeWithAdditionalFields> map) {
+    Preconditions.checkArgument(map != null);
+    this.map = map;
+  }
+  
+  /**
+   * Add an {@link INode} into the {@link INode} map. Replace the old value if 
+   * necessary. 
+   * @param inode The {@link INode} to be added to the map.
+   */
+  public final void put(INode inode) {
+    if (inode instanceof INodeWithAdditionalFields) {
+      map.put((INodeWithAdditionalFields)inode);
+    }
+  }
+  
+  /**
+   * Remove a {@link INode} from the map.
+   * @param inode The {@link INode} to be removed.
+   */
+  public final void remove(INode inode) {
+    map.remove(inode);
+  }
+  
+  /**
+   * @return The size of the map.
+   */
+  public int size() {
+    return map.size();
+  }
+  
+  /**
+   * Get the {@link INode} with the given id from the map.
+   * @param id ID of the {@link INode}.
+   * @return The {@link INode} in the map with the given id. Return null if no 
+   *         such {@link INode} in the map.
+   */
+  public INode get(long id) {
+    INode inode = new INodeWithAdditionalFields(id, null, new PermissionStatus(
+        "", "", new FsPermission((short) 0)), 0, 0) {
+      
+      @Override
+      INode recordModification(Snapshot latest, INodeMap inodeMap)
+          throws QuotaExceededException {
+        return null;
+      }
+      
+      @Override
+      public void destroyAndCollectBlocks(BlocksMapUpdateInfo collectedBlocks,
+          List<INode> removedINodes) {
+        // Nothing to do
+      }
+      
+      @Override
+      public Counts computeQuotaUsage(Counts counts, boolean useCache,
+          int lastSnapshotId) {
+        return null;
+      }
+      
+      @Override
+      public Content.Counts computeContentSummary(Content.Counts counts) {
+        return null;
+      }
+      
+      @Override
+      public Counts cleanSubtree(Snapshot snapshot, Snapshot prior,
+          BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes)
+          throws QuotaExceededException {
+        return null;
+      }
+    };
+      
+    return map.get(inode);
+  }
+  
+  /**
+   * Clear the {@link #map}
+   */
+  public void clear() {
+    map.clear();
+  }
+}

+ 681 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java

@@ -0,0 +1,681 @@
+/**
+ * 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;
+
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * An anonymous reference to an inode.
+ *
+ * This class and its subclasses are used to support multiple access paths.
+ * A file/directory may have multiple access paths when it is stored in some
+ * snapshots and it is renamed/moved to other locations.
+ * 
+ * For example,
+ * (1) Support we have /abc/foo, say the inode of foo is inode(id=1000,name=foo)
+ * (2) create snapshot s0 for /abc
+ * (3) mv /abc/foo /xyz/bar, i.e. inode(id=1000,name=...) is renamed from "foo"
+ *     to "bar" and its parent becomes /xyz.
+ * 
+ * Then, /xyz/bar and /abc/.snapshot/s0/foo are two different access paths to
+ * the same inode, inode(id=1000,name=bar).
+ *
+ * With references, we have the following
+ * - /abc has a child ref(id=1001,name=foo).
+ * - /xyz has a child ref(id=1002) 
+ * - Both ref(id=1001,name=foo) and ref(id=1002) point to another reference,
+ *   ref(id=1003,count=2).
+ * - Finally, ref(id=1003,count=2) points to inode(id=1000,name=bar).
+ * 
+ * Note 1: For a reference without name, e.g. ref(id=1002), it uses the name
+ *         of the referred inode.
+ * Note 2: getParent() always returns the parent in the current state, e.g.
+ *         inode(id=1000,name=bar).getParent() returns /xyz but not /abc.
+ */
+public abstract class INodeReference extends INode {
+  /**
+   * Try to remove the given reference and then return the reference count.
+   * If the given inode is not a reference, return -1;
+   */
+  public static int tryRemoveReference(INode inode) {
+    if (!inode.isReference()) {
+      return -1;
+    }
+    return removeReference(inode.asReference());
+  }
+
+  /**
+   * Remove the given reference and then return the reference count.
+   * If the referred inode is not a WithCount, return -1;
+   */
+  private static int removeReference(INodeReference ref) {
+    final INode referred = ref.getReferredINode();
+    if (!(referred instanceof WithCount)) {
+      return -1;
+    }
+    
+    WithCount wc = (WithCount) referred;
+    wc.removeReference(ref);
+    return wc.getReferenceCount();
+  }
+
+  /**
+   * When destroying a reference node (WithName or DstReference), we call this
+   * method to identify the snapshot which is the latest snapshot before the
+   * reference node's creation. 
+   */
+  static Snapshot getPriorSnapshot(INodeReference ref) {
+    WithCount wc = (WithCount) ref.getReferredINode();
+    WithName wn = null;
+    if (ref instanceof DstReference) {
+      wn = wc.getLastWithName();
+    } else if (ref instanceof WithName) {
+      wn = wc.getPriorWithName((WithName) ref);
+    }
+    if (wn != null) {
+      INode referred = wc.getReferredINode();
+      if (referred instanceof FileWithSnapshot) {
+        return ((FileWithSnapshot) referred).getDiffs().getPrior(
+            wn.lastSnapshotId);
+      } else if (referred instanceof INodeDirectoryWithSnapshot) { 
+        return ((INodeDirectoryWithSnapshot) referred).getDiffs().getPrior(
+            wn.lastSnapshotId);
+      }
+    }
+    return null;
+  }
+  
+  private INode referred;
+  
+  public INodeReference(INode parent, INode referred) {
+    super(parent);
+    this.referred = referred;
+  }
+
+  public final INode getReferredINode() {
+    return referred;
+  }
+
+  public final void setReferredINode(INode referred) {
+    this.referred = referred;
+  }
+  
+  @Override
+  public final boolean isReference() {
+    return true;
+  }
+  
+  @Override
+  public final INodeReference asReference() {
+    return this;
+  }
+
+  @Override
+  public final boolean isFile() {
+    return referred.isFile();
+  }
+  
+  @Override
+  public final INodeFile asFile() {
+    return referred.asFile();
+  }
+  
+  @Override
+  public final boolean isDirectory() {
+    return referred.isDirectory();
+  }
+  
+  @Override
+  public final INodeDirectory asDirectory() {
+    return referred.asDirectory();
+  }
+  
+  @Override
+  public final boolean isSymlink() {
+    return referred.isSymlink();
+  }
+  
+  @Override
+  public final INodeSymlink asSymlink() {
+    return referred.asSymlink();
+  }
+
+  @Override
+  public byte[] getLocalNameBytes() {
+    return referred.getLocalNameBytes();
+  }
+
+  @Override
+  public void setLocalName(byte[] name) {
+    referred.setLocalName(name);
+  }
+
+  @Override
+  public final long getId() {
+    return referred.getId();
+  }
+  
+  @Override
+  public final PermissionStatus getPermissionStatus(Snapshot snapshot) {
+    return referred.getPermissionStatus(snapshot);
+  }
+  
+  @Override
+  public final String getUserName(Snapshot snapshot) {
+    return referred.getUserName(snapshot);
+  }
+  
+  @Override
+  final void setUser(String user) {
+    referred.setUser(user);
+  }
+  
+  @Override
+  public final String getGroupName(Snapshot snapshot) {
+    return referred.getGroupName(snapshot);
+  }
+  
+  @Override
+  final void setGroup(String group) {
+    referred.setGroup(group);
+  }
+  
+  @Override
+  public final FsPermission getFsPermission(Snapshot snapshot) {
+    return referred.getFsPermission(snapshot);
+  }
+  
+  @Override
+  void setPermission(FsPermission permission) {
+    referred.setPermission(permission);
+  }
+  
+  @Override
+  public final long getModificationTime(Snapshot snapshot) {
+    return referred.getModificationTime(snapshot);
+  }
+  
+  @Override
+  public final INode updateModificationTime(long mtime, Snapshot latest,
+      INodeMap inodeMap) throws QuotaExceededException {
+    return referred.updateModificationTime(mtime, latest, inodeMap);
+  }
+  
+  @Override
+  public final void setModificationTime(long modificationTime) {
+    referred.setModificationTime(modificationTime);
+  }
+  
+  @Override
+  public final long getAccessTime(Snapshot snapshot) {
+    return referred.getAccessTime(snapshot);
+  }
+  
+  @Override
+  public final void setAccessTime(long accessTime) {
+    referred.setAccessTime(accessTime);
+  }
+
+  @Override
+  final INode recordModification(Snapshot latest, final INodeMap inodeMap)
+      throws QuotaExceededException {
+    referred.recordModification(latest, inodeMap);
+    // reference is never replaced 
+    return this;
+  }
+
+  @Override // used by WithCount
+  public Quota.Counts cleanSubtree(Snapshot snapshot, Snapshot prior,
+      BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
+      throws QuotaExceededException {
+    return referred.cleanSubtree(snapshot, prior, collectedBlocks,
+        removedINodes);
+  }
+
+  @Override // used by WithCount
+  public void destroyAndCollectBlocks(
+      BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
+    if (removeReference(this) <= 0) {
+      referred.destroyAndCollectBlocks(collectedBlocks, removedINodes);
+    }
+  }
+
+  @Override
+  public Content.Counts computeContentSummary(Content.Counts counts) {
+    return referred.computeContentSummary(counts);
+  }
+
+  @Override
+  public Quota.Counts computeQuotaUsage(Quota.Counts counts, boolean useCache,
+      int lastSnapshotId) {
+    return referred.computeQuotaUsage(counts, useCache, lastSnapshotId);
+  }
+  
+  @Override
+  public final INode getSnapshotINode(Snapshot snapshot) {
+    return referred.getSnapshotINode(snapshot);
+  }
+
+  @Override
+  public final long getNsQuota() {
+    return referred.getNsQuota();
+  }
+
+  @Override
+  public final long getDsQuota() {
+    return referred.getDsQuota();
+  }
+  
+  @Override
+  public final void clear() {
+    super.clear();
+    referred = null;
+  }
+
+  @Override
+  public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix,
+      final Snapshot snapshot) {
+    super.dumpTreeRecursively(out, prefix, snapshot);
+    if (this instanceof DstReference) {
+      out.print(", dstSnapshotId=" + ((DstReference) this).dstSnapshotId);
+    }
+    if (this instanceof WithCount) {
+      out.print(", count=" + ((WithCount)this).getReferenceCount());
+    }
+    out.println();
+    
+    final StringBuilder b = new StringBuilder();
+    for(int i = 0; i < prefix.length(); i++) {
+      b.append(' ');
+    }
+    b.append("->");
+    getReferredINode().dumpTreeRecursively(out, b, snapshot);
+  }
+  
+  public int getDstSnapshotId() {
+    return Snapshot.INVALID_ID;
+  }
+  
+  /** An anonymous reference with reference count. */
+  public static class WithCount extends INodeReference {
+    
+    private final List<WithName> withNameList = new ArrayList<WithName>();
+    
+    /**
+     * Compare snapshot with IDs, where null indicates the current status thus
+     * is greater than any non-null snapshot.
+     */
+    public static final Comparator<WithName> WITHNAME_COMPARATOR
+        = new Comparator<WithName>() {
+      @Override
+      public int compare(WithName left, WithName right) {
+        return left.lastSnapshotId - right.lastSnapshotId;
+      }
+    };
+    
+    public WithCount(INodeReference parent, INode referred) {
+      super(parent, referred);
+      Preconditions.checkArgument(!referred.isReference());
+      referred.setParentReference(this);
+    }
+    
+    public int getReferenceCount() {
+      int count = withNameList.size();
+      if (getParentReference() != null) {
+        count++;
+      }
+      return count;
+    }
+
+    /** Increment and then return the reference count. */
+    public void addReference(INodeReference ref) {
+      if (ref instanceof WithName) {
+        WithName refWithName = (WithName) ref;
+        int i = Collections.binarySearch(withNameList, refWithName,
+            WITHNAME_COMPARATOR);
+        Preconditions.checkState(i < 0);
+        withNameList.add(-i - 1, refWithName);
+      } else if (ref instanceof DstReference) {
+        setParentReference(ref);
+      }
+    }
+
+    /** Decrement and then return the reference count. */
+    public void removeReference(INodeReference ref) {
+      if (ref instanceof WithName) {
+        int i = Collections.binarySearch(withNameList, (WithName) ref,
+            WITHNAME_COMPARATOR);
+        if (i >= 0) {
+          withNameList.remove(i);
+        }
+      } else if (ref == getParentReference()) {
+        setParent(null);
+      }
+    }
+    
+    WithName getLastWithName() {
+      return withNameList.size() > 0 ? 
+          withNameList.get(withNameList.size() - 1) : null;
+    }
+    
+    WithName getPriorWithName(WithName post) {
+      int i = Collections.binarySearch(withNameList, post, WITHNAME_COMPARATOR);
+      if (i > 0) {
+        return withNameList.get(i - 1);
+      } else if (i == 0 || i == -1) {
+        return null;
+      } else {
+        return withNameList.get(-i - 2);
+      }
+    }
+    
+    @Override
+    public final void addSpaceConsumed(long nsDelta, long dsDelta,
+        boolean verify, int snapshotId) throws QuotaExceededException {
+      INodeReference parentRef = getParentReference();
+      if (parentRef != null) {
+        parentRef.addSpaceConsumed(nsDelta, dsDelta, verify, snapshotId);
+      }
+      addSpaceConsumedToRenameSrc(nsDelta, dsDelta, verify, snapshotId);
+    }
+    
+    @Override
+    public final void addSpaceConsumedToRenameSrc(long nsDelta, long dsDelta,
+        boolean verify, int snapshotId) throws QuotaExceededException {
+      if (snapshotId != Snapshot.INVALID_ID) {
+        for (INodeReference.WithName withName : withNameList) {
+          if (withName.getLastSnapshotId() >= snapshotId) {
+            withName.addSpaceConsumed(nsDelta, dsDelta, verify, snapshotId);
+            break;
+          }
+        }
+      }
+    }
+  }
+  
+  /** A reference with a fixed name. */
+  public static class WithName extends INodeReference {
+
+    private final byte[] name;
+
+    /**
+     * The id of the last snapshot in the src tree when this WithName node was 
+     * generated. When calculating the quota usage of the referred node, only 
+     * the files/dirs existing when this snapshot was taken will be counted for 
+     * this WithName node and propagated along its ancestor path.
+     */
+    private final int lastSnapshotId;
+    
+    public WithName(INodeDirectory parent, WithCount referred, byte[] name,
+        int lastSnapshotId) {
+      super(parent, referred);
+      this.name = name;
+      this.lastSnapshotId = lastSnapshotId;
+      referred.addReference(this);
+    }
+
+    @Override
+    public final byte[] getLocalNameBytes() {
+      return name;
+    }
+
+    @Override
+    public final void setLocalName(byte[] name) {
+      throw new UnsupportedOperationException("Cannot set name: " + getClass()
+          + " is immutable.");
+    }
+    
+    public int getLastSnapshotId() {
+      return lastSnapshotId;
+    }
+    
+    @Override
+    public final Content.Counts computeContentSummary(Content.Counts counts) {
+      //only count diskspace for WithName
+      final Quota.Counts q = Quota.Counts.newInstance();
+      computeQuotaUsage(q, false, lastSnapshotId);
+      counts.add(Content.DISKSPACE, q.get(Quota.DISKSPACE));
+      return counts;
+    }
+
+    @Override
+    public final Quota.Counts computeQuotaUsage(Quota.Counts counts,
+        boolean useCache, int lastSnapshotId) {
+      // if this.lastSnapshotId < lastSnapshotId, the rename of the referred 
+      // node happened before the rename of its ancestor. This should be 
+      // impossible since for WithName node we only count its children at the 
+      // time of the rename. 
+      Preconditions.checkState(this.lastSnapshotId >= lastSnapshotId);
+      final INode referred = this.getReferredINode().asReference()
+          .getReferredINode();
+      // we cannot use cache for the referred node since its cached quota may
+      // have already been updated by changes in the current tree
+      return referred.computeQuotaUsage(counts, false, this.lastSnapshotId);
+    }
+    
+    @Override
+    public Quota.Counts cleanSubtree(final Snapshot snapshot, Snapshot prior,
+        final BlocksMapUpdateInfo collectedBlocks,
+        final List<INode> removedINodes) throws QuotaExceededException {
+      // since WithName node resides in deleted list acting as a snapshot copy,
+      // the parameter snapshot must be non-null
+      Preconditions.checkArgument(snapshot != null);
+      // if prior is null, we need to check snapshot belonging to the previous
+      // WithName instance
+      if (prior == null) {
+        prior = getPriorSnapshot(this);
+      }
+
+      Quota.Counts counts = getReferredINode().cleanSubtree(snapshot, prior,
+          collectedBlocks, removedINodes);
+      INodeReference ref = getReferredINode().getParentReference();
+      if (ref != null) {
+        ref.addSpaceConsumed(-counts.get(Quota.NAMESPACE),
+            -counts.get(Quota.DISKSPACE), true, Snapshot.INVALID_ID);
+      }
+      return counts;
+    }
+    
+    @Override
+    public void destroyAndCollectBlocks(BlocksMapUpdateInfo collectedBlocks,
+        final List<INode> removedINodes) {
+      Snapshot snapshot = getSelfSnapshot();
+      if (removeReference(this) <= 0) {
+        getReferredINode().destroyAndCollectBlocks(collectedBlocks,
+            removedINodes);
+      } else {
+        Snapshot prior = getPriorSnapshot(this);
+        INode referred = getReferredINode().asReference().getReferredINode();
+        
+        if (snapshot != null) {
+          if (prior != null && snapshot.getId() <= prior.getId()) {
+            // the snapshot to be deleted has been deleted while traversing 
+            // the src tree of the previous rename operation. This usually 
+            // happens when rename's src and dst are under the same 
+            // snapshottable directory. E.g., the following operation sequence:
+            // 1. create snapshot s1 on /test
+            // 2. rename /test/foo/bar to /test/foo2/bar
+            // 3. create snapshot s2 on /test
+            // 4. delete snapshot s2
+            return;
+          }
+          try {
+            Quota.Counts counts = referred.cleanSubtree(snapshot, prior,
+                collectedBlocks, removedINodes);
+            INodeReference ref = getReferredINode().getParentReference();
+            if (ref != null) {
+              ref.addSpaceConsumed(-counts.get(Quota.NAMESPACE),
+                  -counts.get(Quota.DISKSPACE), true, Snapshot.INVALID_ID);
+            }
+          } catch (QuotaExceededException e) {
+            LOG.error("should not exceed quota while snapshot deletion", e);
+          }
+        }
+      }
+    }
+    
+    private Snapshot getSelfSnapshot() {
+      INode referred = getReferredINode().asReference().getReferredINode();
+      Snapshot snapshot = null;
+      if (referred instanceof FileWithSnapshot) {
+        snapshot = ((FileWithSnapshot) referred).getDiffs().getPrior(
+            lastSnapshotId);
+      } else if (referred instanceof INodeDirectoryWithSnapshot) {
+        snapshot = ((INodeDirectoryWithSnapshot) referred).getDiffs().getPrior(
+            lastSnapshotId);
+      }
+      return snapshot;
+    }
+  }
+  
+  public static class DstReference extends INodeReference {
+    /**
+     * Record the latest snapshot of the dst subtree before the rename. For
+     * later operations on the moved/renamed files/directories, if the latest
+     * snapshot is after this dstSnapshot, changes will be recorded to the
+     * latest snapshot. Otherwise changes will be recorded to the snapshot
+     * belonging to the src of the rename.
+     * 
+     * {@link Snapshot#INVALID_ID} means no dstSnapshot (e.g., src of the
+     * first-time rename).
+     */
+    private final int dstSnapshotId;
+    
+    @Override
+    public final int getDstSnapshotId() {
+      return dstSnapshotId;
+    }
+    
+    public DstReference(INodeDirectory parent, WithCount referred,
+        final int dstSnapshotId) {
+      super(parent, referred);
+      this.dstSnapshotId = dstSnapshotId;
+      referred.addReference(this);
+    }
+    
+    @Override
+    public Quota.Counts cleanSubtree(Snapshot snapshot, Snapshot prior,
+        BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes)
+        throws QuotaExceededException {
+      if (snapshot == null && prior == null) {
+        Quota.Counts counts = Quota.Counts.newInstance();
+        this.computeQuotaUsage(counts, true);
+        destroyAndCollectBlocks(collectedBlocks, removedINodes);
+        return counts;
+      } else {
+        // if prior is null, we need to check snapshot belonging to the previous
+        // WithName instance
+        if (prior == null) {
+          prior = getPriorSnapshot(this);
+        }
+        if (snapshot != null && snapshot.equals(prior)) {
+          return Quota.Counts.newInstance();
+        }
+        return getReferredINode().cleanSubtree(snapshot, prior,
+            collectedBlocks, removedINodes);
+      }
+    }
+    
+    /**
+     * {@inheritDoc}
+     * <br/>
+     * To destroy a DstReference node, we first remove its link with the 
+     * referred node. If the reference number of the referred node is <= 0, we 
+     * destroy the subtree of the referred node. Otherwise, we clean the 
+     * referred node's subtree and delete everything created after the last 
+     * rename operation, i.e., everything outside of the scope of the prior 
+     * WithName nodes.
+     */
+    @Override
+    public void destroyAndCollectBlocks(
+        BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
+      if (removeReference(this) <= 0) {
+        getReferredINode().destroyAndCollectBlocks(collectedBlocks,
+            removedINodes);
+      } else {
+        // we will clean everything, including files, directories, and 
+        // snapshots, that were created after this prior snapshot
+        Snapshot prior = getPriorSnapshot(this);
+        // prior must be non-null, otherwise we do not have any previous 
+        // WithName nodes, and the reference number will be 0.
+        Preconditions.checkState(prior != null);
+        // identify the snapshot created after prior
+        Snapshot snapshot = getSelfSnapshot(prior);
+        
+        INode referred = getReferredINode().asReference().getReferredINode();
+        if (referred instanceof FileWithSnapshot) {
+          // if referred is a file, it must be a FileWithSnapshot since we did
+          // recordModification before the rename
+          FileWithSnapshot sfile = (FileWithSnapshot) referred;
+          // make sure we mark the file as deleted
+          sfile.deleteCurrentFile();
+          if (snapshot != null) {
+            try {
+              referred.cleanSubtree(snapshot, prior, collectedBlocks,
+                  removedINodes);
+            } catch (QuotaExceededException e) {
+              LOG.error("should not exceed quota while snapshot deletion", e);
+            }
+          }
+        } else if (referred instanceof INodeDirectoryWithSnapshot) {
+          // similarly, if referred is a directory, it must be an
+          // INodeDirectoryWithSnapshot
+          INodeDirectoryWithSnapshot sdir = 
+              (INodeDirectoryWithSnapshot) referred;
+          try {
+            INodeDirectoryWithSnapshot.destroyDstSubtree(sdir, snapshot, prior,
+                collectedBlocks, removedINodes);
+          } catch (QuotaExceededException e) {
+            LOG.error("should not exceed quota while snapshot deletion", e);
+          }
+        }
+      }
+    }
+    
+    private Snapshot getSelfSnapshot(final Snapshot prior) {
+      WithCount wc = (WithCount) getReferredINode().asReference();
+      INode referred = wc.getReferredINode();
+      Snapshot lastSnapshot = null;
+      if (referred instanceof FileWithSnapshot) {
+        lastSnapshot = ((FileWithSnapshot) referred).getDiffs()
+            .getLastSnapshot(); 
+      } else if (referred instanceof INodeDirectoryWithSnapshot) {
+        lastSnapshot = ((INodeDirectoryWithSnapshot) referred)
+            .getLastSnapshot();
+      }
+      if (lastSnapshot != null && !lastSnapshot.equals(prior)) {
+        return lastSnapshot;
+      } else {
+        return null;
+      }
+    }
+  }
+}

+ 58 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java

@@ -17,28 +17,55 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
+import java.io.PrintWriter;
+import java.util.List;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 
 /**
 /**
  * An {@link INode} representing a symbolic link.
  * An {@link INode} representing a symbolic link.
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
-public class INodeSymlink extends INode {
+public class INodeSymlink extends INodeWithAdditionalFields {
   private final byte[] symlink; // The target URI
   private final byte[] symlink; // The target URI
 
 
-  INodeSymlink(long id, String value, long mtime, long atime,
-      PermissionStatus permissions) {
-    super(id, permissions, mtime, atime);
-    this.symlink = DFSUtil.string2Bytes(value);
+  INodeSymlink(long id, byte[] name, PermissionStatus permissions,
+      long mtime, long atime, String symlink) {
+    super(id, name, permissions, mtime, atime);
+    this.symlink = DFSUtil.string2Bytes(symlink);
+  }
+  
+  INodeSymlink(INodeSymlink that) {
+    super(that);
+    this.symlink = that.symlink;
   }
   }
 
 
+  @Override
+  INode recordModification(Snapshot latest, final INodeMap inodeMap)
+      throws QuotaExceededException {
+    if (isInLatestSnapshot(latest)) {
+      INodeDirectory parent = getParent();
+      parent.saveChild2Snapshot(this, latest, new INodeSymlink(this), inodeMap);
+    }
+    return this;
+  }
+
+  /** @return true unconditionally. */
   @Override
   @Override
   public boolean isSymlink() {
   public boolean isSymlink() {
     return true;
     return true;
   }
   }
 
 
+  /** @return this object. */
+  @Override
+  public INodeSymlink asSymlink() {
+    return this;
+  }
+
   public String getSymlinkString() {
   public String getSymlinkString() {
     return DFSUtil.bytes2String(symlink);
     return DFSUtil.bytes2String(symlink);
   }
   }
@@ -46,21 +73,39 @@ public class INodeSymlink extends INode {
   public byte[] getSymlink() {
   public byte[] getSymlink() {
     return symlink;
     return symlink;
   }
   }
+  
+  @Override
+  public Quota.Counts cleanSubtree(final Snapshot snapshot, Snapshot prior,
+      final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
+    if (snapshot == null && prior == null) {
+      destroyAndCollectBlocks(collectedBlocks, removedINodes);
+    }
+    return Quota.Counts.newInstance(1, 0);
+  }
+  
+  @Override
+  public void destroyAndCollectBlocks(final BlocksMapUpdateInfo collectedBlocks,
+      final List<INode> removedINodes) {
+    removedINodes.add(this);
+  }
 
 
   @Override
   @Override
-  DirCounts spaceConsumedInTree(DirCounts counts) {
-    counts.nsCount += 1;
+  public Quota.Counts computeQuotaUsage(Quota.Counts counts,
+      boolean updateCache, int lastSnapshotId) {
+    counts.add(Quota.NAMESPACE, 1);
     return counts;
     return counts;
   }
   }
-  
+
   @Override
   @Override
-  int collectSubtreeBlocksAndClear(BlocksMapUpdateInfo info) {
-    return 1;
+  public Content.Counts computeContentSummary(final Content.Counts counts) {
+    counts.add(Content.SYMLINK, 1);
+    return counts;
   }
   }
 
 
   @Override
   @Override
-  long[] computeContentSummary(long[] summary) {
-    summary[1]++; // Increment the file count
-    return summary;
+  public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix,
+      final Snapshot snapshot) {
+    super.dumpTreeRecursively(out, prefix, snapshot);
+    out.println();
   }
   }
 }
 }

+ 257 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java

@@ -0,0 +1,257 @@
+/**
+ * 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;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+import org.apache.hadoop.hdfs.util.LightWeightGSet.LinkedElement;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * {@link INode} with additional fields including id, name, permission,
+ * access time and modification time.
+ */
+@InterfaceAudience.Private
+public abstract class INodeWithAdditionalFields extends INode
+    implements LinkedElement {
+  private static enum PermissionStatusFormat {
+    MODE(0, 16),
+    GROUP(MODE.OFFSET + MODE.LENGTH, 25),
+    USER(GROUP.OFFSET + GROUP.LENGTH, 23);
+
+    final int OFFSET;
+    final int LENGTH; //bit length
+    final long MASK;
+
+    PermissionStatusFormat(int offset, int length) {
+      OFFSET = offset;
+      LENGTH = length;
+      MASK = ((-1L) >>> (64 - LENGTH)) << OFFSET;
+    }
+
+    long retrieve(long record) {
+      return (record & MASK) >>> OFFSET;
+    }
+
+    long combine(long bits, long record) {
+      return (record & ~MASK) | (bits << OFFSET);
+    }
+
+    /** Encode the {@link PermissionStatus} to a long. */
+    static long toLong(PermissionStatus ps) {
+      long permission = 0L;
+      final int user = SerialNumberManager.INSTANCE.getUserSerialNumber(
+          ps.getUserName());
+      permission = USER.combine(user, permission);
+      final int group = SerialNumberManager.INSTANCE.getGroupSerialNumber(
+          ps.getGroupName());
+      permission = GROUP.combine(group, permission);
+      final int mode = ps.getPermission().toShort();
+      permission = MODE.combine(mode, permission);
+      return permission;
+    }
+  }
+
+  /** The inode id. */
+  final private long id;
+  /**
+   *  The inode name is in java UTF8 encoding; 
+   *  The name in HdfsFileStatus should keep the same encoding as this.
+   *  if this encoding is changed, implicitly getFileInfo and listStatus in
+   *  clientProtocol are changed; The decoding at the client
+   *  side should change accordingly.
+   */
+  private byte[] name = null;
+  /** 
+   * Permission encoded using {@link PermissionStatusFormat}.
+   * Codes other than {@link #clonePermissionStatus(INodeWithAdditionalFields)}
+   * and {@link #updatePermissionStatus(PermissionStatusFormat, long)}
+   * should not modify it.
+   */
+  private long permission = 0L;
+  /** The last modification time*/
+  private long modificationTime = 0L;
+  /** The last access time*/
+  private long accessTime = 0L;
+
+  /** For implementing {@link LinkedElement}. */
+  private LinkedElement next = null;
+
+  private INodeWithAdditionalFields(INode parent, long id, byte[] name,
+      long permission, long modificationTime, long accessTime) {
+    super(parent);
+    this.id = id;
+    this.name = name;
+    this.permission = permission;
+    this.modificationTime = modificationTime;
+    this.accessTime = accessTime;
+  }
+
+  INodeWithAdditionalFields(long id, byte[] name, PermissionStatus permissions,
+      long modificationTime, long accessTime) {
+    this(null, id, name, PermissionStatusFormat.toLong(permissions),
+        modificationTime, accessTime);
+  }
+  
+  /** @param other Other node to be copied */
+  INodeWithAdditionalFields(INodeWithAdditionalFields other) {
+    this(other.getParentReference() != null ? other.getParentReference()
+        : other.getParent(), other.getId(), other.getLocalNameBytes(),
+        other.permission, other.modificationTime, other.accessTime);
+  }
+
+  @Override
+  public void setNext(LinkedElement next) {
+    this.next = next;
+  }
+  
+  @Override
+  public LinkedElement getNext() {
+    return next;
+  }
+
+  /** Get inode id */
+  public final long getId() {
+    return this.id;
+  }
+
+  @Override
+  public final byte[] getLocalNameBytes() {
+    return name;
+  }
+  
+  @Override
+  public final void setLocalName(byte[] name) {
+    this.name = name;
+  }
+
+  /** Clone the {@link PermissionStatus}. */
+  final void clonePermissionStatus(INodeWithAdditionalFields that) {
+    this.permission = that.permission;
+  }
+
+  @Override
+  final PermissionStatus getPermissionStatus(Snapshot snapshot) {
+    return new PermissionStatus(getUserName(snapshot), getGroupName(snapshot),
+        getFsPermission(snapshot));
+  }
+
+  private final void updatePermissionStatus(PermissionStatusFormat f, long n) {
+    this.permission = f.combine(n, permission);
+  }
+
+  @Override
+  final String getUserName(Snapshot snapshot) {
+    if (snapshot != null) {
+      return getSnapshotINode(snapshot).getUserName();
+    }
+
+    int n = (int)PermissionStatusFormat.USER.retrieve(permission);
+    return SerialNumberManager.INSTANCE.getUser(n);
+  }
+
+  @Override
+  final void setUser(String user) {
+    int n = SerialNumberManager.INSTANCE.getUserSerialNumber(user);
+    updatePermissionStatus(PermissionStatusFormat.USER, n);
+  }
+
+  @Override
+  final String getGroupName(Snapshot snapshot) {
+    if (snapshot != null) {
+      return getSnapshotINode(snapshot).getGroupName();
+    }
+
+    int n = (int)PermissionStatusFormat.GROUP.retrieve(permission);
+    return SerialNumberManager.INSTANCE.getGroup(n);
+  }
+
+  @Override
+  final void setGroup(String group) {
+    int n = SerialNumberManager.INSTANCE.getGroupSerialNumber(group);
+    updatePermissionStatus(PermissionStatusFormat.GROUP, n);
+  }
+
+  @Override
+  final FsPermission getFsPermission(Snapshot snapshot) {
+    if (snapshot != null) {
+      return getSnapshotINode(snapshot).getFsPermission();
+    }
+
+    return new FsPermission(
+        (short)PermissionStatusFormat.MODE.retrieve(permission));
+  }
+
+  final short getFsPermissionShort() {
+    return (short)PermissionStatusFormat.MODE.retrieve(permission);
+  }
+  @Override
+  void setPermission(FsPermission permission) {
+    final short mode = permission.toShort();
+    updatePermissionStatus(PermissionStatusFormat.MODE, mode);
+  }
+
+  @Override
+  final long getModificationTime(Snapshot snapshot) {
+    if (snapshot != null) {
+      return getSnapshotINode(snapshot).getModificationTime(null);
+    }
+
+    return this.modificationTime;
+  }
+
+
+  /** Update modification time if it is larger than the current value. */
+  public final INode updateModificationTime(long mtime, Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
+    Preconditions.checkState(isDirectory());
+    if (mtime <= modificationTime) {
+      return this;
+    }
+    return setModificationTime(mtime, latest, inodeMap);
+  }
+
+  final void cloneModificationTime(INodeWithAdditionalFields that) {
+    this.modificationTime = that.modificationTime;
+  }
+
+  @Override
+  public final void setModificationTime(long modificationTime) {
+    this.modificationTime = modificationTime;
+  }
+
+  @Override
+  final long getAccessTime(Snapshot snapshot) {
+    if (snapshot != null) {
+      return getSnapshotINode(snapshot).getAccessTime(null);
+    }
+
+    return accessTime;
+  }
+
+  /**
+   * Set last access time of inode.
+   */
+  public final void setAccessTime(long accessTime) {
+    this.accessTime = accessTime;
+  }
+}

+ 417 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java

@@ -0,0 +1,417 @@
+/**
+ * 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;
+
+import java.util.Arrays;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Contains INodes information resolved from a given path.
+ */
+public class INodesInPath {
+  public static final Log LOG = LogFactory.getLog(INodesInPath.class);
+
+  /**
+   * @return true if path component is {@link HdfsConstants#DOT_SNAPSHOT_DIR}
+   */
+  private static boolean isDotSnapshotDir(byte[] pathComponent) {
+    return pathComponent == null ? false
+        : Arrays.equals(HdfsConstants.DOT_SNAPSHOT_DIR_BYTES, pathComponent);
+  }
+
+  /**
+   * Given some components, create a path name.
+   * @param components The path components
+   * @param start index
+   * @param end index
+   * @return concatenated path
+   */
+  private static String constructPath(byte[][] components, int start, int end) {
+    StringBuilder buf = new StringBuilder();
+    for (int i = start; i < end; i++) {
+      buf.append(DFSUtil.bytes2String(components[i]));
+      if (i < end - 1) {
+        buf.append(Path.SEPARATOR);
+      }
+    }
+    return buf.toString();
+  }
+
+  static INodesInPath resolve(final INodeDirectory startingDir,
+      final byte[][] components) throws UnresolvedLinkException {
+    return resolve(startingDir, components, components.length, false);
+  }
+
+  /**
+   * Retrieve existing INodes from a path. If existing is big enough to store
+   * all path components (existing and non-existing), then existing INodes
+   * will be stored starting from the root INode into existing[0]; if
+   * existing is not big enough to store all path components, then only the
+   * last existing and non existing INodes will be stored so that
+   * existing[existing.length-1] refers to the INode of the final component.
+   * 
+   * An UnresolvedPathException is always thrown when an intermediate path 
+   * component refers to a symbolic link. If the final path component refers 
+   * to a symbolic link then an UnresolvedPathException is only thrown if
+   * resolveLink is true.  
+   * 
+   * <p>
+   * Example: <br>
+   * Given the path /c1/c2/c3 where only /c1/c2 exists, resulting in the
+   * following path components: ["","c1","c2","c3"],
+   * 
+   * <p>
+   * <code>getExistingPathINodes(["","c1","c2"], [?])</code> should fill the
+   * array with [c2] <br>
+   * <code>getExistingPathINodes(["","c1","c2","c3"], [?])</code> should fill the
+   * array with [null]
+   * 
+   * <p>
+   * <code>getExistingPathINodes(["","c1","c2"], [?,?])</code> should fill the
+   * array with [c1,c2] <br>
+   * <code>getExistingPathINodes(["","c1","c2","c3"], [?,?])</code> should fill
+   * the array with [c2,null]
+   * 
+   * <p>
+   * <code>getExistingPathINodes(["","c1","c2"], [?,?,?,?])</code> should fill
+   * the array with [rootINode,c1,c2,null], <br>
+   * <code>getExistingPathINodes(["","c1","c2","c3"], [?,?,?,?])</code> should
+   * fill the array with [rootINode,c1,c2,null]
+   * 
+   * @param startingDir the starting directory
+   * @param components array of path component name
+   * @param numOfINodes number of INodes to return
+   * @param resolveLink indicates whether UnresolvedLinkException should
+   *        be thrown when the path refers to a symbolic link.
+   * @return the specified number of existing INodes in the path
+   */
+  static INodesInPath resolve(final INodeDirectory startingDir,
+      final byte[][] components, final int numOfINodes, 
+      final boolean resolveLink) throws UnresolvedLinkException {
+    Preconditions.checkArgument(startingDir.compareTo(components[0]) == 0);
+
+    INode curNode = startingDir;
+    final INodesInPath existing = new INodesInPath(components, numOfINodes);
+    int count = 0;
+    int index = numOfINodes - components.length;
+    if (index > 0) {
+      index = 0;
+    }
+    while (count < components.length && curNode != null) {
+      final boolean lastComp = (count == components.length - 1);      
+      if (index >= 0) {
+        existing.addNode(curNode);
+      }
+      final boolean isRef = curNode.isReference();
+      final boolean isDir = curNode.isDirectory();
+      final INodeDirectory dir = isDir? curNode.asDirectory(): null;  
+      if (!isRef && isDir && dir instanceof INodeDirectoryWithSnapshot) {
+        //if the path is a non-snapshot path, update the latest snapshot.
+        if (!existing.isSnapshot()) {
+          existing.updateLatestSnapshot(
+              ((INodeDirectoryWithSnapshot)dir).getLastSnapshot());
+        }
+      } else if (isRef && isDir && !lastComp) {
+        // If the curNode is a reference node, need to check its dstSnapshot:
+        // 1. if the existing snapshot is no later than the dstSnapshot (which
+        // is the latest snapshot in dst before the rename), the changes 
+        // should be recorded in previous snapshots (belonging to src).
+        // 2. however, if the ref node is already the last component, we still 
+        // need to know the latest snapshot among the ref node's ancestors, 
+        // in case of processing a deletion operation. Thus we do not overwrite
+        // the latest snapshot if lastComp is true. In case of the operation is
+        // a modification operation, we do a similar check in corresponding 
+        // recordModification method.
+        if (!existing.isSnapshot()) {
+          int dstSnapshotId = curNode.asReference().getDstSnapshotId();
+          Snapshot latest = existing.getLatestSnapshot();
+          if (latest == null ||  // no snapshot in dst tree of rename
+              dstSnapshotId >= latest.getId()) { // the above scenario 
+            Snapshot lastSnapshot = null;
+            if (curNode.isDirectory()
+                && curNode.asDirectory() instanceof INodeDirectoryWithSnapshot) {
+              lastSnapshot = ((INodeDirectoryWithSnapshot) curNode
+                  .asDirectory()).getLastSnapshot();
+            }
+            existing.setSnapshot(lastSnapshot);
+          }
+        }
+      }
+      if (curNode.isSymlink() && (!lastComp || (lastComp && resolveLink))) {
+        final String path = constructPath(components, 0, components.length);
+        final String preceding = constructPath(components, 0, count);
+        final String remainder =
+          constructPath(components, count + 1, components.length);
+        final String link = DFSUtil.bytes2String(components[count]);
+        final String target = curNode.asSymlink().getSymlinkString();
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("UnresolvedPathException " +
+            " path: " + path + " preceding: " + preceding +
+            " count: " + count + " link: " + link + " target: " + target +
+            " remainder: " + remainder);
+        }
+        throw new UnresolvedPathException(path, preceding, remainder, target);
+      }
+      if (lastComp || !isDir) {
+        break;
+      }
+      final byte[] childName = components[count + 1];
+      
+      // check if the next byte[] in components is for ".snapshot"
+      if (isDotSnapshotDir(childName)
+          && isDir && dir instanceof INodeDirectoryWithSnapshot) {
+        // skip the ".snapshot" in components
+        count++;
+        index++;
+        existing.isSnapshot = true;
+        if (index >= 0) { // decrease the capacity by 1 to account for .snapshot
+          existing.capacity--;
+        }
+        // check if ".snapshot" is the last element of components
+        if (count == components.length - 1) {
+          break;
+        }
+        // Resolve snapshot root
+        final Snapshot s = ((INodeDirectorySnapshottable)dir).getSnapshot(
+            components[count + 1]);
+        if (s == null) {
+          //snapshot not found
+          curNode = null;
+        } else {
+          curNode = s.getRoot();
+          existing.setSnapshot(s);
+        }
+        if (index >= -1) {
+          existing.snapshotRootIndex = existing.numNonNull;
+        }
+      } else {
+        // normal case, and also for resolving file/dir under snapshot root
+        curNode = dir.getChild(childName, existing.getPathSnapshot());
+      }
+      count++;
+      index++;
+    }
+    return existing;
+  }
+
+  private final byte[][] path;
+  /**
+   * Array with the specified number of INodes resolved for a given path.
+   */
+  private INode[] inodes;
+  /**
+   * Indicate the number of non-null elements in {@link #inodes}
+   */
+  private int numNonNull;
+  /**
+   * The path for a snapshot file/dir contains the .snapshot thus makes the
+   * length of the path components larger the number of inodes. We use
+   * the capacity to control this special case.
+   */
+  private int capacity;
+  /**
+   * true if this path corresponds to a snapshot
+   */
+  private boolean isSnapshot;
+  /**
+   * Index of {@link INodeDirectoryWithSnapshot} for snapshot path, else -1
+   */
+  private int snapshotRootIndex;
+  /**
+   * For snapshot paths, it is the reference to the snapshot; or null if the
+   * snapshot does not exist. For non-snapshot paths, it is the reference to
+   * the latest snapshot found in the path; or null if no snapshot is found.
+   */
+  private Snapshot snapshot = null; 
+
+  private INodesInPath(byte[][] path, int number) {
+    this.path = path;
+    assert (number >= 0);
+    inodes = new INode[number];
+    capacity = number;
+    numNonNull = 0;
+    isSnapshot = false;
+    snapshotRootIndex = -1;
+  }
+
+  /**
+   * For non-snapshot paths, return the latest snapshot found in the path.
+   * For snapshot paths, return null.
+   */
+  public Snapshot getLatestSnapshot() {
+    return isSnapshot? null: snapshot;
+  }
+  
+  /**
+   * For snapshot paths, return the snapshot specified in the path.
+   * For non-snapshot paths, return null.
+   */
+  public Snapshot getPathSnapshot() {
+    return isSnapshot? snapshot: null;
+  }
+
+  private void setSnapshot(Snapshot s) {
+    snapshot = s;
+  }
+  
+  private void updateLatestSnapshot(Snapshot s) {
+    if (snapshot == null
+        || (s != null && Snapshot.ID_COMPARATOR.compare(snapshot, s) < 0)) {
+      snapshot = s;
+    }
+  }
+
+  /**
+   * @return the whole inodes array including the null elements.
+   */
+  INode[] getINodes() {
+    if (capacity < inodes.length) {
+      INode[] newNodes = new INode[capacity];
+      System.arraycopy(inodes, 0, newNodes, 0, capacity);
+      inodes = newNodes;
+    }
+    return inodes;
+  }
+  
+  /**
+   * @return the i-th inode if i >= 0;
+   *         otherwise, i < 0, return the (length + i)-th inode.
+   */
+  public INode getINode(int i) {
+    return inodes[i >= 0? i: inodes.length + i];
+  }
+  
+  /** @return the last inode. */
+  public INode getLastINode() {
+    return inodes[inodes.length - 1];
+  }
+
+  byte[] getLastLocalName() {
+    return path[path.length - 1];
+  }
+  
+  /**
+   * @return index of the {@link INodeDirectoryWithSnapshot} in
+   *         {@link #inodes} for snapshot path, else -1.
+   */
+  int getSnapshotRootIndex() {
+    return this.snapshotRootIndex;
+  }
+  
+  /**
+   * @return isSnapshot true for a snapshot path
+   */
+  boolean isSnapshot() {
+    return this.isSnapshot;
+  }
+  
+  /**
+   * Add an INode at the end of the array
+   */
+  private void addNode(INode node) {
+    inodes[numNonNull++] = node;
+  }
+  
+  void setINode(int i, INode inode) {
+    inodes[i >= 0? i: inodes.length + i] = inode;
+  }
+  
+  void setLastINode(INode last) {
+    inodes[inodes.length - 1] = last;
+  }
+  
+  /**
+   * @return The number of non-null elements
+   */
+  int getNumNonNull() {
+    return numNonNull;
+  }
+  
+  private static String toString(INode inode) {
+    return inode == null? null: inode.getLocalName();
+  }
+
+  @Override
+  public String toString() {
+    return toString(true);
+  }
+
+  private String toString(boolean vaildateObject) {
+    if (vaildateObject) {
+      vaildate();
+    }
+
+    final StringBuilder b = new StringBuilder(getClass().getSimpleName())
+        .append(": path = ").append(DFSUtil.byteArray2PathString(path))
+        .append("\n  inodes = ");
+    if (inodes == null) {
+      b.append("null");
+    } else if (inodes.length == 0) {
+      b.append("[]");
+    } else {
+      b.append("[").append(toString(inodes[0]));
+      for(int i = 1; i < inodes.length; i++) {
+        b.append(", ").append(toString(inodes[i]));
+      }
+      b.append("], length=").append(inodes.length);
+    }
+    b.append("\n  numNonNull = ").append(numNonNull)
+     .append("\n  capacity   = ").append(capacity)
+     .append("\n  isSnapshot        = ").append(isSnapshot)
+     .append("\n  snapshotRootIndex = ").append(snapshotRootIndex)
+     .append("\n  snapshot          = ").append(snapshot);
+    return b.toString();
+  }
+
+  void vaildate() {
+    // check parent up to snapshotRootIndex or numNonNull
+    final int n = snapshotRootIndex >= 0? snapshotRootIndex + 1: numNonNull;  
+    int i = 0;
+    if (inodes[i] != null) {
+      for(i++; i < n && inodes[i] != null; i++) {
+        final INodeDirectory parent_i = inodes[i].getParent();
+        final INodeDirectory parent_i_1 = inodes[i-1].getParent();
+        if (parent_i != inodes[i-1] &&
+            (parent_i_1 == null || !parent_i_1.isSnapshottable()
+                || parent_i != parent_i_1)) {
+          throw new AssertionError(
+              "inodes[" + i + "].getParent() != inodes[" + (i-1)
+              + "]\n  inodes[" + i + "]=" + inodes[i].toDetailString()
+              + "\n  inodes[" + (i-1) + "]=" + inodes[i-1].toDetailString()
+              + "\n this=" + toString(false));
+        }
+      }
+    }
+    if (i != n) {
+      throw new AssertionError("i = " + i + " != " + n
+          + ", this=" + toString(false));
+    }
+  }
+}

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

@@ -256,7 +256,9 @@ public class LeaseManager {
     private String findPath(INodeFileUnderConstruction pendingFile) {
     private String findPath(INodeFileUnderConstruction pendingFile) {
       try {
       try {
         for (String src : paths) {
         for (String src : paths) {
-          if (fsnamesystem.dir.getINode(src) == pendingFile) {
+          INode node = fsnamesystem.dir.getINode(src);
+          if (node == pendingFile
+              || (node.isFile() && node.asFile() == pendingFile)) {
             return src;
             return src;
           }
           }
         }
         }

+ 62 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -65,6 +65,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -72,6 +73,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
@@ -1104,4 +1106,64 @@ class NameNodeRpcServer implements NamenodeProtocols {
   public DataEncryptionKey getDataEncryptionKey() throws IOException {
   public DataEncryptionKey getDataEncryptionKey() throws IOException {
     return namesystem.getBlockManager().generateDataEncryptionKey();
     return namesystem.getBlockManager().generateDataEncryptionKey();
   }
   }
+
+  @Override
+  public String createSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException {
+    if (!checkPathLength(snapshotRoot)) {
+      throw new IOException("createSnapshot: Pathname too long.  Limit "
+          + MAX_PATH_LENGTH + " characters, " + MAX_PATH_DEPTH + " levels.");
+    }
+    metrics.incrCreateSnapshotOps();
+    return namesystem.createSnapshot(snapshotRoot, snapshotName);
+  }
+  
+  @Override
+  public void deleteSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException {
+    metrics.incrDeleteSnapshotOps();
+    namesystem.deleteSnapshot(snapshotRoot, snapshotName);
+  }
+
+  @Override
+  // Client Protocol
+  public void allowSnapshot(String snapshotRoot) throws IOException {
+    metrics.incrAllowSnapshotOps();
+    namesystem.allowSnapshot(snapshotRoot);
+  }
+
+  @Override
+  // Client Protocol
+  public void disallowSnapshot(String snapshot) throws IOException {
+    metrics.incrDisAllowSnapshotOps();
+    namesystem.disallowSnapshot(snapshot);
+  }
+
+  @Override
+  public void renameSnapshot(String snapshotRoot, String snapshotOldName,
+      String snapshotNewName) throws IOException {
+    if (snapshotNewName == null || snapshotNewName.isEmpty()) {
+      throw new IOException("The new snapshot name is null or empty.");
+    }
+    metrics.incrRenameSnapshotOps();
+    namesystem.renameSnapshot(snapshotRoot, snapshotOldName, snapshotNewName);
+  }
+
+  @Override // Client Protocol
+  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+      throws IOException {
+    SnapshottableDirectoryStatus[] status = namesystem
+        .getSnapshottableDirListing();
+    metrics.incrListSnapshottableDirOps();
+    return status;
+  }
+
+  @Override
+  public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
+      String earlierSnapshotName, String laterSnapshotName) throws IOException {
+    SnapshotDiffReport report = namesystem.getSnapshotDiffReport(snapshotRoot,
+        earlierSnapshotName, laterSnapshotName);
+    metrics.incrSnapshotDiffReportOps();
+    return report;
+  }
 }
 }

+ 13 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java

@@ -775,7 +775,16 @@ class NamenodeJspHelper {
       }
       }
     }
     }
   }
   }
-  
+
+  private static String getLocalParentDir(INode inode) {
+    final INode parent = inode.isRoot() ? inode : inode.getParent();
+    String parentDir = "";
+    if (parent != null) {
+      parentDir = parent.getFullPathName();
+    }
+    return (parentDir != null) ? parentDir : "";
+  }
+
   // utility class used in block_info_xml.jsp
   // utility class used in block_info_xml.jsp
   static class XMLBlockInfo {
   static class XMLBlockInfo {
     final Block block;
     final Block block;
@@ -790,7 +799,7 @@ class NamenodeJspHelper {
         this.inode = null;
         this.inode = null;
       } else {
       } else {
         this.block = new Block(blockId);
         this.block = new Block(blockId);
-        this.inode = (INodeFile) blockManager.getBlockCollection(block);
+        this.inode = ((INode)blockManager.getBlockCollection(block)).asFile();
       }
       }
     }
     }
 
 
@@ -817,7 +826,7 @@ class NamenodeJspHelper {
           doc.endTag();
           doc.endTag();
 
 
           doc.startTag("local_directory");
           doc.startTag("local_directory");
-          doc.pcdata(inode.getLocalParentDir());
+          doc.pcdata(getLocalParentDir(inode));
           doc.endTag();
           doc.endTag();
 
 
           doc.startTag("user_name");
           doc.startTag("user_name");
@@ -849,7 +858,7 @@ class NamenodeJspHelper {
           doc.endTag();
           doc.endTag();
 
 
           doc.startTag("replication");
           doc.startTag("replication");
-          doc.pcdata(""+inode.getBlockReplication());
+          doc.pcdata(""+inode.getFileReplication());
           doc.endTag();
           doc.endTag();
 
 
           doc.startTag("disk_space_consumed");
           doc.startTag("disk_space_consumed");

+ 64 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Quota.java

@@ -0,0 +1,64 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hdfs.util.EnumCounters;
+
+/** Quota types. */
+public enum Quota {
+  /** The namespace usage, i.e. the number of name objects. */
+  NAMESPACE,
+  /** The diskspace usage in bytes including replication. */
+  DISKSPACE;
+
+  /** Counters for quota counts. */
+  public static class Counts extends EnumCounters<Quota> {
+    /** @return a new counter with the given namespace and diskspace usages. */
+    public static Counts newInstance(long namespace, long diskspace) {
+      final Counts c = new Counts();
+      c.set(NAMESPACE, namespace);
+      c.set(DISKSPACE, diskspace);
+      return c;
+    }
+
+    public static Counts newInstance() {
+      return newInstance(0, 0);
+    }
+    
+    Counts() {
+      super(Quota.values());
+    }
+  }
+
+  /**
+   * Is quota violated?
+   * The quota is violated if quota is set and usage > quota. 
+   */
+  static boolean isViolated(final long quota, final long usage) {
+    return quota >= 0 && usage > quota;
+  }
+
+  /**
+   * Is quota violated?
+   * The quota is violated if quota is set, delta > 0 and usage + delta > quota.
+   */
+  static boolean isViolated(final long quota, final long usage,
+      final long delta) {
+    return quota >= 0 && delta > 0 && usage > quota - delta;
+  }
+}

+ 43 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java

@@ -57,6 +57,20 @@ public class NameNodeMetrics {
   @Metric MutableCounterLong createSymlinkOps;
   @Metric MutableCounterLong createSymlinkOps;
   @Metric MutableCounterLong getLinkTargetOps;
   @Metric MutableCounterLong getLinkTargetOps;
   @Metric MutableCounterLong filesInGetListingOps;
   @Metric MutableCounterLong filesInGetListingOps;
+  @Metric("Number of allowSnapshot operations")
+  MutableCounterLong allowSnapshotOps;
+  @Metric("Number of disallowSnapshot operations")
+  MutableCounterLong disallowSnapshotOps;
+  @Metric("Number of createSnapshot operations")
+  MutableCounterLong createSnapshotOps;
+  @Metric("Number of deleteSnapshot operations")
+  MutableCounterLong deleteSnapshotOps;
+  @Metric("Number of renameSnapshot operations")
+  MutableCounterLong renameSnapshotOps;
+  @Metric("Number of listSnapshottableDirectory operations")
+  MutableCounterLong listSnapshottableDirOps;
+  @Metric("Number of snapshotDiffReport operations")
+  MutableCounterLong snapshotDiffReportOps;
 
 
   @Metric("Journal transactions") MutableRate transactions;
   @Metric("Journal transactions") MutableRate transactions;
   @Metric("Journal syncs") MutableRate syncs;
   @Metric("Journal syncs") MutableRate syncs;
@@ -131,7 +145,7 @@ public class NameNodeMetrics {
     filesRenamed.incr();
     filesRenamed.incr();
   }
   }
 
 
-  public void incrFilesDeleted(int delta) {
+  public void incrFilesDeleted(long delta) {
     filesDeleted.incr(delta);
     filesDeleted.incr(delta);
   }
   }
 
 
@@ -159,6 +173,34 @@ public class NameNodeMetrics {
     getLinkTargetOps.incr();
     getLinkTargetOps.incr();
   }
   }
 
 
+  public void incrAllowSnapshotOps() {
+    allowSnapshotOps.incr();
+  }
+  
+  public void incrDisAllowSnapshotOps() {
+    disallowSnapshotOps.incr();
+  }
+  
+  public void incrCreateSnapshotOps() {
+    createSnapshotOps.incr();
+  }
+  
+  public void incrDeleteSnapshotOps() {
+    deleteSnapshotOps.incr();
+  }
+  
+  public void incrRenameSnapshotOps() {
+    renameSnapshotOps.incr();
+  }
+  
+  public void incrListSnapshottableDirOps() {
+    listSnapshottableDirOps.incr();
+  }
+  
+  public void incrSnapshotDiffReportOps() {
+    snapshotDiffReportOps.incr();
+  }
+  
   public void addTransaction(long latency) {
   public void addTransaction(long latency) {
     transactions.add(latency);
     transactions.add(latency);
   }
   }

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

@@ -0,0 +1,144 @@
+/**
+ * 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.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
+import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * The difference of an inode between in two snapshots.
+ * {@link AbstractINodeDiffList} 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<Integer> {
+
+  /** The snapshot will be obtained after this diff is applied. */
+  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 Integer that) {
+    return Snapshot.ID_INTEGER_COMPARATOR.compare(this.snapshot.getId(), that);
+  }
+
+  /** @return the snapshot object of this diff. */
+  public final Snapshot getSnapshot() {
+    return snapshot;
+  }
+  
+  final void setSnapshot(Snapshot snapshot) {
+    this.snapshot = snapshot;
+  }
+
+  /** @return the posterior diff. */
+  final D getPosterior() {
+    return posteriorDiff;
+  }
+
+  /** @return the posterior diff. */
+  final void setPosterior(D posterior) {
+    posteriorDiff = posterior;
+  }
+
+  /** Save the INode state to the snapshot if it is not done already. */
+  void saveSnapshotCopy(N snapshotCopy, N currentINode) {
+    Preconditions.checkState(snapshotINode == null, "Expected snapshotINode to be null");
+    snapshotINode = snapshotCopy;
+  }
+
+  /** @return the inode corresponding to the snapshot. */
+  N getSnapshotINode() {
+    // get from this diff, then the posterior diff
+    // and then null for 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 null;
+      }
+    }
+  }
+
+  /** Combine the posterior diff and collect blocks for deletion. */
+  abstract Quota.Counts combinePosteriorAndCollectBlocks(final N currentINode,
+      final D posterior, final BlocksMapUpdateInfo collectedBlocks,
+      final List<INode> removedINodes);
+  
+  /**
+   * Delete and clear self.
+   * @param currentINode The inode where the deletion happens.
+   * @param collectedBlocks Used to collect blocks for deletion.
+   * @return quota usage delta
+   */
+  abstract Quota.Counts destroyDiffAndCollectBlocks(final N currentINode,
+      final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes);
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + ": " + snapshot + " (post="
+        + (posteriorDiff == null? null: posteriorDiff.snapshot) + ")";
+  }
+
+  void writeSnapshot(DataOutput out) throws IOException {
+    // Assume the snapshot is recorded before, write id only.
+    out.writeInt(snapshot.getId());
+  }
+  
+  abstract void write(DataOutput out, ReferenceMap referenceMap
+      ) throws IOException;
+}

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

@@ -0,0 +1,318 @@
+/**
+ * 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.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
+import org.apache.hadoop.hdfs.server.namenode.Quota;
+
+/**
+ * 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}. */
+  public final List<D> asList() {
+    return Collections.unmodifiableList(diffs);
+  }
+  
+  /** Get the size of the list and then clear it. */
+  public void clear() {
+    diffs.clear();
+  }
+
+  /** @return an {@link AbstractINodeDiff}. */
+  abstract D createDiff(Snapshot snapshot, N currentINode);
+
+  /** @return a snapshot copy of the current inode. */  
+  abstract N createSnapshotCopy(N currentINode);
+
+  /**
+   * Delete a snapshot. 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 prior The snapshot taken before the to-be-deleted snapshot
+   * @param collectedBlocks Used to collect information for blocksMap update
+   * @return delta in namespace. 
+   */
+  public final Quota.Counts deleteSnapshotDiff(final Snapshot snapshot,
+      Snapshot prior, final N currentINode,
+      final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
+      throws QuotaExceededException {
+    int snapshotIndex = Collections.binarySearch(diffs, snapshot.getId());
+    
+    Quota.Counts counts = Quota.Counts.newInstance();
+    D removed = null;
+    if (snapshotIndex == 0) {
+      if (prior != null) {
+        // set the snapshot to latestBefore
+        diffs.get(snapshotIndex).setSnapshot(prior);
+      } else {
+        removed = diffs.remove(0);
+        counts.add(Quota.NAMESPACE, 1);
+        // We add 1 to the namespace quota usage since we delete a diff. 
+        // The quota change will be propagated to 
+        // 1) ancestors in the current tree, and 
+        // 2) src tree of any renamed ancestor.
+        // Because for 2) we do not calculate the number of diff for quota 
+        // usage, we need to compensate this diff change for 2)
+        currentINode.addSpaceConsumedToRenameSrc(1, 0, false, snapshot.getId());
+        counts.add(removed.destroyDiffAndCollectBlocks(currentINode,
+            collectedBlocks, removedINodes));
+      }
+    } else if (snapshotIndex > 0) {
+      final AbstractINodeDiff<N, D> previous = diffs.get(snapshotIndex - 1);
+      if (!previous.getSnapshot().equals(prior)) {
+        diffs.get(snapshotIndex).setSnapshot(prior);
+      } else {
+        // combine the to-be-removed diff with its previous diff
+        removed = diffs.remove(snapshotIndex);
+        counts.add(Quota.NAMESPACE, 1);
+        currentINode.addSpaceConsumedToRenameSrc(1, 0, false, snapshot.getId());
+        if (previous.snapshotINode == null) {
+          previous.snapshotINode = removed.snapshotINode;
+        } else if (removed.snapshotINode != null) {
+          removed.snapshotINode.clear();
+        }
+        counts.add(previous.combinePosteriorAndCollectBlocks(
+            currentINode, removed, collectedBlocks, removedINodes));
+        previous.setPosterior(removed.getPosterior());
+        removed.setPosterior(null);
+      }
+    }
+    return counts;
+  }
+
+  /** Add an {@link AbstractINodeDiff} for the given snapshot. */
+  final D addDiff(Snapshot latest, N currentINode)
+      throws QuotaExceededException {
+    currentINode.addSpaceConsumed(1, 0, true, Snapshot.INVALID_ID);
+    return addLast(createDiff(latest, currentINode));
+  }
+
+  /** Append the diff at the end of the list. */
+  private final D addLast(D diff) {
+    final D last = getLast();
+    diffs.add(diff);
+    if (last != null) {
+      last.setPosterior(diff);
+    }
+    return diff;
+  }
+  
+  /** Add the diff to the beginning of the list. */
+  final void addFirst(D diff) {
+    final D first = diffs.isEmpty()? null: diffs.get(0);
+    diffs.add(0, diff);
+    diff.setPosterior(first);
+  }
+
+  /** @return the last diff. */
+  public final D getLast() {
+    final int n = diffs.size();
+    return n == 0? null: diffs.get(n - 1);
+  }
+
+  /** @return the last snapshot. */
+  public final Snapshot getLastSnapshot() {
+    final AbstractINodeDiff<N, D> last = getLast();
+    return last == null? null: last.getSnapshot();
+  }
+  
+  /**
+   * Find the latest snapshot before a given snapshot.
+   * @param anchorId The returned snapshot's id must be <= or < this given 
+   *                 snapshot id.
+   * @param exclusive True means the returned snapshot's id must be < the given
+   *                  id, otherwise <=.
+   * @return The latest snapshot before the given snapshot.
+   */
+  private final Snapshot getPrior(int anchorId, boolean exclusive) {
+    if (anchorId == Snapshot.INVALID_ID) {
+      return getLastSnapshot();
+    }
+    final int i = Collections.binarySearch(diffs, anchorId);
+    if (exclusive) { // must be the one before
+      if (i == -1 || i == 0) {
+        return null;
+      } else {
+        int priorIndex = i > 0 ? i - 1 : -i - 2;
+        return diffs.get(priorIndex).getSnapshot();
+      }
+    } else { // the one, or the one before if not existing
+      if (i >= 0) {
+        return diffs.get(i).getSnapshot();
+      } else if (i < -1) {
+        return diffs.get(-i - 2).getSnapshot();
+      } else { // i == -1
+        return null;
+      }
+    }
+  }
+  
+  public final Snapshot getPrior(int snapshotId) {
+    return getPrior(snapshotId, false);
+  }
+  
+  /**
+   * Update the prior snapshot.
+   */
+  final Snapshot updatePrior(Snapshot snapshot, Snapshot prior) {
+    int id = snapshot == null ? Snapshot.INVALID_ID : snapshot.getId();
+    Snapshot s = getPrior(id, true);
+    if (s != null && 
+        (prior == null || Snapshot.ID_COMPARATOR.compare(s, prior) > 0)) {
+      return s;
+    }
+    return prior;
+  }
+
+  /**
+   * @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. 
+   */
+  public final D getDiff(Snapshot snapshot) {
+    return getDiffById(snapshot == null ? 
+        Snapshot.INVALID_ID : snapshot.getId());
+  }
+  
+  private final D getDiffById(final int snapshotId) {
+    if (snapshotId == Snapshot.INVALID_ID) {
+      return null;
+    }
+    final int i = Collections.binarySearch(diffs, snapshotId);
+    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;
+    }
+  }
+  
+  /**
+   * Search for the snapshot whose id is 1) no less than the given id, 
+   * and 2) most close to the given id.
+   */
+  public final Snapshot getSnapshotById(final int snapshotId) {
+    D diff = getDiffById(snapshotId);
+    return diff == null ? null : diff.getSnapshot();
+  }
+  
+  /**
+   * Check if changes have happened between two snapshots.
+   * @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 earlier, Snapshot later) {
+    final int size = diffs.size();
+    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 (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
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * @return the inode corresponding to the given snapshot.
+   *         Note that the current inode is returned if there is no change
+   *         between the given snapshot and the current state. 
+   */
+  N getSnapshotINode(final Snapshot snapshot, final N currentINode) {
+    final D diff = getDiff(snapshot);
+    final N inode = diff == null? null: diff.getSnapshotINode();
+    return inode == null? currentINode: inode;
+  }
+
+  /**
+   * 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, N currentINode)
+      throws QuotaExceededException {
+    final D last = getLast();
+    if (last != null
+        && Snapshot.ID_COMPARATOR.compare(last.getSnapshot(), latest) >= 0) {
+      return last;
+    } else {
+      try {
+        return addDiff(latest, currentINode);
+      } catch(NSQuotaExceededException e) {
+        e.setMessagePrefix("Failed to record modification for snapshot");
+        throw e;
+      }
+    }
+  }
+
+  /** Save the snapshot copy to the latest snapshot. */
+  public void saveSelf2Snapshot(Snapshot latest, N currentINode, N snapshotCopy)
+      throws QuotaExceededException {
+    if (latest != null) {
+      D diff = checkAndAddLatestSnapshotDiff(latest, currentINode);
+      if (diff.snapshotINode == null) {
+        if (snapshotCopy == null) {
+          snapshotCopy = createSnapshotCopy(currentINode);
+        }
+        diff.saveSnapshotCopy(snapshotCopy, currentINode);
+      }
+    }
+  }
+  
+  @Override
+  public Iterator<D> iterator() {
+    return diffs.iterator();
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + ": " + diffs;
+  }
+}

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

@@ -0,0 +1,57 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiff;
+
+/**
+ * A list of FileDiffs for storing snapshot data.
+ */
+@InterfaceAudience.Private
+public class FileDiffList
+    extends AbstractINodeDiffList<INodeFile, FileDiff> {
+
+  
+  @Override
+  FileDiff createDiff(Snapshot snapshot, INodeFile file) {
+    return new FileDiff(snapshot, file);
+  }
+
+  @Override
+  INodeFile createSnapshotCopy(INodeFile currentINode) {
+    if (currentINode instanceof INodeFileUnderConstructionWithSnapshot) {
+      final INodeFileUnderConstruction uc = 
+          (INodeFileUnderConstruction) currentINode;
+      
+      final INodeFileUnderConstruction copy = new INodeFileUnderConstruction(
+          uc, uc.getClientName(), uc.getClientMachine(), uc.getClientNode());
+      
+      copy.setBlocks(null);
+      return copy;
+    } else {
+      final INodeFile copy = new INodeFile(currentINode);
+      copy.setBlocks(null);
+      return copy;
+    }
+  }
+}

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

@@ -0,0 +1,213 @@
+/**
+ * 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.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
+import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
+
+/**
+ * {@link INodeFile} with a link to the next element.
+ * The link of all the snapshot files and the original file form a circular
+ * linked list so that all elements are accessible by any of the elements.
+ */
+@InterfaceAudience.Private
+public interface FileWithSnapshot {
+  /**
+   * The difference of an {@link INodeFile} between two snapshots.
+   */
+  public static class FileDiff extends AbstractINodeDiff<INodeFile, FileDiff> {
+    /** The file size at snapshot creation time. */
+    private final long fileSize;
+
+    FileDiff(Snapshot snapshot, INodeFile file) {
+      super(snapshot, null, null);
+      fileSize = file.computeFileSize();
+    }
+
+    /** Constructor used by FSImage loading */
+    FileDiff(Snapshot snapshot, INodeFile snapshotINode,
+        FileDiff posteriorDiff, long fileSize) {
+      super(snapshot, snapshotINode, posteriorDiff);
+      this.fileSize = fileSize;
+    }
+
+    /** @return the file size in the snapshot. */
+    public long getFileSize() {
+      return fileSize;
+    }
+
+    private static Quota.Counts updateQuotaAndCollectBlocks(
+        INodeFile currentINode, FileDiff removed,
+        BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
+      FileWithSnapshot sFile = (FileWithSnapshot) currentINode;
+      long oldDiskspace = currentINode.diskspaceConsumed();
+      if (removed.snapshotINode != null) {
+        short replication = removed.snapshotINode.getFileReplication();
+        short currentRepl = currentINode.getBlockReplication();
+        if (currentRepl == 0) {
+          oldDiskspace = currentINode.computeFileSize(true, true) * replication;
+        } else if (replication > currentRepl) {  
+          oldDiskspace = oldDiskspace / currentINode.getBlockReplication()
+              * replication;
+        }
+      }
+      
+      Util.collectBlocksAndClear(sFile, collectedBlocks, removedINodes);
+      
+      long dsDelta = oldDiskspace - currentINode.diskspaceConsumed();
+      return Quota.Counts.newInstance(0, dsDelta);
+    }
+    
+    @Override
+    Quota.Counts combinePosteriorAndCollectBlocks(INodeFile currentINode,
+        FileDiff posterior, BlocksMapUpdateInfo collectedBlocks,
+        final List<INode> removedINodes) {
+      return updateQuotaAndCollectBlocks(currentINode, posterior,
+          collectedBlocks, removedINodes);
+    }
+    
+    @Override
+    public String toString() {
+      return super.toString() + " fileSize=" + fileSize + ", rep="
+          + (snapshotINode == null? "?": snapshotINode.getFileReplication());
+    }
+
+    @Override
+    void write(DataOutput out, ReferenceMap referenceMap) throws IOException {
+      writeSnapshot(out);
+      out.writeLong(fileSize);
+
+      // write snapshotINode
+      if (snapshotINode != null) {
+        out.writeBoolean(true);
+        FSImageSerialization.writeINodeFile(snapshotINode, out, true);
+      } else {
+        out.writeBoolean(false);
+      }
+    }
+
+    @Override
+    Quota.Counts destroyDiffAndCollectBlocks(INodeFile currentINode,
+        BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
+      return updateQuotaAndCollectBlocks(currentINode, this,
+          collectedBlocks, removedINodes);
+    }
+  }
+
+  /** @return the {@link INodeFile} view of this object. */
+  public INodeFile asINodeFile();
+
+  /** @return the file diff list. */
+  public FileDiffList getDiffs();
+
+  /** Is the current file deleted? */
+  public boolean isCurrentFileDeleted();
+  
+  /** Delete the file from the current tree */
+  public void deleteCurrentFile();
+
+  /** Utility methods for the classes which implement the interface. */
+  public static class Util {
+    /** 
+     * @return block replication, which is the max file replication among
+     *         the file and the diff list.
+     */
+    public static short getBlockReplication(final FileWithSnapshot file) {
+      short max = file.isCurrentFileDeleted()? 0
+          : file.asINodeFile().getFileReplication();
+      for(FileDiff d : file.getDiffs()) {
+        if (d.snapshotINode != null) {
+          final short replication = d.snapshotINode.getFileReplication();
+          if (replication > max) {
+            max = replication;
+          }
+        }
+      }
+      return max;
+    }
+
+    /**
+     * If some blocks at the end of the block list no longer belongs to
+     * any inode, collect them and update the block list.
+     */
+    static void collectBlocksAndClear(final FileWithSnapshot file,
+        final BlocksMapUpdateInfo info, final List<INode> removedINodes) {
+      // check if everything is deleted.
+      if (file.isCurrentFileDeleted()
+          && file.getDiffs().asList().isEmpty()) {
+        file.asINodeFile().destroyAndCollectBlocks(info, removedINodes);
+        return;
+      }
+
+      // find max file size.
+      final long max;
+      if (file.isCurrentFileDeleted()) {
+        final FileDiff last = file.getDiffs().getLast();
+        max = last == null? 0: last.fileSize;
+      } else { 
+        max = file.asINodeFile().computeFileSize();
+      }
+
+      collectBlocksBeyondMax(file, max, info);
+    }
+
+    private static void collectBlocksBeyondMax(final FileWithSnapshot file,
+        final long max, final BlocksMapUpdateInfo collectedBlocks) {
+      final BlockInfo[] oldBlocks = file.asINodeFile().getBlocks();
+      if (oldBlocks != null) {
+        //find the minimum n such that the size of the first n blocks > max
+        int n = 0;
+        for(long size = 0; n < oldBlocks.length && max > size; n++) {
+          size += oldBlocks[n].getNumBytes();
+        }
+        
+        // starting from block n, the data is beyond max.
+        if (n < oldBlocks.length) {
+          // resize the array.  
+          final BlockInfo[] newBlocks;
+          if (n == 0) {
+            newBlocks = null;
+          } else {
+            newBlocks = new BlockInfo[n];
+            System.arraycopy(oldBlocks, 0, newBlocks, 0, n);
+          }
+          
+          // set new blocks
+          file.asINodeFile().setBlocks(newBlocks);
+
+          // collect the blocks beyond max.  
+          if (collectedBlocks != null) {
+            for(; n < oldBlocks.length; n++) {
+              collectedBlocks.addDeleteBlock(oldBlocks[n]);
+            }
+          }
+        }
+      }
+    }
+  }
+}

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

@@ -0,0 +1,529 @@
+/**
+ * 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.io.IOException;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
+import org.apache.hadoop.hdfs.server.namenode.Content;
+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.INodeMap;
+import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.util.Diff.ListType;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
+import org.apache.hadoop.util.Time;
+
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.SignedBytes;
+
+/**
+ * Directories where taking snapshots is allowed.
+ * 
+ * Like other {@link INode} subclasses, this class is synchronized externally
+ * by the namesystem and FSDirectory locks.
+ */
+@InterfaceAudience.Private
+public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
+  /** Limit the number of snapshot per snapshottable directory. */
+  static final int SNAPSHOT_LIMIT = 1 << 16;
+
+  /** Cast INode to INodeDirectorySnapshottable. */
+  static public INodeDirectorySnapshottable valueOf(
+      INode inode, String src) throws IOException {
+    final INodeDirectory dir = INodeDirectory.valueOf(inode, src);
+    if (!dir.isSnapshottable()) {
+      throw new SnapshotException(
+          "Directory is not a snapshottable directory: " + src);
+    }
+    return (INodeDirectorySnapshottable)dir;
+  }
+  
+  /**
+   * A class describing the difference between snapshots of a snapshottable
+   * directory.
+   */
+  public static class SnapshotDiffInfo {
+    /** Compare two inodes based on their full names */
+    public static final Comparator<INode> INODE_COMPARATOR = 
+        new Comparator<INode>() {
+      @Override
+      public int compare(INode left, INode right) {
+        if (left == null) {
+          return right == null ? 0 : -1;
+        } else {
+          if (right == null) {
+            return 1;
+          } else {
+            int cmp = compare(left.getParent(), right.getParent());
+            return cmp == 0 ? SignedBytes.lexicographicalComparator().compare(
+                left.getLocalNameBytes(), right.getLocalNameBytes()) : cmp;
+          }
+        }
+      }
+    };
+    
+    /** The root directory of the snapshots */
+    private final INodeDirectorySnapshottable snapshotRoot;
+    /** The starting point of the difference */
+    private final Snapshot from;
+    /** The end point of the difference */
+    private final Snapshot to;
+    /**
+     * A map recording modified INodeFile and INodeDirectory and their relative
+     * path corresponding to the snapshot root. Sorted based on their names.
+     */ 
+    private final SortedMap<INode, byte[][]> diffMap = 
+        new TreeMap<INode, byte[][]>(INODE_COMPARATOR);
+    /**
+     * A map capturing the detailed difference about file creation/deletion.
+     * Each key indicates a directory whose children have been changed between
+     * the two snapshots, while its associated value is a {@link ChildrenDiff}
+     * storing the changes (creation/deletion) happened to the children (files).
+     */
+    private final Map<INodeDirectoryWithSnapshot, ChildrenDiff> dirDiffMap = 
+        new HashMap<INodeDirectoryWithSnapshot, ChildrenDiff>();
+    
+    SnapshotDiffInfo(INodeDirectorySnapshottable snapshotRoot, Snapshot start,
+        Snapshot end) {
+      this.snapshotRoot = snapshotRoot;
+      this.from = start;
+      this.to = end;
+    }
+    
+    /** Add a dir-diff pair */
+    private void addDirDiff(INodeDirectoryWithSnapshot dir,
+        byte[][] relativePath, ChildrenDiff diff) {
+      dirDiffMap.put(dir, diff);
+      diffMap.put(dir, relativePath);
+    }
+    
+    /** Add a modified file */ 
+    private void addFileDiff(INodeFile file, byte[][] relativePath) {
+      diffMap.put(file, relativePath);
+    }
+    
+    /** @return True if {@link #from} is earlier than {@link #to} */
+    private boolean isFromEarlier() {
+      return Snapshot.ID_COMPARATOR.compare(from, to) < 0;
+    }
+    
+    /**
+     * Generate a {@link SnapshotDiffReport} based on detailed diff information.
+     * @return A {@link SnapshotDiffReport} describing the difference
+     */
+    public SnapshotDiffReport generateReport() {
+      List<DiffReportEntry> diffReportList = new ArrayList<DiffReportEntry>();
+      for (INode node : diffMap.keySet()) {
+        diffReportList.add(new DiffReportEntry(DiffType.MODIFY, diffMap
+            .get(node)));
+        if (node.isDirectory()) {
+          ChildrenDiff dirDiff = dirDiffMap.get(node);
+          List<DiffReportEntry> subList = dirDiff.generateReport(
+              diffMap.get(node), (INodeDirectoryWithSnapshot) node,
+              isFromEarlier());
+          diffReportList.addAll(subList);
+        }
+      }
+      return new SnapshotDiffReport(snapshotRoot.getFullPathName(),
+          Snapshot.getSnapshotName(from), Snapshot.getSnapshotName(to),
+          diffReportList);
+    }
+  }
+
+  /**
+   * Snapshots of this directory in ascending order of snapshot names.
+   * Note that snapshots in ascending order of snapshot id are stored in
+   * {@link INodeDirectoryWithSnapshot}.diffs (a private field).
+   */
+  private final List<Snapshot> snapshotsByNames = new ArrayList<Snapshot>();
+
+  /**
+   * @return {@link #snapshotsByNames}
+   */
+  ReadOnlyList<Snapshot> getSnapshotsByNames() {
+    return ReadOnlyList.Util.asReadOnlyList(this.snapshotsByNames);
+  }
+  
+  /** Number of snapshots allowed. */
+  private int snapshotQuota = SNAPSHOT_LIMIT;
+
+  public INodeDirectorySnapshottable(INodeDirectory dir) {
+    super(dir, true, dir instanceof INodeDirectoryWithSnapshot ? 
+        ((INodeDirectoryWithSnapshot) dir).getDiffs(): null);
+  }
+  
+  /** @return the number of existing snapshots. */
+  public int getNumSnapshots() {
+    return snapshotsByNames.size();
+  }
+  
+  private int searchSnapshot(byte[] snapshotName) {
+    return Collections.binarySearch(snapshotsByNames, snapshotName);
+  }
+
+  /** @return the snapshot with the given name. */
+  public Snapshot getSnapshot(byte[] snapshotName) {
+    final int i = searchSnapshot(snapshotName);
+    return i < 0? null: snapshotsByNames.get(i);
+  }
+  
+  /** @return {@link #snapshotsByNames} as a {@link ReadOnlyList} */
+  public ReadOnlyList<Snapshot> getSnapshotList() {
+    return ReadOnlyList.Util.asReadOnlyList(snapshotsByNames);
+  }
+  
+  /**
+   * Rename a snapshot
+   * @param path
+   *          The directory path where the snapshot was taken. Used for
+   *          generating exception message.
+   * @param oldName
+   *          Old name of the snapshot
+   * @param newName
+   *          New name the snapshot will be renamed to
+   * @throws SnapshotException
+   *           Throw SnapshotException when either the snapshot with the old
+   *           name does not exist or a snapshot with the new name already
+   *           exists
+   */
+  public void renameSnapshot(String path, String oldName, String newName)
+      throws SnapshotException {
+    if (newName.equals(oldName)) {
+      return;
+    }
+    final int indexOfOld = searchSnapshot(DFSUtil.string2Bytes(oldName));
+    if (indexOfOld < 0) {
+      throw new SnapshotException("The snapshot " + oldName
+          + " does not exist for directory " + path);
+    } else {
+      final byte[] newNameBytes = DFSUtil.string2Bytes(newName);
+      int indexOfNew = searchSnapshot(newNameBytes);
+      if (indexOfNew > 0) {
+        throw new SnapshotException("The snapshot " + newName
+            + " already exists for directory " + path);
+      }
+      // remove the one with old name from snapshotsByNames
+      Snapshot snapshot = snapshotsByNames.remove(indexOfOld);
+      final INodeDirectory ssRoot = snapshot.getRoot();
+      ssRoot.setLocalName(newNameBytes);
+      indexOfNew = -indexOfNew - 1;
+      if (indexOfNew <= indexOfOld) {
+        snapshotsByNames.add(indexOfNew, snapshot);
+      } else { // indexOfNew > indexOfOld
+        snapshotsByNames.add(indexOfNew - 1, snapshot);
+      }
+    }
+  }
+
+  public int getSnapshotQuota() {
+    return snapshotQuota;
+  }
+
+  public void setSnapshotQuota(int snapshotQuota) {
+    if (snapshotQuota < 0) {
+      throw new HadoopIllegalArgumentException(
+          "Cannot set snapshot quota to " + snapshotQuota + " < 0");
+    }
+    this.snapshotQuota = snapshotQuota;
+  }
+
+  @Override
+  public boolean isSnapshottable() {
+    return true;
+  }
+  
+  /**
+   * Simply add a snapshot into the {@link #snapshotsByNames}. Used by FSImage
+   * loading.
+   */
+  void addSnapshot(Snapshot snapshot) {
+    this.snapshotsByNames.add(snapshot);
+  }
+
+  /** Add a snapshot. */
+  Snapshot addSnapshot(int id, String name) throws SnapshotException,
+      QuotaExceededException {
+    //check snapshot quota
+    final int n = getNumSnapshots();
+    if (n + 1 > snapshotQuota) {
+      throw new SnapshotException("Failed to add snapshot: there are already "
+          + n + " snapshot(s) and the snapshot quota is "
+          + snapshotQuota);
+    }
+    final Snapshot s = new Snapshot(id, name, this);
+    final byte[] nameBytes = s.getRoot().getLocalNameBytes();
+    final int i = searchSnapshot(nameBytes);
+    if (i >= 0) {
+      throw new SnapshotException("Failed to add snapshot: there is already a "
+          + "snapshot with the same name \"" + Snapshot.getSnapshotName(s) + "\".");
+    }
+
+    final DirectoryDiff d = getDiffs().addDiff(s, this);
+    d.snapshotINode = s.getRoot();
+    snapshotsByNames.add(-i - 1, s);
+
+    //set modification time
+    updateModificationTime(Time.now(), null, null);
+    s.getRoot().setModificationTime(getModificationTime(), null, null);
+    return s;
+  }
+  
+  /**
+   * Remove the snapshot with the given name from {@link #snapshotsByNames},
+   * 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
+   * @return The removed snapshot. Null if no snapshot with the given name 
+   *         exists.
+   */
+  Snapshot removeSnapshot(String snapshotName,
+      BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
+      throws SnapshotException {
+    final int i = searchSnapshot(DFSUtil.string2Bytes(snapshotName));
+    if (i < 0) {
+      throw new SnapshotException("Cannot delete snapshot " + snapshotName
+          + " from path " + this.getFullPathName()
+          + ": the snapshot does not exist.");
+    } else {
+      final Snapshot snapshot = snapshotsByNames.remove(i);
+      Snapshot prior = Snapshot.findLatestSnapshot(this, snapshot);
+      try {
+        Quota.Counts counts = cleanSubtree(snapshot, prior, collectedBlocks,
+            removedINodes);
+        INodeDirectory parent = getParent();
+        if (parent != null) {
+          // there will not be any WithName node corresponding to the deleted 
+          // snapshot, thus only update the quota usage in the current tree
+          parent.addSpaceConsumed(-counts.get(Quota.NAMESPACE),
+              -counts.get(Quota.DISKSPACE), true, Snapshot.INVALID_ID);
+        }
+      } catch(QuotaExceededException e) {
+        LOG.error("BUG: removeSnapshot increases namespace usage.", e);
+      }
+      return snapshot;
+    }
+  }
+  
+  @Override
+  public Content.Counts computeContentSummary(final Content.Counts counts) {
+    super.computeContentSummary(counts);
+    counts.add(Content.SNAPSHOT, snapshotsByNames.size());
+    counts.add(Content.SNAPSHOTTABLE_DIRECTORY, 1);
+    return counts;
+  }
+
+  /**
+   * Compute the difference between two snapshots (or a snapshot and the current
+   * directory) of the directory.
+   * 
+   * @param from The name of the start point of the comparison. Null indicating
+   *          the current tree.
+   * @param to The name of the end point. Null indicating the current tree.
+   * @return The difference between the start/end points.
+   * @throws SnapshotException If there is no snapshot matching the starting
+   *           point, or if endSnapshotName is not null but cannot be identified
+   *           as a previous snapshot.
+   */
+  SnapshotDiffInfo computeDiff(final String from, final String to)
+      throws SnapshotException {
+    Snapshot fromSnapshot = getSnapshotByName(from);
+    Snapshot toSnapshot = getSnapshotByName(to);
+    // if the start point is equal to the end point, return null
+    if (from.equals(to)) {
+      return null;
+    }
+    SnapshotDiffInfo diffs = new SnapshotDiffInfo(this, fromSnapshot,
+        toSnapshot);
+    computeDiffRecursively(this, new ArrayList<byte[]>(), diffs);
+    return diffs;
+  }
+  
+  /**
+   * Find the snapshot matching the given name.
+   * 
+   * @param snapshotName The name of the snapshot.
+   * @return The corresponding snapshot. Null if snapshotName is null or empty.
+   * @throws SnapshotException If snapshotName is not null or empty, but there
+   *           is no snapshot matching the name.
+   */
+  private Snapshot getSnapshotByName(String snapshotName)
+      throws SnapshotException {
+    Snapshot s = null;
+    if (snapshotName != null && !snapshotName.isEmpty()) {
+      final int index = searchSnapshot(DFSUtil.string2Bytes(snapshotName));
+      if (index < 0) {
+        throw new SnapshotException("Cannot find the snapshot of directory "
+            + this.getFullPathName() + " with name " + snapshotName);
+      }
+      s = snapshotsByNames.get(index);
+    }
+    return s;
+  }
+  
+  /**
+   * Recursively compute the difference between snapshots under a given
+   * directory/file.
+   * @param node The directory/file under which the diff is computed. 
+   * @param parentPath Relative path (corresponding to the snapshot root) of 
+   *                   the node's parent.
+   * @param diffReport data structure used to store the diff.
+   */
+  private void computeDiffRecursively(INode node, List<byte[]> parentPath,
+      SnapshotDiffInfo diffReport) {
+    ChildrenDiff diff = new ChildrenDiff();
+    byte[][] relativePath = parentPath.toArray(new byte[parentPath.size()][]);
+    if (node.isDirectory()) {
+      INodeDirectory dir = node.asDirectory();
+      if (dir instanceof INodeDirectoryWithSnapshot) {
+        INodeDirectoryWithSnapshot sdir = (INodeDirectoryWithSnapshot) dir;
+        boolean change = sdir.computeDiffBetweenSnapshots(
+            diffReport.from, diffReport.to, diff);
+        if (change) {
+          diffReport.addDirDiff(sdir, relativePath, diff);
+        }
+      }
+      ReadOnlyList<INode> children = dir.getChildrenList(diffReport
+          .isFromEarlier() ? diffReport.to : diffReport.from);
+      for (INode child : children) {
+        final byte[] name = child.getLocalNameBytes();
+        if (diff.searchIndex(ListType.CREATED, name) < 0
+            && diff.searchIndex(ListType.DELETED, name) < 0) {
+          parentPath.add(name);
+          computeDiffRecursively(child, parentPath, diffReport);
+          parentPath.remove(parentPath.size() - 1);
+        }
+      }
+    } else if (node.isFile() && node.asFile() instanceof FileWithSnapshot) {
+      FileWithSnapshot file = (FileWithSnapshot) node.asFile();
+      Snapshot earlierSnapshot = diffReport.isFromEarlier() ? diffReport.from
+          : diffReport.to;
+      Snapshot laterSnapshot = diffReport.isFromEarlier() ? diffReport.to
+          : diffReport.from;
+      boolean change = file.getDiffs().changedBetweenSnapshots(earlierSnapshot,
+          laterSnapshot);
+      if (change) {
+        diffReport.addFileDiff(file.asINodeFile(), relativePath);
+      }
+    }
+  }
+  
+  /**
+   * Replace itself with {@link INodeDirectoryWithSnapshot} or
+   * {@link INodeDirectory} depending on the latest snapshot.
+   */
+  INodeDirectory replaceSelf(final Snapshot latest, final INodeMap inodeMap)
+      throws QuotaExceededException {
+    if (latest == null) {
+      Preconditions.checkState(getLastSnapshot() == null,
+          "latest == null but getLastSnapshot() != null, this=%s", this);
+      return replaceSelf4INodeDirectory(inodeMap);
+    } else {
+      return replaceSelf4INodeDirectoryWithSnapshot(inodeMap)
+          .recordModification(latest, null);
+    }
+  }
+
+  @Override
+  public String toDetailString() {
+    return super.toDetailString() + ", snapshotsByNames=" + snapshotsByNames;
+  }
+
+  @Override
+  public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix,
+      Snapshot snapshot) {
+    super.dumpTreeRecursively(out, prefix, snapshot);
+
+    if (snapshot == null) {
+      out.println();
+      out.print(prefix);
+
+      out.print("Snapshot of ");
+      final String name = getLocalName();
+      out.print(name.isEmpty()? "/": name);
+      out.print(": quota=");
+      out.print(getSnapshotQuota());
+
+      int n = 0;
+      for(DirectoryDiff diff : getDiffs()) {
+        if (diff.isSnapshotRoot()) {
+          n++;
+        }
+      }
+      Preconditions.checkState(n == snapshotsByNames.size());
+      out.print(", #snapshot=");
+      out.println(n);
+
+      dumpTreeRecursively(out, prefix, new Iterable<SnapshotAndINode>() {
+        @Override
+        public Iterator<SnapshotAndINode> iterator() {
+          return new Iterator<SnapshotAndINode>() {
+            final Iterator<DirectoryDiff> i = getDiffs().iterator();
+            private DirectoryDiff next = findNext();
+  
+            private DirectoryDiff findNext() {
+              for(; i.hasNext(); ) {
+                final DirectoryDiff diff = i.next();
+                if (diff.isSnapshotRoot()) {
+                  return diff;
+                }
+              }
+              return null;
+            }
+
+            @Override
+            public boolean hasNext() {
+              return next != null;
+            }
+  
+            @Override
+            public SnapshotAndINode next() {
+              final Snapshot s = next.snapshot;
+              final SnapshotAndINode pair = new SnapshotAndINode(s);
+              next = findNext();
+              return pair;
+            }
+  
+            @Override
+            public void remove() {
+              throw new UnsupportedOperationException();
+            }
+          };
+        }
+      });
+    }
+  }
+}

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

@@ -0,0 +1,895 @@
+/**
+ * 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.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
+import org.apache.hadoop.hdfs.server.namenode.Content;
+import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryWithQuota;
+import org.apache.hadoop.hdfs.server.namenode.INodeMap;
+import org.apache.hadoop.hdfs.server.namenode.INodeReference;
+import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
+import org.apache.hadoop.hdfs.util.Diff;
+import org.apache.hadoop.hdfs.util.Diff.Container;
+import org.apache.hadoop.hdfs.util.Diff.ListType;
+import org.apache.hadoop.hdfs.util.Diff.UndoInfo;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * The directory with snapshots. It maintains a list of snapshot diffs for
+ * storing snapshot data. When there are modifications to the directory, the old
+ * data is stored in the latest snapshot, if there is any.
+ */
+public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
+  /**
+   * The difference between the current state and a previous snapshot
+   * of the children list of an INodeDirectory.
+   */
+  static class ChildrenDiff extends Diff<byte[], INode> {
+    ChildrenDiff() {}
+    
+    private ChildrenDiff(final List<INode> created, final List<INode> deleted) {
+      super(created, deleted);
+    }
+
+    /**
+     * Replace the given child from the created/deleted list.
+     * @return true if the child is replaced; false if the child is not found.
+     */
+    private final boolean replace(final ListType type,
+        final INode oldChild, final INode newChild) {
+      final List<INode> list = getList(type); 
+      final int i = search(list, oldChild.getLocalNameBytes());
+      if (i < 0) {
+        return false;
+      }
+
+      final INode removed = list.set(i, newChild);
+      Preconditions.checkState(removed == oldChild);
+      return true;
+    }
+
+    private final boolean removeChild(ListType type, final INode child) {
+      final List<INode> list = getList(type);
+      final int i = searchIndex(type, child.getLocalNameBytes());
+      if (i >= 0 && list.get(i) == child) {
+        list.remove(i);
+        return true;
+      }
+      return false;
+    }
+    
+    /** clear the created list */
+    private Quota.Counts destroyCreatedList(
+        final INodeDirectoryWithSnapshot currentINode,
+        final BlocksMapUpdateInfo collectedBlocks,
+        final List<INode> removedINodes) {
+      Quota.Counts counts = Quota.Counts.newInstance();
+      final List<INode> createdList = getList(ListType.CREATED);
+      for (INode c : createdList) {
+        c.computeQuotaUsage(counts, true);
+        c.destroyAndCollectBlocks(collectedBlocks, removedINodes);
+        // c should be contained in the children list, remove it
+        currentINode.removeChild(c);
+      }
+      createdList.clear();
+      return counts;
+    }
+    
+    /** clear the deleted list */
+    private Quota.Counts destroyDeletedList(
+        final BlocksMapUpdateInfo collectedBlocks,
+        final List<INode> removedINodes) {
+      Quota.Counts counts = Quota.Counts.newInstance();
+      final List<INode> deletedList = getList(ListType.DELETED);
+      for (INode d : deletedList) {
+        d.computeQuotaUsage(counts, false);
+        d.destroyAndCollectBlocks(collectedBlocks, removedINodes);
+      }
+      deletedList.clear();
+      return counts;
+    }
+    
+    /** Serialize {@link #created} */
+    private void writeCreated(DataOutput out) throws IOException {
+      final List<INode> created = getList(ListType.CREATED);
+      out.writeInt(created.size());
+      for (INode node : created) {
+        // For INode in created list, we only need to record its local name 
+        byte[] name = node.getLocalNameBytes();
+        out.writeShort(name.length);
+        out.write(name);
+      }
+    }
+    
+    /** Serialize {@link #deleted} */
+    private void writeDeleted(DataOutput out,
+        ReferenceMap referenceMap) throws IOException {
+      final List<INode> deleted = getList(ListType.DELETED);
+      out.writeInt(deleted.size());
+      for (INode node : deleted) {
+        FSImageSerialization.saveINode2Image(node, out, true, referenceMap);
+      }
+    }
+    
+    /** Serialize to out */
+    private void write(DataOutput out, ReferenceMap referenceMap
+        ) throws IOException {
+      writeCreated(out);
+      writeDeleted(out, referenceMap);    
+    }
+
+    /** Get the list of INodeDirectory contained in the deleted list */
+    private void getDirsInDeleted(List<INodeDirectory> dirList) {
+      for (INode node : getList(ListType.DELETED)) {
+        if (node.isDirectory()) {
+          dirList.add(node.asDirectory());
+        }
+      }
+    }
+    
+    /**
+     * Interpret the diff and generate a list of {@link DiffReportEntry}.
+     * @param parentPath The relative path of the parent.
+     * @param parent The directory that the diff belongs to.
+     * @param fromEarlier True indicates {@code diff=later-earlier}, 
+     *                    False indicates {@code diff=earlier-later}
+     * @return A list of {@link DiffReportEntry} as the diff report.
+     */
+    public List<DiffReportEntry> generateReport(byte[][] parentPath,
+        INodeDirectoryWithSnapshot parent, boolean fromEarlier) {
+      List<DiffReportEntry> cList = new ArrayList<DiffReportEntry>();
+      List<DiffReportEntry> dList = new ArrayList<DiffReportEntry>();
+      int c = 0, d = 0;
+      List<INode> created = getList(ListType.CREATED);
+      List<INode> deleted = getList(ListType.DELETED);
+      byte[][] fullPath = new byte[parentPath.length + 1][];
+      System.arraycopy(parentPath, 0, fullPath, 0, parentPath.length);
+      for (; c < created.size() && d < deleted.size(); ) {
+        INode cnode = created.get(c);
+        INode dnode = deleted.get(d);
+        if (cnode.compareTo(dnode.getLocalNameBytes()) == 0) {
+          fullPath[fullPath.length - 1] = cnode.getLocalNameBytes();
+          if (cnode.isSymlink() && dnode.isSymlink()) {
+            dList.add(new DiffReportEntry(DiffType.MODIFY, fullPath));
+          } else {
+            // must be the case: delete first and then create an inode with the
+            // same name
+            cList.add(new DiffReportEntry(DiffType.CREATE, fullPath));
+            dList.add(new DiffReportEntry(DiffType.DELETE, fullPath));
+          }
+          c++;
+          d++;
+        } else if (cnode.compareTo(dnode.getLocalNameBytes()) < 0) {
+          fullPath[fullPath.length - 1] = cnode.getLocalNameBytes();
+          cList.add(new DiffReportEntry(fromEarlier ? DiffType.CREATE
+              : DiffType.DELETE, fullPath));
+          c++;
+        } else {
+          fullPath[fullPath.length - 1] = dnode.getLocalNameBytes();
+          dList.add(new DiffReportEntry(fromEarlier ? DiffType.DELETE
+              : DiffType.CREATE, fullPath));
+          d++;
+        }
+      }
+      for (; d < deleted.size(); d++) {
+        fullPath[fullPath.length - 1] = deleted.get(d).getLocalNameBytes();
+        dList.add(new DiffReportEntry(fromEarlier ? DiffType.DELETE
+            : DiffType.CREATE, fullPath));
+      }
+      for (; c < created.size(); c++) {
+        fullPath[fullPath.length - 1] = created.get(c).getLocalNameBytes();
+        cList.add(new DiffReportEntry(fromEarlier ? DiffType.CREATE
+            : DiffType.DELETE, fullPath));
+      }
+      dList.addAll(cList);
+      return dList;
+    }
+  }
+  
+  /**
+   * The difference of an {@link INodeDirectory} between two snapshots.
+   */
+  public static class DirectoryDiff extends
+      AbstractINodeDiff<INodeDirectory, DirectoryDiff> {
+    /** The size of the children list at snapshot creation time. */
+    private final int childrenSize;
+    /** The children list diff. */
+    private final ChildrenDiff diff;
+
+    private DirectoryDiff(Snapshot snapshot, INodeDirectory dir) {
+      super(snapshot, null, null);
+
+      this.childrenSize = dir.getChildrenList(null).size();
+      this.diff = new ChildrenDiff();
+    }
+
+    /** Constructor used by FSImage loading */
+    DirectoryDiff(Snapshot snapshot, INodeDirectory snapshotINode,
+        DirectoryDiff posteriorDiff, int childrenSize,
+        List<INode> createdList, List<INode> deletedList) {
+      super(snapshot, snapshotINode, posteriorDiff);
+      this.childrenSize = childrenSize;
+      this.diff = new ChildrenDiff(createdList, deletedList);
+    }
+    
+    ChildrenDiff getChildrenDiff() {
+      return diff;
+    }
+    
+    /** Is the inode the root of the snapshot? */
+    boolean isSnapshotRoot() {
+      return snapshotINode == snapshot.getRoot();
+    }
+    
+    @Override
+    Quota.Counts combinePosteriorAndCollectBlocks(
+        final INodeDirectory currentDir, final DirectoryDiff posterior,
+        final BlocksMapUpdateInfo collectedBlocks,
+        final List<INode> removedINodes) {
+      final Quota.Counts counts = Quota.Counts.newInstance();
+      diff.combinePosterior(posterior.diff, new Diff.Processor<INode>() {
+        /** Collect blocks for deleted files. */
+        @Override
+        public void process(INode inode) {
+          if (inode != null) {
+            inode.computeQuotaUsage(counts, false);
+            inode.destroyAndCollectBlocks(collectedBlocks, removedINodes);
+          }
+        }
+      });
+      return counts;
+    }
+
+    /**
+     * @return The children list of a directory in a snapshot.
+     *         Since the snapshot is read-only, the logical view of the list is
+     *         never changed although the internal data structure may mutate.
+     */
+    ReadOnlyList<INode> getChildrenList(final INodeDirectory currentDir) {
+      return new ReadOnlyList<INode>() {
+        private List<INode> children = null;
+
+        private List<INode> initChildren() {
+          if (children == null) {
+            final ChildrenDiff combined = new ChildrenDiff();
+            for(DirectoryDiff d = DirectoryDiff.this; d != null; d = d.getPosterior()) {
+              combined.combinePosterior(d.diff, null);
+            }
+            children = combined.apply2Current(ReadOnlyList.Util.asList(
+                currentDir.getChildrenList(null)));
+          }
+          return children;
+        }
+
+        @Override
+        public Iterator<INode> iterator() {
+          return initChildren().iterator();
+        }
+    
+        @Override
+        public boolean isEmpty() {
+          return childrenSize == 0;
+        }
+    
+        @Override
+        public int size() {
+          return childrenSize;
+        }
+    
+        @Override
+        public INode get(int i) {
+          return initChildren().get(i);
+        }
+      };
+    }
+
+    /** @return the child with the given name. */
+    INode getChild(byte[] name, boolean checkPosterior,
+        INodeDirectory currentDir) {
+      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
+          return returned.getElement(); 
+        } else if (!checkPosterior) {
+          // Since checkPosterior is false, return null, i.e. not found.   
+          return null;
+        } else if (d.getPosterior() == null) {
+          // no more posterior diff, get from current inode.
+          return currentDir.getChild(name, null);
+        }
+      }
+    }
+    
+    @Override
+    public String toString() {
+      return super.toString() + " childrenSize=" + childrenSize + ", " + diff;
+    }
+    
+    @Override
+    void write(DataOutput out, ReferenceMap referenceMap) throws IOException {
+      writeSnapshot(out);
+      out.writeInt(childrenSize);
+
+      // write snapshotINode
+      if (isSnapshotRoot()) {
+        out.writeBoolean(true);
+      } else {
+        out.writeBoolean(false);
+        if (snapshotINode != null) {
+          out.writeBoolean(true);
+          FSImageSerialization.writeINodeDirectory(snapshotINode, out);
+        } else {
+          out.writeBoolean(false);
+        }
+      }
+      // Write diff. Node need to write poseriorDiff, since diffs is a list.
+      diff.write(out, referenceMap);
+    }
+
+    @Override
+    Quota.Counts destroyDiffAndCollectBlocks(INodeDirectory currentINode,
+        BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
+      // this diff has been deleted
+      Quota.Counts counts = Quota.Counts.newInstance();
+      counts.add(diff.destroyDeletedList(collectedBlocks, removedINodes));
+      return counts;
+    }
+  }
+
+  /** A list of directory diffs. */
+  public static class DirectoryDiffList
+      extends AbstractINodeDiffList<INodeDirectory, DirectoryDiff> {
+
+    @Override
+    DirectoryDiff createDiff(Snapshot snapshot, INodeDirectory currentDir) {
+      return new DirectoryDiff(snapshot, currentDir);
+    }
+
+    @Override
+    INodeDirectory createSnapshotCopy(INodeDirectory currentDir) {
+      final INodeDirectory copy = currentDir.isQuotaSet()?
+          new INodeDirectoryWithQuota(currentDir, false,
+              currentDir.getNsQuota(), currentDir.getDsQuota())
+        : new INodeDirectory(currentDir, false);
+      copy.clearChildren();
+      return copy;
+    }
+
+    /** Replace the given child in the created/deleted list, if there is any. */
+    private boolean replaceChild(final ListType type, final INode oldChild,
+        final INode newChild) {
+      final List<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)) {
+          return true;
+        }
+      }
+      return false;
+    }
+    
+    /** Remove the given child in the created/deleted list, if there is any. */
+    private boolean removeChild(final ListType type, final INode child) {
+      final List<DirectoryDiff> diffList = asList();
+      for(int i = diffList.size() - 1; i >= 0; i--) {
+        final ChildrenDiff diff = diffList.get(i).diff;
+        if (diff.removeChild(type, child)) {
+          return true;
+        }
+      }
+      return false;
+    }
+  }
+
+  /**
+   * Compute the difference between Snapshots.
+   * 
+   * @param fromSnapshot Start point of the diff computation. Null indicates
+   *          current tree.
+   * @param toSnapshot End point of the diff computation. Null indicates current
+   *          tree.
+   * @param diff Used to capture the changes happening to the children. Note
+   *          that the diff still represents (later_snapshot - earlier_snapshot)
+   *          although toSnapshot can be before fromSnapshot.
+   * @return Whether changes happened between the startSnapshot and endSnaphsot.
+   */
+  boolean computeDiffBetweenSnapshots(Snapshot fromSnapshot,
+      Snapshot toSnapshot, ChildrenDiff diff) {
+    Snapshot earlier = fromSnapshot;
+    Snapshot later = toSnapshot;
+    if (Snapshot.ID_COMPARATOR.compare(fromSnapshot, toSnapshot) > 0) {
+      earlier = toSnapshot;
+      later = fromSnapshot;
+    }
+    
+    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, earlier.getId());
+    int laterDiffIndex = later == null ? size : Collections
+        .binarySearch(difflist, later.getId());
+    earlierDiffIndex = earlierDiffIndex < 0 ? (-earlierDiffIndex - 1)
+        : earlierDiffIndex;
+    laterDiffIndex = laterDiffIndex < 0 ? (-laterDiffIndex - 1)
+        : laterDiffIndex;
+    
+    boolean dirMetadataChanged = false;
+    INodeDirectory dirCopy = null;
+    for (int i = earlierDiffIndex; i < laterDiffIndex; i++) {
+      DirectoryDiff sdiff = difflist.get(i);
+      diff.combinePosterior(sdiff.diff, null);
+      if (dirMetadataChanged == false && sdiff.snapshotINode != null) {
+        if (dirCopy == null) {
+          dirCopy = sdiff.snapshotINode;
+        } else if (!dirCopy.metadataEquals(sdiff.snapshotINode)) {
+          dirMetadataChanged = true;
+        }
+      }
+    }
+
+    if (!diff.isEmpty() || dirMetadataChanged) {
+      return true;
+    } else if (dirCopy != null) {
+      for (int i = laterDiffIndex; i < size; i++) {
+        if (!dirCopy.metadataEquals(difflist.get(i).snapshotINode)) {
+          return true;
+        }
+      }
+      return !dirCopy.metadataEquals(this);
+    } else {
+      return false;
+    }
+  }
+
+  /** Diff list sorted by snapshot IDs, i.e. in chronological order. */
+  private final DirectoryDiffList diffs;
+
+  public INodeDirectoryWithSnapshot(INodeDirectory that) {
+    this(that, true, that instanceof INodeDirectoryWithSnapshot?
+        ((INodeDirectoryWithSnapshot)that).getDiffs(): null);
+  }
+
+  INodeDirectoryWithSnapshot(INodeDirectory that, boolean adopt,
+      DirectoryDiffList diffs) {
+    super(that, adopt, that.getNsQuota(), that.getDsQuota());
+    this.diffs = diffs != null? diffs: new DirectoryDiffList();
+  }
+
+  /** @return the last snapshot. */
+  public Snapshot getLastSnapshot() {
+    return diffs.getLastSnapshot();
+  }
+
+  /** @return the snapshot diff list. */
+  public DirectoryDiffList getDiffs() {
+    return diffs;
+  }
+
+  @Override
+  public INodeDirectory getSnapshotINode(Snapshot snapshot) {
+    return diffs.getSnapshotINode(snapshot, this);
+  }
+
+  @Override
+  public INodeDirectoryWithSnapshot recordModification(final Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
+    if (isInLatestSnapshot(latest) && !shouldRecordInSrcSnapshot(latest)) {
+      return saveSelf2Snapshot(latest, null);
+    }
+    return this;
+  }
+
+  /** Save the snapshot copy to the latest snapshot. */
+  public INodeDirectoryWithSnapshot saveSelf2Snapshot(
+      final Snapshot latest, final INodeDirectory snapshotCopy)
+          throws QuotaExceededException {
+    diffs.saveSelf2Snapshot(latest, this, snapshotCopy);
+    return this;
+  }
+
+  @Override
+  public INode saveChild2Snapshot(final INode child, final Snapshot latest,
+      final INode snapshotCopy, final INodeMap inodeMap)
+      throws QuotaExceededException {
+    Preconditions.checkArgument(!child.isDirectory(),
+        "child is a directory, child=%s", child);
+    if (latest == null) {
+      return child;
+    }
+
+    final DirectoryDiff diff = diffs.checkAndAddLatestSnapshotDiff(latest, this);
+    if (diff.getChild(child.getLocalNameBytes(), false, this) != null) {
+      // it was already saved in the latest snapshot earlier.  
+      return child;
+    }
+
+    diff.diff.modify(snapshotCopy, child);
+    return child;
+  }
+
+  @Override
+  public boolean addChild(INode inode, boolean setModTime, Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
+    ChildrenDiff diff = null;
+    Integer undoInfo = null;
+    if (isInLatestSnapshot(latest)) {
+      diff = diffs.checkAndAddLatestSnapshotDiff(latest, this).diff;
+      undoInfo = diff.create(inode);
+    }
+    final boolean added = super.addChild(inode, setModTime, null, inodeMap);
+    if (!added && undoInfo != null) {
+      diff.undoCreate(inode, undoInfo);
+    }
+    return added; 
+  }
+
+  @Override
+  public boolean removeChild(INode child, Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
+    ChildrenDiff diff = null;
+    UndoInfo<INode> undoInfo = null;
+    // For a directory that is not a renamed node, if isInLatestSnapshot returns
+    // false, the directory is not in the latest snapshot, thus we do not need
+    // to record the removed child in any snapshot.
+    // For a directory that was moved/renamed, note that if the directory is in
+    // any of the previous snapshots, we will create a reference node for the 
+    // directory while rename, and isInLatestSnapshot will return true in that
+    // scenario (if all previous snapshots have been deleted, isInLatestSnapshot
+    // still returns false). Thus if isInLatestSnapshot returns false, the 
+    // directory node cannot be in any snapshot (not in current tree, nor in 
+    // previous src tree). Thus we do not need to record the removed child in 
+    // any snapshot.
+    if (isInLatestSnapshot(latest)) {
+      diff = diffs.checkAndAddLatestSnapshotDiff(latest, this).diff;
+      undoInfo = diff.delete(child);
+    }
+    final boolean removed = removeChild(child);
+    if (undoInfo != null) {
+      if (!removed) {
+        //remove failed, undo
+        diff.undoDelete(child, undoInfo);
+      }
+    }
+    return removed;
+  }
+  
+  @Override
+  public void replaceChild(final INode oldChild, final INode newChild,
+      final INodeMap inodeMap) {
+    super.replaceChild(oldChild, newChild, inodeMap);
+    diffs.replaceChild(ListType.CREATED, oldChild, newChild);
+  }
+  
+  /**
+   * This method is usually called by the undo section of rename.
+   * 
+   * Before calling this function, in the rename operation, we replace the
+   * original src node (of the rename operation) with a reference node (WithName
+   * instance) in both the children list and a created list, delete the
+   * reference node from the children list, and add it to the corresponding
+   * deleted list.
+   * 
+   * To undo the above operations, we have the following steps in particular:
+   * 
+   * <pre>
+   * 1) remove the WithName node from the deleted list (if it exists) 
+   * 2) replace the WithName node in the created list with srcChild 
+   * 3) add srcChild back as a child of srcParent. Note that we already add 
+   * the node into the created list of a snapshot diff in step 2, we do not need
+   * to add srcChild to the created list of the latest snapshot.
+   * </pre>
+   * 
+   * We do not need to update quota usage because the old child is in the 
+   * deleted list before. 
+   * 
+   * @param oldChild
+   *          The reference node to be removed/replaced
+   * @param newChild
+   *          The node to be added back
+   * @param latestSnapshot
+   *          The latest snapshot. Note this may not be the last snapshot in the
+   *          {@link #diffs}, since the src tree of the current rename operation
+   *          may be the dst tree of a previous rename.
+   * @throws QuotaExceededException should not throw this exception
+   */
+  public void undoRename4ScrParent(final INodeReference oldChild,
+      final INode newChild, Snapshot latestSnapshot)
+      throws QuotaExceededException {
+    diffs.removeChild(ListType.DELETED, oldChild);
+    diffs.replaceChild(ListType.CREATED, oldChild, newChild);
+    // pass null for inodeMap since the parent node will not get replaced when
+    // undoing rename
+    addChild(newChild, true, null, null);
+  }
+  
+  /**
+   * Undo the rename operation for the dst tree, i.e., if the rename operation
+   * (with OVERWRITE option) removes a file/dir from the dst tree, add it back
+   * and delete possible record in the deleted list.  
+   */
+  public void undoRename4DstParent(final INode deletedChild,
+      Snapshot latestSnapshot) throws QuotaExceededException {
+    boolean removeDeletedChild = diffs.removeChild(ListType.DELETED,
+        deletedChild);
+    // pass null for inodeMap since the parent node will not get replaced when
+    // undoing rename
+    final boolean added = addChild(deletedChild, true, removeDeletedChild ? null
+        : latestSnapshot, null);
+    // update quota usage if adding is successfully and the old child has not
+    // been stored in deleted list before
+    if (added && !removeDeletedChild) {
+      final Quota.Counts counts = deletedChild.computeQuotaUsage();
+      addSpaceConsumed(counts.get(Quota.NAMESPACE),
+          counts.get(Quota.DISKSPACE), false, Snapshot.INVALID_ID);
+    }
+  }
+
+  @Override
+  public ReadOnlyList<INode> getChildrenList(Snapshot snapshot) {
+    final DirectoryDiff diff = diffs.getDiff(snapshot);
+    return diff != null? diff.getChildrenList(this): super.getChildrenList(null);
+  }
+
+  @Override
+  public INode getChild(byte[] name, Snapshot snapshot) {
+    final DirectoryDiff diff = diffs.getDiff(snapshot);
+    return diff != null? diff.getChild(name, true, this): super.getChild(name, null);
+  }
+
+  @Override
+  public String toDetailString() {
+    return super.toDetailString() + ", " + diffs;
+  }
+  
+  /**
+   * Get all the directories that are stored in some snapshot but not in the
+   * current children list. These directories are equivalent to the directories
+   * stored in the deletes lists.
+   */
+  public void getSnapshotDirectory(List<INodeDirectory> snapshotDir) {
+    for (DirectoryDiff sdiff : diffs) {
+      sdiff.getChildrenDiff().getDirsInDeleted(snapshotDir);
+    }
+  }
+
+  @Override
+  public Quota.Counts cleanSubtree(final Snapshot snapshot, Snapshot prior,
+      final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
+      throws QuotaExceededException {
+    Quota.Counts counts = Quota.Counts.newInstance();
+    if (snapshot == null) { // delete the current directory
+      recordModification(prior, null);
+      // delete everything in created list
+      DirectoryDiff lastDiff = diffs.getLast();
+      if (lastDiff != null) {
+        counts.add(lastDiff.diff.destroyCreatedList(this, collectedBlocks,
+            removedINodes));
+      }
+    } else {
+      // update prior
+      prior = getDiffs().updatePrior(snapshot, prior);
+      counts.add(getDiffs().deleteSnapshotDiff(snapshot, prior, this, 
+          collectedBlocks, removedINodes));
+      if (prior != null) {
+        DirectoryDiff priorDiff = this.getDiffs().getDiff(prior);
+        if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
+          // For files/directories created between "prior" and "snapshot", 
+          // we need to clear snapshot copies for "snapshot". Note that we must
+          // use null as prior in the cleanSubtree call. Files/directories that
+          // were created before "prior" will be covered by the later 
+          // cleanSubtreeRecursively call.
+          for (INode cNode : priorDiff.getChildrenDiff().getList(
+              ListType.CREATED)) {
+            counts.add(cNode.cleanSubtree(snapshot, null, collectedBlocks,
+                removedINodes));
+          }
+          // When a directory is moved from the deleted list of the posterior
+          // diff to the deleted list of this diff, we need to destroy its
+          // descendants that were 1) created after taking this diff and 2)
+          // deleted after taking posterior diff.
+
+          // For files moved from posterior's deleted list, we also need to
+          // delete its snapshot copy associated with the posterior snapshot.
+          for (INode dNode : priorDiff.getChildrenDiff().getList(
+              ListType.DELETED)) {
+            counts.add(cleanDeletedINode(dNode, snapshot, prior,
+                collectedBlocks, removedINodes));
+          }
+        }
+      }
+    }
+    counts.add(cleanSubtreeRecursively(snapshot, prior, collectedBlocks,
+        removedINodes));
+    
+    if (isQuotaSet()) {
+      this.addSpaceConsumed2Cache(-counts.get(Quota.NAMESPACE),
+          -counts.get(Quota.DISKSPACE));
+    }
+    return counts;
+  }
+  
+  /**
+   * Clean an inode while we move it from the deleted list of post to the
+   * deleted list of prior.
+   * @param inode The inode to clean.
+   * @param post The post snapshot.
+   * @param prior The prior snapshot.
+   * @param collectedBlocks Used to collect blocks for later deletion.
+   * @return Quota usage update.
+   */
+  private static Quota.Counts cleanDeletedINode(INode inode, final Snapshot post, 
+      final Snapshot prior, final BlocksMapUpdateInfo collectedBlocks, 
+      final List<INode> removedINodes) throws QuotaExceededException {
+    Quota.Counts counts = Quota.Counts.newInstance();
+    Deque<INode> queue = new ArrayDeque<INode>();
+    queue.addLast(inode);
+    while (!queue.isEmpty()) {
+      INode topNode = queue.pollFirst();
+      if (topNode instanceof INodeReference.WithName) {
+        INodeReference.WithName wn = (INodeReference.WithName) topNode;
+        if (wn.getLastSnapshotId() >= post.getId()) {
+          wn.cleanSubtree(post, prior, collectedBlocks, removedINodes);
+        }
+        // For DstReference node, since the node is not in the created list of
+        // prior, we should treat it as regular file/dir
+      } else if (topNode.isFile()
+          && topNode.asFile() instanceof FileWithSnapshot) {
+        FileWithSnapshot fs = (FileWithSnapshot) topNode.asFile();
+        counts.add(fs.getDiffs().deleteSnapshotDiff(post, prior,
+            topNode.asFile(), collectedBlocks, removedINodes));
+      } else if (topNode.isDirectory()) {
+        INodeDirectory dir = topNode.asDirectory();
+        if (dir instanceof INodeDirectoryWithSnapshot) {
+          // delete files/dirs created after prior. Note that these
+          // files/dirs, along with inode, were deleted right after post.
+          INodeDirectoryWithSnapshot sdir = (INodeDirectoryWithSnapshot) dir;
+          DirectoryDiff priorDiff = sdir.getDiffs().getDiff(prior);
+          if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
+            counts.add(priorDiff.diff.destroyCreatedList(sdir,
+                collectedBlocks, removedINodes));
+          }
+        }
+        for (INode child : dir.getChildrenList(prior)) {
+          queue.addLast(child);
+        }
+      }
+    }
+    return counts;
+  }
+
+  @Override
+  public void destroyAndCollectBlocks(
+      final BlocksMapUpdateInfo collectedBlocks, 
+      final List<INode> removedINodes) {
+    // destroy its diff list
+    for (DirectoryDiff diff : diffs) {
+      diff.destroyDiffAndCollectBlocks(this, collectedBlocks, removedINodes);
+    }
+    diffs.clear();
+    super.destroyAndCollectBlocks(collectedBlocks, removedINodes);
+  }
+
+  @Override
+  public final Quota.Counts computeQuotaUsage(Quota.Counts counts,
+      boolean useCache, int lastSnapshotId) {
+    if ((useCache && isQuotaSet()) || lastSnapshotId == Snapshot.INVALID_ID) {
+      return super.computeQuotaUsage(counts, useCache, lastSnapshotId);
+    }
+    
+    Snapshot lastSnapshot = diffs.getSnapshotById(lastSnapshotId);
+    
+    ReadOnlyList<INode> childrenList = getChildrenList(lastSnapshot);
+    for (INode child : childrenList) {
+      child.computeQuotaUsage(counts, useCache, lastSnapshotId);
+    }
+    
+    counts.add(Quota.NAMESPACE, 1);
+    return counts;
+  }
+  
+  @Override
+  public Quota.Counts computeQuotaUsage4CurrentDirectory(Quota.Counts counts) {
+    super.computeQuotaUsage4CurrentDirectory(counts);
+    for(DirectoryDiff d : diffs) {
+      for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
+        deleted.computeQuotaUsage(counts, false, Snapshot.INVALID_ID);
+      }
+    }
+    counts.add(Quota.NAMESPACE, diffs.asList().size());
+    return counts;
+  }
+
+  @Override
+  public Content.Counts computeContentSummary(final Content.Counts counts) {
+    super.computeContentSummary(counts);
+    computeContentSummary4Snapshot(counts);
+    return counts;
+  }
+
+  private void computeContentSummary4Snapshot(final Content.Counts counts) {
+    for(DirectoryDiff d : diffs) {
+      for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
+        deleted.computeContentSummary(counts);
+      }
+    }
+    counts.add(Content.DIRECTORY, diffs.asList().size());
+  }
+  
+  /**
+   * Destroy a subtree under a DstReference node.
+   */
+  public static void destroyDstSubtree(INode inode, final Snapshot snapshot,
+      final Snapshot prior, final BlocksMapUpdateInfo collectedBlocks,
+      final List<INode> removedINodes) throws QuotaExceededException {
+    Preconditions.checkArgument(prior != null);
+    if (inode.isReference()) {
+      if (inode instanceof INodeReference.WithName && snapshot != null) {
+        // this inode has been renamed before the deletion of the DstReference
+        // subtree
+        inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes);
+      } else { 
+        // for DstReference node, continue this process to its subtree
+        destroyDstSubtree(inode.asReference().getReferredINode(), snapshot,
+            prior, collectedBlocks, removedINodes);
+      }
+    } else if (inode.isFile() && snapshot != null) {
+      inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes);
+    } else if (inode.isDirectory()) {
+      if (inode instanceof INodeDirectoryWithSnapshot) {
+        INodeDirectoryWithSnapshot sdir = (INodeDirectoryWithSnapshot) inode;
+        DirectoryDiffList diffList = sdir.getDiffs();
+        if (snapshot != null) {
+          diffList.deleteSnapshotDiff(snapshot, prior, sdir, collectedBlocks,
+              removedINodes);
+        }
+        DirectoryDiff priorDiff = diffList.getDiff(prior);
+        if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
+          priorDiff.diff.destroyCreatedList(sdir, collectedBlocks,
+              removedINodes);
+        }
+      }
+      for (INode child : inode.asDirectory().getChildrenList(prior)) {
+        destroyDstSubtree(child, snapshot, prior, collectedBlocks,
+            removedINodes);
+      }
+    }
+  }
+}

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

@@ -0,0 +1,126 @@
+/**
+ * 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.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
+import org.apache.hadoop.hdfs.server.namenode.INodeMap;
+import org.apache.hadoop.hdfs.server.namenode.Quota;
+
+/**
+ * Represent an {@link INodeFileUnderConstruction} that is snapshotted.
+ */
+@InterfaceAudience.Private
+public class INodeFileUnderConstructionWithSnapshot
+    extends INodeFileUnderConstruction implements FileWithSnapshot {
+  private final FileDiffList diffs;
+  private boolean isCurrentFileDeleted = false;
+
+  INodeFileUnderConstructionWithSnapshot(final INodeFile f,
+      final String clientName,
+      final String clientMachine,
+      final DatanodeDescriptor clientNode,
+      final FileDiffList diffs) {
+    super(f, clientName, clientMachine, clientNode);
+    this.diffs = diffs != null? diffs: new FileDiffList();
+  }
+
+  /**
+   * Construct an {@link INodeFileUnderConstructionWithSnapshot} based on an
+   * {@link INodeFileUnderConstruction}.
+   * 
+   * @param f The given {@link INodeFileUnderConstruction} instance
+   */
+  public INodeFileUnderConstructionWithSnapshot(INodeFileUnderConstruction f,
+      final FileDiffList diffs) {
+    this(f, f.getClientName(), f.getClientMachine(), f.getClientNode(), diffs);
+  }
+  
+  @Override
+  protected INodeFileWithSnapshot toINodeFile(final long mtime) {
+    assertAllBlocksComplete();
+    final long atime = getModificationTime();
+    final INodeFileWithSnapshot f = new INodeFileWithSnapshot(this, getDiffs());
+    f.setModificationTime(mtime);
+    f.setAccessTime(atime);
+    return f;
+  }
+
+  @Override
+  public boolean isCurrentFileDeleted() {
+    return isCurrentFileDeleted;
+  }
+  
+  @Override
+  public void deleteCurrentFile() {
+    isCurrentFileDeleted = true;
+  }
+
+  @Override
+  public INodeFile getSnapshotINode(Snapshot snapshot) {
+    return diffs.getSnapshotINode(snapshot, this);
+  }
+
+  @Override
+  public INodeFileUnderConstructionWithSnapshot recordModification(
+      final Snapshot latest, final INodeMap inodeMap)
+      throws QuotaExceededException {
+    if (isInLatestSnapshot(latest) && !shouldRecordInSrcSnapshot(latest)) {
+      diffs.saveSelf2Snapshot(latest, this, null);
+    }
+    return this;
+  }
+
+  @Override
+  public INodeFile asINodeFile() {
+    return this;
+  }
+
+  @Override
+  public FileDiffList getDiffs() {
+    return diffs;
+  }
+
+  @Override
+  public Quota.Counts cleanSubtree(final Snapshot snapshot, Snapshot prior,
+      final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
+      throws QuotaExceededException {
+    if (snapshot == null) { // delete the current file
+      recordModification(prior, null);
+      isCurrentFileDeleted = true;
+      Util.collectBlocksAndClear(this, collectedBlocks, removedINodes);
+      return Quota.Counts.newInstance();
+    } else { // delete a snapshot
+      prior = getDiffs().updatePrior(snapshot, prior);
+      return diffs.deleteSnapshotDiff(snapshot, prior, this, collectedBlocks,
+          removedINodes);
+    }
+  }
+
+  @Override
+  public String toDetailString() {
+    return super.toDetailString()
+        + (isCurrentFileDeleted()? " (DELETED), ": ", ") + diffs;
+  }
+}

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

@@ -0,0 +1,113 @@
+/**
+ * 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.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.INodeMap;
+import org.apache.hadoop.hdfs.server.namenode.Quota;
+
+/**
+ * Represent an {@link INodeFile} that is snapshotted.
+ */
+@InterfaceAudience.Private
+public class INodeFileWithSnapshot extends INodeFile
+    implements FileWithSnapshot {
+  private final FileDiffList diffs;
+  private boolean isCurrentFileDeleted = false;
+
+  public INodeFileWithSnapshot(INodeFile f) {
+    this(f, f instanceof FileWithSnapshot?
+        ((FileWithSnapshot)f).getDiffs(): null);
+  }
+
+  public INodeFileWithSnapshot(INodeFile f, FileDiffList diffs) {
+    super(f);
+    this.diffs = diffs != null? diffs: new FileDiffList();
+  }
+
+  @Override
+  public INodeFileUnderConstructionWithSnapshot toUnderConstruction(
+      final String clientName,
+      final String clientMachine,
+      final DatanodeDescriptor clientNode) {
+    return new INodeFileUnderConstructionWithSnapshot(this,
+        clientName, clientMachine, clientNode, getDiffs());
+  }
+
+  @Override
+  public boolean isCurrentFileDeleted() {
+    return isCurrentFileDeleted;
+  }
+  
+  @Override
+  public void deleteCurrentFile() {
+    isCurrentFileDeleted = true;
+  }
+
+  @Override
+  public INodeFile getSnapshotINode(Snapshot snapshot) {
+    return diffs.getSnapshotINode(snapshot, this);
+  }
+
+  @Override
+  public INodeFileWithSnapshot recordModification(final Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
+    if (isInLatestSnapshot(latest) && !shouldRecordInSrcSnapshot(latest)) {
+      diffs.saveSelf2Snapshot(latest, this, null);
+    }
+    return this;
+  }
+
+  @Override
+  public INodeFile asINodeFile() {
+    return this;
+  }
+
+  @Override
+  public FileDiffList getDiffs() {
+    return diffs;
+  }
+
+  @Override
+  public Quota.Counts cleanSubtree(final Snapshot snapshot, Snapshot prior,
+      final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
+      throws QuotaExceededException {
+    if (snapshot == null) { // delete the current file
+      recordModification(prior, null);
+      isCurrentFileDeleted = true;
+      Util.collectBlocksAndClear(this, collectedBlocks, removedINodes);
+      return Quota.Counts.newInstance();
+    } else { // delete a snapshot
+      prior = getDiffs().updatePrior(snapshot, prior);
+      return diffs.deleteSnapshotDiff(snapshot, prior, this, collectedBlocks,
+          removedINodes);
+    }
+  }
+
+  @Override
+  public String toDetailString() {
+    return super.toDetailString()
+        + (isCurrentFileDeleted()? "(DELETED), ": ", ") + diffs;
+  }
+}

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

@@ -0,0 +1,215 @@
+/**
+ * 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.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Comparator;
+import java.util.Date;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.namenode.FSImageFormat;
+import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
+
+/** Snapshot of a sub-tree in the namesystem. */
+@InterfaceAudience.Private
+public class Snapshot implements Comparable<byte[]> {
+  public static final int INVALID_ID = -1;
+  
+  /**
+   * The pattern for generating the default snapshot name.
+   * E.g. s20130412-151029.033
+   */
+  private static final String DEFAULT_SNAPSHOT_NAME_PATTERN = "'s'yyyyMMdd-HHmmss.SSS";
+  
+  public static String generateDefaultSnapshotName() {
+    return new SimpleDateFormat(DEFAULT_SNAPSHOT_NAME_PATTERN).format(new Date());
+  }
+
+  public static String getSnapshotPath(String snapshottableDir,
+      String snapshotRelativePath) {
+    final StringBuilder b = new StringBuilder(snapshottableDir);
+    if (b.charAt(b.length() - 1) != Path.SEPARATOR_CHAR) {
+      b.append(Path.SEPARATOR);
+    }
+    return b.append(HdfsConstants.DOT_SNAPSHOT_DIR)
+        .append(Path.SEPARATOR)
+        .append(snapshotRelativePath)
+        .toString();
+  }
+  
+  /** 
+   * Get the name of the given snapshot. 
+   * @param s The given snapshot.
+   * @return The name of the snapshot, or an empty string if {@code s} is null
+   */
+  static String getSnapshotName(Snapshot s) {
+    return s != null ? s.getRoot().getLocalName() : "";
+  }
+
+  /**
+   * 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 - right; 
+      }
+    }
+  };
+
+  /**
+   * Find the latest snapshot that 1) covers the given inode (which means the
+   * snapshot was either taken on the inode or taken on an ancestor of the
+   * inode), and 2) was taken before the given snapshot (if the given snapshot 
+   * is not null).
+   * 
+   * @param inode the given inode that the returned snapshot needs to cover
+   * @param anchor the returned snapshot should be taken before this snapshot.
+   * @return the latest snapshot covers the given inode and was taken before the
+   *         the given snapshot (if it is not null).
+   */
+  public static Snapshot findLatestSnapshot(INode inode, Snapshot anchor) {
+    Snapshot latest = null;
+    for(; inode != null; inode = inode.getParent()) {
+      if (inode.isDirectory()) {
+        final INodeDirectory dir = inode.asDirectory();
+        if (dir instanceof INodeDirectoryWithSnapshot) {
+          latest = ((INodeDirectoryWithSnapshot) dir).getDiffs().updatePrior(
+              anchor, latest);
+        }
+      }
+    }
+    return latest;
+  }
+  
+  static Snapshot read(DataInput in, FSImageFormat.Loader loader)
+      throws IOException {
+    final int snapshotId = in.readInt();
+    final INode root = loader.loadINodeWithLocalName(false, in);
+    return new Snapshot(snapshotId, root.asDirectory(), null);
+  }
+
+  /** The root directory of the snapshot. */
+  static public class Root extends INodeDirectory {
+    Root(INodeDirectory other) {
+      super(other, false);
+    }
+
+    @Override
+    public ReadOnlyList<INode> getChildrenList(Snapshot snapshot) {
+      return getParent().getChildrenList(snapshot);
+    }
+
+    @Override
+    public INode getChild(byte[] name, Snapshot snapshot) {
+      return getParent().getChild(name, snapshot);
+    }
+    
+    @Override
+    public String getFullPathName() {
+      return getSnapshotPath(getParent().getFullPathName(), getLocalName());
+    }
+  }
+
+  /** Snapshot ID. */
+  private final int id;
+  /** The root directory of the snapshot. */
+  private final Root root;
+
+  Snapshot(int id, String name, INodeDirectorySnapshottable dir) {
+    this(id, dir, dir);
+    this.root.setLocalName(DFSUtil.string2Bytes(name));
+  }
+
+  Snapshot(int id, INodeDirectory dir, INodeDirectorySnapshottable parent) {
+    this.id = id;
+    this.root = new Root(dir);
+
+    this.root.setParent(parent);
+  }
+  
+  public int getId() {
+    return id;
+  }
+
+  /** @return the root directory of the snapshot. */
+  public Root getRoot() {
+    return root;
+  }
+
+  @Override
+  public int compareTo(byte[] bytes) {
+    return root.compareTo(bytes);
+  }
+  
+  @Override
+  public boolean equals(Object that) {
+    if (this == that) {
+      return true;
+    } else if (that == null || !(that instanceof Snapshot)) {
+      return false;
+    }
+    return this.id == ((Snapshot)that).id;
+  }
+  
+  @Override
+  public int hashCode() {
+    return id;
+  }
+  
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "." + root.getLocalName() + "(id=" + id + ")";
+  }
+  
+  /** Serialize the fields to out */
+  void write(DataOutput out) throws IOException {
+    out.writeInt(id);
+    // write root
+    FSImageSerialization.writeINodeDirectory(root, out);
+  }
+}

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

@@ -0,0 +1,33 @@
+/**
+ * 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.security.AccessControlException;
+
+/** Snapshot access related exception. */
+public class SnapshotAccessControlException extends AccessControlException {
+  private static final long serialVersionUID = 1L;
+
+  public SnapshotAccessControlException(final String message) {
+    super(message);
+  }
+
+  public SnapshotAccessControlException(final Throwable cause) {
+    super(cause);
+  }
+}

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

@@ -0,0 +1,33 @@
+/**
+ * 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.io.IOException;
+
+/** Snapshot related exception. */
+public class SnapshotException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public SnapshotException(final String message) {
+    super(message);
+  }
+
+  public SnapshotException(final Throwable cause) {
+    super(cause);
+  }
+}

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

@@ -0,0 +1,361 @@
+/**
+ * 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.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.server.namenode.FSImageFormat;
+import org.apache.hadoop.hdfs.server.namenode.FSImageFormat.Loader;
+import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
+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.INodeReference;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiffList;
+import org.apache.hadoop.hdfs.tools.snapshot.SnapshotDiff;
+import org.apache.hadoop.hdfs.util.Diff.ListType;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
+
+/**
+ * A helper class defining static methods for reading/writing snapshot related
+ * information from/to FSImage.
+ */
+public class SnapshotFSImageFormat {
+  /**
+   * Save snapshots and snapshot quota for a snapshottable directory.
+   * @param current The directory that the snapshots belongs to.
+   * @param out The {@link DataOutput} to write.
+   * @throws IOException
+   */
+  public static void saveSnapshots(INodeDirectorySnapshottable current,
+      DataOutput out) throws IOException {
+    // list of snapshots in snapshotsByNames
+    ReadOnlyList<Snapshot> snapshots = current.getSnapshotsByNames();
+    out.writeInt(snapshots.size());
+    for (Snapshot s : snapshots) {
+      // write the snapshot id
+      out.writeInt(s.getId());
+    }
+    // snapshot quota
+    out.writeInt(current.getSnapshotQuota());
+  }
+  
+  /**
+   * Save SnapshotDiff list for an INodeDirectoryWithSnapshot.
+   * @param sNode The directory that the SnapshotDiff list belongs to.
+   * @param out The {@link DataOutput} to write.
+   */
+  private static <N extends INode, D extends AbstractINodeDiff<N, D>>
+      void saveINodeDiffs(final AbstractINodeDiffList<N, D> diffs,
+      final DataOutput out, ReferenceMap referenceMap) throws IOException {
+    // Record the diffs in reversed order, so that we can find the correct
+    // reference for INodes in the created list when loading the FSImage
+    if (diffs == null) {
+      out.writeInt(-1); // no diffs
+    } else {
+      final List<D> list = diffs.asList();
+      final int size = list.size();
+      out.writeInt(size);
+      for (int i = size - 1; i >= 0; i--) {
+        list.get(i).write(out, referenceMap);
+      }
+    }
+  }
+  
+  public static void saveDirectoryDiffList(final INodeDirectory dir,
+      final DataOutput out, final ReferenceMap referenceMap
+      ) throws IOException {
+    saveINodeDiffs(dir instanceof INodeDirectoryWithSnapshot?
+        ((INodeDirectoryWithSnapshot)dir).getDiffs(): null, out, referenceMap);
+  }
+  
+  public static void saveFileDiffList(final INodeFile file,
+      final DataOutput out) throws IOException {
+    saveINodeDiffs(file instanceof FileWithSnapshot?
+        ((FileWithSnapshot)file).getDiffs(): null, out, null);
+  }
+
+  public static FileDiffList loadFileDiffList(DataInput in,
+      FSImageFormat.Loader loader) throws IOException {
+    final int size = in.readInt();
+    if (size == -1) {
+      return null;
+    } else {
+      final FileDiffList diffs = new FileDiffList();
+      FileDiff posterior = null;
+      for(int i = 0; i < size; i++) {
+        final FileDiff d = loadFileDiff(posterior, in, loader);
+        diffs.addFirst(d);
+        posterior = d;
+      }
+      return diffs;
+    }
+  }
+
+  private static FileDiff loadFileDiff(FileDiff posterior, DataInput in,
+      FSImageFormat.Loader loader) throws IOException {
+    // 1. Read the full path of the Snapshot root to identify the Snapshot
+    final Snapshot snapshot = loader.getSnapshot(in);
+
+    // 2. Load file size
+    final long fileSize = in.readLong();
+    
+    // 3. Load snapshotINode 
+    final INodeFile snapshotINode = in.readBoolean()?
+        loader.loadINodeWithLocalName(true, in).asFile(): null;
+    
+    return new FileDiff(snapshot, snapshotINode, posterior, fileSize);
+  }
+
+  /**
+   * Load a node stored in the created list from fsimage.
+   * @param createdNodeName The name of the created node.
+   * @param parent The directory that the created list belongs to.
+   * @return The created node.
+   */
+  private static INode loadCreated(byte[] createdNodeName,
+      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 (DirectoryDiff postDiff : parent.getDiffs()) {
+      final INode d = postDiff.getChildrenDiff().search(ListType.DELETED,
+          createdNodeName);
+      if (d != null) {
+        return d;
+      } // else go to the next SnapshotDiff
+    } 
+    // use the current child
+    INode currentChild = parent.getChild(createdNodeName, null);
+    if (currentChild == null) {
+      throw new IOException("Cannot find an INode associated with the INode "
+          + DFSUtil.bytes2String(createdNodeName)
+          + " in created list while loading FSImage.");
+    }
+    return currentChild;
+  }
+  
+  /**
+   * Load the created list from fsimage.
+   * @param parent The directory that the created list belongs to.
+   * @param in The {@link DataInput} to read.
+   * @return The created list.
+   */
+  private static List<INode> loadCreatedList(INodeDirectoryWithSnapshot parent,
+      DataInput in) throws IOException {
+    // read the size of the created list
+    int createdSize = in.readInt();
+    List<INode> createdList = new ArrayList<INode>(createdSize);
+    for (int i = 0; i < createdSize; i++) {
+      byte[] createdNodeName = FSImageSerialization.readLocalName(in);
+      INode created = loadCreated(createdNodeName, parent);
+      createdList.add(created);
+    }
+    return createdList;
+  }
+    
+  /**
+   * Load the deleted list from the fsimage.
+   * 
+   * @param parent The directory that the deleted list belongs to.
+   * @param createdList The created list associated with the deleted list in 
+   *                    the same Diff.
+   * @param in The {@link DataInput} to read.
+   * @param loader The {@link Loader} instance.
+   * @return The deleted list.
+   */
+  private static List<INode> loadDeletedList(INodeDirectoryWithSnapshot parent,
+      List<INode> createdList, DataInput in, FSImageFormat.Loader loader)
+      throws IOException {
+    int deletedSize = in.readInt();
+    List<INode> deletedList = new ArrayList<INode>(deletedSize);
+    for (int i = 0; i < deletedSize; i++) {
+      final INode deleted = loader.loadINodeWithLocalName(true, in);
+      deletedList.add(deleted);
+      // set parent: the parent field of an INode in the deleted list is not 
+      // useful, but set the parent here to be consistent with the original 
+      // fsdir tree.
+      deleted.setParent(parent);
+    }
+    return deletedList;
+  }
+  
+  /**
+   * Load snapshots and snapshotQuota for a Snapshottable directory.
+   * @param snapshottableParent The snapshottable directory for loading.
+   * @param numSnapshots The number of snapshots that the directory has.
+   * @param in The {@link DataInput} instance to read.
+   * @param loader The {@link Loader} instance that this loading procedure is 
+   *               using.
+   */
+  public static void loadSnapshotList(
+      INodeDirectorySnapshottable snapshottableParent, int numSnapshots,
+      DataInput in, FSImageFormat.Loader loader) throws IOException {
+    for (int i = 0; i < numSnapshots; i++) {
+      // read snapshots
+      final Snapshot s = loader.getSnapshot(in);
+      s.getRoot().setParent(snapshottableParent);
+      snapshottableParent.addSnapshot(s);
+    }
+    int snapshotQuota = in.readInt();
+    snapshottableParent.setSnapshotQuota(snapshotQuota);
+  }
+  
+  /**
+   * Load the {@link SnapshotDiff} list for the INodeDirectoryWithSnapshot
+   * directory.
+   * @param dir The snapshottable directory for loading.
+   * @param in The {@link DataInput} instance to read.
+   * @param loader The {@link Loader} instance that this loading procedure is 
+   *               using.
+   */
+  public static void loadDirectoryDiffList(INodeDirectory dir,
+      DataInput in, FSImageFormat.Loader loader) throws IOException {
+    final int size = in.readInt();
+    if (dir instanceof INodeDirectoryWithSnapshot) {
+      INodeDirectoryWithSnapshot withSnapshot = (INodeDirectoryWithSnapshot)dir;
+      DirectoryDiffList diffs = withSnapshot.getDiffs();
+      for (int i = 0; i < size; i++) {
+        diffs.addFirst(loadDirectoryDiff(withSnapshot, in, loader));
+      }
+    }
+  }
+  
+  /**
+   * Load the snapshotINode field of {@link SnapshotDiff}.
+   * @param snapshot The Snapshot associated with the {@link SnapshotDiff}.
+   * @param in The {@link DataInput} to read.
+   * @param loader The {@link Loader} instance that this loading procedure is 
+   *               using.
+   * @return The snapshotINode.
+   */
+  private static INodeDirectory loadSnapshotINodeInDirectoryDiff(
+      Snapshot snapshot, DataInput in, FSImageFormat.Loader loader)
+      throws IOException {
+    // read the boolean indicating whether snapshotINode == Snapshot.Root
+    boolean useRoot = in.readBoolean();      
+    if (useRoot) {
+      return snapshot.getRoot();
+    } else {
+      // another boolean is used to indicate whether snapshotINode is non-null
+      return in.readBoolean()?
+          loader.loadINodeWithLocalName(true, in).asDirectory(): null;
+    }
+  }
+   
+  /**
+   * Load {@link DirectoryDiff} from fsimage.
+   * @param parent The directory that the SnapshotDiff belongs to.
+   * @param in The {@link DataInput} instance to read.
+   * @param loader The {@link Loader} instance that this loading procedure is 
+   *               using.
+   * @return A {@link DirectoryDiff}.
+   */
+  private static DirectoryDiff loadDirectoryDiff(
+      INodeDirectoryWithSnapshot parent, DataInput in,
+      FSImageFormat.Loader loader) throws IOException {
+    // 1. Read the full path of the Snapshot root to identify the Snapshot
+    final Snapshot snapshot = loader.getSnapshot(in);
+
+    // 2. Load DirectoryDiff#childrenSize
+    int childrenSize = in.readInt();
+    
+    // 3. Load DirectoryDiff#snapshotINode 
+    INodeDirectory snapshotINode = loadSnapshotINodeInDirectoryDiff(snapshot,
+        in, loader);
+    
+    // 4. Load the created list in SnapshotDiff#Diff
+    List<INode> createdList = loadCreatedList(parent, in);
+    
+    // 5. Load the deleted list in SnapshotDiff#Diff
+    List<INode> deletedList = loadDeletedList(parent, createdList, in, loader);
+    
+    // 6. Compose the SnapshotDiff
+    List<DirectoryDiff> diffs = parent.getDiffs().asList();
+    DirectoryDiff sdiff = new DirectoryDiff(snapshot, snapshotINode,
+        diffs.isEmpty() ? null : diffs.get(0),
+        childrenSize, createdList, deletedList);
+    return sdiff;
+  }
+  
+
+  /** A reference map for fsimage serialization. */
+  public static class ReferenceMap {
+    /**
+     * Used to indicate whether the reference node itself has been saved
+     */
+    private final Map<Long, INodeReference.WithCount> referenceMap
+        = new HashMap<Long, INodeReference.WithCount>();
+    /**
+     * Used to record whether the subtree of the reference node has been saved 
+     */
+    private final Map<Long, Long> dirMap = new HashMap<Long, Long>();
+
+    public void writeINodeReferenceWithCount(
+        INodeReference.WithCount withCount, DataOutput out,
+        boolean writeUnderConstruction) throws IOException {
+      final INode referred = withCount.getReferredINode();
+      final long id = withCount.getId();
+      final boolean firstReferred = !referenceMap.containsKey(id);
+      out.writeBoolean(firstReferred);
+
+      if (firstReferred) {
+        FSImageSerialization.saveINode2Image(referred, out,
+            writeUnderConstruction, this);
+        referenceMap.put(id, withCount);
+      } else {
+        out.writeLong(id);
+      }
+    }
+    
+    public boolean toProcessSubtree(long id) {
+      if (dirMap.containsKey(id)) {
+        return false;
+      } else {
+        dirMap.put(id, id);
+        return true;
+      }
+    }
+    
+    public INodeReference.WithCount loadINodeReferenceWithCount(
+        boolean isSnapshotINode, DataInput in, FSImageFormat.Loader loader
+        ) throws IOException {
+      final boolean firstReferred = in.readBoolean();
+
+      final INodeReference.WithCount withCount;
+      if (firstReferred) {
+        final INode referred = loader.loadINodeWithLocalName(isSnapshotINode, in);
+        withCount = new INodeReference.WithCount(null, referred);
+        referenceMap.put(withCount.getId(), withCount);
+      } else {
+        final long id = in.readLong();
+        withCount = referenceMap.get(id);
+      }
+      return withCount;
+    }
+  }
+}

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

@@ -0,0 +1,370 @@
+/**
+ * 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.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSImageFormat;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable.SnapshotDiffInfo;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Manage snapshottable directories and their snapshots.
+ * 
+ * This class includes operations that create, access, modify snapshots and/or
+ * snapshot-related data. In general, the locking structure of snapshot
+ * operations is: <br>
+ * 
+ * 1. Lock the {@link FSNamesystem} lock in {@link FSNamesystem} before calling
+ * into {@link SnapshotManager} methods.<br>
+ * 2. Lock the {@link FSDirectory} lock for the {@link SnapshotManager} methods
+ * if necessary.
+ */
+public class SnapshotManager implements SnapshotStats {
+  private boolean allowNestedSnapshots = false;
+  private final FSDirectory fsdir;
+  private static final int SNAPSHOT_ID_BIT_WIDTH = 24;
+
+  private final AtomicInteger numSnapshots = new AtomicInteger();
+
+  private int snapshotCounter = 0;
+  
+  /** All snapshottable directories in the namesystem. */
+  private final Map<Long, INodeDirectorySnapshottable> snapshottables
+      = new HashMap<Long, INodeDirectorySnapshottable>();
+
+  public SnapshotManager(final FSDirectory fsdir) {
+    this.fsdir = fsdir;
+  }
+
+  /** Used in tests only */
+  void setAllowNestedSnapshots(boolean allowNestedSnapshots) {
+    this.allowNestedSnapshots = allowNestedSnapshots;
+  }
+
+  private void checkNestedSnapshottable(INodeDirectory dir, String path)
+      throws SnapshotException {
+    if (allowNestedSnapshots) {
+      return;
+    }
+
+    for(INodeDirectorySnapshottable s : snapshottables.values()) {
+      if (s.isAncestorDirectory(dir)) {
+        throw new SnapshotException(
+            "Nested snapshottable directories not allowed: path=" + path
+            + ", the subdirectory " + s.getFullPathName()
+            + " is already a snapshottable directory.");
+      }
+      if (dir.isAncestorDirectory(s)) {
+        throw new SnapshotException(
+            "Nested snapshottable directories not allowed: path=" + path
+            + ", the ancestor " + s.getFullPathName()
+            + " is already a snapshottable directory.");
+      }
+    }
+  }
+
+  /**
+   * Set the given directory as a snapshottable directory.
+   * If the path is already a snapshottable directory, update the quota.
+   */
+  public void setSnapshottable(final String path, boolean checkNestedSnapshottable)
+      throws IOException {
+    final INodesInPath iip = fsdir.getINodesInPath4Write(path);
+    final INodeDirectory d = INodeDirectory.valueOf(iip.getLastINode(), path);
+    if (checkNestedSnapshottable) {
+      checkNestedSnapshottable(d, path);
+    }
+
+
+    final INodeDirectorySnapshottable s;
+    if (d.isSnapshottable()) {
+      //The directory is already a snapshottable directory.
+      s = (INodeDirectorySnapshottable)d; 
+      s.setSnapshotQuota(INodeDirectorySnapshottable.SNAPSHOT_LIMIT);
+    } else {
+      s = d.replaceSelf4INodeDirectorySnapshottable(iip.getLatestSnapshot(),
+          fsdir.getINodeMap());
+    }
+    addSnapshottable(s);
+  }
+  
+  /** Add the given snapshottable directory to {@link #snapshottables}. */
+  public void addSnapshottable(INodeDirectorySnapshottable dir) {
+    snapshottables.put(dir.getId(), dir);
+  }
+
+  /** Remove the given snapshottable directory from {@link #snapshottables}. */
+  private void removeSnapshottable(INodeDirectorySnapshottable s) {
+    final INodeDirectorySnapshottable removed = snapshottables.remove(s.getId());
+    Preconditions.checkState(s == removed);
+  }
+  
+  /** Remove snapshottable directories from {@link #snapshottables} */
+  public void removeSnapshottable(List<INodeDirectorySnapshottable> toRemove) {
+    if (toRemove != null) {
+      for (INodeDirectorySnapshottable s : toRemove) {
+        removeSnapshottable(s);
+      }
+    }
+  }
+
+  /**
+   * Set the given snapshottable directory to non-snapshottable.
+   * 
+   * @throws SnapshotException if there are snapshots in the directory.
+   */
+  public void resetSnapshottable(final String path) throws IOException {
+    final INodesInPath iip = fsdir.getINodesInPath4Write(path);
+    final INodeDirectorySnapshottable s = INodeDirectorySnapshottable.valueOf(
+        iip.getLastINode(), path);
+    if (s.getNumSnapshots() > 0) {
+      throw new SnapshotException("The directory " + path + " has snapshot(s). "
+          + "Please redo the operation after removing all the snapshots.");
+    }
+
+    if (s == fsdir.getRoot()) {
+      if (s.getSnapshotQuota() == 0) {
+        throw new SnapshotException("Root is not a snapshottable directory");
+      }
+      s.setSnapshotQuota(0); 
+    } else {
+      s.replaceSelf(iip.getLatestSnapshot(), fsdir.getINodeMap());
+    }
+    removeSnapshottable(s);
+  }
+
+  /**
+  * Find the source root directory where the snapshot will be taken
+  * for a given path.
+  *
+  * @param path The directory path where the snapshot will be taken.
+  * @return Snapshottable directory.
+  * @throws IOException
+  *           Throw IOException when the given path does not lead to an
+  *           existing snapshottable directory.
+  */
+  public INodeDirectorySnapshottable getSnapshottableRoot(final String path
+      ) throws IOException {
+    final INodesInPath i = fsdir.getINodesInPath4Write(path);
+    return INodeDirectorySnapshottable.valueOf(i.getLastINode(), path);
+  }
+
+  /**
+   * Create a snapshot of the given path.
+   * It is assumed that the caller will perform synchronization.
+   *
+   * @param path
+   *          The directory path where the snapshot will be taken.
+   * @param snapshotName
+   *          The name of the snapshot.
+   * @throws IOException
+   *           Throw IOException when 1) the given path does not lead to an
+   *           existing snapshottable directory, and/or 2) there exists a
+   *           snapshot with the given name for the directory, and/or 3)
+   *           snapshot number exceeds quota
+   */
+  public String createSnapshot(final String path, String snapshotName
+      ) throws IOException {
+    INodeDirectorySnapshottable srcRoot = getSnapshottableRoot(path);
+
+    if (snapshotCounter == getMaxSnapshotID()) {
+      // We have reached the maximum allowable snapshot ID and since we don't
+      // handle rollover we will fail all subsequent snapshot creation
+      // requests.
+      //
+      throw new SnapshotException(
+          "Failed to create the snapshot. The FileSystem has run out of " +
+          "snapshot IDs and ID rollover is not supported.");
+    }
+
+    srcRoot.addSnapshot(snapshotCounter, snapshotName);
+      
+    //create success, update id
+    snapshotCounter++;
+    numSnapshots.getAndIncrement();
+    return Snapshot.getSnapshotPath(path, snapshotName);
+  }
+  
+  /**
+   * Delete a snapshot for a snapshottable directory
+   * @param path Path to the directory where the snapshot was taken
+   * @param snapshotName Name of the snapshot to be deleted
+   * @param collectedBlocks Used to collect information to update blocksMap 
+   * @throws IOException
+   */
+  public void deleteSnapshot(final String path, final String snapshotName,
+      BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
+      throws IOException {
+    // parse the path, and check if the path is a snapshot path
+    // the INodeDirectorySnapshottable#valueOf method will throw Exception 
+    // if the path is not for a snapshottable directory
+    INodeDirectorySnapshottable srcRoot = getSnapshottableRoot(path);
+    srcRoot.removeSnapshot(snapshotName, collectedBlocks, removedINodes);
+    numSnapshots.getAndDecrement();
+  }
+
+  /**
+   * Rename the given snapshot
+   * @param path
+   *          The directory path where the snapshot was taken
+   * @param oldSnapshotName
+   *          Old name of the snapshot
+   * @param newSnapshotName
+   *          New name of the snapshot
+   * @throws IOException
+   *           Throw IOException when 1) the given path does not lead to an
+   *           existing snapshottable directory, and/or 2) the snapshot with the
+   *           old name does not exist for the directory, and/or 3) there exists
+   *           a snapshot with the new name for the directory
+   */
+  public void renameSnapshot(final String path, final String oldSnapshotName,
+      final String newSnapshotName) throws IOException {
+    // Find the source root directory path where the snapshot was taken.
+    // All the check for path has been included in the valueOf method.
+    final INodeDirectorySnapshottable srcRoot
+        = INodeDirectorySnapshottable.valueOf(fsdir.getINode(path), path);
+    // Note that renameSnapshot and createSnapshot are synchronized externally
+    // through FSNamesystem's write lock
+    srcRoot.renameSnapshot(path, oldSnapshotName, newSnapshotName);
+  }
+  
+  @Override
+  public int getNumSnapshottableDirs() {
+    return snapshottables.size();
+  }
+
+  @Override
+  public int getNumSnapshots() {
+    return numSnapshots.get();
+  }
+  
+  /**
+   * Write {@link #snapshotCounter}, {@link #numSnapshots},
+   * and all snapshots to the DataOutput.
+   */
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(snapshotCounter);
+    out.writeInt(numSnapshots.get());
+
+    // write all snapshots.
+    for(INodeDirectorySnapshottable snapshottableDir : snapshottables.values()) {
+      for(Snapshot s : snapshottableDir.getSnapshotsByNames()) {
+        s.write(out);
+      }
+    }
+  }
+  
+  /**
+   * Read values of {@link #snapshotCounter}, {@link #numSnapshots}, and
+   * all snapshots from the DataInput
+   */
+  public Map<Integer, Snapshot> read(DataInput in, FSImageFormat.Loader loader
+      ) throws IOException {
+    snapshotCounter = in.readInt();
+    numSnapshots.set(in.readInt());
+    
+    // read snapshots
+    final Map<Integer, Snapshot> snapshotMap = new HashMap<Integer, Snapshot>();
+    for(int i = 0; i < numSnapshots.get(); i++) {
+      final Snapshot s = Snapshot.read(in, loader);
+      snapshotMap.put(s.getId(), s);
+    }
+    return snapshotMap;
+  }
+  
+  /**
+   * List all the snapshottable directories that are owned by the current user.
+   * @param userName Current user name.
+   * @return Snapshottable directories that are owned by the current user,
+   *         represented as an array of {@link SnapshottableDirectoryStatus}. If
+   *         {@code userName} is null, return all the snapshottable dirs.
+   */
+  public SnapshottableDirectoryStatus[] getSnapshottableDirListing(
+      String userName) {
+    if (snapshottables.isEmpty()) {
+      return null;
+    }
+    
+    List<SnapshottableDirectoryStatus> statusList = 
+        new ArrayList<SnapshottableDirectoryStatus>();
+    for (INodeDirectorySnapshottable dir : snapshottables.values()) {
+      if (userName == null || userName.equals(dir.getUserName())) {
+        SnapshottableDirectoryStatus status = new SnapshottableDirectoryStatus(
+            dir.getModificationTime(), dir.getAccessTime(),
+            dir.getFsPermission(), dir.getUserName(), dir.getGroupName(),
+            dir.getLocalNameBytes(), dir.getId(), dir.getNumSnapshots(),
+            dir.getSnapshotQuota(), dir.getParent() == null ? 
+                DFSUtil.EMPTY_BYTES : 
+                DFSUtil.string2Bytes(dir.getParent().getFullPathName()));
+        statusList.add(status);
+      }
+    }
+    Collections.sort(statusList, SnapshottableDirectoryStatus.COMPARATOR);
+    return statusList.toArray(
+        new SnapshottableDirectoryStatus[statusList.size()]);
+  }
+  
+  /**
+   * Compute the difference between two snapshots of a directory, or between a
+   * snapshot of the directory and its current tree.
+   */
+  public SnapshotDiffInfo diff(final String path, final String from,
+      final String to) throws IOException {
+    if ((from == null || from.isEmpty())
+        && (to == null || to.isEmpty())) {
+      // both fromSnapshot and toSnapshot indicate the current tree
+      return null;
+    }
+
+    // Find the source root directory path where the snapshots were taken.
+    // All the check for path has been included in the valueOf method.
+    INodesInPath inodesInPath = fsdir.getINodesInPath4Write(path.toString());
+    final INodeDirectorySnapshottable snapshotRoot = INodeDirectorySnapshottable
+        .valueOf(inodesInPath.getLastINode(), path);
+    
+    return snapshotRoot.computeDiff(from, to);
+  }
+
+  /**
+   * Returns the maximum allowable snapshot ID based on the bit width of the
+   * snapshot ID.
+   *
+   * @return maximum allowable snapshot ID.
+   */
+   public int getMaxSnapshotID() {
+    return ((1 << SNAPSHOT_ID_BIT_WIDTH) - 1);
+  }
+}

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

@@ -0,0 +1,36 @@
+/**
+ * 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;
+
+/**
+ * This is an interface used to retrieve statistic information related to
+ * snapshots
+ */
+public interface SnapshotStats {
+
+  /**
+   * @return The number of snapshottale directories in the system 
+   */
+  public int getNumSnapshottableDirs();
+  
+  /**
+   * @return The number of directories that have been snapshotted
+   */
+  public int getNumSnapshots();
+  
+}

+ 62 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java

@@ -406,6 +406,30 @@ public class DFSAdmin extends FsShell {
     System.out.println("Safe mode is " + (inSafeMode ? "ON" : "OFF"));
     System.out.println("Safe mode is " + (inSafeMode ? "ON" : "OFF"));
   }
   }
 
 
+  /**
+   * Allow snapshot on a directory.
+   * Usage: java DFSAdmin -allowSnapshot snapshotDir
+   * @param argv List of of command line parameters.
+   * @exception IOException
+   */
+  public void allowSnapshot(String[] argv) throws IOException {   
+    DistributedFileSystem dfs = getDFS();
+    dfs.allowSnapshot(new Path(argv[1]));
+    System.out.println("Allowing snaphot on " + argv[1] + " succeeded");
+  }
+  
+  /**
+   * Allow snapshot on a directory.
+   * Usage: java DFSAdmin -disallowSnapshot snapshotDir
+   * @param argv List of of command line parameters.
+   * @exception IOException
+   */
+  public void disallowSnapshot(String[] argv) throws IOException {  
+    DistributedFileSystem dfs = getDFS();
+    dfs.disallowSnapshot(new Path(argv[1]));
+    System.out.println("Disallowing snaphot on " + argv[1] + " succeeded");
+  }
+  
   /**
   /**
    * Command to ask the namenode to save the namespace.
    * Command to ask the namenode to save the namespace.
    * Usage: java DFSAdmin -saveNamespace
    * Usage: java DFSAdmin -saveNamespace
@@ -547,6 +571,8 @@ public class DFSAdmin extends FsShell {
       "\t[-deleteBlockPool datanodehost:port blockpoolId [force]]\n"+
       "\t[-deleteBlockPool datanodehost:port blockpoolId [force]]\n"+
       "\t[-setBalancerBandwidth <bandwidth>]\n" +
       "\t[-setBalancerBandwidth <bandwidth>]\n" +
       "\t[-fetchImage <local directory>]\n" +
       "\t[-fetchImage <local directory>]\n" +
+      "\t[-allowSnapshot <snapshotDir>]\n" +
+      "\t[-disallowSnapshot <snapshotDir>]\n" +
       "\t[-help [cmd]]\n";
       "\t[-help [cmd]]\n";
 
 
     String report ="-report: \tReports basic filesystem information and statistics.\n";
     String report ="-report: \tReports basic filesystem information and statistics.\n";
@@ -637,6 +663,12 @@ public class DFSAdmin extends FsShell {
       "\tDownloads the most recent fsimage from the Name Node and saves it in" +
       "\tDownloads the most recent fsimage from the Name Node and saves it in" +
       "\tthe specified local directory.\n";
       "\tthe specified local directory.\n";
     
     
+    String allowSnapshot = "-allowSnapshot <snapshotDir>:\n" +
+        "\tAllow snapshots to be taken on a directory.\n";
+    
+    String disallowSnapshot = "-disallowSnapshot <snapshotDir>:\n" +
+        "\tDo not allow snapshots to be taken on a directory any more.\n";
+    
     String help = "-help [cmd]: \tDisplays help for the given command or all commands if none\n" +
     String help = "-help [cmd]: \tDisplays help for the given command or all commands if none\n" +
       "\t\tis specified.\n";
       "\t\tis specified.\n";
 
 
@@ -680,6 +712,10 @@ public class DFSAdmin extends FsShell {
       System.out.println(setBalancerBandwidth);
       System.out.println(setBalancerBandwidth);
     } else if ("fetchImage".equals(cmd)) {
     } else if ("fetchImage".equals(cmd)) {
       System.out.println(fetchImage);
       System.out.println(fetchImage);
+    } else if ("allowSnapshot".equalsIgnoreCase(cmd)) {
+      System.out.println(allowSnapshot);
+    } else if ("disallowSnapshot".equalsIgnoreCase(cmd)) {
+      System.out.println(disallowSnapshot);
     } else if ("help".equals(cmd)) {
     } else if ("help".equals(cmd)) {
       System.out.println(help);
       System.out.println(help);
     } else {
     } else {
@@ -704,6 +740,8 @@ public class DFSAdmin extends FsShell {
       System.out.println(deleteBlockPool);
       System.out.println(deleteBlockPool);
       System.out.println(setBalancerBandwidth);
       System.out.println(setBalancerBandwidth);
       System.out.println(fetchImage);
       System.out.println(fetchImage);
+      System.out.println(allowSnapshot);
+      System.out.println(disallowSnapshot);
       System.out.println(help);
       System.out.println(help);
       System.out.println();
       System.out.println();
       ToolRunner.printGenericCommandUsage(System.out);
       ToolRunner.printGenericCommandUsage(System.out);
@@ -879,7 +917,13 @@ public class DFSAdmin extends FsShell {
                          + " [-report]");
                          + " [-report]");
     } else if ("-safemode".equals(cmd)) {
     } else if ("-safemode".equals(cmd)) {
       System.err.println("Usage: java DFSAdmin"
       System.err.println("Usage: java DFSAdmin"
-                         + " [-safemode enter | leave | get | wait]");
+          + " [-safemode enter | leave | get | wait]");
+    } else if ("-allowSnapshot".equalsIgnoreCase(cmd)) {
+      System.err.println("Usage: java DFSAdmin"
+          + " [-allowSnapshot <snapshotDir>]");
+    } else if ("-disallowSnapshot".equalsIgnoreCase(cmd)) {
+      System.err.println("Usage: java DFSAdmin"
+          + " [-disallowSnapshot <snapshotDir>]");
     } else if ("-saveNamespace".equals(cmd)) {
     } else if ("-saveNamespace".equals(cmd)) {
       System.err.println("Usage: java DFSAdmin"
       System.err.println("Usage: java DFSAdmin"
                          + " [-saveNamespace]");
                          + " [-saveNamespace]");
@@ -938,7 +982,9 @@ public class DFSAdmin extends FsShell {
       System.err.println("Usage: java DFSAdmin");
       System.err.println("Usage: java DFSAdmin");
       System.err.println("Note: Administrative commands can only be run as the HDFS superuser.");
       System.err.println("Note: Administrative commands can only be run as the HDFS superuser.");
       System.err.println("           [-report]");
       System.err.println("           [-report]");
-      System.err.println("           [-safemode enter | leave | get | wait]");
+      System.err.println("           [-safemode enter | leave | get | wait]"); 
+      System.err.println("           [-allowSnapshot <snapshotDir>]");
+      System.err.println("           [-disallowSnapshot <snapshotDir>]");
       System.err.println("           [-saveNamespace]");
       System.err.println("           [-saveNamespace]");
       System.err.println("           [-rollEdits]");
       System.err.println("           [-rollEdits]");
       System.err.println("           [-restoreFailedStorage true|false|check]");
       System.err.println("           [-restoreFailedStorage true|false|check]");
@@ -988,6 +1034,16 @@ public class DFSAdmin extends FsShell {
         printUsage(cmd);
         printUsage(cmd);
         return exitCode;
         return exitCode;
       }
       }
+    } else if ("-allowSnapshot".equalsIgnoreCase(cmd)) {
+      if (argv.length != 2) {
+        printUsage(cmd);
+        return exitCode;
+      }
+    } else if ("-disallowSnapshot".equalsIgnoreCase(cmd)) {
+      if (argv.length != 2) {
+        printUsage(cmd);
+        return exitCode;
+      }
     } else if ("-report".equals(cmd)) {
     } else if ("-report".equals(cmd)) {
       if (argv.length != 1) {
       if (argv.length != 1) {
         printUsage(cmd);
         printUsage(cmd);
@@ -1079,6 +1135,10 @@ public class DFSAdmin extends FsShell {
         report();
         report();
       } else if ("-safemode".equals(cmd)) {
       } else if ("-safemode".equals(cmd)) {
         setSafeMode(argv, i);
         setSafeMode(argv, i);
+      } else if ("-allowSnapshot".equalsIgnoreCase(cmd)) {
+        allowSnapshot(argv);
+      } else if ("-disallowSnapshot".equalsIgnoreCase(cmd)) {
+        disallowSnapshot(argv);
       } else if ("-saveNamespace".equals(cmd)) {
       } else if ("-saveNamespace".equals(cmd)) {
         exitCode = saveNamespace();
         exitCode = saveNamespace();
       } else if ("-rollEdits".equals(cmd)) {
       } else if ("-rollEdits".equals(cmd)) {

+ 262 - 42
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java

@@ -22,6 +22,8 @@ import java.io.IOException;
 import java.text.DateFormat;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -30,6 +32,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
+import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.tools.offlineImageViewer.ImageVisitor.ImageElement;
 import org.apache.hadoop.hdfs.tools.offlineImageViewer.ImageVisitor.ImageElement;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.WritableUtils;
@@ -123,8 +126,11 @@ class ImageLoaderCurrent implements ImageLoader {
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
   private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
   private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
       -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39,
       -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39,
-      -40, -41, -42};
+      -40, -41, -42, -43};
   private int imageVersion = 0;
   private int imageVersion = 0;
+  
+  private final Map<Long, String> subtreeMap = new HashMap<Long, String>();
+  private final Map<Long, String> dirNodeMap = new HashMap<Long, String>();
 
 
   /* (non-Javadoc)
   /* (non-Javadoc)
    * @see ImageLoader#canProcessVersion(int)
    * @see ImageLoader#canProcessVersion(int)
@@ -162,11 +168,18 @@ class ImageLoaderCurrent implements ImageLoader {
       if (LayoutVersion.supports(Feature.STORED_TXIDS, imageVersion)) {
       if (LayoutVersion.supports(Feature.STORED_TXIDS, imageVersion)) {
         v.visit(ImageElement.TRANSACTION_ID, in.readLong());
         v.visit(ImageElement.TRANSACTION_ID, in.readLong());
       }
       }
-
+      
       if (LayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion)) {
       if (LayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion)) {
         v.visit(ImageElement.LAST_INODE_ID, in.readLong());
         v.visit(ImageElement.LAST_INODE_ID, in.readLong());
       }
       }
       
       
+      boolean supportSnapshot = LayoutVersion.supports(Feature.SNAPSHOT,
+          imageVersion);
+      if (supportSnapshot) {
+        v.visit(ImageElement.SNAPSHOT_COUNTER, in.readInt());
+        v.visit(ImageElement.NUM_SNAPSHOTS_TOTAL, in.readInt());
+      }
+      
       if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imageVersion)) {
       if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imageVersion)) {
         boolean isCompressed = in.readBoolean();
         boolean isCompressed = in.readBoolean();
         v.visit(ImageElement.IS_COMPRESSED, String.valueOf(isCompressed));
         v.visit(ImageElement.IS_COMPRESSED, String.valueOf(isCompressed));
@@ -183,7 +196,9 @@ class ImageLoaderCurrent implements ImageLoader {
           in = new DataInputStream(codec.createInputStream(in));
           in = new DataInputStream(codec.createInputStream(in));
         }
         }
       }
       }
-      processINodes(in, v, numInodes, skipBlocks);
+      processINodes(in, v, numInodes, skipBlocks, supportSnapshot);
+      subtreeMap.clear();
+      dirNodeMap.clear();
 
 
       processINodesUC(in, v, skipBlocks);
       processINodesUC(in, v, skipBlocks);
 
 
@@ -271,6 +286,12 @@ class ImageLoaderCurrent implements ImageLoader {
       byte [] name = FSImageSerialization.readBytes(in);
       byte [] name = FSImageSerialization.readBytes(in);
       String n = new String(name, "UTF8");
       String n = new String(name, "UTF8");
       v.visit(ImageElement.INODE_PATH, n);
       v.visit(ImageElement.INODE_PATH, n);
+      
+      if (LayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion)) {
+        long inodeId = in.readLong();
+        v.visit(ImageElement.INODE_ID, inodeId);
+      }
+      
       v.visit(ImageElement.REPLICATION, in.readShort());
       v.visit(ImageElement.REPLICATION, in.readShort());
       v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
       v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
 
 
@@ -360,16 +381,22 @@ class ImageLoaderCurrent implements ImageLoader {
    * @param v Visitor to walk over INodes
    * @param v Visitor to walk over INodes
    * @param numInodes Number of INodes stored in file
    * @param numInodes Number of INodes stored in file
    * @param skipBlocks Process all the blocks within the INode?
    * @param skipBlocks Process all the blocks within the INode?
+   * @param supportSnapshot Whether or not the imageVersion supports snapshot
    * @throws VisitException
    * @throws VisitException
    * @throws IOException
    * @throws IOException
    */
    */
   private void processINodes(DataInputStream in, ImageVisitor v,
   private void processINodes(DataInputStream in, ImageVisitor v,
-      long numInodes, boolean skipBlocks) throws IOException {
+      long numInodes, boolean skipBlocks, boolean supportSnapshot)
+      throws IOException {
     v.visitEnclosingElement(ImageElement.INODES,
     v.visitEnclosingElement(ImageElement.INODES,
         ImageElement.NUM_INODES, numInodes);
         ImageElement.NUM_INODES, numInodes);
     
     
     if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION, imageVersion)) {
     if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION, imageVersion)) {
-      processLocalNameINodes(in, v, numInodes, skipBlocks);
+      if (!supportSnapshot) {
+        processLocalNameINodes(in, v, numInodes, skipBlocks);
+      } else {
+        processLocalNameINodesWithSnapshot(in, v, skipBlocks);
+      }
     } else { // full path name
     } else { // full path name
       processFullNameINodes(in, v, numInodes, skipBlocks);
       processFullNameINodes(in, v, numInodes, skipBlocks);
     }
     }
@@ -390,7 +417,7 @@ class ImageLoaderCurrent implements ImageLoader {
   private void processLocalNameINodes(DataInputStream in, ImageVisitor v,
   private void processLocalNameINodes(DataInputStream in, ImageVisitor v,
       long numInodes, boolean skipBlocks) throws IOException {
       long numInodes, boolean skipBlocks) throws IOException {
     // process root
     // process root
-    processINode(in, v, skipBlocks, "");
+    processINode(in, v, skipBlocks, "", false);
     numInodes--;
     numInodes--;
     while (numInodes > 0) {
     while (numInodes > 0) {
       numInodes -= processDirectory(in, v, skipBlocks);
       numInodes -= processDirectory(in, v, skipBlocks);
@@ -400,40 +427,172 @@ class ImageLoaderCurrent implements ImageLoader {
   private int processDirectory(DataInputStream in, ImageVisitor v,
   private int processDirectory(DataInputStream in, ImageVisitor v,
      boolean skipBlocks) throws IOException {
      boolean skipBlocks) throws IOException {
     String parentName = FSImageSerialization.readString(in);
     String parentName = FSImageSerialization.readString(in);
+    return processChildren(in, v, skipBlocks, parentName);
+  }
+  
+  /**
+   * Process image with local path name and snapshot support
+   * 
+   * @param in image stream
+   * @param v visitor
+   * @param skipBlocks skip blocks or not
+   */
+  private void processLocalNameINodesWithSnapshot(DataInputStream in,
+      ImageVisitor v, boolean skipBlocks) throws IOException {
+    // process root
+    processINode(in, v, skipBlocks, "", false);
+    processDirectoryWithSnapshot(in, v, skipBlocks);
+  }
+  
+  /**
+   * Process directories when snapshot is supported.
+   */
+  private void processDirectoryWithSnapshot(DataInputStream in, ImageVisitor v,
+      boolean skipBlocks) throws IOException {
+    // 1. load dir node id
+    long inodeId = in.readLong();
+    
+    String dirName = dirNodeMap.get(inodeId);
+    String oldValue = subtreeMap.put(inodeId, dirName);
+    if (oldValue != null) { // the subtree has been visited
+      return;
+    }
+    
+    // 2. load possible snapshots
+    processSnapshots(in, v, dirName);
+    // 3. load children nodes
+    processChildren(in, v, skipBlocks, dirName);
+    // 4. load possible directory diff list
+    processDirectoryDiffList(in, v, dirName);
+    // recursively process sub-directories
+    final int numSubTree = in.readInt();
+    for (int i = 0; i < numSubTree; i++) {
+      processDirectoryWithSnapshot(in, v, skipBlocks);
+    }
+  }
+  
+  /**
+   * Process snapshots of a snapshottable directory
+   */
+  private void processSnapshots(DataInputStream in, ImageVisitor v,
+      String rootName) throws IOException {
+    final int numSnapshots = in.readInt();
+    if (numSnapshots >= 0) {
+      v.visitEnclosingElement(ImageElement.SNAPSHOTS,
+          ImageElement.NUM_SNAPSHOTS, numSnapshots);
+      for (int i = 0; i < numSnapshots; i++) {
+        // process snapshot
+        v.visitEnclosingElement(ImageElement.SNAPSHOT);
+        v.visit(ImageElement.SNAPSHOT_ID, in.readInt());
+        // process root of snapshot
+        v.visitEnclosingElement(ImageElement.SNAPSHOT_ROOT);
+        processINode(in, v, true, rootName, false);
+        v.leaveEnclosingElement();
+        v.leaveEnclosingElement();
+      }
+      v.visit(ImageElement.SNAPSHOT_QUOTA, in.readInt());
+      v.leaveEnclosingElement();
+    }
+  }
+  
+  private void processDirectoryDiffList(DataInputStream in, ImageVisitor v,
+      String currentINodeName) throws IOException {
+    final int numDirDiff = in.readInt();
+    if (numDirDiff >= 0) {
+      v.visitEnclosingElement(ImageElement.SNAPSHOT_DIR_DIFFS,
+          ImageElement.NUM_SNAPSHOT_DIR_DIFF, numDirDiff);
+      for (int i = 0; i < numDirDiff; i++) {
+        // process directory diffs in reverse chronological oder
+        processDirectoryDiff(in, v, currentINodeName); 
+      }
+      v.leaveEnclosingElement();
+    }
+  }
+  
+  private void processDirectoryDiff(DataInputStream in, ImageVisitor v,
+      String currentINodeName) throws IOException {
+    v.visitEnclosingElement(ImageElement.SNAPSHOT_DIR_DIFF);
+    String snapshot = FSImageSerialization.readString(in);
+    v.visit(ImageElement.SNAPSHOT_DIFF_SNAPSHOTROOT, snapshot);
+    v.visit(ImageElement.SNAPSHOT_DIR_DIFF_CHILDREN_SIZE, in.readInt());
+    
+    // process snapshotINode
+    boolean useRoot = in.readBoolean();
+    if (!useRoot) {
+      if (in.readBoolean()) {
+        v.visitEnclosingElement(ImageElement.SNAPSHOT_DIFF_SNAPSHOTINODE);
+        processINode(in, v, true, currentINodeName, true);
+        v.leaveEnclosingElement();
+      }
+    }
+    
+    // process createdList
+    int createdSize = in.readInt();
+    v.visitEnclosingElement(ImageElement.SNAPSHOT_DIR_DIFF_CREATEDLIST,
+        ImageElement.SNAPSHOT_DIR_DIFF_CREATEDLIST_SIZE, createdSize);
+    for (int i = 0; i < createdSize; i++) {
+      String createdNode = FSImageSerialization.readString(in);
+      v.visit(ImageElement.SNAPSHOT_DIR_DIFF_CREATED_INODE, createdNode);
+    }
+    v.leaveEnclosingElement();
+    
+    // process deletedList
+    int deletedSize = in.readInt();
+    v.visitEnclosingElement(ImageElement.SNAPSHOT_DIR_DIFF_DELETEDLIST,
+        ImageElement.SNAPSHOT_DIR_DIFF_DELETEDLIST_SIZE, deletedSize);
+    for (int i = 0; i < deletedSize; i++) {
+      v.visitEnclosingElement(ImageElement.SNAPSHOT_DIR_DIFF_DELETED_INODE);
+      processINode(in, v, false, currentINodeName, true);
+      v.leaveEnclosingElement();
+    }
+    v.leaveEnclosingElement();
+    v.leaveEnclosingElement();
+  }
+  
+  /** Process children under a directory */
+  private int processChildren(DataInputStream in, ImageVisitor v,
+      boolean skipBlocks, String parentName) throws IOException {
     int numChildren = in.readInt();
     int numChildren = in.readInt();
-    for (int i=0; i<numChildren; i++) {
-      processINode(in, v, skipBlocks, parentName);
+    for (int i = 0; i < numChildren; i++) {
+      processINode(in, v, skipBlocks, parentName, false);
     }
     }
     return numChildren;
     return numChildren;
   }
   }
   
   
-   /**
-    * Process image with full path name
-    * 
-    * @param in image stream
-    * @param v visitor
-    * @param numInodes number of indoes to read
-    * @param skipBlocks skip blocks or not
-    * @throws IOException if there is any error occurs
-    */
-   private void processFullNameINodes(DataInputStream in, ImageVisitor v,
-       long numInodes, boolean skipBlocks) throws IOException {
-     for(long i = 0; i < numInodes; i++) {
-       processINode(in, v, skipBlocks, null);
-     }
-   }
-   
-   /**
-    * Process an INode
-    * 
-    * @param in image stream
-    * @param v visitor
-    * @param skipBlocks skip blocks or not
-    * @param parentName the name of its parent node
-    * @throws IOException
-    */
+  /**
+   * Process image with full path name
+   * 
+   * @param in image stream
+   * @param v visitor
+   * @param numInodes number of indoes to read
+   * @param skipBlocks skip blocks or not
+   * @throws IOException if there is any error occurs
+   */
+  private void processFullNameINodes(DataInputStream in, ImageVisitor v,
+      long numInodes, boolean skipBlocks) throws IOException {
+    for(long i = 0; i < numInodes; i++) {
+      processINode(in, v, skipBlocks, null, false);
+    }
+  }
+ 
+  /**
+   * Process an INode
+   * 
+   * @param in image stream
+   * @param v visitor
+   * @param skipBlocks skip blocks or not
+   * @param parentName the name of its parent node
+   * @param isSnapshotCopy whether or not the inode is a snapshot copy
+   * @throws IOException
+   */
   private void processINode(DataInputStream in, ImageVisitor v,
   private void processINode(DataInputStream in, ImageVisitor v,
-      boolean skipBlocks, String parentName) throws IOException {
+      boolean skipBlocks, String parentName, boolean isSnapshotCopy)
+      throws IOException {
+    boolean supportSnapshot = 
+        LayoutVersion.supports(Feature.SNAPSHOT, imageVersion);
+    boolean supportInodeId = 
+        LayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion);
+    
     v.visitEnclosingElement(ImageElement.INODE);
     v.visitEnclosingElement(ImageElement.INODE);
     String pathName = FSImageSerialization.readString(in);
     String pathName = FSImageSerialization.readString(in);
     if (parentName != null) {  // local name
     if (parentName != null) {  // local name
@@ -443,9 +602,11 @@ class ImageLoaderCurrent implements ImageLoader {
       }
       }
     }
     }
 
 
+    long inodeId = INodeId.GRANDFATHER_INODE_ID;
     v.visit(ImageElement.INODE_PATH, pathName);
     v.visit(ImageElement.INODE_PATH, pathName);
-    if (LayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion)) {
-      v.visit(ImageElement.INODE_ID, in.readLong());
+    if (supportInodeId) {
+      inodeId = in.readLong();
+      v.visit(ImageElement.INODE_ID, inodeId);
     }
     }
     v.visit(ImageElement.REPLICATION, in.readShort());
     v.visit(ImageElement.REPLICATION, in.readShort());
     v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
     v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
@@ -455,21 +616,80 @@ class ImageLoaderCurrent implements ImageLoader {
     int numBlocks = in.readInt();
     int numBlocks = in.readInt();
 
 
     processBlocks(in, v, numBlocks, skipBlocks);
     processBlocks(in, v, numBlocks, skipBlocks);
-
-    // File or directory
-    if (numBlocks > 0 || numBlocks == -1) {
+    
+    if (numBlocks > 0) { // File
+      if (supportSnapshot) {
+        // process file diffs
+        processFileDiffList(in, v, parentName);
+        if (isSnapshotCopy) {
+          boolean underConstruction = in.readBoolean();
+          if (underConstruction) {
+            v.visit(ImageElement.CLIENT_NAME,
+                FSImageSerialization.readString(in));
+            v.visit(ImageElement.CLIENT_MACHINE,
+                FSImageSerialization.readString(in));
+          }
+        }
+      }
+    } else if (numBlocks == -1) { // Directory
+      if (supportSnapshot && supportInodeId) {
+        dirNodeMap.put(inodeId, pathName);
+      }
       v.visit(ImageElement.NS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
       v.visit(ImageElement.NS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
       if (LayoutVersion.supports(Feature.DISKSPACE_QUOTA, imageVersion))
       if (LayoutVersion.supports(Feature.DISKSPACE_QUOTA, imageVersion))
         v.visit(ImageElement.DS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
         v.visit(ImageElement.DS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
-    }
-    if (numBlocks == -2) {
+      if (supportSnapshot) {
+        boolean snapshottable = in.readBoolean();
+        if (!snapshottable) {
+          boolean withSnapshot = in.readBoolean();
+          v.visit(ImageElement.IS_WITHSNAPSHOT_DIR, Boolean.toString(withSnapshot));
+        } else {
+          v.visit(ImageElement.IS_SNAPSHOTTABLE_DIR, Boolean.toString(snapshottable));
+        }
+      }
+    } else if (numBlocks == -2) {
       v.visit(ImageElement.SYMLINK, Text.readString(in));
       v.visit(ImageElement.SYMLINK, Text.readString(in));
+    } else if (numBlocks == -3) { // reference node
+      final boolean isWithName = in.readBoolean();
+      int snapshotId = in.readInt();
+      if (isWithName) {
+        v.visit(ImageElement.SNAPSHOT_LAST_SNAPSHOT_ID, snapshotId);
+      } else {
+        v.visit(ImageElement.SNAPSHOT_DST_SNAPSHOT_ID, snapshotId);
+      }
+      
+      final boolean firstReferred = in.readBoolean();
+      if (firstReferred) {
+        v.visitEnclosingElement(ImageElement.SNAPSHOT_REF_INODE);
+        processINode(in, v, skipBlocks, parentName, isSnapshotCopy);
+        v.leaveEnclosingElement();  // referred inode    
+      } else {
+        v.visit(ImageElement.SNAPSHOT_REF_INODE_ID, in.readLong());
+      }
     }
     }
 
 
     processPermission(in, v);
     processPermission(in, v);
     v.leaveEnclosingElement(); // INode
     v.leaveEnclosingElement(); // INode
   }
   }
-
+  
+  private void processFileDiffList(DataInputStream in, ImageVisitor v,
+      String currentINodeName) throws IOException {
+    final int size = in.readInt();
+    if (size >= 0) {
+      v.visitEnclosingElement(ImageElement.SNAPSHOT_FILE_DIFFS,
+          ImageElement.NUM_SNAPSHOT_FILE_DIFF, size);
+      String snapshot = FSImageSerialization.readString(in);
+      v.visit(ImageElement.SNAPSHOT_DIFF_SNAPSHOTROOT, snapshot);
+      v.visit(ImageElement.SNAPSHOT_FILE_SIZE, in.readLong());
+      if (in.readBoolean()) {
+        v.visitEnclosingElement(ImageElement.SNAPSHOT_DIFF_SNAPSHOTINODE);
+        processINode(in, v, true, currentINodeName, true);
+        v.leaveEnclosingElement();
+      }
+      v.leaveEnclosingElement();
+    }
+  }
+  
   /**
   /**
    * Helper method to format dates during processing.
    * Helper method to format dates during processing.
    * @param date Date as read from image file
    * @param date Date as read from image file

+ 32 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.java

@@ -82,7 +82,38 @@ abstract class ImageVisitor {
     DELEGATION_TOKEN_IDENTIFIER_MASTER_KEY_ID,
     DELEGATION_TOKEN_IDENTIFIER_MASTER_KEY_ID,
     TRANSACTION_ID,
     TRANSACTION_ID,
     LAST_INODE_ID,
     LAST_INODE_ID,
-    INODE_ID
+    INODE_ID,
+
+    SNAPSHOT_COUNTER,
+    NUM_SNAPSHOTS_TOTAL,
+    NUM_SNAPSHOTS,
+    SNAPSHOTS,
+    SNAPSHOT,
+    SNAPSHOT_ID,
+    SNAPSHOT_ROOT,
+    SNAPSHOT_QUOTA,
+    NUM_SNAPSHOT_DIR_DIFF,
+    SNAPSHOT_DIR_DIFFS,
+    SNAPSHOT_DIR_DIFF,
+    SNAPSHOT_DIFF_SNAPSHOTROOT,
+    SNAPSHOT_DIR_DIFF_CHILDREN_SIZE,
+    SNAPSHOT_DIFF_SNAPSHOTINODE,
+    SNAPSHOT_DIR_DIFF_CREATEDLIST,
+    SNAPSHOT_DIR_DIFF_CREATEDLIST_SIZE,
+    SNAPSHOT_DIR_DIFF_CREATED_INODE,
+    SNAPSHOT_DIR_DIFF_DELETEDLIST,
+    SNAPSHOT_DIR_DIFF_DELETEDLIST_SIZE,
+    SNAPSHOT_DIR_DIFF_DELETED_INODE,
+    IS_SNAPSHOTTABLE_DIR,
+    IS_WITHSNAPSHOT_DIR,
+    SNAPSHOT_FILE_DIFFS,
+    SNAPSHOT_FILE_DIFF,
+    NUM_SNAPSHOT_FILE_DIFF,
+    SNAPSHOT_FILE_SIZE,
+    SNAPSHOT_DST_SNAPSHOT_ID,
+    SNAPSHOT_LAST_SNAPSHOT_ID,
+    SNAPSHOT_REF_INODE_ID,
+    SNAPSHOT_REF_INODE
   }
   }
   
   
   /**
   /**

+ 58 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/snapshot/LsSnapshottableDir.java

@@ -0,0 +1,58 @@
+/**
+ * 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.tools.snapshot;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+
+/**
+ * A tool used to list all snapshottable directories that are owned by the 
+ * current user. The tool returns all the snapshottable directories if the user
+ * is a super user.
+ */
+@InterfaceAudience.Private
+public class LsSnapshottableDir {
+  public static void main(String[] argv) throws IOException {
+    String description = "LsSnapshottableDir: \n" +
+        "\tGet the list of snapshottable directories that are owned by the current user.\n" +
+        "\tReturn all the snapshottable directories if the current user is a super user.\n";
+
+    if(argv.length != 0) {
+      System.err.println("Usage: \n" + description);
+      System.exit(1);
+    }
+    
+    Configuration conf = new Configuration();
+    FileSystem fs = FileSystem.get(conf);
+    if (! (fs instanceof DistributedFileSystem)) {
+      System.err.println(
+          "LsSnapshottableDir can only be used in DistributedFileSystem");
+      System.exit(1);
+    }
+    DistributedFileSystem dfs = (DistributedFileSystem) fs;
+    
+    SnapshottableDirectoryStatus[] stats = dfs.getSnapshottableDirListing();
+    SnapshottableDirectoryStatus.print(stats, System.out);
+  }
+
+}

+ 90 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/snapshot/SnapshotDiff.java

@@ -0,0 +1,90 @@
+/**
+ * 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.tools.snapshot;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+
+/**
+ * A tool used to get the difference report between two snapshots, or between
+ * a snapshot and the current status of a directory. 
+ * <pre>
+ * Usage: SnapshotDiff snapshotDir from to
+ * For from/to, users can use "." to present the current status, and use 
+ * ".snapshot/snapshot_name" to present a snapshot, where ".snapshot/" can be 
+ * omitted.
+ * </pre>
+ */
+@InterfaceAudience.Private
+public class SnapshotDiff {
+  private static String getSnapshotName(String name) {
+    if (Path.CUR_DIR.equals(name)) { // current directory
+      return "";
+    }
+    final int i;
+    if (name.startsWith(HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR)) {
+      i = 0;
+    } else if (name.startsWith(
+        HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR + Path.SEPARATOR)) {
+      i = 1;
+    } else {
+      return name;
+    }
+
+    // get the snapshot name
+    return name.substring(i + HdfsConstants.DOT_SNAPSHOT_DIR.length() + 1);
+  }
+  
+  public static void main(String[] argv) throws IOException {
+    String description = "SnapshotDiff <snapshotDir> <from> <to>:\n" +
+    "\tGet the difference between two snapshots, \n" + 
+    "\tor between a snapshot and the current tree of a directory.\n" +
+    "\tFor <from>/<to>, users can use \".\" to present the current status,\n" +
+    "\tand use \".snapshot/snapshot_name\" to present a snapshot,\n" +
+    "\twhere \".snapshot/\" can be omitted\n";
+    
+    if(argv.length != 3) {
+      System.err.println("Usage: \n" + description);
+      System.exit(1);
+    }
+    
+    Configuration conf = new Configuration();
+    FileSystem fs = FileSystem.get(conf);
+    if (! (fs instanceof DistributedFileSystem)) {
+      System.err.println(
+          "SnapshotDiff can only be used in DistributedFileSystem");
+      System.exit(1);
+    }
+    DistributedFileSystem dfs = (DistributedFileSystem) fs;
+    
+    Path snapshotRoot = new Path(argv[0]);
+    String fromSnapshot = getSnapshotName(argv[1]);
+    String toSnapshot = getSnapshotName(argv[2]);
+    SnapshotDiffReport diffReport = dfs.getSnapshotDiffReport(snapshotRoot,
+        fromSnapshot, toSnapshot);
+    System.out.println(diffReport.toString());
+  }
+
+}

+ 485 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/Diff.java

@@ -0,0 +1,485 @@
+/**
+ * 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.util;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * The difference between the current state and a previous state of a list.
+ * 
+ * Given a previous state of a set and a sequence of create, delete and modify
+ * operations such that the current state of the set can be obtained by applying
+ * the operations on the previous state, the following algorithm construct the
+ * difference between the current state and the previous state of the set.
+ * 
+ * <pre>
+ * Two lists are maintained in the algorithm:
+ * - c-list for newly created elements
+ * - d-list for the deleted elements
+ *
+ * Denote the state of an element by the following
+ *   (0, 0): neither in c-list nor d-list
+ *   (c, 0): in c-list but not in d-list
+ *   (0, d): in d-list but not in c-list
+ *   (c, d): in both c-list and d-list
+ *
+ * For each case below, ( , ) at the end shows the result state of the element.
+ *
+ * Case 1. Suppose the element i is NOT in the previous state.           (0, 0)
+ *   1.1. create i in current: add it to c-list                          (c, 0)
+ *   1.1.1. create i in current and then create: impossible
+ *   1.1.2. create i in current and then delete: remove it from c-list   (0, 0)
+ *   1.1.3. create i in current and then modify: replace it in c-list    (c', 0)
+ *
+ *   1.2. delete i from current: impossible
+ *
+ *   1.3. modify i in current: impossible
+ *
+ * Case 2. Suppose the element i is ALREADY in the previous state.       (0, 0)
+ *   2.1. create i in current: impossible
+ *
+ *   2.2. delete i from current: add it to d-list                        (0, d)
+ *   2.2.1. delete i from current and then create: add it to c-list      (c, d)
+ *   2.2.2. delete i from current and then delete: impossible
+ *   2.2.2. delete i from current and then modify: impossible
+ *
+ *   2.3. modify i in current: put it in both c-list and d-list          (c, d)
+ *   2.3.1. modify i in current and then create: impossible
+ *   2.3.2. modify i in current and then delete: remove it from c-list   (0, d)
+ *   2.3.3. modify i in current and then modify: replace it in c-list    (c', d)
+ * </pre>
+ *
+ * @param <K> The key type.
+ * @param <E> The element type, which must implement {@link Element} interface.
+ */
+public class Diff<K, E extends Diff.Element<K>> {
+  public static enum ListType {
+    CREATED, DELETED
+  }
+
+  /** An interface for the elements in a {@link Diff}. */
+  public static interface Element<K> extends Comparable<K> {
+    /** @return the key of this object. */
+    public K getKey();
+  }
+
+  /** An interface for passing a method in order to process elements. */
+  public static interface Processor<E> {
+    /** Process the given element. */
+    public void process(E element);
+  }
+
+  /** Containing exactly one element. */
+  public static class Container<E> {
+    private final E element;
+
+    private Container(E element) {
+      this.element = element;
+    }
+
+    /** @return the element. */
+    public E getElement() {
+      return element;
+    }
+  }
+  
+  /** 
+   * Undo information for some operations such as delete(E)
+   * and {@link Diff#modify(Element, Element)}.
+   */
+  public static class UndoInfo<E> {
+    private final int createdInsertionPoint;
+    private final E trashed;
+    private final Integer deletedInsertionPoint;
+    
+    private UndoInfo(final int createdInsertionPoint, final E trashed,
+        final Integer deletedInsertionPoint) {
+      this.createdInsertionPoint = createdInsertionPoint;
+      this.trashed = trashed;
+      this.deletedInsertionPoint = deletedInsertionPoint;
+    }
+    
+    public E getTrashedElement() {
+      return trashed;
+    }
+  }
+
+  private static final int DEFAULT_ARRAY_INITIAL_CAPACITY = 4;
+
+  /**
+   * Search the element from the list.
+   * @return -1 if the list is null; otherwise, return the insertion point
+   *    defined in {@link Collections#binarySearch(List, Object)}.
+   *    Note that, when the list is null, -1 is the correct insertion point.
+   */
+  protected static <K, E extends Comparable<K>> int search(
+      final List<E> elements, final K name) {
+    return elements == null? -1: Collections.binarySearch(elements, name);
+  }
+
+  private static <E> void remove(final List<E> elements, final int i,
+      final E expected) {
+    final E removed = elements.remove(-i - 1);
+    Preconditions.checkState(removed == expected,
+        "removed != expected=%s, removed=%s.", expected, removed);
+  }
+
+  /** c-list: element(s) created in current. */
+  private List<E> created;
+  /** d-list: element(s) deleted from current. */
+  private List<E> deleted;
+  
+  protected Diff() {}
+
+  protected Diff(final List<E> created, final List<E> deleted) {
+    this.created = created;
+    this.deleted = deleted;
+  }
+
+  /** @return the created list, which is never null. */
+  public List<E> getList(final ListType type) {
+    final List<E> list = type == ListType.CREATED? created: deleted;
+    return list == null? Collections.<E>emptyList(): list;
+  }
+
+  public int searchIndex(final ListType type, final K name) {
+    return search(getList(type), name);
+  }
+
+  /**
+   * @return null if the element is not found;
+   *         otherwise, return the element in the created/deleted list.
+   */
+  public E search(final ListType type, final K name) {
+    final List<E> list = getList(type); 
+    final int c = search(list, name);
+    return c < 0 ? null : list.get(c);
+  }
+  
+  /** @return true if no changes contained in the diff */
+  public boolean isEmpty() {
+    return (created == null || created.isEmpty())
+        && (deleted == null || deleted.isEmpty());
+  }
+  
+  /**
+   * Insert the given element to the created/deleted list.
+   * @param i the insertion point defined
+   *          in {@link Collections#binarySearch(List, Object)}
+   */
+  private void insert(final ListType type, final E element, final int i) {
+    List<E> list = type == ListType.CREATED? created: deleted; 
+    if (i >= 0) {
+      throw new AssertionError("Element already exists: element=" + element
+          + ", " + type + "=" + list);
+    }
+    if (list == null) {
+      list = new ArrayList<E>(DEFAULT_ARRAY_INITIAL_CAPACITY);
+      if (type == ListType.CREATED) {
+        created = list;
+      } else if (type == ListType.DELETED){
+        deleted = list;
+      }
+    }
+    list.add(-i - 1, element);
+  }
+
+  /**
+   * Create an element in current state.
+   * @return the c-list insertion point for undo.
+   */
+  public int create(final E element) {
+    final int c = search(created, element.getKey());
+    insert(ListType.CREATED, element, c);
+    return c;
+  }
+
+  /**
+   * 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);
+  }
+
+  /**
+   * Delete an element from current state.
+   * @return the undo information.
+   */
+  public UndoInfo<E> delete(final E element) {
+    final int c = search(created, element.getKey());
+    E previous = null;
+    Integer d = null;
+    if (c >= 0) {
+      // remove a newly created element
+      previous = created.remove(c);
+    } else {
+      // not in c-list, it must be in previous
+      d = search(deleted, element.getKey());
+      insert(ListType.DELETED, element, d);
+    }
+    return new UndoInfo<E>(c, previous, d);
+  }
+  
+  /**
+   * 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;
+    if (c >= 0) {
+      created.add(c, undoInfo.trashed);
+    } else {
+      remove(deleted, undoInfo.deletedInsertionPoint, element);
+    }
+  }
+
+  /**
+   * Modify an element in current state.
+   * @return the undo information.
+   */
+  public UndoInfo<E> modify(final E oldElement, final E newElement) {
+    Preconditions.checkArgument(oldElement != newElement,
+        "They are the same object: oldElement == newElement = %s", newElement);
+    Preconditions.checkArgument(oldElement.compareTo(newElement.getKey()) == 0,
+        "The names do not match: oldElement=%s, newElement=%s",
+        oldElement, newElement);
+    final int c = search(created, newElement.getKey());
+    E previous = null;
+    Integer d = null;
+    if (c >= 0) {
+      // Case 1.1.3 and 2.3.3: element is already in c-list,
+      previous = created.set(c, newElement);
+      
+      // For previous != oldElement, set it to oldElement
+      previous = oldElement;
+    } else {
+      d = search(deleted, oldElement.getKey());
+      if (d < 0) {
+        // Case 2.3: neither in c-list nor d-list
+        insert(ListType.CREATED, newElement, c);
+        insert(ListType.DELETED, oldElement, d);
+      }
+    }
+    return new UndoInfo<E>(c, previous, d);
+  }
+
+  /**
+   * 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) {
+    final int c = undoInfo.createdInsertionPoint;
+    if (c >= 0) {
+      created.set(c, undoInfo.trashed);
+    } else {
+      final int d = undoInfo.deletedInsertionPoint;
+      if (d < 0) {
+        remove(created, c, newElement);
+        remove(deleted, d, oldElement);
+      }
+    }
+  }
+
+  /**
+   * Find an element in the previous state.
+   * 
+   * @return null if the element cannot be determined in the previous state
+   *         since no change is recorded and it should be determined in the
+   *         current state; otherwise, return a {@link Container} containing the
+   *         element in the previous state. Note that the element can possibly
+   *         be null which means that the element is not found in the previous
+   *         state.
+   */
+  public Container<E> accessPrevious(final K name) {
+    return accessPrevious(name, created, deleted);
+  }
+
+  private static <K, E extends Diff.Element<K>> Container<E> accessPrevious(
+      final K name, final List<E> clist, final List<E> dlist) {
+    final int d = search(dlist, name);
+    if (d >= 0) {
+      // the element was in previous and was once deleted in current.
+      return new Container<E>(dlist.get(d));
+    } else {
+      final int c = search(clist, name);
+      // When c >= 0, the element in current is a newly created element.
+      return c < 0? null: new Container<E>(null);
+    }
+  }
+
+  /**
+   * Find an element in the current state.
+   * 
+   * @return null if the element cannot be determined in the current state since
+   *         no change is recorded and it should be determined in the previous
+   *         state; otherwise, return a {@link Container} containing the element in
+   *         the current state. Note that the element can possibly be null which
+   *         means that the element is not found in the current state.
+   */
+  public Container<E> accessCurrent(K name) {
+    return accessPrevious(name, deleted, created);
+  }
+
+  /**
+   * Apply this diff to previous state in order to obtain current state.
+   * @return the current state of the list.
+   */
+  public List<E> apply2Previous(final List<E> previous) {
+    return apply2Previous(previous,
+        getList(ListType.CREATED), getList(ListType.DELETED));
+  }
+
+  private static <K, E extends Diff.Element<K>> List<E> apply2Previous(
+      final List<E> previous, final List<E> clist, final List<E> dlist) {
+    // Assumptions:
+    // (A1) All lists are sorted.
+    // (A2) All elements in dlist must be in previous.
+    // (A3) All elements in clist must be not in tmp = previous - dlist.
+    final List<E> tmp = new ArrayList<E>(previous.size() - dlist.size());
+    {
+      // tmp = previous - dlist
+      final Iterator<E> i = previous.iterator();
+      for(E deleted : dlist) {
+        E e = i.next(); //since dlist is non-empty, e must exist by (A2).
+        int cmp = 0;
+        for(; (cmp = e.compareTo(deleted.getKey())) < 0; e = i.next()) {
+          tmp.add(e);
+        }
+        Preconditions.checkState(cmp == 0); // check (A2)
+      }
+      for(; i.hasNext(); ) {
+        tmp.add(i.next());
+      }
+    }
+
+    final List<E> current = new ArrayList<E>(tmp.size() + clist.size());
+    {
+      // current = tmp + clist
+      final Iterator<E> tmpIterator = tmp.iterator();
+      final Iterator<E> cIterator = clist.iterator();
+
+      E t = tmpIterator.hasNext()? tmpIterator.next(): null;
+      E c = cIterator.hasNext()? cIterator.next(): null;
+      for(; t != null || c != null; ) {
+        final int cmp = c == null? 1
+            : t == null? -1
+            : c.compareTo(t.getKey());
+
+        if (cmp < 0) {
+          current.add(c);
+          c = cIterator.hasNext()? cIterator.next(): null;
+        } else if (cmp > 0) {
+          current.add(t);
+          t = tmpIterator.hasNext()? tmpIterator.next(): null;
+        } else {
+          throw new AssertionError("Violated assumption (A3).");
+        }
+      }
+    }
+    return current;
+  }
+
+  /**
+   * Apply the reverse of this diff to current state in order
+   * to obtain the previous state.
+   * @return the previous state of the list.
+   */
+  public List<E> apply2Current(final List<E> current) {
+    return apply2Previous(current,
+        getList(ListType.DELETED), getList(ListType.CREATED));
+  }
+  
+  /**
+   * Combine this diff with a posterior diff.  We have the following cases:
+   * 
+   * <pre>
+   * 1. For (c, 0) in the posterior diff, check the element in this diff:
+   * 1.1 (c', 0)  in this diff: impossible
+   * 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 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 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 modify(E, E).
+   * </pre>
+   * 
+   * @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.
+   */
+  public void combinePosterior(final Diff<K, E> posterior,
+      final Processor<E> deletedProcesser) {
+    final Iterator<E> createdIterator = posterior.getList(ListType.CREATED).iterator();
+    final Iterator<E> deletedIterator = posterior.getList(ListType.DELETED).iterator();
+
+    E c = createdIterator.hasNext()? createdIterator.next(): null;
+    E d = deletedIterator.hasNext()? deletedIterator.next(): null;
+
+    for(; c != null || d != null; ) {
+      final int cmp = c == null? 1
+          : d == null? -1
+          : c.compareTo(d.getKey());
+      if (cmp < 0) {
+        // case 1: only in c-list
+        create(c);
+        c = createdIterator.hasNext()? createdIterator.next(): null;
+      } else if (cmp > 0) {
+        // case 2: only in d-list
+        final UndoInfo<E> ui = delete(d);
+        if (deletedProcesser != null) {
+          deletedProcesser.process(ui.trashed);
+        }
+        d = deletedIterator.hasNext()? deletedIterator.next(): null;
+      } else {
+        // case 3: in both c-list and d-list 
+        final UndoInfo<E> ui = modify(d, c);
+        if (deletedProcesser != null) {
+          deletedProcesser.process(ui.trashed);
+        }
+        c = createdIterator.hasNext()? createdIterator.next(): null;
+        d = deletedIterator.hasNext()? deletedIterator.next(): null;
+      }
+    }
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName()
+        +  "{created=" + getList(ListType.CREATED)
+        + ", deleted=" + getList(ListType.DELETED) + "}";
+  }
+}

+ 170 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/EnumCounters.java

@@ -0,0 +1,170 @@
+/**
+ * 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.util;
+
+import java.util.HashMap;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Counters for an enum type.
+ * 
+ * For example, suppose there is an enum type
+ * <pre>
+ * enum Fruit { APPLE, ORANGE, GRAPE }
+ * </pre>
+ * An {@link EnumCounters} object can be created for counting the numbers of
+ * APPLE, ORANGLE and GRAPE.
+ *
+ * @param <E> the enum type
+ */
+public class EnumCounters<E extends Enum<E>> {
+  /** An array of enum constants. */
+  private final E[] enumConstants;
+  /** The counter array, counters[i] corresponds to the enumConstants[i]. */
+  private final long[] counters;
+
+  /**
+   * Construct counters for the given enum constants.
+   * @param enumConstants an array of enum constants such that, 
+   *                      for all i, enumConstants[i].ordinal() == i.
+   */
+  public EnumCounters(final E[] enumConstants) {
+    for(int i = 0; i < enumConstants.length; i++) {
+      Preconditions.checkArgument(enumConstants[i].ordinal() == i);
+    }
+    this.enumConstants = enumConstants;
+    this.counters = new long[enumConstants.length];
+  }
+  
+  /** @return the value of counter e. */
+  public final long get(final E e) {
+    return counters[e.ordinal()];
+  }
+
+  /** Negate all counters. */
+  public final void negation() {
+    for(int i = 0; i < counters.length; i++) {
+      counters[i] = -counters[i];
+    }
+  }
+  
+  /** Set counter e to the given value. */
+  public final void set(final E e, final long value) {
+    counters[e.ordinal()] = value;
+  }
+
+  /** Add the given value to counter e. */
+  public final void add(final E e, final long value) {
+    counters[e.ordinal()] += value;
+  }
+
+  /** Add that counters to this counters. */
+  public final void add(final EnumCounters<E> that) {
+    for(int i = 0; i < counters.length; i++) {
+      this.counters[i] += that.counters[i];
+    }
+  }
+
+  /** Subtract the given value from counter e. */
+  public final void subtract(final E e, final long value) {
+    counters[e.ordinal()] -= value;
+  }
+
+  /** Subtract that counters from this counters. */
+  public final void subtract(final EnumCounters<E> that) {
+    for(int i = 0; i < counters.length; i++) {
+      this.counters[i] -= that.counters[i];
+    }
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder b = new StringBuilder();
+    for(int i = 0; i < counters.length; i++) {
+      final String name = enumConstants[i].name();
+      b.append(name).append("=").append(counters[i]).append(", ");
+    }
+    return b.substring(0, b.length() - 2);
+  }
+
+  /**
+   * A factory for creating counters.
+   * 
+   * @param <E> the enum type
+   * @param <C> the counter type
+   */
+  public static interface Factory<E extends Enum<E>,
+                                  C extends EnumCounters<E>> {
+    /** Create a new counters instance. */
+    public C newInstance(); 
+  }
+
+  /**
+   * A key-value map which maps the keys to {@link EnumCounters}.
+   * Note that null key is supported.
+   *
+   * @param <K> the key type
+   * @param <E> the enum type
+   * @param <C> the counter type
+   */
+  public static class Map<K, E extends Enum<E>, C extends EnumCounters<E>> {
+    /** The factory for creating counters. */
+    private final Factory<E, C> factory;
+    /** Key-to-Counts map. */
+    private final java.util.Map<K, C> counts = new HashMap<K, C>();
+    
+    /** Construct a map. */
+    public Map(final Factory<E, C> factory) {
+      this.factory = factory;
+    }
+
+    /** @return the counters for the given key. */
+    public final C getCounts(final K key) {
+      C c = counts.get(key);
+      if (c == null) {
+        c = factory.newInstance();
+        counts.put(key, c); 
+      }
+      return c;
+    }
+    
+    /** @return the sum of the values of all the counters. */
+    public final C sum() {
+      final C sum = factory.newInstance();
+      for(C c : counts.values()) {
+        sum.add(c);
+      }
+      return sum;
+    }
+    
+    /** @return the sum of the values of all the counters for e. */
+    public final long sum(final E e) {
+      long sum = 0;
+      for(C c : counts.values()) {
+        sum += c.get(e);
+      }
+      return sum;
+    }
+
+    @Override
+    public String toString() {
+      return counts.toString();
+    }
+  }
+}

+ 240 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ReadOnlyList.java

@@ -0,0 +1,240 @@
+/**
+ * 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.util;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * A {@link ReadOnlyList} is a unmodifiable list,
+ * which supports read-only operations.
+ * 
+ * @param <E> The type of the list elements.
+ */
+@InterfaceAudience.Private
+public interface ReadOnlyList<E> extends Iterable<E> {
+  /**
+   * Is this an empty list?
+   */
+  boolean isEmpty();
+
+  /**
+   * @return the size of this list.
+   */
+  int size();
+
+  /**
+   * @return the i-th element.
+   */
+  E get(int i);
+  
+  /**
+   * Utilities for {@link ReadOnlyList}
+   */
+  public static class Util {
+    /** @return an empty list. */
+    public static <E> ReadOnlyList<E> emptyList() {
+      return ReadOnlyList.Util.asReadOnlyList(Collections.<E>emptyList());
+    }
+
+    /**
+     * The same as {@link Collections#binarySearch(List, Object)}
+     * except that the list is a {@link ReadOnlyList}.
+     *
+     * @return the insertion point defined
+     *         in {@link Collections#binarySearch(List, Object)}.
+     */
+    public static <K, E extends Comparable<K>> int binarySearch(
+        final ReadOnlyList<E> list, final K key) {
+      int lower = 0;
+      for(int upper = list.size() - 1; lower <= upper; ) {
+        final int mid = (upper + lower) >>> 1;
+
+        final int d = list.get(mid).compareTo(key);
+        if (d == 0) {
+          return mid;
+        } else if (d > 0) {
+          upper = mid - 1;
+        } else {
+          lower = mid + 1;
+        }
+      }
+      return -(lower + 1);
+    }
+
+    /**
+     * @return a {@link ReadOnlyList} view of the given list.
+     */
+    public static <E> ReadOnlyList<E> asReadOnlyList(final List<E> list) {
+      return new ReadOnlyList<E>() {
+        @Override
+        public Iterator<E> iterator() {
+          return list.iterator();
+        }
+
+        @Override
+        public boolean isEmpty() {
+          return list.isEmpty();
+        }
+
+        @Override
+        public int size() {
+          return list.size();
+        }
+
+        @Override
+        public E get(int i) {
+          return list.get(i);
+        }
+      };
+    }
+
+    /**
+     * @return a {@link List} view of the given list.
+     */
+    public static <E> List<E> asList(final ReadOnlyList<E> list) {
+      return new List<E>() {
+        @Override
+        public Iterator<E> iterator() {
+          return list.iterator();
+        }
+
+        @Override
+        public boolean isEmpty() {
+          return list.isEmpty();
+        }
+
+        @Override
+        public int size() {
+          return list.size();
+        }
+
+        @Override
+        public E get(int i) {
+          return list.get(i);
+        }
+
+        @Override
+        public Object[] toArray() {
+          final Object[] a = new Object[size()];
+          for(int i = 0; i < a.length; i++) {
+            a[i] = get(i);
+          }
+          return a;
+        }
+
+        //All methods below are not supported.
+
+        @Override
+        public boolean add(E e) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void add(int index, E element) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean addAll(Collection<? extends E> c) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean addAll(int index, Collection<? extends E> c) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void clear() {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean contains(Object o) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean containsAll(Collection<?> c) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public int indexOf(Object o) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public int lastIndexOf(Object o) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public ListIterator<E> listIterator() {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public ListIterator<E> listIterator(int index) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean remove(Object o) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public E remove(int index) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean removeAll(Collection<?> c) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean retainAll(Collection<?> c) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public E set(int index, E element) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public List<E> subList(int fromIndex, int toIndex) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public <T> T[] toArray(T[] a) {
+          throw new UnsupportedOperationException();
+        }
+      };
+    }
+  }
+}

+ 1 - 18
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -45,21 +45,16 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.DelegationTokenRenewer;
 import org.apache.hadoop.fs.DelegationTokenRenewer;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.ByteRangeInputStream;
 import org.apache.hadoop.hdfs.ByteRangeInputStream;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
@@ -95,7 +90,6 @@ import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryUtils;
 import org.apache.hadoop.io.retry.RetryUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
@@ -351,18 +345,7 @@ public class WebHdfsFileSystem extends FileSystem
       return ioe;
       return ioe;
     }
     }
 
 
-    final RemoteException re = (RemoteException)ioe;
-    return re.unwrapRemoteException(AccessControlException.class,
-        InvalidToken.class,
-        AuthenticationException.class,
-        AuthorizationException.class,
-        FileAlreadyExistsException.class,
-        FileNotFoundException.class,
-        ParentNotDirectoryException.class,
-        UnresolvedPathException.class,
-        SafeModeException.class,
-        DSQuotaExceededException.class,
-        NSQuotaExceededException.class);
+    return ((RemoteException)ioe).unwrapRemoteException();
   }
   }
 
 
   /**
   /**

+ 69 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -212,6 +212,21 @@ message GetListingResponseProto {
   optional DirectoryListingProto dirList = 1;
   optional DirectoryListingProto dirList = 1;
 }
 }
 
 
+message GetSnapshottableDirListingRequestProto { // no input parameters
+}
+message GetSnapshottableDirListingResponseProto {
+  optional SnapshottableDirectoryListingProto snapshottableDirList = 1;
+}
+
+message GetSnapshotDiffReportRequestProto {
+  required string snapshotRoot = 1;
+  required string fromSnapshot = 2;
+  required string toSnapshot = 3;
+}
+message GetSnapshotDiffReportResponseProto {
+  required SnapshotDiffReportProto diffReport = 1;
+}
+
 message RenewLeaseRequestProto {
 message RenewLeaseRequestProto {
   required string clientName = 1;
   required string clientName = 1;
 }
 }
@@ -434,6 +449,46 @@ message GetDataEncryptionKeyResponseProto {
   optional DataEncryptionKeyProto dataEncryptionKey = 1;
   optional DataEncryptionKeyProto dataEncryptionKey = 1;
 }
 }
 
 
+message CreateSnapshotRequestProto {
+  required string snapshotRoot = 1;
+  optional string snapshotName = 2;
+}
+
+message CreateSnapshotResponseProto {
+  required string snapshotPath = 1;
+}
+
+message RenameSnapshotRequestProto {
+  required string snapshotRoot = 1;
+  required string snapshotOldName = 2;
+  required string snapshotNewName = 3;
+}
+
+message RenameSnapshotResponseProto { // void response
+}
+
+message AllowSnapshotRequestProto {
+  required string snapshotRoot = 1;
+}
+
+message AllowSnapshotResponseProto {
+}
+
+message DisallowSnapshotRequestProto {
+  required string snapshotRoot = 1;
+}
+
+message DisallowSnapshotResponseProto {
+}
+
+message DeleteSnapshotRequestProto {
+  required string snapshotRoot = 1;
+  required string snapshotName = 2;
+}
+
+message DeleteSnapshotResponseProto { // void response
+}
+
 service ClientNamenodeProtocol {
 service ClientNamenodeProtocol {
   rpc getBlockLocations(GetBlockLocationsRequestProto)
   rpc getBlockLocations(GetBlockLocationsRequestProto)
       returns(GetBlockLocationsResponseProto);
       returns(GetBlockLocationsResponseProto);
@@ -507,6 +562,20 @@ service ClientNamenodeProtocol {
       returns(SetBalancerBandwidthResponseProto);
       returns(SetBalancerBandwidthResponseProto);
   rpc getDataEncryptionKey(GetDataEncryptionKeyRequestProto)
   rpc getDataEncryptionKey(GetDataEncryptionKeyRequestProto)
       returns(GetDataEncryptionKeyResponseProto);
       returns(GetDataEncryptionKeyResponseProto);
+  rpc createSnapshot(CreateSnapshotRequestProto)
+      returns(CreateSnapshotResponseProto);
+  rpc renameSnapshot(RenameSnapshotRequestProto)
+      returns(RenameSnapshotResponseProto);
+  rpc allowSnapshot(AllowSnapshotRequestProto)
+      returns(AllowSnapshotResponseProto);
+  rpc disallowSnapshot(DisallowSnapshotRequestProto)
+      returns(DisallowSnapshotResponseProto);   
+  rpc getSnapshottableDirListing(GetSnapshottableDirListingRequestProto)
+      returns(GetSnapshottableDirListingResponseProto);
+  rpc deleteSnapshot(DeleteSnapshotRequestProto)
+      returns(DeleteSnapshotResponseProto);
+  rpc getSnapshotDiffReport(GetSnapshotDiffReportRequestProto)
+      returns(GetSnapshotDiffReportResponseProto);
   rpc isFileClosed(IsFileClosedRequestProto)
   rpc isFileClosed(IsFileClosedRequestProto)
       returns(IsFileClosedResponseProto);
       returns(IsFileClosedResponseProto);
 }
 }

+ 55 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto

@@ -145,7 +145,7 @@ message LocatedBlocksProto {
 
 
 
 
 /**
 /**
- * Status of a file, directory  or symlink
+ * Status of a file, directory or symlink
  * Optionally includes a file's block locations if requested by client on the rpc call.
  * Optionally includes a file's block locations if requested by client on the rpc call.
  */
  */
 message HdfsFileStatusProto {
 message HdfsFileStatusProto {
@@ -209,6 +209,46 @@ message DirectoryListingProto {
   required uint32 remainingEntries  = 2;
   required uint32 remainingEntries  = 2;
 }
 }
 
 
+/**
+ * Status of a snapshottable directory: besides the normal information for 
+ * a directory status, also include snapshot quota, number of snapshots, and
+ * the full path of the parent directory. 
+ */
+message SnapshottableDirectoryStatusProto {
+  required HdfsFileStatusProto dirStatus = 1;
+
+  // Fields specific for snapshottable directory
+  required uint32 snapshot_quota = 2;
+  required uint32 snapshot_number = 3;
+  required bytes parent_fullpath = 4;
+}
+
+/**
+ * Snapshottable directory listing
+ */
+message SnapshottableDirectoryListingProto {
+  repeated SnapshottableDirectoryStatusProto snapshottableDirListing = 1;
+}
+
+/**
+ * Snapshot diff report entry
+ */
+message SnapshotDiffReportEntryProto {
+  required bytes fullpath = 1;
+  required string modificationLabel = 2;
+}
+
+/**
+ * Snapshot diff report
+ */
+message SnapshotDiffReportProto {
+  // full path of the directory where snapshots were taken
+  required string snapshotRoot = 1;
+  required string fromSnapshot = 2;
+  required string toSnapshot = 3;
+  repeated SnapshotDiffReportEntryProto diffReportEntries = 4;
+}
+
 /**
 /**
  * Common node information shared by all the nodes in the cluster
  * Common node information shared by all the nodes in the cluster
  */
  */
@@ -374,3 +414,17 @@ message VersionResponseProto {
   required NamespaceInfoProto info = 1;
   required NamespaceInfoProto info = 1;
 }
 }
 
 
+/**
+ * Information related to a snapshot
+ * TODO: add more information
+ */
+message SnapshotInfoProto {
+  required string snapshotName = 1;
+  required string snapshotRoot = 2;
+  required FsPermissionProto permission = 3;
+  required string owner = 4;
+  required string group = 5;
+  required string createTime = 6;
+  // TODO: do we need access time?
+}
+

+ 262 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/xdoc/HdfsSnapshots.xml

@@ -0,0 +1,262 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<document xmlns="http://maven.apache.org/XDOC/2.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
+
+  <properties>
+    <title>HFDS Snapshots</title>
+  </properties>
+
+  <body>
+
+  <h1>HDFS Snapshots</h1>
+  <macro name="toc">
+    <param name="section" value="0"/>
+    <param name="fromDepth" value="0"/>
+    <param name="toDepth" value="4"/>
+  </macro>
+
+  <section name="Overview" id="Overview">
+  <p>
+    HDFS Snapshots are read-only point-in-time copies of the file system.
+    Snapshots can be taken on a subtree of the file system or the entire file system.
+    Some common use cases of snapshots are data backup, protection against user errors
+    and disaster recovery.
+  </p>
+
+  <p>
+    The implementation of HDFS Snapshots is efficient:
+  </p>
+  <ul>
+    <li>Snapshot creation is instantaneous:
+        the cost is <em>O(1)</em> excluding the inode lookup time.</li>
+    <li>Additional memory is used only when modifications are made relative to a snapshot:
+        memory usage is <em>O(M)</em>,
+        where <em>M</em> is the number of modified files/directories.</li>
+    <li>Blocks in datanodes are not copied:
+        the snapshot files record the block list and the file size.
+        There is no data copying.</li>
+    <li>Snapshots do not adversely affect regular HDFS operations:
+        modifications are recorded in reverse chronological order
+        so that the current data can be accessed directly.
+        The snapshot data is computed by subtracting the modifications
+        from the current data.</li>
+  </ul>
+
+  <subsection name="Snapshottable Directories" id="SnapshottableDirectories">
+  <p>
+    Snapshots can be taken on any directory once the directory has been set as
+    <em>snapshottable</em>.
+    A snapshottable directory is able to accommodate 65,536 simultaneous snapshots.
+    There is no limit on the number of snapshottable directories.
+    Administrators may set any directory to be snapshottable.
+    If there are snapshots in a snapshottable directory,
+    the directory can be neither deleted nor renamed
+    before all the snapshots are deleted.
+  </p>
+<!--
+  <p>
+    Nested snapshottable directories are currently not allowed.
+    In other words, a directory cannot be set to snapshottable
+    if one of its ancestors is a snapshottable directory.
+  </p>
+-->
+  </subsection>
+
+  <subsection name="Snapshot Paths" id="SnapshotPaths">
+  <p>
+    For a snapshottable directory,
+    the path component <em>".snapshot"</em> is used for accessing its snapshots.
+    Suppose <code>/foo</code> is a snapshottable directory,
+    <code>/foo/bar</code> is a file/directory in <code>/foo</code>,
+    and <code>/foo</code> has a snapshot <code>s0</code>.
+    Then, the path <source>/foo/.snapshot/s0/bar</source>
+    refers to the snapshot copy of <code>/foo/bar</code>.
+    The usual API and CLI can work with the ".snapshot" paths.
+    The following are some examples.
+  </p>
+  <ul>
+    <li>Listing all the snapshots under a snapshottable directory:
+      <source>hdfs dfs -ls /foo/.snapshot</source></li>
+    <li>Listing the files in snapshot <code>s0</code>:
+      <source>hdfs dfs -ls /foo/.snapshot/s0</source></li>
+    <li>Copying a file from snapshot <code>s0</code>:
+      <source>hdfs dfs -cp /foo/.snapshot/s0/bar /tmp</source></li>
+  </ul>
+  <p>
+    The name ".snapshot" is now a reserved file name in HDFS
+    so that users cannot create a file/directory with ".snapshot" as the name.
+    If ".snapshot" is used in a previous version of HDFS, it must be renamed before upgrade;
+    otherwise, upgrade will fail. 
+  </p>
+  </subsection>
+  </section>
+
+  <section name="Snapshot Operations" id="SnapshotOperations">
+  <subsection name="Administrator Operations" id="AdministratorOperations">
+  <p>
+    The operations described in this section require superuser privilege.
+  </p>
+
+  <h4>Allow Snapshots</h4>
+  <p>
+    Allowing snapshots of a directory to be created.
+    If the operation completes successfully, the directory becomes snapshottable.
+  </p>
+  <ul>
+    <li>Command:
+      <source>hdfs dfsadmin -allowSnapshot &lt;path&gt;</source></li>
+    <li>Arguments:<table>
+      <tr><td>path</td><td>The path of the snapshottable directory.</td></tr>
+    </table></li>
+  </ul>
+  <p>
+    See also the corresponding Java API
+    <code>void allowSnapshot(Path path)</code> in <code>HdfsAdmin</code>.
+  </p>
+
+  <h4>Disallow Snapshots</h4>
+  <p>
+    Disallowing snapshots of a directory to be created.
+    All snapshots of the directory must be deleted before disallowing snapshots.
+  </p>
+  <ul>
+    <li>Command:
+      <source>hdfs dfsadmin -disallowSnapshot &lt;path&gt;</source></li>
+    <li>Arguments:<table>
+      <tr><td>path</td><td>The path of the snapshottable directory.</td></tr>
+    </table></li>
+  </ul>
+  <p>
+    See also the corresponding Java API
+    <code>void disallowSnapshot(Path path)</code> in <code>HdfsAdmin</code>.
+  </p>
+  </subsection>
+
+  <subsection name="User Operations" id="UserOperations">
+  <p>
+    The section describes user operations.
+    Note that HDFS superuser can perform all the operations
+    without satisfying the permission requirement in the individual operations.
+  </p>
+
+  <h4>Create Snapshots</h4>
+  <p>
+    Create a snapshot of a snapshottable directory.
+    This operation requires owner privilege of the snapshottable directory.
+  </p>
+  <ul>
+    <li>Command:
+      <source>hdfs dfs -createSnapshot &lt;path&gt; [&lt;snapshotName&gt;]</source></li>
+    <li>Arguments:<table>
+      <tr><td>path</td><td>The path of the snapshottable directory.</td></tr>
+      <tr><td>snapshotName</td><td>
+        The snapshot name, which is an optional argument.
+        When it is omitted, a default name is generated using a timestamp with the format
+        <code>"'s'yyyyMMdd-HHmmss.SSS"</code>, e.g. "s20130412-151029.033".
+      </td></tr>
+    </table></li>
+  </ul>
+  <p>
+    See also the corresponding Java API
+    <code>Path createSnapshot(Path path)</code> and
+    <code>Path createSnapshot(Path path, String snapshotName)</code>
+    in <a href="../../api/org/apache/hadoop/fs/FileSystem.html"><code>FileSystem</code></a>.
+    The snapshot path is returned in these methods.
+  </p>
+
+  <h4>Delete Snapshots</h4>
+  <p>
+    Delete a snapshot of from a snapshottable directory.
+    This operation requires owner privilege of the snapshottable directory.
+  </p>
+  <ul>
+    <li>Command:
+      <source>hdfs dfs -deleteSnapshot &lt;path&gt; &lt;snapshotName&gt;</source></li>
+    <li>Arguments:<table>
+      <tr><td>path</td><td>The path of the snapshottable directory.</td></tr>
+      <tr><td>snapshotName</td><td>The snapshot name.</td></tr>
+    </table></li>
+  </ul>
+  <p>
+    See also the corresponding Java API
+    <code>void deleteSnapshot(Path path, String snapshotName)</code>
+    in <a href="../../api/org/apache/hadoop/fs/FileSystem.html"><code>FileSystem</code></a>.
+  </p>
+
+  <h4>Rename Snapshots</h4>
+  <p>
+    Rename a snapshot.
+    This operation requires owner privilege of the snapshottable directory.
+  </p>
+  <ul>
+    <li>Command:
+      <source>hdfs dfs -renameSnapshot &lt;path&gt; &lt;oldName&gt; &lt;newName&gt;</source></li>
+    <li>Arguments:<table>
+      <tr><td>path</td><td>The path of the snapshottable directory.</td></tr>
+      <tr><td>oldName</td><td>The old snapshot name.</td></tr>
+      <tr><td>newName</td><td>The new snapshot name.</td></tr>
+    </table></li>
+  </ul>
+  <p>
+    See also the corresponding Java API
+    <code>void renameSnapshot(Path path, String oldName, String newName)</code>
+    in <a href="../../api/org/apache/hadoop/fs/FileSystem.html"><code>FileSystem</code></a>.
+  </p>
+
+  <h4>Get Snapshottable Directory Listing</h4>
+  <p>
+    Get all the snapshottable directories where the current user has permission to take snapshtos.
+  </p>
+  <ul>
+    <li>Command:
+      <source>hdfs lsSnapshottableDir</source></li>
+    <li>Arguments: none</li>
+  </ul>
+  <p>
+    See also the corresponding Java API
+    <code>SnapshottableDirectoryStatus[] getSnapshottableDirectoryListing()</code>
+    in <code>DistributedFileSystem</code>.
+  </p>
+
+  <h4>Get Snapshots Difference Report</h4>
+  <p>
+    Get the differences between two snapshots.
+    This operation requires read access privilege for all files/directories in both snapshots.
+  </p>
+  <ul>
+    <li>Command:
+      <source>hdfs snapshotDiff &lt;path&gt; &lt;fromSnapshot&gt; &lt;toSnapshot&gt;</source></li>
+    <li>Arguments:<table>
+      <tr><td>path</td><td>The path of the snapshottable directory.</td></tr>
+      <tr><td>fromSnapshot</td><td>The name of the starting snapshot.</td></tr>
+      <tr><td>toSnapshot</td><td>The name of the ending snapshot.</td></tr>
+    </table></li>
+  </ul>
+  <p>
+    See also the corresponding Java API
+    <code>SnapshotDiffReport getSnapshotDiffReport(Path path, String fromSnapshot, String toSnapshot)</code>
+    in <code>DistributedFileSystem</code>.
+  </p>
+
+  </subsection>
+  </section>
+
+  </body>
+</document>

+ 19 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -600,6 +600,25 @@ public class DFSTestUtil {
     IOUtils.copyBytes(is, os, s.length(), true);
     IOUtils.copyBytes(is, os, s.length(), true);
   }
   }
   
   
+  /**
+   * Append specified length of bytes to a given file
+   * @param fs The file system
+   * @param p Path of the file to append
+   * @param length Length of bytes to append to the file
+   * @throws IOException
+   */
+  public static void appendFile(FileSystem fs, Path p, int length)
+      throws IOException {
+    assert fs.exists(p);
+    assert length >= 0;
+    byte[] toAppend = new byte[length];
+    Random random = new Random();
+    random.nextBytes(toAppend);
+    FSDataOutputStream out = fs.append(p);
+    out.write(toAppend);
+    out.close();
+  }
+  
   /**
   /**
    * @return url content as string (UTF-8 encoding assumed)
    * @return url content as string (UTF-8 encoding assumed)
    */
    */

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
 
 
 /** A class for testing quota-related commands */
 /** A class for testing quota-related commands */
@@ -169,15 +170,13 @@ public class TestQuota {
       fout = dfs.create(childFile1, replication);
       fout = dfs.create(childFile1, replication);
       
       
       // 10.s: but writing fileLen bytes should result in an quota exception
       // 10.s: but writing fileLen bytes should result in an quota exception
-      hasException = false;
       try {
       try {
         fout.write(new byte[fileLen]);
         fout.write(new byte[fileLen]);
         fout.close();
         fout.close();
+        Assert.fail();
       } catch (QuotaExceededException e) {
       } catch (QuotaExceededException e) {
-        hasException = true;
         IOUtils.closeStream(fout);
         IOUtils.closeStream(fout);
       }
       }
-      assertTrue(hasException);
       
       
       //delete the file
       //delete the file
       dfs.delete(childFile1, false);
       dfs.delete(childFile1, false);

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

@@ -63,7 +63,7 @@ public class CreateEditsLog {
     PermissionStatus p = new PermissionStatus("joeDoe", "people",
     PermissionStatus p = new PermissionStatus("joeDoe", "people",
                                       new FsPermission((short)0777));
                                       new FsPermission((short)0777));
     INodeDirectory dirInode = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID,
     INodeDirectory dirInode = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID,
-        p, 0L);
+        null, p, 0L);
     editLog.logMkDir(BASE_PATH, dirInode);
     editLog.logMkDir(BASE_PATH, dirInode);
     long blockSize = 10;
     long blockSize = 10;
     BlockInfo[] blocks = new BlockInfo[blocksPerFile];
     BlockInfo[] blocks = new BlockInfo[blocksPerFile];
@@ -92,7 +92,7 @@ public class CreateEditsLog {
       // Log the new sub directory in edits
       // Log the new sub directory in edits
       if ((iF % nameGenerator.getFilesPerDirectory())  == 0) {
       if ((iF % nameGenerator.getFilesPerDirectory())  == 0) {
         String currentDir = nameGenerator.getCurrentDir();
         String currentDir = nameGenerator.getCurrentDir();
-        dirInode = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID, p, 0L);
+        dirInode = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID, null, p, 0L);
         editLog.logMkDir(currentDir, dirInode);
         editLog.logMkDir(currentDir, dirInode);
       }
       }
       editLog.logOpenFile(filePath, new INodeFileUnderConstruction(
       editLog.logOpenFile(filePath, new INodeFileUnderConstruction(

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java

@@ -47,6 +47,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirType;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirType;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -217,8 +218,8 @@ public abstract class FSImageTestUtil {
         FsPermission.createImmutable((short)0755));
         FsPermission.createImmutable((short)0755));
     for (int i = 1; i <= numDirs; i++) {
     for (int i = 1; i <= numDirs; i++) {
       String dirName = "dir" + i;
       String dirName = "dir" + i;
-      INodeDirectory dir = new INodeDirectory(newInodeId + i - 1, dirName,
-          perms);
+      INodeDirectory dir = new INodeDirectory(newInodeId + i - 1,
+          DFSUtil.string2Bytes(dirName), perms, 0L);
       editLog.logMkDir("/" + dirName, dir);
       editLog.logMkDir("/" + dirName, dir);
     }
     }
     editLog.logSync();
     editLog.logSync();

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java

@@ -155,6 +155,19 @@ public class OfflineEditsViewerHelper {
     // OP_MKDIR 3
     // OP_MKDIR 3
     Path pathDirectoryMkdir = new Path("/directory_mkdir");
     Path pathDirectoryMkdir = new Path("/directory_mkdir");
     dfs.mkdirs(pathDirectoryMkdir);
     dfs.mkdirs(pathDirectoryMkdir);
+    // OP_ALLOW_SNAPSHOT 29
+    dfs.allowSnapshot(pathDirectoryMkdir);
+    // OP_DISALLOW_SNAPSHOT 30
+    dfs.disallowSnapshot(pathDirectoryMkdir);
+    // OP_CREATE_SNAPSHOT 26
+    String ssName = "snapshot1";
+    dfs.allowSnapshot(pathDirectoryMkdir);
+    dfs.createSnapshot(pathDirectoryMkdir, ssName);
+    // OP_RENAME_SNAPSHOT 28
+    String ssNewName = "snapshot2";
+    dfs.renameSnapshot(pathDirectoryMkdir, ssName, ssNewName);
+    // OP_DELETE_SNAPSHOT 27
+    dfs.deleteSnapshot(pathDirectoryMkdir, ssNewName);
     // OP_SET_REPLICATION 4
     // OP_SET_REPLICATION 4
     s = dfs.create(pathFileCreate);
     s = dfs.create(pathFileCreate);
     s.close();
     s.close();

+ 9 - 25
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.java

@@ -51,7 +51,6 @@ public class TestFSDirectory {
 
 
   private final Path sub11 = new Path(sub1, "sub11");
   private final Path sub11 = new Path(sub1, "sub11");
   private final Path file3 = new Path(sub11, "file3");
   private final Path file3 = new Path(sub11, "file3");
-  private final Path file4 = new Path(sub1, "z_file4");
   private final Path file5 = new Path(sub1, "z_file5");
   private final Path file5 = new Path(sub1, "z_file5");
 
 
   private final Path sub2 = new Path(dir, "sub2");
   private final Path sub2 = new Path(dir, "sub2");
@@ -107,27 +106,13 @@ public class TestFSDirectory {
 
 
     for(; (line = in.readLine()) != null; ) {
     for(; (line = in.readLine()) != null; ) {
       line = line.trim();
       line = line.trim();
-      Assert.assertTrue(line.startsWith(INodeDirectory.DUMPTREE_LAST_ITEM)
-          || line.startsWith(INodeDirectory.DUMPTREE_EXCEPT_LAST_ITEM));
-      checkClassName(line);
+      if (!line.isEmpty() && !line.contains("snapshot")) {
+        Assert.assertTrue("line=" + line,
+            line.startsWith(INodeDirectory.DUMPTREE_LAST_ITEM)
+            || line.startsWith(INodeDirectory.DUMPTREE_EXCEPT_LAST_ITEM));
+        checkClassName(line);
+      }
     }
     }
-
-    LOG.info("Create a new file " + file4);
-    DFSTestUtil.createFile(hdfs, file4, 1024, REPLICATION, seed);
-
-    final StringBuffer b2 = root.dumpTreeRecursively();
-    System.out.println("b2=" + b2);
-
-    int i = 0;
-    int j = b1.length() - 1;
-    for(; b1.charAt(i) == b2.charAt(i); i++);
-    int k = b2.length() - 1;
-    for(; b1.charAt(j) == b2.charAt(k); j--, k--);
-    final String diff = b2.substring(i, k + 1);
-    System.out.println("i=" + i + ", j=" + j + ", k=" + k);
-    System.out.println("diff=" + diff);
-    Assert.assertTrue(i > j);
-    Assert.assertTrue(diff.contains(file4.getName()));
   }
   }
   
   
   @Test
   @Test
@@ -135,7 +120,7 @@ public class TestFSDirectory {
     fsdir.reset();
     fsdir.reset();
     Assert.assertFalse(fsdir.isReady());
     Assert.assertFalse(fsdir.isReady());
     final INodeDirectory root = (INodeDirectory) fsdir.getINode("/");
     final INodeDirectory root = (INodeDirectory) fsdir.getINode("/");
-    Assert.assertTrue(root.getChildrenList().isEmpty());
+    Assert.assertTrue(root.getChildrenList(null).isEmpty());
     fsdir.imageLoadComplete();
     fsdir.imageLoadComplete();
     Assert.assertTrue(fsdir.isReady());
     Assert.assertTrue(fsdir.isReady());
   }
   }
@@ -144,8 +129,7 @@ public class TestFSDirectory {
     int i = line.lastIndexOf('(');
     int i = line.lastIndexOf('(');
     int j = line.lastIndexOf('@');
     int j = line.lastIndexOf('@');
     final String classname = line.substring(i+1, j);
     final String classname = line.substring(i+1, j);
-    Assert.assertTrue(classname.equals(INodeFile.class.getSimpleName())
-        || classname.equals(INodeDirectory.class.getSimpleName())
-        || classname.equals(INodeDirectoryWithQuota.class.getSimpleName()));
+    Assert.assertTrue(classname.startsWith(INodeFile.class.getSimpleName())
+        || classname.startsWith(INodeDirectory.class.getSimpleName()));
   }
   }
 }
 }

+ 453 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java

@@ -0,0 +1,453 @@
+/**
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+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.SnapshotTestHelper;
+import org.apache.hadoop.hdfs.util.Canceler;
+import org.apache.log4j.Level;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test FSImage save/load when Snapshot is supported
+ */
+public class TestFSImageWithSnapshot {
+  {
+    SnapshotTestHelper.disableLogs();
+    ((Log4JLogger)INode.LOG).getLogger().setLevel(Level.ALL);
+  }
+
+  static final long seed = 0;
+  static final short REPLICATION = 3;
+  static final int BLOCKSIZE = 1024;
+  static final long txid = 1;
+
+  private final Path dir = new Path("/TestSnapshot");
+  private static String testDir =
+      System.getProperty("test.build.data", "build/test/data");
+  
+  Configuration conf;
+  MiniDFSCluster cluster;
+  FSNamesystem fsn;
+  DistributedFileSystem hdfs;
+  
+  @Before
+  public void setUp() throws Exception {
+    conf = new Configuration();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
+        .build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Create a temp fsimage file for testing.
+   * @param dir The directory where the fsimage file resides
+   * @param imageTxId The transaction id of the fsimage
+   * @return The file of the image file
+   */
+  private File getImageFile(String dir, long imageTxId) {
+    return new File(dir, String.format("%s_%019d", NameNodeFile.IMAGE,
+        imageTxId));
+  }
+  
+  /** 
+   * Create a temp file for dumping the fsdir
+   * @param dir directory for the temp file
+   * @param suffix suffix of of the temp file
+   * @return the temp file
+   */
+  private File getDumpTreeFile(String dir, String suffix) {
+    return new File(dir, String.format("dumpTree_%s", suffix));
+  }
+  
+  /** 
+   * Dump the fsdir tree to a temp file
+   * @param fileSuffix suffix of the temp file for dumping
+   * @return the temp file
+   */
+  private File dumpTree2File(String fileSuffix) throws IOException {
+    File file = getDumpTreeFile(testDir, fileSuffix);
+    SnapshotTestHelper.dumpTree2File(fsn.getFSDirectory(), file);
+    return file;
+  }
+  
+  /** Append a file without closing the output stream */
+  private HdfsDataOutputStream appendFileWithoutClosing(Path file, int length)
+      throws IOException {
+    byte[] toAppend = new byte[length];
+    Random random = new Random();
+    random.nextBytes(toAppend);
+    HdfsDataOutputStream out = (HdfsDataOutputStream) hdfs.append(file);
+    out.write(toAppend);
+    return out;
+  }
+  
+  /** Save the fsimage to a temp file */
+  private File saveFSImageToTempFile() throws IOException {
+    SaveNamespaceContext context = new SaveNamespaceContext(fsn, txid,
+        new Canceler());
+    FSImageFormat.Saver saver = new FSImageFormat.Saver(context);
+    FSImageCompression compression = FSImageCompression.createCompression(conf);
+    File imageFile = getImageFile(testDir, txid);
+    fsn.readLock();
+    try {
+      saver.save(imageFile, compression);
+    } finally {
+      fsn.readUnlock();
+    }
+    return imageFile;
+  }
+  
+  /** Load the fsimage from a temp file */
+  private void loadFSImageFromTempFile(File imageFile) throws IOException {
+    FSImageFormat.Loader loader = new FSImageFormat.Loader(conf, fsn);
+    fsn.writeLock();
+    fsn.getFSDirectory().writeLock();
+    try {
+      loader.load(imageFile);
+      FSImage.updateCountForQuota(
+          (INodeDirectoryWithQuota)fsn.getFSDirectory().getINode("/"));
+    } finally {
+      fsn.getFSDirectory().writeUnlock();
+      fsn.writeUnlock();
+    }
+  }
+  
+  /**
+   * Testing steps:
+   * <pre>
+   * 1. Creating/modifying directories/files while snapshots are being taken.
+   * 2. Dump the FSDirectory tree of the namesystem.
+   * 3. Save the namesystem to a temp file (FSImage saving).
+   * 4. Restart the cluster and format the namesystem.
+   * 5. Load the namesystem from the temp file (FSImage loading).
+   * 6. Dump the FSDirectory again and compare the two dumped string.
+   * </pre>
+   */
+  @Test
+  public void testSaveLoadImage() throws Exception {
+    int s = 0;
+    // make changes to the namesystem
+    hdfs.mkdirs(dir);
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s" + ++s);
+    Path sub1 = new Path(dir, "sub1");
+    hdfs.mkdirs(sub1);
+    hdfs.setPermission(sub1, new FsPermission((short)0777));
+    Path sub11 = new Path(sub1, "sub11");
+    hdfs.mkdirs(sub11);
+    checkImage(s);
+
+    hdfs.createSnapshot(dir, "s" + ++s);
+    Path sub1file1 = new Path(sub1, "sub1file1");
+    Path sub1file2 = new Path(sub1, "sub1file2");
+    DFSTestUtil.createFile(hdfs, sub1file1, BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, sub1file2, BLOCKSIZE, REPLICATION, seed);
+    checkImage(s);
+    
+    hdfs.createSnapshot(dir, "s" + ++s);
+    Path sub2 = new Path(dir, "sub2");
+    Path sub2file1 = new Path(sub2, "sub2file1");
+    Path sub2file2 = new Path(sub2, "sub2file2");
+    DFSTestUtil.createFile(hdfs, sub2file1, BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, sub2file2, BLOCKSIZE, REPLICATION, seed);
+    checkImage(s);
+
+    hdfs.createSnapshot(dir, "s" + ++s);
+    hdfs.setReplication(sub1file1, (short) (REPLICATION - 1));
+    hdfs.delete(sub1file2, true);
+    hdfs.setOwner(sub2, "dr.who", "unknown");
+    hdfs.delete(sub2file1, true);
+    checkImage(s);
+    
+    hdfs.createSnapshot(dir, "s" + ++s);
+    Path sub1_sub2file2 = new Path(sub1, "sub2file2");
+    hdfs.rename(sub2file2, sub1_sub2file2);
+    
+    hdfs.rename(sub1file1, sub2file1);
+    checkImage(s);
+    
+    hdfs.rename(sub2file1, sub2file2);
+    checkImage(s);
+  }
+
+  void checkImage(int s) throws IOException {
+    final String name = "s" + s;
+
+    // dump the fsdir tree
+    File fsnBefore = dumpTree2File(name + "_before");
+    
+    // save the namesystem to a temp file
+    File imageFile = saveFSImageToTempFile();
+    
+    long numSdirBefore = fsn.getNumSnapshottableDirs();
+    long numSnapshotBefore = fsn.getNumSnapshots();
+    SnapshottableDirectoryStatus[] dirBefore = hdfs.getSnapshottableDirListing();
+
+    // shutdown the cluster
+    cluster.shutdown();
+
+    // dump the fsdir tree
+    File fsnBetween = dumpTree2File(name + "_between");
+    SnapshotTestHelper.compareDumpedTreeInFile(fsnBefore, fsnBetween, true);
+
+    // restart the cluster, and format the cluster
+    cluster = new MiniDFSCluster.Builder(conf).format(true)
+        .numDataNodes(REPLICATION).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+    
+    // load the namesystem from the temp file
+    loadFSImageFromTempFile(imageFile);
+    
+    // dump the fsdir tree again
+    File fsnAfter = dumpTree2File(name + "_after");
+    
+    // compare two dumped tree
+    SnapshotTestHelper.compareDumpedTreeInFile(fsnBefore, fsnAfter, true);
+    
+    long numSdirAfter = fsn.getNumSnapshottableDirs();
+    long numSnapshotAfter = fsn.getNumSnapshots();
+    SnapshottableDirectoryStatus[] dirAfter = hdfs.getSnapshottableDirListing();
+    
+    Assert.assertEquals(numSdirBefore, numSdirAfter);
+    Assert.assertEquals(numSnapshotBefore, numSnapshotAfter);
+    Assert.assertEquals(dirBefore.length, dirAfter.length);
+    List<String> pathListBefore = new ArrayList<String>();
+    for (SnapshottableDirectoryStatus sBefore : dirBefore) {
+      pathListBefore.add(sBefore.getFullPath().toString());
+    }
+    for (SnapshottableDirectoryStatus sAfter : dirAfter) {
+      Assert.assertTrue(pathListBefore.contains(sAfter.getFullPath().toString()));
+    }
+  }
+  
+  /**
+   * Test the fsimage saving/loading while file appending.
+   */
+  @Test (timeout=60000)
+  public void testSaveLoadImageWithAppending() throws Exception {
+    Path sub1 = new Path(dir, "sub1");
+    Path sub1file1 = new Path(sub1, "sub1file1");
+    Path sub1file2 = new Path(sub1, "sub1file2");
+    DFSTestUtil.createFile(hdfs, sub1file1, BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, sub1file2, BLOCKSIZE, REPLICATION, seed);
+    
+    // 1. create snapshot s0
+    hdfs.allowSnapshot(dir);
+    hdfs.createSnapshot(dir, "s0");
+    
+    // 2. create snapshot s1 before appending sub1file1 finishes
+    HdfsDataOutputStream out = appendFileWithoutClosing(sub1file1, BLOCKSIZE);
+    out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
+    // also append sub1file2
+    DFSTestUtil.appendFile(hdfs, sub1file2, BLOCKSIZE);
+    hdfs.createSnapshot(dir, "s1");
+    out.close();
+    
+    // 3. create snapshot s2 before appending finishes
+    out = appendFileWithoutClosing(sub1file1, BLOCKSIZE);
+    out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
+    hdfs.createSnapshot(dir, "s2");
+    out.close();
+    
+    // 4. save fsimage before appending finishes
+    out = appendFileWithoutClosing(sub1file1, BLOCKSIZE);
+    out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
+    // dump fsdir
+    File fsnBefore = dumpTree2File("before");
+    // save the namesystem to a temp file
+    File imageFile = saveFSImageToTempFile();
+    
+    // 5. load fsimage and compare
+    // first restart the cluster, and format the cluster
+    out.close();
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).format(true)
+        .numDataNodes(REPLICATION).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+    // then load the fsimage
+    loadFSImageFromTempFile(imageFile);
+    
+    // dump the fsdir tree again
+    File fsnAfter = dumpTree2File("after");
+    
+    // compare two dumped tree
+    SnapshotTestHelper.compareDumpedTreeInFile(fsnBefore, fsnAfter, true);
+  }
+  
+  /**
+   * Test the fsimage loading while there is file under construction.
+   */
+  @Test (timeout=60000)
+  public void testLoadImageWithAppending() throws Exception {
+    Path sub1 = new Path(dir, "sub1");
+    Path sub1file1 = new Path(sub1, "sub1file1");
+    Path sub1file2 = new Path(sub1, "sub1file2");
+    DFSTestUtil.createFile(hdfs, sub1file1, BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, sub1file2, BLOCKSIZE, REPLICATION, seed);
+    
+    hdfs.allowSnapshot(dir);
+    hdfs.createSnapshot(dir, "s0");
+    
+    HdfsDataOutputStream out = appendFileWithoutClosing(sub1file1, BLOCKSIZE);
+    out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));      
+    
+    // save namespace and restart cluster
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.saveNamespace();
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).format(false)
+        .numDataNodes(REPLICATION).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+  }
+  
+  /**
+   * Test fsimage loading when 1) there is an empty file loaded from fsimage,
+   * and 2) there is later an append operation to be applied from edit log.
+   */
+  @Test (timeout=60000)
+  public void testLoadImageWithEmptyFile() throws Exception {
+    // create an empty file
+    Path file = new Path(dir, "file");
+    FSDataOutputStream out = hdfs.create(file);
+    out.close();
+    
+    // save namespace
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.saveNamespace();
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    
+    // append to the empty file
+    out = hdfs.append(file);
+    out.write(1);
+    out.close();
+    
+    // restart cluster
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).format(false)
+        .numDataNodes(REPLICATION).build();
+    cluster.waitActive();
+    hdfs = cluster.getFileSystem();
+    
+    FileStatus status = hdfs.getFileStatus(file);
+    assertEquals(1, status.getLen());
+  }
+  
+  /**
+   * Testing a special case with snapshots. When the following steps happen:
+   * <pre>
+   * 1. Take snapshot s1 on dir.
+   * 2. Create new dir and files under subsubDir, which is descendant of dir.
+   * 3. Take snapshot s2 on dir.
+   * 4. Delete subsubDir.
+   * 5. Delete snapshot s2.
+   * </pre>
+   * When we merge the diff from s2 to s1 (since we deleted s2), we need to make
+   * sure all the files/dirs created after s1 should be destroyed. Otherwise
+   * we may save these files/dirs to the fsimage, and cause FileNotFound 
+   * Exception while loading fsimage.  
+   */
+  @Test (timeout=300000)
+  public void testSaveLoadImageAfterSnapshotDeletion()
+      throws Exception {
+    // create initial dir and subdir
+    Path dir = new Path("/dir");
+    Path subDir = new Path(dir, "subdir");
+    Path subsubDir = new Path(subDir, "subsubdir");
+    hdfs.mkdirs(subsubDir);
+    
+    // take snapshots on subdir and dir
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s1");
+    
+    // create new dir under initial dir
+    Path newDir = new Path(subsubDir, "newdir");
+    Path newFile = new Path(newDir, "newfile");
+    hdfs.mkdirs(newDir);
+    DFSTestUtil.createFile(hdfs, newFile, BLOCKSIZE, REPLICATION, seed);
+    
+    // create another snapshot
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s2");
+    
+    // delete subsubdir
+    hdfs.delete(subsubDir, true);
+    
+    // delete snapshot s2
+    hdfs.deleteSnapshot(dir, "s2");
+    
+    // restart cluster
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
+        .format(false).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+    
+    // save namespace to fsimage
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.saveNamespace();
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).format(false)
+        .numDataNodes(REPLICATION).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+  }
+}

+ 11 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java

@@ -31,9 +31,12 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.FSLimitException.IllegalNameException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
@@ -103,6 +106,7 @@ public class TestFsLimits {
     addChildWithName("333", null);
     addChildWithName("333", null);
     addChildWithName("4444", null);
     addChildWithName("4444", null);
     addChildWithName("55555", null);
     addChildWithName("55555", null);
+    addChildWithName(HdfsConstants.DOT_SNAPSHOT_DIR, IllegalNameException.class);
   }
   }
 
 
   @Test
   @Test
@@ -142,6 +146,7 @@ public class TestFsLimits {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_DIRECTORY_ITEMS_KEY, 2);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_DIRECTORY_ITEMS_KEY, 2);
     fsIsReady = false;
     fsIsReady = false;
     
     
+    addChildWithName(HdfsConstants.DOT_SNAPSHOT_DIR, IllegalNameException.class);
     addChildWithName("1", null);
     addChildWithName("1", null);
     addChildWithName("22", null);
     addChildWithName("22", null);
     addChildWithName("333", null);
     addChildWithName("333", null);
@@ -154,13 +159,15 @@ public class TestFsLimits {
     if (fs == null) fs = new MockFSDirectory();
     if (fs == null) fs = new MockFSDirectory();
 
 
     INode child = new INodeDirectory(getMockNamesystem().allocateNewInodeId(),
     INode child = new INodeDirectory(getMockNamesystem().allocateNewInodeId(),
-        name, perms);
-    child.setLocalName(name);
+        DFSUtil.string2Bytes(name), perms, 0L);
     
     
     Class<?> generated = null;
     Class<?> generated = null;
     try {
     try {
-      fs.verifyFsLimits(inodes, 1, child);
-      rootInode.addChild(child, false);
+      fs.verifyMaxComponentLength(child.getLocalNameBytes(), inodes, 1);
+      fs.verifyMaxDirItems(inodes, 1);
+      fs.verifyINodeName(child.getLocalNameBytes());
+
+      rootInode.addChild(child);
     } catch (QuotaExceededException e) {
     } catch (QuotaExceededException e) {
       generated = e.getClass();
       generated = e.getClass();
     }
     }

+ 38 - 66
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -52,6 +52,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.junit.Test;
 import org.junit.Test;
@@ -63,10 +64,15 @@ public class TestINodeFile {
   static final short BLOCKBITS = 48;
   static final short BLOCKBITS = 48;
   static final long BLKSIZE_MAXVALUE = ~(0xffffL << BLOCKBITS);
   static final long BLKSIZE_MAXVALUE = ~(0xffffL << BLOCKBITS);
 
 
-  private String userName = "Test";
+  private final PermissionStatus perm = new PermissionStatus(
+      "userName", null, FsPermission.getDefault());
   private short replication;
   private short replication;
   private long preferredBlockSize;
   private long preferredBlockSize;
 
 
+  INodeFile createINodeFile(short replication, long preferredBlockSize) {
+    return new INodeFile(INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
+        null, replication, preferredBlockSize);
+  }
   /**
   /**
    * Test for the Replication value. Sets a value and checks if it was set
    * Test for the Replication value. Sets a value and checks if it was set
    * correct.
    * correct.
@@ -75,11 +81,9 @@ public class TestINodeFile {
   public void testReplication () {
   public void testReplication () {
     replication = 3;
     replication = 3;
     preferredBlockSize = 128*1024*1024;
     preferredBlockSize = 128*1024*1024;
-    INodeFile inf = new INodeFile(INodeId.GRANDFATHER_INODE_ID,
-        new PermissionStatus(userName, null, FsPermission.getDefault()), null,
-        replication, 0L, 0L, preferredBlockSize);
+    INodeFile inf = createINodeFile(replication, preferredBlockSize);
     assertEquals("True has to be returned in this case", replication,
     assertEquals("True has to be returned in this case", replication,
-                 inf.getBlockReplication());
+                 inf.getFileReplication());
   }
   }
 
 
   /**
   /**
@@ -92,9 +96,7 @@ public class TestINodeFile {
               throws IllegalArgumentException {
               throws IllegalArgumentException {
     replication = -1;
     replication = -1;
     preferredBlockSize = 128*1024*1024;
     preferredBlockSize = 128*1024*1024;
-    new INodeFile(INodeId.GRANDFATHER_INODE_ID, new PermissionStatus(userName,
-        null, FsPermission.getDefault()), null, replication, 0L, 0L,
-        preferredBlockSize);
+    createINodeFile(replication, preferredBlockSize);
   }
   }
 
 
   /**
   /**
@@ -105,9 +107,7 @@ public class TestINodeFile {
   public void testPreferredBlockSize () {
   public void testPreferredBlockSize () {
     replication = 3;
     replication = 3;
     preferredBlockSize = 128*1024*1024;
     preferredBlockSize = 128*1024*1024;
-    INodeFile inf = new INodeFile(INodeId.GRANDFATHER_INODE_ID,
-        new PermissionStatus(userName, null, FsPermission.getDefault()), null,
-        replication, 0L, 0L, preferredBlockSize);
+    INodeFile inf = createINodeFile(replication, preferredBlockSize);
    assertEquals("True has to be returned in this case", preferredBlockSize,
    assertEquals("True has to be returned in this case", preferredBlockSize,
         inf.getPreferredBlockSize());
         inf.getPreferredBlockSize());
  }
  }
@@ -116,9 +116,7 @@ public class TestINodeFile {
   public void testPreferredBlockSizeUpperBound () {
   public void testPreferredBlockSizeUpperBound () {
     replication = 3;
     replication = 3;
     preferredBlockSize = BLKSIZE_MAXVALUE;
     preferredBlockSize = BLKSIZE_MAXVALUE;
-    INodeFile inf = new INodeFile(INodeId.GRANDFATHER_INODE_ID,
-        new PermissionStatus(userName, null, FsPermission.getDefault()), null,
-        replication, 0L, 0L, preferredBlockSize);
+    INodeFile inf = createINodeFile(replication, preferredBlockSize);
     assertEquals("True has to be returned in this case", BLKSIZE_MAXVALUE,
     assertEquals("True has to be returned in this case", BLKSIZE_MAXVALUE,
                  inf.getPreferredBlockSize());
                  inf.getPreferredBlockSize());
   }
   }
@@ -133,9 +131,7 @@ public class TestINodeFile {
               throws IllegalArgumentException {
               throws IllegalArgumentException {
     replication = 3;
     replication = 3;
     preferredBlockSize = -1;
     preferredBlockSize = -1;
-    new INodeFile(INodeId.GRANDFATHER_INODE_ID, new PermissionStatus(userName,
-        null, FsPermission.getDefault()), null, replication, 0L, 0L,
-        preferredBlockSize);
+    createINodeFile(replication, preferredBlockSize);
   } 
   } 
 
 
   /**
   /**
@@ -148,41 +144,31 @@ public class TestINodeFile {
               throws IllegalArgumentException {
               throws IllegalArgumentException {
     replication = 3;
     replication = 3;
     preferredBlockSize = BLKSIZE_MAXVALUE+1;
     preferredBlockSize = BLKSIZE_MAXVALUE+1;
-    new INodeFile(INodeId.GRANDFATHER_INODE_ID, new PermissionStatus(userName,
-        null, FsPermission.getDefault()), null, replication, 0L, 0L,
-        preferredBlockSize);
+    createINodeFile(replication, preferredBlockSize);
  }
  }
 
 
   @Test
   @Test
   public void testGetFullPathName() {
   public void testGetFullPathName() {
-    PermissionStatus perms = new PermissionStatus(
-      userName, null, FsPermission.getDefault());
-
     replication = 3;
     replication = 3;
     preferredBlockSize = 128*1024*1024;
     preferredBlockSize = 128*1024*1024;
-    INodeFile inf = new INodeFile(INodeId.GRANDFATHER_INODE_ID, perms, null,
-        replication, 0L, 0L, preferredBlockSize);
-    inf.setLocalName("f");
+    INodeFile inf = createINodeFile(replication, preferredBlockSize);
+    inf.setLocalName(DFSUtil.string2Bytes("f"));
 
 
     INodeDirectory root = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID,
     INodeDirectory root = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID,
-        INodeDirectory.ROOT_NAME, perms);
-    INodeDirectory dir = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID, "d",
-        perms);
+        INodeDirectory.ROOT_NAME, perm, 0L);
+    INodeDirectory dir = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID,
+        DFSUtil.string2Bytes("d"), perm, 0L);
 
 
     assertEquals("f", inf.getFullPathName());
     assertEquals("f", inf.getFullPathName());
-    assertEquals("", inf.getLocalParentDir());
 
 
-    dir.addChild(inf, false);
+    dir.addChild(inf);
     assertEquals("d"+Path.SEPARATOR+"f", inf.getFullPathName());
     assertEquals("d"+Path.SEPARATOR+"f", inf.getFullPathName());
-    assertEquals("d", inf.getLocalParentDir());
     
     
-    root.addChild(dir, false);
+    root.addChild(dir);
     assertEquals(Path.SEPARATOR+"d"+Path.SEPARATOR+"f", inf.getFullPathName());
     assertEquals(Path.SEPARATOR+"d"+Path.SEPARATOR+"f", inf.getFullPathName());
     assertEquals(Path.SEPARATOR+"d", dir.getFullPathName());
     assertEquals(Path.SEPARATOR+"d", dir.getFullPathName());
 
 
     assertEquals(Path.SEPARATOR, root.getFullPathName());
     assertEquals(Path.SEPARATOR, root.getFullPathName());
-    assertEquals(Path.SEPARATOR, root.getLocalParentDir());
-    
   }
   }
   
   
   /**
   /**
@@ -214,10 +200,14 @@ public class TestINodeFile {
       // Check the full path name of the INode associating with the file
       // Check the full path name of the INode associating with the file
       INode fnode = fsdir.getINode(file.toString());
       INode fnode = fsdir.getINode(file.toString());
       assertEquals(file.toString(), fnode.getFullPathName());
       assertEquals(file.toString(), fnode.getFullPathName());
-
+      
       // Call FSDirectory#unprotectedSetQuota which calls
       // Call FSDirectory#unprotectedSetQuota which calls
       // INodeDirectory#replaceChild
       // INodeDirectory#replaceChild
       dfs.setQuota(dir, Long.MAX_VALUE - 1, replication * fileLen * 10);
       dfs.setQuota(dir, Long.MAX_VALUE - 1, replication * fileLen * 10);
+      INode dirNode = fsdir.getINode(dir.toString());
+      assertEquals(dir.toString(), dirNode.getFullPathName());
+      assertTrue(dirNode instanceof INodeDirectoryWithQuota);
+      
       final Path newDir = new Path("/newdir");
       final Path newDir = new Path("/newdir");
       final Path newFile = new Path(newDir, "file");
       final Path newFile = new Path(newDir, "file");
       // Also rename dir
       // Also rename dir
@@ -235,27 +225,14 @@ public class TestINodeFile {
   }
   }
   
   
   @Test
   @Test
-  public void testAppendBlocks() {
+  public void testConcatBlocks() {
     INodeFile origFile = createINodeFiles(1, "origfile")[0];
     INodeFile origFile = createINodeFiles(1, "origfile")[0];
     assertEquals("Number of blocks didn't match", origFile.numBlocks(), 1L);
     assertEquals("Number of blocks didn't match", origFile.numBlocks(), 1L);
 
 
     INodeFile[] appendFiles =   createINodeFiles(4, "appendfile");
     INodeFile[] appendFiles =   createINodeFiles(4, "appendfile");
-    origFile.appendBlocks(appendFiles, getTotalBlocks(appendFiles));
+    origFile.concatBlocks(appendFiles);
     assertEquals("Number of blocks didn't match", origFile.numBlocks(), 5L);
     assertEquals("Number of blocks didn't match", origFile.numBlocks(), 5L);
   }
   }
-
-  /** 
-   * Gives the count of blocks for a given number of files
-   * @param files Array of INode files
-   * @return total count of blocks
-   */
-  private int getTotalBlocks(INodeFile[] files) {
-    int nBlocks=0;
-    for(int i=0; i < files.length; i++) {
-       nBlocks += files[i].numBlocks();
-    }
-    return nBlocks;
-  }
   
   
   /** 
   /** 
    * Creates the required number of files with one block each
    * Creates the required number of files with one block each
@@ -270,11 +247,9 @@ public class TestINodeFile {
     preferredBlockSize = 128 * 1024 * 1024;
     preferredBlockSize = 128 * 1024 * 1024;
     INodeFile[] iNodes = new INodeFile[nCount];
     INodeFile[] iNodes = new INodeFile[nCount];
     for (int i = 0; i < nCount; i++) {
     for (int i = 0; i < nCount; i++) {
-      PermissionStatus perms = new PermissionStatus(userName, null,
-          FsPermission.getDefault());
-      iNodes[i] = new INodeFile(i, perms, null, replication, 0L, 0L,
+      iNodes[i] = new INodeFile(i, null, perm, 0L, 0L, null, replication,
           preferredBlockSize);
           preferredBlockSize);
-      iNodes[i].setLocalName(fileNamePrefix +  Integer.toString(i));
+      iNodes[i].setLocalName(DFSUtil.string2Bytes(fileNamePrefix + i));
       BlockInfo newblock = new BlockInfo(replication);
       BlockInfo newblock = new BlockInfo(replication);
       iNodes[i].addBlock(newblock);
       iNodes[i].addBlock(newblock);
     }
     }
@@ -290,8 +265,6 @@ public class TestINodeFile {
   @Test
   @Test
   public void testValueOf () throws IOException {
   public void testValueOf () throws IOException {
     final String path = "/testValueOf";
     final String path = "/testValueOf";
-    final PermissionStatus perm = new PermissionStatus(
-        userName, null, FsPermission.getDefault());
     final short replication = 3;
     final short replication = 3;
 
 
     {//cast from null
     {//cast from null
@@ -323,8 +296,7 @@ public class TestINodeFile {
     }
     }
 
 
     {//cast from INodeFile
     {//cast from INodeFile
-      final INode from = new INodeFile(INodeId.GRANDFATHER_INODE_ID, perm,
-          null, replication, 0L, 0L, preferredBlockSize);
+      final INode from = createINodeFile(replication, preferredBlockSize);
 
 
      //cast to INodeFile, should success
      //cast to INodeFile, should success
       final INodeFile f = INodeFile.valueOf(from, path);
       final INodeFile f = INodeFile.valueOf(from, path);
@@ -371,8 +343,8 @@ public class TestINodeFile {
     }
     }
 
 
     {//cast from INodeDirectory
     {//cast from INodeDirectory
-      final INode from = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID, perm,
-          0L);
+      final INode from = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID, null,
+          perm, 0L);
 
 
       //cast to INodeFile, should fail
       //cast to INodeFile, should fail
       try {
       try {
@@ -816,13 +788,13 @@ public class TestINodeFile {
   /**
   /**
    * For a given path, build a tree of INodes and return the leaf node.
    * For a given path, build a tree of INodes and return the leaf node.
    */
    */
-  private INode createTreeOfInodes(String path) {
+  private INode createTreeOfInodes(String path) throws QuotaExceededException {
     byte[][] components = INode.getPathComponents(path);
     byte[][] components = INode.getPathComponents(path);
     FsPermission perm = FsPermission.createImmutable((short)0755);
     FsPermission perm = FsPermission.createImmutable((short)0755);
     PermissionStatus permstatus = PermissionStatus.createImmutable("", "", perm);
     PermissionStatus permstatus = PermissionStatus.createImmutable("", "", perm);
     
     
     long id = 0;
     long id = 0;
-    INodeDirectory prev = new INodeDirectory(++id, "", permstatus);
+    INodeDirectory prev = new INodeDirectory(++id, new byte[0], permstatus, 0);
     INodeDirectory dir = null;
     INodeDirectory dir = null;
     for (byte[] component : components) {
     for (byte[] component : components) {
       if (component.length == 0) {
       if (component.length == 0) {
@@ -830,7 +802,7 @@ public class TestINodeFile {
       }
       }
       System.out.println("Adding component " + DFSUtil.bytes2String(component));
       System.out.println("Adding component " + DFSUtil.bytes2String(component));
       dir = new INodeDirectory(++id, component, permstatus, 0);
       dir = new INodeDirectory(++id, component, permstatus, 0);
-      prev.addChild(dir, false);
+      prev.addChild(dir, false, null, null);
       prev = dir;
       prev = dir;
     }
     }
     return dir; // Last Inode in the chain
     return dir; // Last Inode in the chain
@@ -848,7 +820,7 @@ public class TestINodeFile {
    * Test for {@link FSDirectory#getPathComponents(INode)}
    * Test for {@link FSDirectory#getPathComponents(INode)}
    */
    */
   @Test
   @Test
-  public void testGetPathFromInode() {
+  public void testGetPathFromInode() throws QuotaExceededException {
     String path = "/a/b/c";
     String path = "/a/b/c";
     INode inode = createTreeOfInodes(path);
     INode inode = createTreeOfInodes(path);
     byte[][] expected = INode.getPathComponents(path);
     byte[][] expected = INode.getPathComponents(path);
@@ -860,7 +832,7 @@ public class TestINodeFile {
    * Tests for {@link FSDirectory#resolvePath(String, byte[][], FSDirectory)}
    * Tests for {@link FSDirectory#resolvePath(String, byte[][], FSDirectory)}
    */
    */
   @Test
   @Test
-  public void testInodePath() throws FileNotFoundException {
+  public void testInodePath() throws IOException {
     // For a non .inodes path the regular components are returned
     // For a non .inodes path the regular components are returned
     String path = "/a/b/c";
     String path = "/a/b/c";
     INode inode = createTreeOfInodes(path);
     INode inode = createTreeOfInodes(path);

+ 438 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java

@@ -0,0 +1,438 @@
+/**
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.FileNotFoundException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/** Test snapshot related operations. */
+public class TestSnapshotPathINodes {
+  private static final long seed = 0;
+  private static final short REPLICATION = 3;
+
+  static private final Path dir = new Path("/TestSnapshot");
+  
+  static private final Path sub1 = new Path(dir, "sub1");
+  static private final Path file1 = new Path(sub1, "file1");
+  static private final Path file2 = new Path(sub1, "file2");
+
+  static private Configuration conf;
+  static private MiniDFSCluster cluster;
+  static private FSNamesystem fsn;
+  static private FSDirectory fsdir;
+
+  static private DistributedFileSystem hdfs;
+
+  @BeforeClass
+  static public void setUp() throws Exception {
+    conf = new Configuration();
+    cluster = new MiniDFSCluster.Builder(conf)
+      .numDataNodes(REPLICATION)
+      .build();
+    cluster.waitActive();
+    
+    fsn = cluster.getNamesystem();
+    fsdir = fsn.getFSDirectory();
+    
+    hdfs = cluster.getFileSystem();
+    DFSTestUtil.createFile(hdfs, file1, 1024, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, file2, 1024, REPLICATION, seed);
+  }
+
+  @AfterClass
+  static public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /** Test allow-snapshot operation. */
+  @Test (timeout=15000)
+  public void testAllowSnapshot() throws Exception {
+    final String pathStr = sub1.toString();
+    final INode before = fsdir.getINode(pathStr);
+    
+    // Before a directory is snapshottable
+    Assert.assertTrue(before instanceof INodeDirectory);
+    Assert.assertFalse(before instanceof INodeDirectorySnapshottable);
+
+    // After a directory is snapshottable
+    final Path path = new Path(pathStr);
+    hdfs.allowSnapshot(path);
+    {
+      final INode after = fsdir.getINode(pathStr);
+      Assert.assertTrue(after instanceof INodeDirectorySnapshottable);
+    }
+    
+    hdfs.disallowSnapshot(path);
+    {
+      final INode after = fsdir.getINode(pathStr);
+      Assert.assertTrue(after instanceof INodeDirectory);
+      Assert.assertFalse(after instanceof INodeDirectorySnapshottable);
+    }
+  }
+  
+  static Snapshot getSnapshot(INodesInPath inodesInPath, String name) {
+    if (name == null) {
+      return null;
+    }
+    final int i = inodesInPath.getSnapshotRootIndex() - 1;
+    final INode inode = inodesInPath.getINodes()[i];
+    return ((INodeDirectorySnapshottable)inode).getSnapshot(
+        DFSUtil.string2Bytes(name)); 
+  }
+
+  static void assertSnapshot(INodesInPath inodesInPath, boolean isSnapshot,
+      final Snapshot snapshot, int index) {
+    assertEquals(isSnapshot, inodesInPath.isSnapshot());
+    assertEquals(index, inodesInPath.getSnapshotRootIndex());
+    assertEquals(isSnapshot? snapshot: null, inodesInPath.getPathSnapshot());
+    assertEquals(isSnapshot? null: snapshot, inodesInPath.getLatestSnapshot());
+    if (isSnapshot && index >= 0) {
+      assertEquals(Snapshot.Root.class, inodesInPath.getINodes()[index].getClass());
+    }
+  }
+
+  static void assertINodeFile(INode inode, Path path) {
+    assertEquals(path.getName(), inode.getLocalName());
+    assertEquals(INodeFile.class, inode.getClass());
+  }
+
+  /** 
+   * Test {@link INodeDirectory#getExistingPathINodes(byte[][], int, boolean)} 
+   * for normal (non-snapshot) file.
+   */
+  @Test (timeout=15000)
+  public void testNonSnapshotPathINodes() throws Exception {
+    // Get the inodes by resolving the path of a normal file
+    String[] names = INode.getPathNames(file1.toString());
+    byte[][] components = INode.getPathComponents(names);
+    INodesInPath nodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
+    INode[] inodes = nodesInPath.getINodes();
+    // The number of inodes should be equal to components.length
+    assertEquals(inodes.length, components.length);
+    // The returned nodesInPath should be non-snapshot
+    assertSnapshot(nodesInPath, false, null, -1);
+
+    // The last INode should be associated with file1
+    assertTrue("file1=" + file1 + ", nodesInPath=" + nodesInPath,
+        inodes[components.length - 1] != null);
+    assertEquals(inodes[components.length - 1].getFullPathName(),
+        file1.toString());
+    assertEquals(inodes[components.length - 2].getFullPathName(),
+        sub1.toString());
+    assertEquals(inodes[components.length - 3].getFullPathName(),
+        dir.toString());
+    
+    // Call getExistingPathINodes and request only one INode. This is used
+    // when identifying the INode for a given path.
+    nodesInPath = INodesInPath.resolve(fsdir.rootDir, components, 1, false);
+    inodes = nodesInPath.getINodes();
+    assertEquals(inodes.length, 1);
+    assertSnapshot(nodesInPath, false, null, -1);
+    assertEquals(inodes[0].getFullPathName(), file1.toString());
+    
+    // Call getExistingPathINodes and request 2 INodes. This is usually used
+    // when identifying the parent INode of a given path.
+    nodesInPath = INodesInPath.resolve(fsdir.rootDir, components, 2, false);
+    inodes = nodesInPath.getINodes();
+    assertEquals(inodes.length, 2);
+    assertSnapshot(nodesInPath, false, null, -1);
+    assertEquals(inodes[1].getFullPathName(), file1.toString());
+    assertEquals(inodes[0].getFullPathName(), sub1.toString());
+  }
+  
+  /** 
+   * Test {@link INodeDirectory#getExistingPathINodes(byte[][], int, boolean)} 
+   * for snapshot file.
+   */
+  @Test (timeout=15000)
+  public void testSnapshotPathINodes() throws Exception {
+    // Create a snapshot for the dir, and check the inodes for the path
+    // pointing to a snapshot file
+    hdfs.allowSnapshot(sub1);
+    hdfs.createSnapshot(sub1, "s1");
+    // The path when accessing the snapshot file of file1 is
+    // /TestSnapshot/sub1/.snapshot/s1/file1
+    String snapshotPath = sub1.toString() + "/.snapshot/s1/file1";
+    String[] names = INode.getPathNames(snapshotPath);
+    byte[][] components = INode.getPathComponents(names);
+    INodesInPath nodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
+    INode[] inodes = nodesInPath.getINodes();
+    // Length of inodes should be (components.length - 1), since we will ignore
+    // ".snapshot" 
+    assertEquals(inodes.length, components.length - 1);
+    // SnapshotRootIndex should be 3: {root, Testsnapshot, sub1, s1, file1}
+    final Snapshot snapshot = getSnapshot(nodesInPath, "s1");
+    assertSnapshot(nodesInPath, true, snapshot, 3);
+    // Check the INode for file1 (snapshot file)
+    INode snapshotFileNode = inodes[inodes.length - 1]; 
+    assertINodeFile(snapshotFileNode, file1);
+    assertTrue(snapshotFileNode.getParent() instanceof 
+        INodeDirectoryWithSnapshot);
+    
+    // Call getExistingPathINodes and request only one INode.
+    nodesInPath = INodesInPath.resolve(fsdir.rootDir, components, 1, false);
+    inodes = nodesInPath.getINodes();
+    assertEquals(inodes.length, 1);
+    // The snapshotroot (s1) is not included in inodes. Thus the
+    // snapshotRootIndex should be -1.
+    assertSnapshot(nodesInPath, true, snapshot, -1);
+    // Check the INode for file1 (snapshot file)
+    assertINodeFile(nodesInPath.getLastINode(), file1);
+    
+    // Call getExistingPathINodes and request 2 INodes.
+    nodesInPath = INodesInPath.resolve(fsdir.rootDir, components, 2, false);
+    inodes = nodesInPath.getINodes();
+    assertEquals(inodes.length, 2);
+    // There should be two INodes in inodes: s1 and snapshot of file1. Thus the
+    // SnapshotRootIndex should be 0.
+    assertSnapshot(nodesInPath, true, snapshot, 0);
+    assertINodeFile(nodesInPath.getLastINode(), file1);
+    
+    // Resolve the path "/TestSnapshot/sub1/.snapshot"  
+    String dotSnapshotPath = sub1.toString() + "/.snapshot";
+    names = INode.getPathNames(dotSnapshotPath);
+    components = INode.getPathComponents(names);
+    nodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
+    inodes = nodesInPath.getINodes();
+    // The number of INodes returned should be components.length - 1 since we
+    // will ignore ".snapshot"
+    assertEquals(inodes.length, components.length - 1);
+
+    // No SnapshotRoot dir is included in the resolved inodes  
+    assertSnapshot(nodesInPath, true, snapshot, -1);
+    // The last INode should be the INode for sub1
+    final INode last = nodesInPath.getLastINode();
+    assertEquals(last.getFullPathName(), sub1.toString());
+    assertFalse(last instanceof INodeFileWithSnapshot);
+    
+    String[] invalidPathComponent = {"invalidDir", "foo", ".snapshot", "bar"};
+    Path invalidPath = new Path(invalidPathComponent[0]);
+    for(int i = 1; i < invalidPathComponent.length; i++) {
+      invalidPath = new Path(invalidPath, invalidPathComponent[i]);
+      try {
+        hdfs.getFileStatus(invalidPath);
+        Assert.fail();
+      } catch(FileNotFoundException fnfe) {
+        System.out.println("The exception is expected: " + fnfe);
+      }
+    }
+  }
+  
+  /** 
+   * Test {@link INodeDirectory#getExistingPathINodes(byte[][], int, boolean)} 
+   * for snapshot file after deleting the original file.
+   */
+  @Test (timeout=15000)
+  public void testSnapshotPathINodesAfterDeletion() throws Exception {
+    // Create a snapshot for the dir, and check the inodes for the path
+    // pointing to a snapshot file
+    hdfs.allowSnapshot(sub1);
+    hdfs.createSnapshot(sub1, "s2");
+    
+    // Delete the original file /TestSnapshot/sub1/file1
+    hdfs.delete(file1, false);
+    
+    final Snapshot snapshot;
+    {
+      // Resolve the path for the snapshot file
+      // /TestSnapshot/sub1/.snapshot/s2/file1
+      String snapshotPath = sub1.toString() + "/.snapshot/s2/file1";
+      String[] names = INode.getPathNames(snapshotPath);
+      byte[][] components = INode.getPathComponents(names);
+      INodesInPath nodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
+      INode[] inodes = nodesInPath.getINodes();
+      // Length of inodes should be (components.length - 1), since we will ignore
+      // ".snapshot" 
+      assertEquals(inodes.length, components.length - 1);
+      // SnapshotRootIndex should be 3: {root, Testsnapshot, sub1, s2, file1}
+      snapshot = getSnapshot(nodesInPath, "s2");
+      assertSnapshot(nodesInPath, true, snapshot, 3);
+  
+      // Check the INode for file1 (snapshot file)
+      final INode inode = inodes[inodes.length - 1];
+      assertEquals(file1.getName(), inode.getLocalName());
+      assertEquals(INodeFileWithSnapshot.class, inode.getClass());
+    }
+
+    // Check the INodes for path /TestSnapshot/sub1/file1
+    String[] names = INode.getPathNames(file1.toString());
+    byte[][] components = INode.getPathComponents(names);
+    INodesInPath nodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
+    INode[] inodes = nodesInPath.getINodes();
+    // The length of inodes should be equal to components.length
+    assertEquals(inodes.length, components.length);
+    // The number of non-null elements should be components.length - 1 since
+    // file1 has been deleted
+    assertEquals(nodesInPath.getNumNonNull(), components.length - 1);
+    // The returned nodesInPath should be non-snapshot
+    assertSnapshot(nodesInPath, false, snapshot, -1);
+    // The last INode should be null, and the one before should be associated
+    // with sub1
+    assertNull(inodes[components.length - 1]);
+    assertEquals(inodes[components.length - 2].getFullPathName(),
+        sub1.toString());
+    assertEquals(inodes[components.length - 3].getFullPathName(),
+        dir.toString());
+  }
+  
+  static private Snapshot s4;
+
+  /** 
+   * Test {@link INodeDirectory#getExistingPathINodes(byte[][], int, boolean)} 
+   * for snapshot file while adding a new file after snapshot.
+   */
+  @Test (timeout=15000)
+  public void testSnapshotPathINodesWithAddedFile() throws Exception {
+    // Create a snapshot for the dir, and check the inodes for the path
+    // pointing to a snapshot file
+    hdfs.allowSnapshot(sub1);
+    hdfs.createSnapshot(sub1, "s4");
+    
+    // Add a new file /TestSnapshot/sub1/file3
+    final Path file3 = new Path(sub1, "file3");
+    DFSTestUtil.createFile(hdfs, file3, 1024, REPLICATION, seed);
+  
+    {
+      // Check the inodes for /TestSnapshot/sub1/.snapshot/s4/file3
+      String snapshotPath = sub1.toString() + "/.snapshot/s4/file3";
+      String[] names = INode.getPathNames(snapshotPath);
+      byte[][] components = INode.getPathComponents(names);
+      INodesInPath nodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
+      INode[] inodes = nodesInPath.getINodes();
+      // Length of inodes should be (components.length - 1), since we will ignore
+      // ".snapshot" 
+      assertEquals(inodes.length, components.length - 1);
+      // The number of non-null inodes should be components.length - 2, since
+      // snapshot of file3 does not exist
+      assertEquals(nodesInPath.getNumNonNull(), components.length - 2);
+      s4 = getSnapshot(nodesInPath, "s4");
+
+      // SnapshotRootIndex should still be 3: {root, Testsnapshot, sub1, s4, null}
+      assertSnapshot(nodesInPath, true, s4, 3);
+  
+      // Check the last INode in inodes, which should be null
+      assertNull(inodes[inodes.length - 1]);
+    }
+
+    // Check the inodes for /TestSnapshot/sub1/file3
+    String[] names = INode.getPathNames(file3.toString());
+    byte[][] components = INode.getPathComponents(names);
+    INodesInPath nodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
+    INode[] inodes = nodesInPath.getINodes();
+    // The number of inodes should be equal to components.length
+    assertEquals(inodes.length, components.length);
+
+    // The returned nodesInPath should be non-snapshot
+    assertSnapshot(nodesInPath, false, s4, -1);
+
+    // The last INode should be associated with file3
+    assertEquals(inodes[components.length - 1].getFullPathName(),
+        file3.toString());
+    assertEquals(inodes[components.length - 2].getFullPathName(),
+        sub1.toString());
+    assertEquals(inodes[components.length - 3].getFullPathName(),
+        dir.toString());
+  }
+  
+  /** 
+   * Test {@link INodeDirectory#getExistingPathINodes(byte[][], int, boolean)} 
+   * for snapshot file while modifying file after snapshot.
+   */
+  @Test (timeout=15000)
+  public void testSnapshotPathINodesAfterModification() throws Exception {
+    //file1 was deleted, create it again.
+    DFSTestUtil.createFile(hdfs, file1, 1024, REPLICATION, seed);
+
+    // First check the INode for /TestSnapshot/sub1/file1
+    String[] names = INode.getPathNames(file1.toString());
+    byte[][] components = INode.getPathComponents(names);
+    INodesInPath nodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
+    INode[] inodes = nodesInPath.getINodes();
+    // The number of inodes should be equal to components.length
+    assertEquals(inodes.length, components.length);
+    assertSnapshot(nodesInPath, false, s4, -1);
+
+    // The last INode should be associated with file1
+    assertEquals(inodes[components.length - 1].getFullPathName(),
+        file1.toString());
+    
+    // Create a snapshot for the dir, and check the inodes for the path
+    // pointing to a snapshot file
+    hdfs.allowSnapshot(sub1);
+    hdfs.createSnapshot(sub1, "s3");
+    
+    // Modify file1
+    DFSTestUtil.appendFile(hdfs, file1, "the content for appending");
+
+    // Check the INodes for snapshot of file1
+    String snapshotPath = sub1.toString() + "/.snapshot/s3/file1";
+    names = INode.getPathNames(snapshotPath);
+    components = INode.getPathComponents(names);
+    INodesInPath ssNodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
+    INode[] ssInodes = ssNodesInPath.getINodes();
+    // Length of ssInodes should be (components.length - 1), since we will
+    // ignore ".snapshot" 
+    assertEquals(ssInodes.length, components.length - 1);
+    final Snapshot s3 = getSnapshot(ssNodesInPath, "s3");
+    assertSnapshot(ssNodesInPath, true, s3, 3);
+    // Check the INode for snapshot of file1
+    INode snapshotFileNode = ssInodes[ssInodes.length - 1]; 
+    assertEquals(snapshotFileNode.getLocalName(), file1.getName());
+    assertTrue(snapshotFileNode instanceof INodeFileWithSnapshot);
+    // The modification time of the snapshot INode should be the same with the
+    // original INode before modification
+    assertEquals(inodes[inodes.length - 1].getModificationTime(),
+        snapshotFileNode.getModificationTime(ssNodesInPath.getPathSnapshot()));
+
+    // Check the INode for /TestSnapshot/sub1/file1 again
+    names = INode.getPathNames(file1.toString());
+    components = INode.getPathComponents(names);
+    INodesInPath newNodesInPath = INodesInPath.resolve(fsdir.rootDir, components);
+    assertSnapshot(newNodesInPath, false, s3, -1);
+    INode[] newInodes = newNodesInPath.getINodes();
+    // The number of inodes should be equal to components.length
+    assertEquals(newInodes.length, components.length);
+    // The last INode should be associated with file1
+    final int last = components.length - 1;
+    assertEquals(newInodes[last].getFullPathName(), file1.toString());
+    // The modification time of the INode for file3 should have been changed
+    Assert.assertFalse(inodes[last].getModificationTime()
+        == newInodes[last].getModificationTime());
+  }
+}

+ 480 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java

@@ -0,0 +1,480 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage;
+import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+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.LeaseManager;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.ipc.ProtobufRpcEngine.Server;
+import org.apache.hadoop.metrics2.impl.MetricsSystemImpl;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.log4j.Level;
+import org.junit.Assert;
+
+/**
+ * Helper for writing snapshot related tests
+ */
+public class SnapshotTestHelper {
+  public static final Log LOG = LogFactory.getLog(SnapshotTestHelper.class);
+
+  /** Disable the logs that are not very useful for snapshot related tests. */
+  public static void disableLogs() {
+    final String[] lognames = {
+        "org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceScanner",
+        "org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl",
+        "org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetAsyncDiskService",
+    };
+    for(String n : lognames) {
+      setLevel2OFF(LogFactory.getLog(n));
+    }
+    
+    setLevel2OFF(LogFactory.getLog(UserGroupInformation.class));
+    setLevel2OFF(LogFactory.getLog(BlockManager.class));
+    setLevel2OFF(LogFactory.getLog(FSNamesystem.class));
+    setLevel2OFF(LogFactory.getLog(DirectoryScanner.class));
+    setLevel2OFF(LogFactory.getLog(MetricsSystemImpl.class));
+    
+    setLevel2OFF(DataBlockScanner.LOG);
+    setLevel2OFF(HttpServer.LOG);
+    setLevel2OFF(DataNode.LOG);
+    setLevel2OFF(BlockPoolSliceStorage.LOG);
+    setLevel2OFF(LeaseManager.LOG);
+    setLevel2OFF(NameNode.stateChangeLog);
+    setLevel2OFF(NameNode.blockStateChangeLog);
+    setLevel2OFF(DFSClient.LOG);
+    setLevel2OFF(Server.LOG);
+  }
+
+  static void setLevel2OFF(Object log) {
+    ((Log4JLogger)log).getLogger().setLevel(Level.OFF);
+  }
+
+  private SnapshotTestHelper() {
+    // Cannot be instantinatied
+  }
+
+  public static Path getSnapshotRoot(Path snapshottedDir, String snapshotName) {
+    return new Path(snapshottedDir, HdfsConstants.DOT_SNAPSHOT_DIR + "/"
+        + snapshotName);
+  }
+
+  public static Path getSnapshotPath(Path snapshottedDir, String snapshotName,
+      String fileLocalName) {
+    return new Path(getSnapshotRoot(snapshottedDir, snapshotName),
+        fileLocalName);
+  }
+
+  /**
+   * Create snapshot for a dir using a given snapshot name
+   * 
+   * @param hdfs DistributedFileSystem instance
+   * @param snapshotRoot The dir to be snapshotted
+   * @param snapshotName The name of the snapshot
+   * @return The path of the snapshot root
+   */
+  public static Path createSnapshot(DistributedFileSystem hdfs,
+      Path snapshotRoot, String snapshotName) throws Exception {
+    LOG.info("createSnapshot " + snapshotName + " for " + snapshotRoot);
+    assertTrue(hdfs.exists(snapshotRoot));
+    hdfs.allowSnapshot(snapshotRoot);
+    hdfs.createSnapshot(snapshotRoot, snapshotName);
+    // set quota to a large value for testing counts
+    hdfs.setQuota(snapshotRoot, Long.MAX_VALUE-1, Long.MAX_VALUE-1);
+    return SnapshotTestHelper.getSnapshotRoot(snapshotRoot, snapshotName);
+  }
+
+  /**
+   * Check the functionality of a snapshot.
+   * 
+   * @param hdfs DistributedFileSystem instance
+   * @param snapshotRoot The root of the snapshot
+   * @param snapshottedDir The snapshotted directory
+   */
+  public static void checkSnapshotCreation(DistributedFileSystem hdfs,
+      Path snapshotRoot, Path snapshottedDir) throws Exception {
+    // Currently we only check if the snapshot was created successfully
+    assertTrue(hdfs.exists(snapshotRoot));
+    // Compare the snapshot with the current dir
+    FileStatus[] currentFiles = hdfs.listStatus(snapshottedDir);
+    FileStatus[] snapshotFiles = hdfs.listStatus(snapshotRoot);
+    assertEquals("snapshottedDir=" + snapshottedDir
+        + ", snapshotRoot=" + snapshotRoot,
+        currentFiles.length, snapshotFiles.length);
+  }
+  
+  /**
+   * Compare two dumped trees that are stored in two files. The following is an
+   * example of the dumped tree:
+   * 
+   * <pre>
+   * information of root
+   * +- the first child of root (e.g., /foo)
+   *   +- the first child of /foo
+   *   ...
+   *   \- the last child of /foo (e.g., /foo/bar)
+   *     +- the first child of /foo/bar
+   *     ...
+   *   snapshots of /foo
+   *   +- snapshot s_1
+   *   ...
+   *   \- snapshot s_n
+   * +- second child of root
+   *   ...
+   * \- last child of root
+   * 
+   * The following information is dumped for each inode:
+   * localName (className@hashCode) parent permission group user
+   * 
+   * Specific information for different types of INode: 
+   * {@link INodeDirectory}:childrenSize 
+   * {@link INodeFile}: fileSize, block list. Check {@link BlockInfo#toString()} 
+   * and {@link BlockInfoUnderConstruction#toString()} for detailed information.
+   * {@link FileWithSnapshot}: next link
+   * </pre>
+   * @see INode#dumpTreeRecursively()
+   */
+  public static void compareDumpedTreeInFile(File file1, File file2,
+      boolean compareQuota) throws IOException {
+    try {
+      compareDumpedTreeInFile(file1, file2, compareQuota, false);
+    } catch(Throwable t) {
+      LOG.info("FAILED compareDumpedTreeInFile(" + file1 + ", " + file2 + ")", t);
+      compareDumpedTreeInFile(file1, file2, compareQuota, true);
+    }
+  }
+
+  private static void compareDumpedTreeInFile(File file1, File file2,
+      boolean compareQuota, boolean print) throws IOException {
+    if (print) {
+      printFile(file1);
+      printFile(file2);
+    }
+
+    BufferedReader reader1 = new BufferedReader(new FileReader(file1));
+    BufferedReader reader2 = new BufferedReader(new FileReader(file2));
+    try {
+      String line1 = "";
+      String line2 = "";
+      while ((line1 = reader1.readLine()) != null
+          && (line2 = reader2.readLine()) != null) {
+        if (print) {
+          System.out.println();
+          System.out.println("1) " + line1);
+          System.out.println("2) " + line2);
+        }
+        // skip the hashCode part of the object string during the comparison,
+        // also ignore the difference between INodeFile/INodeFileWithSnapshot
+        line1 = line1.replaceAll("INodeFileWithSnapshot", "INodeFile");
+        line2 = line2.replaceAll("INodeFileWithSnapshot", "INodeFile");
+        line1 = line1.replaceAll("@[\\dabcdef]+", "");
+        line2 = line2.replaceAll("@[\\dabcdef]+", "");
+        
+        // skip the replica field of the last block of an
+        // INodeFileUnderConstruction
+        line1 = line1.replaceAll("replicas=\\[.*\\]", "replicas=[]");
+        line2 = line2.replaceAll("replicas=\\[.*\\]", "replicas=[]");
+        
+        if (!compareQuota) {
+          line1 = line1.replaceAll("Quota\\[.*\\]", "Quota[]");
+          line2 = line2.replaceAll("Quota\\[.*\\]", "Quota[]");
+        }
+        
+        // skip the specific fields of BlockInfoUnderConstruction when the node
+        // is an INodeFileSnapshot or an INodeFileUnderConstructionSnapshot
+        if (line1.contains("(INodeFileSnapshot)")
+            || line1.contains("(INodeFileUnderConstructionSnapshot)")) {
+          line1 = line1.replaceAll(
+           "\\{blockUCState=\\w+, primaryNodeIndex=[-\\d]+, replicas=\\[\\]\\}",
+           "");
+          line2 = line2.replaceAll(
+           "\\{blockUCState=\\w+, primaryNodeIndex=[-\\d]+, replicas=\\[\\]\\}",
+           "");
+        }
+        
+        assertEquals(line1, line2);
+      }
+      Assert.assertNull(reader1.readLine());
+      Assert.assertNull(reader2.readLine());
+    } finally {
+      reader1.close();
+      reader2.close();
+    }
+  }
+
+  static void printFile(File f) throws IOException {
+    System.out.println();
+    System.out.println("File: " + f);
+    BufferedReader in = new BufferedReader(new FileReader(f));
+    try {
+      for(String line; (line = in.readLine()) != null; ) {
+        System.out.println(line);
+      }
+    } finally {
+      in.close();
+    }
+  }
+
+  public static void dumpTree2File(FSDirectory fsdir, File f) throws IOException{
+    final PrintWriter out = new PrintWriter(new FileWriter(f, false), true);
+    fsdir.getINode("/").dumpTreeRecursively(out, new StringBuilder(), null);
+    out.close();
+  }
+
+  /**
+   * Generate the path for a snapshot file.
+   * 
+   * @param snapshotRoot of format
+   *          {@literal <snapshottble_dir>/.snapshot/<snapshot_name>}
+   * @param file path to a file
+   * @return The path of the snapshot of the file assuming the file has a
+   *         snapshot under the snapshot root of format
+   *         {@literal <snapshottble_dir>/.snapshot/<snapshot_name>/<path_to_file_inside_snapshot>}
+   *         . Null if the file is not under the directory associated with the
+   *         snapshot root.
+   */
+  static Path getSnapshotFile(Path snapshotRoot, Path file) {
+    Path rootParent = snapshotRoot.getParent();
+    if (rootParent != null && rootParent.getName().equals(".snapshot")) {
+      Path snapshotDir = rootParent.getParent();
+      if (file.toString().contains(snapshotDir.toString())
+          && !file.equals(snapshotDir)) {
+        String fileName = file.toString().substring(
+            snapshotDir.toString().length() + 1);
+        Path snapshotFile = new Path(snapshotRoot, fileName);
+        return snapshotFile;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * A class creating directories trees for snapshot testing. For simplicity,
+   * the directory tree is a binary tree, i.e., each directory has two children
+   * as snapshottable directories.
+   */
+  static class TestDirectoryTree {
+    /** Height of the directory tree */
+    final int height;
+    /** Top node of the directory tree */
+    final Node topNode;
+    /** A map recording nodes for each tree level */
+    final Map<Integer, ArrayList<Node>> levelMap;
+
+    /**
+     * Constructor to build a tree of given {@code height}
+     */
+    TestDirectoryTree(int height, FileSystem fs) throws Exception {
+      this.height = height;
+      this.topNode = new Node(new Path("/TestSnapshot"), 0,
+          null, fs);
+      this.levelMap = new HashMap<Integer, ArrayList<Node>>();
+      addDirNode(topNode, 0);
+      genChildren(topNode, height - 1, fs);
+    }
+
+    /**
+     * Add a node into the levelMap
+     */
+    private void addDirNode(Node node, int atLevel) {
+      ArrayList<Node> list = levelMap.get(atLevel);
+      if (list == null) {
+        list = new ArrayList<Node>();
+        levelMap.put(atLevel, list);
+      }
+      list.add(node);
+    }
+
+    int id = 0;
+    /**
+     * Recursively generate the tree based on the height.
+     * 
+     * @param parent The parent node
+     * @param level The remaining levels to generate
+     * @param fs The FileSystem where to generate the files/dirs
+     * @throws Exception
+     */
+    private void genChildren(Node parent, int level, FileSystem fs)
+        throws Exception {
+      if (level == 0) {
+        return;
+      }
+      parent.leftChild = new Node(new Path(parent.nodePath,
+          "left" + ++id), height - level, parent, fs);
+      parent.rightChild = new Node(new Path(parent.nodePath,
+          "right" + ++id), height - level, parent, fs);
+      addDirNode(parent.leftChild, parent.leftChild.level);
+      addDirNode(parent.rightChild, parent.rightChild.level);
+      genChildren(parent.leftChild, level - 1, fs);
+      genChildren(parent.rightChild, level - 1, fs);
+    }
+
+    /**
+     * Randomly retrieve a node from the directory tree.
+     * 
+     * @param random A random instance passed by user.
+     * @param excludedList Excluded list, i.e., the randomly generated node
+     *          cannot be one of the nodes in this list.
+     * @return a random node from the tree.
+     */
+    Node getRandomDirNode(Random random, List<Node> excludedList) {
+      while (true) {
+        int level = random.nextInt(height);
+        ArrayList<Node> levelList = levelMap.get(level);
+        int index = random.nextInt(levelList.size());
+        Node randomNode = levelList.get(index);
+        if (excludedList == null || !excludedList.contains(randomNode)) {
+          return randomNode;
+        }
+      }
+    }
+
+    /**
+     * The class representing a node in {@link TestDirectoryTree}.
+     * <br>
+     * This contains:
+     * <ul>
+     * <li>Two children representing the two snapshottable directories</li>
+     * <li>A list of files for testing, so that we can check snapshots
+     * after file creation/deletion/modification.</li>
+     * <li>A list of non-snapshottable directories, to test snapshots with
+     * directory creation/deletion. Note that this is needed because the
+     * deletion of a snapshottale directory with snapshots is not allowed.</li>
+     * </ul>
+     */
+    static class Node {
+      /** The level of this node in the directory tree */
+      final int level;
+
+      /** Children */
+      Node leftChild;
+      Node rightChild;
+
+      /** Parent node of the node */
+      final Node parent;
+
+      /** File path of the node */
+      final Path nodePath;
+
+      /**
+       * The file path list for testing snapshots before/after file
+       * creation/deletion/modification
+       */
+      ArrayList<Path> fileList;
+
+      /**
+       * Each time for testing snapshots with file creation, since we do not
+       * want to insert new files into the fileList, we always create the file
+       * that was deleted last time. Thus we record the index for deleted file
+       * in the fileList, and roll the file modification forward in the list.
+       */
+      int nullFileIndex = 0;
+
+      /**
+       * A list of non-snapshottable directories for testing snapshots with
+       * directory creation/deletion
+       */
+      final ArrayList<Node> nonSnapshotChildren;
+
+      Node(Path path, int level, Node parent,
+          FileSystem fs) throws Exception {
+        this.nodePath = path;
+        this.level = level;
+        this.parent = parent;
+        this.nonSnapshotChildren = new ArrayList<Node>();
+        fs.mkdirs(nodePath);
+      }
+
+      /**
+       * Create files and add them in the fileList. Initially the last element
+       * in the fileList is set to null (where we start file creation).
+       */
+      void initFileList(FileSystem fs, String namePrefix, long fileLen,
+          short replication, long seed, int numFiles) throws Exception {
+        fileList = new ArrayList<Path>(numFiles);
+        for (int i = 0; i < numFiles; i++) {
+          Path file = new Path(nodePath, namePrefix + "-f" + i);
+          fileList.add(file);
+          if (i < numFiles - 1) {
+            DFSTestUtil.createFile(fs, file, fileLen, replication, seed);
+          }
+        }
+        nullFileIndex = numFiles - 1;
+      }
+
+      @Override
+      public boolean equals(Object o) {
+        if (o != null && o instanceof Node) {
+          Node node = (Node) o;
+          return node.nodePath.equals(nodePath);
+        }
+        return false;
+      }
+
+      @Override
+      public int hashCode() {
+        return nodePath.hashCode();
+      }
+    }
+  }
+  
+  public static void dumpTree(String message, MiniDFSCluster cluster
+      ) throws UnresolvedLinkException {
+    System.out.println("XXX " + message);
+    cluster.getNameNode().getNamesystem().getFSDirectory().getINode("/"
+        ).dumpTreeRecursively(System.out);
+  }
+}

+ 154 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestDisallowModifyROSnapshot.java

@@ -0,0 +1,154 @@
+/**
+ * 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 static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import java.util.ArrayList;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * This class tests snapshot functionality. One or multiple snapshots are
+ * created. The snapshotted directory is changed and verification is done to
+ * ensure snapshots remain unchanges.
+ */
+public class TestDisallowModifyROSnapshot {
+  private final static Path dir = new Path("/TestSnapshot");
+  private final static Path sub1 = new Path(dir, "sub1");
+  private final static Path sub2 = new Path(dir, "sub2");
+
+  protected static Configuration conf;
+  protected static MiniDFSCluster cluster;
+  protected static FSNamesystem fsn;
+  protected static DistributedFileSystem fs;
+
+  /**
+   * The list recording all previous snapshots. Each element in the array
+   * records a snapshot root.
+   */
+  protected static ArrayList<Path> snapshotList = new ArrayList<Path>();
+  static Path objInSnapshot = null;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    conf = new Configuration();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+
+    fsn = cluster.getNamesystem();
+    fs = cluster.getFileSystem();
+
+    Path path1 = new Path(sub1, "dir1");
+    assertTrue(fs.mkdirs(path1));
+    Path path2 = new Path(sub2, "dir2");
+    assertTrue(fs.mkdirs(path2));
+    SnapshotTestHelper.createSnapshot(fs, sub1, "testSnapshot");
+    objInSnapshot = SnapshotTestHelper.getSnapshotPath(sub1, "testSnapshot",
+        "dir1");
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test(timeout=60000, expected = SnapshotAccessControlException.class)
+  public void testSetReplication() throws Exception {
+    fs.setReplication(objInSnapshot, (short) 1);
+  }
+
+  @Test(timeout=60000, expected = SnapshotAccessControlException.class)
+  public void testSetPermission() throws Exception {
+    fs.setPermission(objInSnapshot, new FsPermission("777"));
+  }
+
+  @Test(timeout=60000, expected = SnapshotAccessControlException.class)
+  public void testSetOwner() throws Exception {
+    fs.setOwner(objInSnapshot, "username", "groupname");
+  }
+
+  @Test (timeout=60000)
+  public void testRename() throws Exception {
+    try {
+      fs.rename(objInSnapshot, new Path("/invalid/path"));
+      fail("Didn't throw SnapshotAccessControlException");
+    } catch (SnapshotAccessControlException e) { /* Ignored */ }
+
+    try {
+      fs.rename(sub2, objInSnapshot);
+      fail("Didn't throw SnapshotAccessControlException");
+    } catch (SnapshotAccessControlException e) { /* Ignored */ }
+
+    try {
+      fs.rename(sub2, objInSnapshot, (Options.Rename) null);
+      fail("Didn't throw SnapshotAccessControlException");
+    } catch (SnapshotAccessControlException e) { /* Ignored */ }
+  }
+
+  @Test(timeout=60000, expected = SnapshotAccessControlException.class)
+  public void testDelete() throws Exception {
+    fs.delete(objInSnapshot, true);
+  }
+
+  @Test(timeout=60000, expected = SnapshotAccessControlException.class)
+  public void testQuota() throws Exception {
+    fs.setQuota(objInSnapshot, 100, 100);
+  }
+
+  @Test(timeout=60000, expected = SnapshotAccessControlException.class)
+  public void testSetTime() throws Exception {
+    fs.setTimes(objInSnapshot, 100, 100);
+  }
+
+  @Test(timeout=60000, expected = SnapshotAccessControlException.class)
+  public void testCreate() throws Exception {
+    @SuppressWarnings("deprecation")
+    DFSClient dfsclient = new DFSClient(conf);
+    dfsclient.create(objInSnapshot.toString(), true);
+  }
+
+  @Test(timeout=60000, expected = SnapshotAccessControlException.class)
+  public void testAppend() throws Exception {
+    fs.append(objInSnapshot, 65535, null);
+  }
+
+  @Test(timeout=60000, expected = SnapshotAccessControlException.class)
+  public void testMkdir() throws Exception {
+    fs.mkdirs(objInSnapshot, new FsPermission("777"));
+  }
+
+  @Test(timeout=60000, expected = SnapshotAccessControlException.class)
+  public void testCreateSymlink() throws Exception {
+    @SuppressWarnings("deprecation")
+    DFSClient dfsclient = new DFSClient(conf);
+    dfsclient.createSymlink(sub2.toString(), "/TestSnapshot/sub1/.snapshot",
+        false);
+  }
+}

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

@@ -0,0 +1,186 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.Random;
+
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
+import org.apache.log4j.Level;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test snapshot functionalities while file appending.
+ */
+public class TestINodeFileUnderConstructionWithSnapshot {
+  {
+    ((Log4JLogger)INode.LOG).getLogger().setLevel(Level.ALL);
+    SnapshotTestHelper.disableLogs();
+  }
+
+  static final long seed = 0;
+  static final short REPLICATION = 3;
+  static final int BLOCKSIZE = 1024;
+
+  private final Path dir = new Path("/TestSnapshot");
+  
+  Configuration conf;
+  MiniDFSCluster cluster;
+  FSNamesystem fsn;
+  DistributedFileSystem hdfs;
+  FSDirectory fsdir;
+  
+  @Before
+  public void setUp() throws Exception {
+    conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
+        .build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    fsdir = fsn.getFSDirectory();
+    hdfs = cluster.getFileSystem();
+    hdfs.mkdirs(dir);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+  
+  /**
+   * Test snapshot after file appending
+   */
+  @Test (timeout=60000)
+  public void testSnapshotAfterAppending() throws Exception {
+    Path file = new Path(dir, "file");
+    // 1. create snapshot --> create file --> append
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
+    DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.appendFile(hdfs, file, BLOCKSIZE);
+    
+    INodeFile fileNode = (INodeFile) fsdir.getINode(file.toString());
+    
+    // 2. create snapshot --> modify the file --> append
+    hdfs.createSnapshot(dir, "s1");
+    hdfs.setReplication(file, (short) (REPLICATION - 1));
+    DFSTestUtil.appendFile(hdfs, file, BLOCKSIZE);
+    
+    // check corresponding inodes
+    fileNode = (INodeFile) fsdir.getINode(file.toString());
+    assertEquals(REPLICATION - 1, fileNode.getFileReplication());
+    assertEquals(BLOCKSIZE * 3, fileNode.computeFileSize());
+
+    // 3. create snapshot --> append
+    hdfs.createSnapshot(dir, "s2");
+    DFSTestUtil.appendFile(hdfs, file, BLOCKSIZE);
+    
+    // check corresponding inodes
+    fileNode = (INodeFile) fsdir.getINode(file.toString());
+    assertEquals(REPLICATION - 1,  fileNode.getFileReplication());
+    assertEquals(BLOCKSIZE * 4, fileNode.computeFileSize());
+  }
+  
+  private HdfsDataOutputStream appendFileWithoutClosing(Path file, int length)
+      throws IOException {
+    byte[] toAppend = new byte[length];
+    Random random = new Random();
+    random.nextBytes(toAppend);
+    HdfsDataOutputStream out = (HdfsDataOutputStream) hdfs.append(file);
+    out.write(toAppend);
+    return out;
+  }
+  
+  /**
+   * Test snapshot during file appending, before the corresponding
+   * {@link FSDataOutputStream} instance closes.
+   */
+  @Test (timeout=60000)
+  public void testSnapshotWhileAppending() throws Exception {
+    Path file = new Path(dir, "file");
+    DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, seed);
+    
+    // 1. append without closing stream --> create snapshot
+    HdfsDataOutputStream out = appendFileWithoutClosing(file, BLOCKSIZE);
+    out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
+    out.close();
+    
+    // check: an INodeFileUnderConstructionWithSnapshot should be stored into s0's
+    // deleted list, with size BLOCKSIZE*2
+    INodeFile fileNode = (INodeFile) fsdir.getINode(file.toString());
+    assertEquals(BLOCKSIZE * 2, fileNode.computeFileSize());
+    INodeDirectorySnapshottable dirNode = (INodeDirectorySnapshottable) fsdir
+        .getINode(dir.toString());
+    DirectoryDiff last = dirNode.getDiffs().getLast();
+    Snapshot s0 = last.snapshot;
+    
+    // 2. append without closing stream
+    out = appendFileWithoutClosing(file, BLOCKSIZE);
+    out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
+    
+    // re-check nodeInDeleted_S0
+    dirNode = (INodeDirectorySnapshottable) fsdir.getINode(dir.toString());
+    assertEquals(BLOCKSIZE * 2, fileNode.computeFileSize(s0));
+    
+    // 3. take snapshot --> close stream
+    hdfs.createSnapshot(dir, "s1");
+    out.close();
+    
+    // check: an INodeFileUnderConstructionWithSnapshot with size BLOCKSIZE*3 should
+    // have been stored in s1's deleted list
+    fileNode = (INodeFile) fsdir.getINode(file.toString());
+    dirNode = (INodeDirectorySnapshottable) fsdir.getINode(dir.toString());
+    last = dirNode.getDiffs().getLast();
+    Snapshot s1 = last.snapshot;
+    assertTrue(fileNode instanceof INodeFileWithSnapshot);
+    assertEquals(BLOCKSIZE * 3, fileNode.computeFileSize(s1));
+    
+    // 4. modify file --> append without closing stream --> take snapshot -->
+    // close stream
+    hdfs.setReplication(file, (short) (REPLICATION - 1));
+    out = appendFileWithoutClosing(file, BLOCKSIZE);
+    hdfs.createSnapshot(dir, "s2");
+    out.close();
+    
+    // re-check the size of nodeInDeleted_S1
+    assertEquals(BLOCKSIZE * 3, fileNode.computeFileSize(s1));
+  }  
+}

+ 368 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestNestedSnapshots.java

@@ -0,0 +1,368 @@
+/**
+ * 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 static org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable.SNAPSHOT_LIMIT;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.Random;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/** Testing nested snapshots. */
+public class TestNestedSnapshots {
+  {
+    SnapshotTestHelper.disableLogs();
+  }
+
+  private static final long SEED = 0;
+  private static Random RANDOM = new Random(SEED);
+
+  private static final short REPLICATION = 3;
+  private static final long BLOCKSIZE = 1024;
+  
+  private static Configuration conf = new Configuration();
+  private static MiniDFSCluster cluster;
+  private static DistributedFileSystem hdfs;
+  
+  @Before
+  public void setUp() throws Exception {
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
+        .build();
+    cluster.waitActive();
+    hdfs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+  
+  /**
+   * Create a snapshot for /test/foo and create another snapshot for
+   * /test/foo/bar.  Files created before the snapshots should appear in both
+   * snapshots and the files created after the snapshots should not appear in
+   * any of the snapshots.  
+   */
+  @Test (timeout=300000)
+  public void testNestedSnapshots() throws Exception {
+    cluster.getNamesystem().getSnapshotManager().setAllowNestedSnapshots(true);
+
+    final Path foo = new Path("/testNestedSnapshots/foo");
+    final Path bar = new Path(foo, "bar");
+    final Path file1 = new Path(bar, "file1");
+    DFSTestUtil.createFile(hdfs, file1, BLOCKSIZE, REPLICATION, SEED);
+    print("create file " + file1);
+
+    final String s1name = "foo-s1";
+    final Path s1path = SnapshotTestHelper.getSnapshotRoot(foo, s1name); 
+    hdfs.allowSnapshot(foo);
+    print("allow snapshot " + foo);
+    hdfs.createSnapshot(foo, s1name);
+    print("create snapshot " + s1name);
+
+    final String s2name = "bar-s2";
+    final Path s2path = SnapshotTestHelper.getSnapshotRoot(bar, s2name); 
+    hdfs.allowSnapshot(bar);
+    print("allow snapshot " + bar);
+    hdfs.createSnapshot(bar, s2name);
+    print("create snapshot " + s2name);
+
+    final Path file2 = new Path(bar, "file2");
+    DFSTestUtil.createFile(hdfs, file2, BLOCKSIZE, REPLICATION, SEED);
+    print("create file " + file2);
+    
+    assertFile(s1path, s2path, file1, true, true, true);
+    assertFile(s1path, s2path, file2, true, false, false);
+
+    //test root
+    final String rootStr = "/";
+    final Path rootPath = new Path(rootStr);
+    hdfs.allowSnapshot(rootPath);
+    print("allow snapshot " + rootStr);
+    final Path rootSnapshot = hdfs.createSnapshot(rootPath);
+    print("create snapshot " + rootSnapshot);
+    hdfs.deleteSnapshot(rootPath, rootSnapshot.getName());
+    print("delete snapshot " + rootSnapshot);
+    hdfs.disallowSnapshot(rootPath);
+    print("disallow snapshot " + rootStr);
+    try {
+      hdfs.disallowSnapshot(rootPath);
+      fail("Expect snapshot exception when disallowing snapshot on root again");
+    } catch (SnapshotException e) {
+      GenericTestUtils.assertExceptionContains(
+          "Root is not a snapshottable directory", e);
+    }
+    
+    //change foo to non-snapshottable
+    hdfs.deleteSnapshot(foo, s1name);
+    hdfs.disallowSnapshot(foo);
+    
+    //test disallow nested snapshots
+    cluster.getNamesystem().getSnapshotManager().setAllowNestedSnapshots(false);
+    try {
+      hdfs.allowSnapshot(rootPath);
+      Assert.fail();
+    } catch(SnapshotException se) {
+      assertNestedSnapshotException(se, "subdirectory");
+    }
+    try {
+      hdfs.allowSnapshot(foo);
+      Assert.fail();
+    } catch(SnapshotException se) {
+      assertNestedSnapshotException(se, "subdirectory");
+    }
+
+    final Path sub1Bar = new Path(bar, "sub1");
+    final Path sub2Bar = new Path(sub1Bar, "sub2");
+    hdfs.mkdirs(sub2Bar);
+    try {
+      hdfs.allowSnapshot(sub1Bar);
+      Assert.fail();
+    } catch(SnapshotException se) {
+      assertNestedSnapshotException(se, "ancestor");
+    }
+    try {
+      hdfs.allowSnapshot(sub2Bar);
+      Assert.fail();
+    } catch(SnapshotException se) {
+      assertNestedSnapshotException(se, "ancestor");
+    }
+  }
+  
+  static void assertNestedSnapshotException(SnapshotException se, String substring) {
+    Assert.assertTrue(se.getMessage().startsWith(
+        "Nested snapshottable directories not allowed"));
+    Assert.assertTrue(se.getMessage().contains(substring));
+  }
+
+  private static void print(String message) throws UnresolvedLinkException {
+    SnapshotTestHelper.dumpTree(message, cluster);
+  }
+
+  private static void assertFile(Path s1, Path s2, Path file,
+      Boolean... expected) throws IOException {
+    final Path[] paths = {
+        file,
+        new Path(s1, "bar/" + file.getName()),
+        new Path(s2, file.getName())
+    };
+    Assert.assertEquals(expected.length, paths.length);
+    for(int i = 0; i < paths.length; i++) {
+      final boolean computed = hdfs.exists(paths[i]);
+      Assert.assertEquals("Failed on " + paths[i], expected[i], computed);
+    }
+  }
+
+  /**
+   * Test the snapshot limit of a single snapshottable directory.
+   * @throws Exception
+   */
+  @Test (timeout=300000)
+  public void testSnapshotLimit() throws Exception {
+    final int step = 1000;
+    final String dirStr = "/testSnapshotLimit/dir";
+    final Path dir = new Path(dirStr);
+    hdfs.mkdirs(dir, new FsPermission((short)0777));
+    hdfs.allowSnapshot(dir);
+
+    int s = 0;
+    for(; s < SNAPSHOT_LIMIT; s++) {
+      final String snapshotName = "s" + s;
+      hdfs.createSnapshot(dir, snapshotName);
+
+      //create a file occasionally 
+      if (s % step == 0) {
+        final Path file = new Path(dirStr, "f" + s);
+        DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, SEED);
+      }
+    }
+
+    try {
+      hdfs.createSnapshot(dir, "s" + s);
+      Assert.fail("Expected to fail to create snapshot, but didn't.");
+    } catch(IOException ioe) {
+      SnapshotTestHelper.LOG.info("The exception is expected.", ioe);
+    }
+
+    for(int f = 0; f < SNAPSHOT_LIMIT; f += step) {
+      final String file = "f" + f;
+      s = RANDOM.nextInt(step);
+      for(; s < SNAPSHOT_LIMIT; s += RANDOM.nextInt(step)) {
+        final Path p = SnapshotTestHelper.getSnapshotPath(dir, "s" + s, file);
+        //the file #f exists in snapshot #s iff s > f.
+        Assert.assertEquals(s > f, hdfs.exists(p));
+      }
+    }
+  }
+
+  @Test (timeout=300000)
+  public void testSnapshotWithQuota() throws Exception {
+    final String dirStr = "/testSnapshotWithQuota/dir";
+    final Path dir = new Path(dirStr);
+    hdfs.mkdirs(dir, new FsPermission((short)0777));
+    hdfs.allowSnapshot(dir);
+
+    // set namespace quota
+    final int NS_QUOTA = 6;
+    hdfs.setQuota(dir, NS_QUOTA, HdfsConstants.QUOTA_DONT_SET);
+
+    // create object to use up the quota.
+    final Path foo = new Path(dir, "foo");
+    final Path f1 = new Path(foo, "f1");
+    DFSTestUtil.createFile(hdfs, f1, BLOCKSIZE, REPLICATION, SEED);
+    {
+      //create a snapshot with default snapshot name
+      final Path snapshotPath = hdfs.createSnapshot(dir);
+
+      //check snapshot path and the default snapshot name
+      final String snapshotName = snapshotPath.getName(); 
+      Assert.assertTrue("snapshotName=" + snapshotName, Pattern.matches(
+          "s\\d\\d\\d\\d\\d\\d\\d\\d-\\d\\d\\d\\d\\d\\d\\.\\d\\d\\d",
+          snapshotName));
+      final Path parent = snapshotPath.getParent();
+      Assert.assertEquals(HdfsConstants.DOT_SNAPSHOT_DIR, parent.getName());
+      Assert.assertEquals(dir, parent.getParent());
+    }
+    final Path f2 = new Path(foo, "f2");
+    DFSTestUtil.createFile(hdfs, f2, BLOCKSIZE, REPLICATION, SEED);
+    
+    try {
+      // normal create file should fail with quota
+      final Path f3 = new Path(foo, "f3");
+      DFSTestUtil.createFile(hdfs, f3, BLOCKSIZE, REPLICATION, SEED);
+      Assert.fail();
+    } catch(NSQuotaExceededException e) {
+      SnapshotTestHelper.LOG.info("The exception is expected.", e);
+    }
+
+    try {
+      // createSnapshot should fail with quota
+      hdfs.createSnapshot(dir);
+      Assert.fail();
+    } catch(NSQuotaExceededException e) {
+      SnapshotTestHelper.LOG.info("The exception is expected.", e);
+    }
+
+    try {
+      // setPermission f1 should fail with quote since it cannot add diff.
+      hdfs.setPermission(f1, new FsPermission((short)0));
+      Assert.fail();
+    } catch(RemoteException e) {
+      Assert.assertSame(NSQuotaExceededException.class,
+          e.unwrapRemoteException().getClass());
+      SnapshotTestHelper.LOG.info("The exception is expected.", e);
+    }
+
+    // setPermission f2 since it was created after the snapshot
+    hdfs.setPermission(f2, new FsPermission((short)0));
+
+    // increase quota and retry the commands.
+    hdfs.setQuota(dir, NS_QUOTA + 2, HdfsConstants.QUOTA_DONT_SET);
+    hdfs.createSnapshot(dir, "s1");
+    hdfs.setPermission(foo, new FsPermission((short)0444));
+  }
+
+  /**
+   * Test {@link Snapshot#ID_COMPARATOR}.
+   */
+  @Test (timeout=300000)
+  public void testIdCmp() {
+    final PermissionStatus perm = PermissionStatus.createImmutable(
+        "user", "group", FsPermission.createImmutable((short)0));
+    final INodeDirectory dir = new INodeDirectory(0,
+        DFSUtil.string2Bytes("foo"), perm, 0L);
+    final INodeDirectorySnapshottable snapshottable
+        = new INodeDirectorySnapshottable(dir);
+    final Snapshot[] snapshots = {
+      new Snapshot(1, "s1", snapshottable),
+      new Snapshot(1, "s1", snapshottable),
+      new Snapshot(2, "s2", snapshottable),
+      new Snapshot(2, "s2", snapshottable),
+    };
+
+    Assert.assertEquals(0, Snapshot.ID_COMPARATOR.compare(null, null));
+    for(Snapshot s : snapshots) {
+      Assert.assertTrue(Snapshot.ID_COMPARATOR.compare(null, s) > 0);
+      Assert.assertTrue(Snapshot.ID_COMPARATOR.compare(s, null) < 0);
+      
+      for(Snapshot t : snapshots) {
+        final int expected = s.getRoot().getLocalName().compareTo(
+            t.getRoot().getLocalName());
+        final int computed = Snapshot.ID_COMPARATOR.compare(s, t);
+        Assert.assertEquals(expected > 0, computed > 0);
+        Assert.assertEquals(expected == 0, computed == 0);
+        Assert.assertEquals(expected < 0, computed < 0);
+      }
+    }
+  }
+  
+  /**
+   * When we have nested snapshottable directories and if we try to reset the
+   * snapshottable descendant back to an regular directory, we need to replace
+   * the snapshottable descendant with an INodeDirectoryWithSnapshot
+   */
+  @Test
+  public void testDisallowNestedSnapshottableDir() throws Exception {
+    cluster.getNamesystem().getSnapshotManager().setAllowNestedSnapshots(true);
+
+    final Path dir = new Path("/dir");
+    final Path sub = new Path(dir, "sub");
+    hdfs.mkdirs(sub);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s1");
+    final Path file = new Path(sub, "file");
+    DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, SEED);
+    
+    FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+    INode subNode = fsdir.getINode(sub.toString());
+    assertTrue(subNode instanceof INodeDirectoryWithSnapshot);
+    
+    hdfs.allowSnapshot(sub);
+    subNode = fsdir.getINode(sub.toString());
+    assertTrue(subNode instanceof INodeDirectorySnapshottable);
+    
+    hdfs.disallowSnapshot(sub);
+    subNode = fsdir.getINode(sub.toString());
+    assertTrue(subNode instanceof INodeDirectoryWithSnapshot);
+  }
+}

+ 1918 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java

@@ -0,0 +1,1918 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.spy;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+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.INodeMap;
+import org.apache.hadoop.hdfs.server.namenode.INodeReference;
+import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
+import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.ChildrenDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
+import org.apache.hadoop.hdfs.util.Diff.ListType;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+/** Testing rename with snapshots. */
+public class TestRenameWithSnapshots {
+  {
+    SnapshotTestHelper.disableLogs();
+  }
+  private static final Log LOG = LogFactory.getLog(TestRenameWithSnapshots.class);
+  
+  private static final long SEED = 0;
+  private static final short REPL = 3;
+  private static final short REPL_1 = 2;
+  private static final short REPL_2 = 1;
+  private static final long BLOCKSIZE = 1024;
+  
+  private static Configuration conf = new Configuration();
+  private static MiniDFSCluster cluster;
+  private static FSNamesystem fsn;
+  private static FSDirectory fsdir;
+  private static DistributedFileSystem hdfs;
+  private static String testDir =
+      System.getProperty("test.build.data", "build/test/data");
+  static private final Path dir = new Path("/testRenameWithSnapshots");
+  static private final Path sub1 = new Path(dir, "sub1");
+  static private final Path file1 = new Path(sub1, "file1");
+  static private final Path file2 = new Path(sub1, "file2");
+  static private final Path file3 = new Path(sub1, "file3");
+  static private final String snap1 = "snap1";
+  static private final String snap2 = "snap2";
+
+  
+  @Before
+  public void setUp() throws Exception {
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL).format(true)
+        .build();
+    cluster.waitActive();
+
+    fsn = cluster.getNamesystem();
+    fsdir = fsn.getFSDirectory();
+
+    hdfs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test (timeout=300000)
+  public void testRenameFromSDir2NonSDir() throws Exception {
+    final String dirStr = "/testRenameWithSnapshot";
+    final String abcStr = dirStr + "/abc";
+    final Path abc = new Path(abcStr);
+    hdfs.mkdirs(abc, new FsPermission((short)0777));
+    hdfs.allowSnapshot(abc);
+
+    final Path foo = new Path(abc, "foo");
+    DFSTestUtil.createFile(hdfs, foo, BLOCKSIZE, REPL, SEED);
+    hdfs.createSnapshot(abc, "s0");
+    
+    try {
+      hdfs.rename(abc, new Path(dirStr, "tmp"));
+      fail("Expect exception since " + abc
+          + " is snapshottable and already has snapshots");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains(abcStr
+          + " is snapshottable and already has snapshots", e);
+    }
+
+    final String xyzStr = dirStr + "/xyz";
+    final Path xyz = new Path(xyzStr);
+    hdfs.mkdirs(xyz, new FsPermission((short)0777));
+    final Path bar = new Path(xyz, "bar");
+    hdfs.rename(foo, bar);
+    
+    final INode fooRef = fsdir.getINode(
+        SnapshotTestHelper.getSnapshotPath(abc, "s0", "foo").toString());
+    Assert.assertTrue(fooRef.isReference());
+    Assert.assertTrue(fooRef.asReference() instanceof INodeReference.WithName);
+
+    final INodeReference.WithCount withCount
+        = (INodeReference.WithCount)fooRef.asReference().getReferredINode();
+    Assert.assertEquals(2, withCount.getReferenceCount());
+
+    final INode barRef = fsdir.getINode(bar.toString());
+    Assert.assertTrue(barRef.isReference());
+
+    Assert.assertSame(withCount, barRef.asReference().getReferredINode());
+    
+    hdfs.delete(bar, false);
+    Assert.assertEquals(1, withCount.getReferenceCount());
+  }
+  
+  private static boolean existsInDiffReport(List<DiffReportEntry> entries,
+      DiffType type, String relativePath) {
+    for (DiffReportEntry entry : entries) {
+      System.out.println("DiffEntry is:" + entry.getType() + "\""
+          + new String(entry.getRelativePath()) + "\"");
+      if ((entry.getType() == type)
+          && ((new String(entry.getRelativePath())).compareTo(relativePath) == 0)) {
+        return true;
+      }
+    }
+    return false;
+  }
+  
+  /**
+   * Rename a file under a snapshottable directory, file does not exist
+   * in a snapshot.
+   */
+  @Test (timeout=60000)
+  public void testRenameFileNotInSnapshot() throws Exception {
+    hdfs.mkdirs(sub1);
+    hdfs.allowSnapshot(sub1);
+    hdfs.createSnapshot(sub1, snap1);
+    DFSTestUtil.createFile(hdfs, file1, BLOCKSIZE, REPL, SEED);
+    hdfs.rename(file1, file2);
+
+    // Query the diff report and make sure it looks as expected.
+    SnapshotDiffReport diffReport = hdfs.getSnapshotDiffReport(sub1, snap1, "");
+    List<DiffReportEntry> entries = diffReport.getDiffList();
+    assertTrue(entries.size() == 2);
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, ""));
+    assertTrue(existsInDiffReport(entries, DiffType.CREATE, file2.getName()));
+  }
+
+  /**
+   * Rename a file under a snapshottable directory, file exists
+   * in a snapshot.
+   */
+  @Test
+  public void testRenameFileInSnapshot() throws Exception {
+    hdfs.mkdirs(sub1);
+    hdfs.allowSnapshot(sub1);
+    DFSTestUtil.createFile(hdfs, file1, BLOCKSIZE, REPL, SEED);
+    hdfs.createSnapshot(sub1, snap1);
+    hdfs.rename(file1, file2);
+
+    // Query the diff report and make sure it looks as expected.
+    SnapshotDiffReport diffReport = hdfs.getSnapshotDiffReport(sub1, snap1, "");
+    System.out.println("DiffList is " + diffReport.toString());
+    List<DiffReportEntry> entries = diffReport.getDiffList();
+    assertTrue(entries.size() == 3);
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, ""));
+    assertTrue(existsInDiffReport(entries, DiffType.CREATE, file2.getName()));
+    assertTrue(existsInDiffReport(entries, DiffType.DELETE, file1.getName()));
+  }
+
+  @Test (timeout=60000)
+  public void testRenameTwiceInSnapshot() throws Exception {
+    hdfs.mkdirs(sub1);
+    hdfs.allowSnapshot(sub1);
+    DFSTestUtil.createFile(hdfs, file1, BLOCKSIZE, REPL, SEED);
+    hdfs.createSnapshot(sub1, snap1);
+    hdfs.rename(file1, file2);
+    
+    hdfs.createSnapshot(sub1, snap2);
+    hdfs.rename(file2, file3);
+
+    SnapshotDiffReport diffReport;
+    
+    // Query the diff report and make sure it looks as expected.
+    diffReport = hdfs.getSnapshotDiffReport(sub1, snap1, snap2);
+    LOG.info("DiffList is " + diffReport.toString());
+    List<DiffReportEntry> entries = diffReport.getDiffList();
+    assertTrue(entries.size() == 3);
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, ""));
+    assertTrue(existsInDiffReport(entries, DiffType.CREATE, file2.getName()));
+    assertTrue(existsInDiffReport(entries, DiffType.DELETE, file1.getName()));
+    
+    diffReport = hdfs.getSnapshotDiffReport(sub1, snap2, "");
+    LOG.info("DiffList is " + diffReport.toString());
+    entries = diffReport.getDiffList();
+    assertTrue(entries.size() == 3);
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, ""));
+    assertTrue(existsInDiffReport(entries, DiffType.CREATE, file3.getName()));
+    assertTrue(existsInDiffReport(entries, DiffType.DELETE, file2.getName()));
+    
+    diffReport = hdfs.getSnapshotDiffReport(sub1, snap1, "");
+    LOG.info("DiffList is " + diffReport.toString());
+    entries = diffReport.getDiffList();
+    assertTrue(entries.size() == 3);
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, ""));
+    assertTrue(existsInDiffReport(entries, DiffType.CREATE, file3.getName()));
+    assertTrue(existsInDiffReport(entries, DiffType.DELETE, file1.getName()));
+  }
+  
+  @Test (timeout=60000)
+  public void testRenameFileInSubDirOfDirWithSnapshot() throws Exception {
+    final Path sub2 = new Path(sub1, "sub2");
+    final Path sub2file1 = new Path(sub2, "sub2file1");
+    final Path sub2file2 = new Path(sub2, "sub2file2");
+    final String sub1snap1 = "sub1snap1";
+    
+    hdfs.mkdirs(sub1);
+    hdfs.mkdirs(sub2);
+    DFSTestUtil.createFile(hdfs, sub2file1, BLOCKSIZE, REPL, SEED);
+    SnapshotTestHelper.createSnapshot(hdfs, sub1, sub1snap1);
+
+    // Rename the file in the subdirectory.
+    hdfs.rename(sub2file1, sub2file2);
+
+    // Query the diff report and make sure it looks as expected.
+    SnapshotDiffReport diffReport = hdfs.getSnapshotDiffReport(sub1, sub1snap1,
+        "");
+    LOG.info("DiffList is \n\"" + diffReport.toString() + "\"");
+    List<DiffReportEntry> entries = diffReport.getDiffList();
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, sub2.getName()));
+    assertTrue(existsInDiffReport(entries, DiffType.CREATE, sub2.getName()
+        + "/" + sub2file2.getName()));
+    assertTrue(existsInDiffReport(entries, DiffType.DELETE, sub2.getName()
+        + "/" + sub2file1.getName()));
+  }
+
+  @Test (timeout=60000)
+  public void testRenameDirectoryInSnapshot() throws Exception {
+    final Path sub2 = new Path(sub1, "sub2");
+    final Path sub3 = new Path(sub1, "sub3");
+    final Path sub2file1 = new Path(sub2, "sub2file1");
+    final String sub1snap1 = "sub1snap1";
+    
+    hdfs.mkdirs(sub1);
+    hdfs.mkdirs(sub2);
+    DFSTestUtil.createFile(hdfs, sub2file1, BLOCKSIZE, REPL, SEED);
+    SnapshotTestHelper.createSnapshot(hdfs, sub1, sub1snap1);
+    
+    // First rename the sub-directory.
+    hdfs.rename(sub2, sub3);
+    
+    // Query the diff report and make sure it looks as expected.
+    SnapshotDiffReport diffReport = hdfs.getSnapshotDiffReport(sub1, sub1snap1,
+        "");
+    LOG.info("DiffList is \n\"" + diffReport.toString() + "\"");
+    List<DiffReportEntry> entries = diffReport.getDiffList();
+    assertEquals(3, entries.size());
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, ""));
+    assertTrue(existsInDiffReport(entries, DiffType.CREATE, sub3.getName()));
+    assertTrue(existsInDiffReport(entries, DiffType.DELETE, sub2.getName()));
+  }
+  
+  /**
+   * After the following steps:
+   * <pre>
+   * 1. Take snapshot s1 on /dir1 at time t1.
+   * 2. Take snapshot s2 on /dir2 at time t2.
+   * 3. Modify the subtree of /dir2/foo/ to make it a dir with snapshots.
+   * 4. Take snapshot s3 on /dir1 at time t3.
+   * 5. Rename /dir2/foo/ to /dir1/foo/.
+   * </pre>
+   * When changes happening on foo, the diff should be recorded in snapshot s2. 
+   */
+  @Test (timeout=60000)
+  public void testRenameDirAcrossSnapshottableDirs() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    hdfs.mkdirs(sdir1);
+    hdfs.mkdirs(sdir2);
+    final Path foo = new Path(sdir2, "foo");
+    final Path bar = new Path(foo, "bar");
+    final Path bar2 = new Path(foo, "bar2");
+    DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
+    DFSTestUtil.createFile(hdfs, bar2, BLOCKSIZE, REPL, SEED);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
+    
+    hdfs.setReplication(bar2, REPL_1);
+    hdfs.delete(bar, true);
+    
+    hdfs.createSnapshot(sdir1, "s3");
+    
+    final Path newfoo = new Path(sdir1, "foo");
+    hdfs.rename(foo, newfoo);
+    
+    // still can visit the snapshot copy of bar through 
+    // /dir2/.snapshot/s2/foo/bar
+    final Path snapshotBar = SnapshotTestHelper.getSnapshotPath(sdir2, "s2",
+        "foo/bar");
+    assertTrue(hdfs.exists(snapshotBar));
+    
+    // delete bar2
+    final Path newBar2 = new Path(newfoo, "bar2");
+    assertTrue(hdfs.exists(newBar2));
+    hdfs.delete(newBar2, true);
+    
+    // /dir2/.snapshot/s2/foo/bar2 should still work
+    final Path bar2_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2",
+        "foo/bar2");
+    assertTrue(hdfs.exists(bar2_s2));
+    FileStatus status = hdfs.getFileStatus(bar2_s2);
+    assertEquals(REPL, status.getReplication());
+    final Path bar2_s3 = SnapshotTestHelper.getSnapshotPath(sdir1, "s3",
+        "foo/bar2");
+    assertFalse(hdfs.exists(bar2_s3));
+  }
+  
+  /**
+   * Rename a single file across snapshottable dirs.
+   */
+  @Test (timeout=60000)
+  public void testRenameFileAcrossSnapshottableDirs() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    hdfs.mkdirs(sdir1);
+    hdfs.mkdirs(sdir2);
+    final Path foo = new Path(sdir2, "foo");
+    DFSTestUtil.createFile(hdfs, foo, BLOCKSIZE, REPL, SEED);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
+    hdfs.createSnapshot(sdir1, "s3");
+    
+    final Path newfoo = new Path(sdir1, "foo");
+    hdfs.rename(foo, newfoo);
+    
+    // change the replication factor of foo
+    hdfs.setReplication(newfoo, REPL_1);
+    
+    // /dir2/.snapshot/s2/foo should still work
+    final Path foo_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2",
+        "foo");
+    assertTrue(hdfs.exists(foo_s2));
+    FileStatus status = hdfs.getFileStatus(foo_s2);
+    assertEquals(REPL, status.getReplication());
+    
+    final Path foo_s3 = SnapshotTestHelper.getSnapshotPath(sdir1, "s3",
+        "foo");
+    assertFalse(hdfs.exists(foo_s3));
+    INodeFileWithSnapshot sfoo = (INodeFileWithSnapshot) fsdir.getINode(
+        newfoo.toString()).asFile();
+    assertEquals("s2", sfoo.getDiffs().getLastSnapshot().getRoot()
+        .getLocalName());
+  }
+  
+  /**
+   * Test renaming a dir and then delete snapshots.
+   */
+  @Test
+  public void testRenameDirAndDeleteSnapshot_1() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    hdfs.mkdirs(sdir1);
+    hdfs.mkdirs(sdir2);
+    final Path foo = new Path(sdir2, "foo");
+    final Path bar = new Path(foo, "bar");
+    final Path bar2 = new Path(foo, "bar2");
+    DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
+    DFSTestUtil.createFile(hdfs, bar2, BLOCKSIZE, REPL, SEED);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
+    hdfs.createSnapshot(sdir1, "s3");
+    
+    final Path newfoo = new Path(sdir1, "foo");
+    hdfs.rename(foo, newfoo);
+    
+    final Path newbar = new Path(newfoo, bar.getName());
+    final Path newbar2 = new Path(newfoo, bar2.getName());
+    final Path newbar3 = new Path(newfoo, "bar3");
+    DFSTestUtil.createFile(hdfs, newbar3, BLOCKSIZE, REPL, SEED);
+    
+    hdfs.createSnapshot(sdir1, "s4");
+    hdfs.delete(newbar, true);
+    hdfs.delete(newbar3, true);
+    
+    assertFalse(hdfs.exists(newbar3));
+    assertFalse(hdfs.exists(bar));
+    final Path bar_s4 = SnapshotTestHelper.getSnapshotPath(sdir1, "s4",
+        "foo/bar");
+    final Path bar3_s4 = SnapshotTestHelper.getSnapshotPath(sdir1, "s4",
+        "foo/bar3");
+    assertTrue(hdfs.exists(bar_s4));
+    assertTrue(hdfs.exists(bar3_s4));
+    
+    hdfs.createSnapshot(sdir1, "s5");
+    hdfs.delete(newbar2, true);
+    assertFalse(hdfs.exists(bar2));
+    final Path bar2_s5 = SnapshotTestHelper.getSnapshotPath(sdir1, "s5",
+        "foo/bar2");
+    assertTrue(hdfs.exists(bar2_s5));
+    
+    // delete snapshot s5. The diff of s5 should be combined to s4
+    hdfs.deleteSnapshot(sdir1, "s5");
+    restartClusterAndCheckImage();
+    assertFalse(hdfs.exists(bar2_s5));
+    final Path bar2_s4 = SnapshotTestHelper.getSnapshotPath(sdir1, "s4",
+        "foo/bar2");
+    assertTrue(hdfs.exists(bar2_s4));
+    
+    // delete snapshot s4. The diff of s4 should be combined to s2 instead of
+    // s3.
+    hdfs.deleteSnapshot(sdir1, "s4");
+    
+    assertFalse(hdfs.exists(bar_s4));
+    Path bar_s3 = SnapshotTestHelper.getSnapshotPath(sdir1, "s3", "foo/bar");
+    assertFalse(hdfs.exists(bar_s3));
+    bar_s3 = SnapshotTestHelper.getSnapshotPath(sdir2, "s3", "foo/bar");
+    assertFalse(hdfs.exists(bar_s3));
+    final Path bar_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2",
+        "foo/bar");
+    assertTrue(hdfs.exists(bar_s2));
+    
+    assertFalse(hdfs.exists(bar2_s4));
+    Path bar2_s3 = SnapshotTestHelper.getSnapshotPath(sdir1, "s3", "foo/bar2");
+    assertFalse(hdfs.exists(bar2_s3));
+    bar2_s3 = SnapshotTestHelper.getSnapshotPath(sdir2, "s3", "foo/bar2");
+    assertFalse(hdfs.exists(bar2_s3));
+    final Path bar2_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2",
+        "foo/bar2");
+    assertTrue(hdfs.exists(bar2_s2));
+    
+    assertFalse(hdfs.exists(bar3_s4));
+    Path bar3_s3 = SnapshotTestHelper.getSnapshotPath(sdir1, "s3", "foo/bar3");
+    assertFalse(hdfs.exists(bar3_s3));
+    bar3_s3 = SnapshotTestHelper.getSnapshotPath(sdir2, "s3", "foo/bar3");
+    assertFalse(hdfs.exists(bar3_s3));
+    final Path bar3_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2",
+        "foo/bar3");
+    assertFalse(hdfs.exists(bar3_s2));
+    
+    // restart the cluster and check fsimage
+    restartClusterAndCheckImage();
+    
+    // delete snapshot s2.
+    hdfs.deleteSnapshot(sdir2, "s2");
+    assertFalse(hdfs.exists(bar_s2));
+    assertFalse(hdfs.exists(bar2_s2));
+    
+    // restart the cluster and check fsimage
+    restartClusterAndCheckImage();
+    hdfs.deleteSnapshot(sdir1, "s3");
+    restartClusterAndCheckImage();
+    hdfs.deleteSnapshot(sdir1, "s1");
+    restartClusterAndCheckImage();
+  }
+  
+  private void restartClusterAndCheckImage() throws IOException {
+    File fsnBefore = new File(testDir, "dumptree_before");
+    File fsnMiddle = new File(testDir, "dumptree_middle");
+    File fsnAfter = new File(testDir, "dumptree_after");
+    
+    SnapshotTestHelper.dumpTree2File(fsdir, fsnBefore);
+    
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).format(false)
+        .numDataNodes(REPL).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    fsdir = fsn.getFSDirectory();
+    hdfs = cluster.getFileSystem();
+    // later check fsnMiddle to see if the edit log is applied correctly 
+    SnapshotTestHelper.dumpTree2File(fsdir, fsnMiddle);
+   
+    // save namespace and restart cluster
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.saveNamespace();
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).format(false)
+        .numDataNodes(REPL).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    fsdir = fsn.getFSDirectory();
+    hdfs = cluster.getFileSystem();
+    // dump the namespace loaded from fsimage
+    SnapshotTestHelper.dumpTree2File(fsdir, fsnAfter);
+    
+    SnapshotTestHelper.compareDumpedTreeInFile(fsnBefore, fsnMiddle, true);
+    SnapshotTestHelper.compareDumpedTreeInFile(fsnBefore, fsnAfter, true);
+  }
+  
+  /**
+   * Test renaming a file and then delete snapshots.
+   */
+  @Test
+  public void testRenameFileAndDeleteSnapshot() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    hdfs.mkdirs(sdir1);
+    hdfs.mkdirs(sdir2);
+    final Path foo = new Path(sdir2, "foo");
+    DFSTestUtil.createFile(hdfs, foo, BLOCKSIZE, REPL, SEED);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
+    hdfs.createSnapshot(sdir1, "s3");
+    
+    final Path newfoo = new Path(sdir1, "foo");
+    hdfs.rename(foo, newfoo);
+    
+    hdfs.setReplication(newfoo, REPL_1);
+    
+    hdfs.createSnapshot(sdir1, "s4");
+    hdfs.setReplication(newfoo, REPL_2);
+    
+    FileStatus status = hdfs.getFileStatus(newfoo);
+    assertEquals(REPL_2, status.getReplication());
+    final Path foo_s4 = SnapshotTestHelper.getSnapshotPath(sdir1, "s4", "foo");
+    status = hdfs.getFileStatus(foo_s4);
+    assertEquals(REPL_1, status.getReplication());
+    
+    hdfs.createSnapshot(sdir1, "s5");
+    final Path foo_s5 = SnapshotTestHelper.getSnapshotPath(sdir1, "s5", "foo");
+    status = hdfs.getFileStatus(foo_s5);
+    assertEquals(REPL_2, status.getReplication());
+    
+    // delete snapshot s5.
+    hdfs.deleteSnapshot(sdir1, "s5");
+    restartClusterAndCheckImage();
+    assertFalse(hdfs.exists(foo_s5));
+    status = hdfs.getFileStatus(foo_s4);
+    assertEquals(REPL_1, status.getReplication());
+    
+    // delete snapshot s4.
+    hdfs.deleteSnapshot(sdir1, "s4");
+    
+    assertFalse(hdfs.exists(foo_s4));
+    Path foo_s3 = SnapshotTestHelper.getSnapshotPath(sdir1, "s3", "foo");
+    assertFalse(hdfs.exists(foo_s3));
+    foo_s3 = SnapshotTestHelper.getSnapshotPath(sdir2, "s3", "foo");
+    assertFalse(hdfs.exists(foo_s3));
+    final Path foo_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2", "foo");
+    assertTrue(hdfs.exists(foo_s2));
+    status = hdfs.getFileStatus(foo_s2);
+    assertEquals(REPL, status.getReplication());
+    
+    INodeFileWithSnapshot snode = (INodeFileWithSnapshot) fsdir.getINode(
+        newfoo.toString()).asFile();
+    assertEquals(1, snode.getDiffs().asList().size());
+    assertEquals("s2", snode.getDiffs().getLastSnapshot().getRoot()
+        .getLocalName());
+    
+    // restart cluster
+    restartClusterAndCheckImage();
+    
+    // delete snapshot s2.
+    hdfs.deleteSnapshot(sdir2, "s2");
+    assertFalse(hdfs.exists(foo_s2));
+    
+    // restart the cluster and check fsimage
+    restartClusterAndCheckImage();
+    hdfs.deleteSnapshot(sdir1, "s3");
+    restartClusterAndCheckImage();
+    hdfs.deleteSnapshot(sdir1, "s1");
+    restartClusterAndCheckImage();
+  }
+  
+  /**
+   * Test rename a dir and a file multiple times across snapshottable 
+   * directories: /dir1/foo -> /dir2/foo -> /dir3/foo -> /dir2/foo -> /dir1/foo
+   * 
+   * Only create snapshots in the beginning (before the rename).
+   */
+  @Test
+  public void testRenameMoreThanOnceAcrossSnapDirs() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    final Path sdir3 = new Path("/dir3");
+    hdfs.mkdirs(sdir1);
+    hdfs.mkdirs(sdir2);
+    hdfs.mkdirs(sdir3);
+    
+    final Path foo_dir1 = new Path(sdir1, "foo");
+    final Path bar1_dir1 = new Path(foo_dir1, "bar1");
+    final Path bar2_dir1 = new Path(sdir1, "bar");
+    DFSTestUtil.createFile(hdfs, bar1_dir1, BLOCKSIZE, REPL, SEED);
+    DFSTestUtil.createFile(hdfs, bar2_dir1, BLOCKSIZE, REPL, SEED);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir3, "s3");
+    
+    // 1. /dir1/foo -> /dir2/foo, /dir1/bar -> /dir2/bar
+    final Path foo_dir2 = new Path(sdir2, "foo");
+    hdfs.rename(foo_dir1, foo_dir2);
+    final Path bar2_dir2 = new Path(sdir2, "bar");
+    hdfs.rename(bar2_dir1, bar2_dir2);
+    
+    // restart the cluster and check fsimage
+    restartClusterAndCheckImage();
+    
+    // modification on /dir2/foo and /dir2/bar
+    final Path bar1_dir2 = new Path(foo_dir2, "bar1");
+    hdfs.setReplication(bar1_dir2, REPL_1);
+    hdfs.setReplication(bar2_dir2, REPL_1);
+    
+    // check
+    final Path bar1_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1",
+        "foo/bar1");
+    final Path bar2_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1",
+        "bar");
+    final Path bar1_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2",
+        "foo/bar1");
+    final Path bar2_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2",
+        "bar");
+    assertTrue(hdfs.exists(bar1_s1));
+    assertTrue(hdfs.exists(bar2_s1));
+    assertFalse(hdfs.exists(bar1_s2));
+    assertFalse(hdfs.exists(bar2_s2));
+    FileStatus statusBar1 = hdfs.getFileStatus(bar1_s1);
+    assertEquals(REPL, statusBar1.getReplication());
+    statusBar1 = hdfs.getFileStatus(bar1_dir2);
+    assertEquals(REPL_1, statusBar1.getReplication());
+    FileStatus statusBar2 = hdfs.getFileStatus(bar2_s1);
+    assertEquals(REPL, statusBar2.getReplication());
+    statusBar2 = hdfs.getFileStatus(bar2_dir2);
+    assertEquals(REPL_1, statusBar2.getReplication());
+    
+    // 2. /dir2/foo -> /dir3/foo, /dir2/bar -> /dir3/bar
+    final Path foo_dir3 = new Path(sdir3, "foo");
+    hdfs.rename(foo_dir2, foo_dir3);
+    final Path bar2_dir3 = new Path(sdir3, "bar");
+    hdfs.rename(bar2_dir2, bar2_dir3);
+    
+    // restart the cluster and check fsimage
+    restartClusterAndCheckImage();
+    
+    // modification on /dir3/foo and /dir3/bar
+    final Path bar1_dir3 = new Path(foo_dir3, "bar1");
+    hdfs.setReplication(bar1_dir3, REPL_2);
+    hdfs.setReplication(bar2_dir3, REPL_2);
+    
+    // check
+    final Path bar1_s3 = SnapshotTestHelper.getSnapshotPath(sdir3, "s3",
+        "foo/bar1");
+    final Path bar2_s3 = SnapshotTestHelper.getSnapshotPath(sdir3, "s3",
+        "bar");
+    assertTrue(hdfs.exists(bar1_s1));
+    assertTrue(hdfs.exists(bar2_s1));
+    assertFalse(hdfs.exists(bar1_s2));
+    assertFalse(hdfs.exists(bar2_s2));
+    assertFalse(hdfs.exists(bar1_s3));
+    assertFalse(hdfs.exists(bar2_s3));
+    statusBar1 = hdfs.getFileStatus(bar1_s1);
+    assertEquals(REPL, statusBar1.getReplication());
+    statusBar1 = hdfs.getFileStatus(bar1_dir3);
+    assertEquals(REPL_2, statusBar1.getReplication());
+    statusBar2 = hdfs.getFileStatus(bar2_s1);
+    assertEquals(REPL, statusBar2.getReplication());
+    statusBar2 = hdfs.getFileStatus(bar2_dir3);
+    assertEquals(REPL_2, statusBar2.getReplication());
+    
+    // 3. /dir3/foo -> /dir2/foo, /dir3/bar -> /dir2/bar
+    hdfs.rename(foo_dir3, foo_dir2);
+    hdfs.rename(bar2_dir3, bar2_dir2);
+    
+    // restart the cluster and check fsimage
+    restartClusterAndCheckImage();
+    
+    // modification on /dir2/foo
+    hdfs.setReplication(bar1_dir2, REPL);
+    hdfs.setReplication(bar2_dir2, REPL);
+    
+    // check
+    assertTrue(hdfs.exists(bar1_s1));
+    assertTrue(hdfs.exists(bar2_s1));
+    assertFalse(hdfs.exists(bar1_s2));
+    assertFalse(hdfs.exists(bar2_s2));
+    assertFalse(hdfs.exists(bar1_s3));
+    assertFalse(hdfs.exists(bar2_s3));
+    statusBar1 = hdfs.getFileStatus(bar1_s1);
+    assertEquals(REPL, statusBar1.getReplication());
+    statusBar1 = hdfs.getFileStatus(bar1_dir2);
+    assertEquals(REPL, statusBar1.getReplication());
+    statusBar2 = hdfs.getFileStatus(bar2_s1);
+    assertEquals(REPL, statusBar2.getReplication());
+    statusBar2 = hdfs.getFileStatus(bar2_dir2);
+    assertEquals(REPL, statusBar2.getReplication());
+    
+    // 4. /dir2/foo -> /dir1/foo, /dir2/bar -> /dir1/bar
+    hdfs.rename(foo_dir2, foo_dir1);
+    hdfs.rename(bar2_dir2, bar2_dir1);
+    
+    // check the internal details
+    INodeReference fooRef = fsdir.getINode4Write(foo_dir1.toString())
+        .asReference();
+    INodeReference.WithCount fooWithCount = (WithCount) fooRef
+        .getReferredINode();
+    // only 2 references: one in deleted list of sdir1, one in created list of
+    // sdir1
+    assertEquals(2, fooWithCount.getReferenceCount());
+    INodeDirectoryWithSnapshot foo = (INodeDirectoryWithSnapshot) fooWithCount
+        .asDirectory();
+    assertEquals(1, foo.getDiffs().asList().size());
+    assertEquals("s1", foo.getLastSnapshot().getRoot().getLocalName());
+    INodeFileWithSnapshot bar1 = (INodeFileWithSnapshot) fsdir.getINode4Write(
+        bar1_dir1.toString()).asFile();
+    assertEquals(1, bar1.getDiffs().asList().size());
+    assertEquals("s1", bar1.getDiffs().getLastSnapshot().getRoot()
+        .getLocalName());
+    
+    INodeReference barRef = fsdir.getINode4Write(bar2_dir1.toString())
+        .asReference();
+    INodeReference.WithCount barWithCount = (WithCount) barRef
+        .getReferredINode();
+    assertEquals(2, barWithCount.getReferenceCount());
+    INodeFileWithSnapshot bar = (INodeFileWithSnapshot) barWithCount.asFile();
+    assertEquals(1, bar.getDiffs().asList().size());
+    assertEquals("s1", bar.getDiffs().getLastSnapshot().getRoot()
+        .getLocalName());
+    
+    // restart the cluster and check fsimage
+    restartClusterAndCheckImage();
+    
+    // delete foo
+    hdfs.delete(foo_dir1, true);
+    hdfs.delete(bar2_dir1, true);
+    
+    // restart the cluster and check fsimage
+    restartClusterAndCheckImage();
+    
+    // check
+    assertTrue(hdfs.exists(bar1_s1));
+    assertTrue(hdfs.exists(bar2_s1));
+    assertFalse(hdfs.exists(bar1_s2));
+    assertFalse(hdfs.exists(bar2_s2));
+    assertFalse(hdfs.exists(bar1_s3));
+    assertFalse(hdfs.exists(bar2_s3));
+    assertFalse(hdfs.exists(foo_dir1));
+    assertFalse(hdfs.exists(bar1_dir1));
+    assertFalse(hdfs.exists(bar2_dir1));
+    statusBar1 = hdfs.getFileStatus(bar1_s1);
+    assertEquals(REPL, statusBar1.getReplication());
+    statusBar2 = hdfs.getFileStatus(bar2_s1);
+    assertEquals(REPL, statusBar2.getReplication());
+    
+    final Path foo_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1", "foo");
+    fooRef = fsdir.getINode(foo_s1.toString()).asReference();
+    fooWithCount = (WithCount) fooRef.getReferredINode();
+    assertEquals(1, fooWithCount.getReferenceCount());
+    
+    barRef = fsdir.getINode(bar2_s1.toString()).asReference();
+    barWithCount = (WithCount) barRef.getReferredINode();
+    assertEquals(1, barWithCount.getReferenceCount());
+  }
+  
+  /**
+   * Test rename a dir multiple times across snapshottable directories: 
+   * /dir1/foo -> /dir2/foo -> /dir3/foo -> /dir2/foo -> /dir1/foo
+   * 
+   * Create snapshots after each rename.
+   */
+  @Test
+  public void testRenameMoreThanOnceAcrossSnapDirs_2() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    final Path sdir3 = new Path("/dir3");
+    hdfs.mkdirs(sdir1);
+    hdfs.mkdirs(sdir2);
+    hdfs.mkdirs(sdir3);
+    
+    final Path foo_dir1 = new Path(sdir1, "foo");
+    final Path bar1_dir1 = new Path(foo_dir1, "bar1");
+    final Path bar_dir1 = new Path(sdir1, "bar");
+    DFSTestUtil.createFile(hdfs, bar1_dir1, BLOCKSIZE, REPL, SEED);
+    DFSTestUtil.createFile(hdfs, bar_dir1, BLOCKSIZE, REPL, SEED);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir3, "s3");
+    
+    // 1. /dir1/foo -> /dir2/foo, /dir1/bar -> /dir2/bar
+    final Path foo_dir2 = new Path(sdir2, "foo");
+    hdfs.rename(foo_dir1, foo_dir2);
+    final Path bar_dir2 = new Path(sdir2, "bar");
+    hdfs.rename(bar_dir1, bar_dir2);
+    
+    // modification on /dir2/foo and /dir2/bar
+    final Path bar1_dir2 = new Path(foo_dir2, "bar1");
+    hdfs.setReplication(bar1_dir2, REPL_1);
+    hdfs.setReplication(bar_dir2, REPL_1);
+    
+    // restart the cluster and check fsimage
+    restartClusterAndCheckImage();
+    
+    // create snapshots
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s11");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s22");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir3, "s33");
+    
+    // 2. /dir2/foo -> /dir3/foo
+    final Path foo_dir3 = new Path(sdir3, "foo");
+    hdfs.rename(foo_dir2, foo_dir3);
+    final Path bar_dir3 = new Path(sdir3, "bar");
+    hdfs.rename(bar_dir2, bar_dir3);
+    
+    // modification on /dir3/foo
+    final Path bar1_dir3 = new Path(foo_dir3, "bar1");
+    hdfs.setReplication(bar1_dir3, REPL_2);
+    hdfs.setReplication(bar_dir3, REPL_2);
+    
+    // restart the cluster and check fsimage
+    restartClusterAndCheckImage();
+    
+    // create snapshots
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s111");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s222");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir3, "s333");
+    
+    // check
+    final Path bar1_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1",
+        "foo/bar1");
+    final Path bar1_s22 = SnapshotTestHelper.getSnapshotPath(sdir2, "s22",
+        "foo/bar1");
+    final Path bar1_s333 = SnapshotTestHelper.getSnapshotPath(sdir3, "s333",
+        "foo/bar1");
+    final Path bar_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1",
+        "bar");
+    final Path bar_s22 = SnapshotTestHelper.getSnapshotPath(sdir2, "s22",
+        "bar");
+    final Path bar_s333 = SnapshotTestHelper.getSnapshotPath(sdir3, "s333",
+        "bar");
+    assertTrue(hdfs.exists(bar1_s1));
+    assertTrue(hdfs.exists(bar1_s22));
+    assertTrue(hdfs.exists(bar1_s333));
+    assertTrue(hdfs.exists(bar_s1));
+    assertTrue(hdfs.exists(bar_s22));
+    assertTrue(hdfs.exists(bar_s333));
+    
+    FileStatus statusBar1 = hdfs.getFileStatus(bar1_s1);
+    assertEquals(REPL, statusBar1.getReplication());
+    statusBar1 = hdfs.getFileStatus(bar1_dir3);
+    assertEquals(REPL_2, statusBar1.getReplication());
+    statusBar1 = hdfs.getFileStatus(bar1_s22);
+    assertEquals(REPL_1, statusBar1.getReplication());
+    statusBar1 = hdfs.getFileStatus(bar1_s333);
+    assertEquals(REPL_2, statusBar1.getReplication());
+    
+    FileStatus statusBar = hdfs.getFileStatus(bar_s1);
+    assertEquals(REPL, statusBar.getReplication());
+    statusBar = hdfs.getFileStatus(bar_dir3);
+    assertEquals(REPL_2, statusBar.getReplication());
+    statusBar = hdfs.getFileStatus(bar_s22);
+    assertEquals(REPL_1, statusBar.getReplication());
+    statusBar = hdfs.getFileStatus(bar_s333);
+    assertEquals(REPL_2, statusBar.getReplication());
+    
+    // 3. /dir3/foo -> /dir2/foo
+    hdfs.rename(foo_dir3, foo_dir2);
+    hdfs.rename(bar_dir3, bar_dir2);
+   
+    // modification on /dir2/foo
+    hdfs.setReplication(bar1_dir2, REPL);
+    hdfs.setReplication(bar_dir2, REPL);
+    
+    // restart the cluster and check fsimage
+    restartClusterAndCheckImage();
+    
+    // create snapshots
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1111");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2222");
+    
+    // check
+    final Path bar1_s2222 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2222",
+        "foo/bar1");
+    final Path bar_s2222 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2222",
+        "bar");
+    assertTrue(hdfs.exists(bar1_s1));
+    assertTrue(hdfs.exists(bar1_s22));
+    assertTrue(hdfs.exists(bar1_s333));
+    assertTrue(hdfs.exists(bar1_s2222));
+    assertTrue(hdfs.exists(bar_s1));
+    assertTrue(hdfs.exists(bar_s22));
+    assertTrue(hdfs.exists(bar_s333));
+    assertTrue(hdfs.exists(bar_s2222));
+    
+    statusBar1 = hdfs.getFileStatus(bar1_s1);
+    assertEquals(REPL, statusBar1.getReplication());
+    statusBar1 = hdfs.getFileStatus(bar1_dir2);
+    assertEquals(REPL, statusBar1.getReplication());
+    statusBar1 = hdfs.getFileStatus(bar1_s22);
+    assertEquals(REPL_1, statusBar1.getReplication());
+    statusBar1 = hdfs.getFileStatus(bar1_s333);
+    assertEquals(REPL_2, statusBar1.getReplication());
+    statusBar1 = hdfs.getFileStatus(bar1_s2222);
+    assertEquals(REPL, statusBar1.getReplication());
+    
+    statusBar = hdfs.getFileStatus(bar_s1);
+    assertEquals(REPL, statusBar.getReplication());
+    statusBar = hdfs.getFileStatus(bar_dir2);
+    assertEquals(REPL, statusBar.getReplication());
+    statusBar = hdfs.getFileStatus(bar_s22);
+    assertEquals(REPL_1, statusBar.getReplication());
+    statusBar = hdfs.getFileStatus(bar_s333);
+    assertEquals(REPL_2, statusBar.getReplication());
+    statusBar = hdfs.getFileStatus(bar_s2222);
+    assertEquals(REPL, statusBar.getReplication());
+    
+    // 4. /dir2/foo -> /dir1/foo
+    hdfs.rename(foo_dir2, foo_dir1);
+    hdfs.rename(bar_dir2, bar_dir1);
+    
+    // check the internal details
+    INodeReference fooRef = fsdir.getINode4Write(foo_dir1.toString())
+        .asReference();
+    INodeReference.WithCount fooWithCount = (WithCount) fooRef.getReferredINode();
+    // 5 references: s1, s22, s333, s2222, current tree of sdir1
+    assertEquals(5, fooWithCount.getReferenceCount());
+    INodeDirectoryWithSnapshot foo = (INodeDirectoryWithSnapshot) fooWithCount
+        .asDirectory();
+    List<DirectoryDiff> fooDiffs = foo.getDiffs().asList();
+    assertEquals(4, fooDiffs.size());
+    assertEquals("s2222", fooDiffs.get(3).snapshot.getRoot().getLocalName());
+    assertEquals("s333", fooDiffs.get(2).snapshot.getRoot().getLocalName());
+    assertEquals("s22", fooDiffs.get(1).snapshot.getRoot().getLocalName());
+    assertEquals("s1", fooDiffs.get(0).snapshot.getRoot().getLocalName());
+    INodeFileWithSnapshot bar1 = (INodeFileWithSnapshot) fsdir.getINode4Write(
+        bar1_dir1.toString()).asFile();
+    List<FileDiff> bar1Diffs = bar1.getDiffs().asList();
+    assertEquals(3, bar1Diffs.size());
+    assertEquals("s333", bar1Diffs.get(2).snapshot.getRoot().getLocalName());
+    assertEquals("s22", bar1Diffs.get(1).snapshot.getRoot().getLocalName());
+    assertEquals("s1", bar1Diffs.get(0).snapshot.getRoot().getLocalName());
+    
+    INodeReference barRef = fsdir.getINode4Write(bar_dir1.toString())
+        .asReference();
+    INodeReference.WithCount barWithCount = (WithCount) barRef.getReferredINode();
+    // 5 references: s1, s22, s333, s2222, current tree of sdir1
+    assertEquals(5, barWithCount.getReferenceCount());
+    INodeFileWithSnapshot bar = (INodeFileWithSnapshot) barWithCount.asFile();
+    List<FileDiff> barDiffs = bar.getDiffs().asList();
+    assertEquals(4, barDiffs.size());
+    assertEquals("s2222", barDiffs.get(3).snapshot.getRoot().getLocalName());
+    assertEquals("s333", barDiffs.get(2).snapshot.getRoot().getLocalName());
+    assertEquals("s22", barDiffs.get(1).snapshot.getRoot().getLocalName());
+    assertEquals("s1", barDiffs.get(0).snapshot.getRoot().getLocalName());
+    
+    // restart the cluster and check fsimage
+    restartClusterAndCheckImage();
+    
+    // delete foo
+    hdfs.delete(foo_dir1, true);
+    hdfs.delete(bar_dir1, true);
+    
+    // restart the cluster and check fsimage
+    restartClusterAndCheckImage();
+    
+    // check
+    final Path bar1_s1111 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1111",
+        "foo/bar1");
+    final Path bar_s1111 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1111",
+        "bar");
+    assertTrue(hdfs.exists(bar1_s1));
+    assertTrue(hdfs.exists(bar1_s22));
+    assertTrue(hdfs.exists(bar1_s333));
+    assertTrue(hdfs.exists(bar1_s2222));
+    assertFalse(hdfs.exists(bar1_s1111));
+    assertTrue(hdfs.exists(bar_s1));
+    assertTrue(hdfs.exists(bar_s22));
+    assertTrue(hdfs.exists(bar_s333));
+    assertTrue(hdfs.exists(bar_s2222));
+    assertFalse(hdfs.exists(bar_s1111));
+    
+    final Path foo_s2222 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2222",
+        "foo");
+    fooRef = fsdir.getINode(foo_s2222.toString()).asReference();
+    fooWithCount = (WithCount) fooRef.getReferredINode();
+    assertEquals(4, fooWithCount.getReferenceCount());
+    foo = (INodeDirectoryWithSnapshot) fooWithCount.asDirectory();
+    fooDiffs = foo.getDiffs().asList();
+    assertEquals(4, fooDiffs.size());
+    assertEquals("s2222", fooDiffs.get(3).snapshot.getRoot().getLocalName());
+    bar1Diffs = bar1.getDiffs().asList();
+    assertEquals(3, bar1Diffs.size());
+    assertEquals("s333", bar1Diffs.get(2).snapshot.getRoot().getLocalName());
+    
+    barRef = fsdir.getINode(bar_s2222.toString()).asReference();
+    barWithCount = (WithCount) barRef.getReferredINode();
+    assertEquals(4, barWithCount.getReferenceCount());
+    bar = (INodeFileWithSnapshot) barWithCount.asFile();
+    barDiffs = bar.getDiffs().asList();
+    assertEquals(4, barDiffs.size());
+    assertEquals("s2222", barDiffs.get(3).snapshot.getRoot().getLocalName());
+  }
+  
+  /**
+   * Test rename from a non-snapshottable dir to a snapshottable dir
+   */
+  @Test (timeout=60000)
+  public void testRenameFromNonSDir2SDir() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    hdfs.mkdirs(sdir1);
+    hdfs.mkdirs(sdir2);
+    final Path foo = new Path(sdir1, "foo");
+    final Path bar = new Path(foo, "bar");
+    DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir2, snap1);
+    
+    final Path newfoo = new Path(sdir2, "foo");
+    hdfs.rename(foo, newfoo);
+    
+    INode fooNode = fsdir.getINode4Write(newfoo.toString());
+    assertTrue(fooNode instanceof INodeDirectory);
+  }
+  
+  /**
+   * Test rename where the src/dst directories are both snapshottable 
+   * directories without snapshots. In such case we need to update the 
+   * snapshottable dir list in SnapshotManager.
+   */
+  @Test (timeout=60000)
+  public void testRenameAndUpdateSnapshottableDirs() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    final Path foo = new Path(sdir1, "foo");
+    final Path bar = new Path(sdir2, "bar");
+    hdfs.mkdirs(foo);
+    hdfs.mkdirs(bar);
+    
+    hdfs.allowSnapshot(foo);
+    SnapshotTestHelper.createSnapshot(hdfs, bar, snap1);
+    assertEquals(2, fsn.getSnapshottableDirListing().length);
+    
+    INodeDirectory fooNode = fsdir.getINode4Write(foo.toString()).asDirectory();
+    long fooId = fooNode.getId();
+    
+    try {
+      hdfs.rename(foo, bar, Rename.OVERWRITE);
+      fail("Expect exception since " + bar
+          + " is snapshottable and already has snapshots");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains(bar.toString()
+          + " is snapshottable and already has snapshots", e);
+    }
+    
+    hdfs.deleteSnapshot(bar, snap1);
+    hdfs.rename(foo, bar, Rename.OVERWRITE);
+    SnapshottableDirectoryStatus[] dirs = fsn.getSnapshottableDirListing();
+    assertEquals(1, dirs.length);
+    assertEquals(bar, dirs[0].getFullPath());
+    assertEquals(fooId, dirs[0].getDirStatus().getFileId());
+  }
+  
+  /**
+   * After rename, delete the snapshot in src
+   */
+  @Test
+  public void testRenameDirAndDeleteSnapshot_2() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    hdfs.mkdirs(sdir1);
+    hdfs.mkdirs(sdir2);
+    final Path foo = new Path(sdir2, "foo");
+    final Path bar = new Path(foo, "bar");
+    DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s3");
+    
+    final Path newfoo = new Path(sdir1, "foo");
+    hdfs.rename(foo, newfoo);
+    
+    // restart the cluster and check fsimage
+    restartClusterAndCheckImage();
+    
+    final Path bar2 = new Path(newfoo, "bar2");
+    DFSTestUtil.createFile(hdfs, bar2, BLOCKSIZE, REPL, SEED);
+    
+    hdfs.createSnapshot(sdir1, "s4");
+    hdfs.delete(newfoo, true);
+    
+    final Path bar2_s4 = SnapshotTestHelper.getSnapshotPath(sdir1, "s4",
+        "foo/bar2");
+    assertTrue(hdfs.exists(bar2_s4));
+    final Path bar_s4 = SnapshotTestHelper.getSnapshotPath(sdir1, "s4",
+        "foo/bar");
+    assertTrue(hdfs.exists(bar_s4));
+        
+    // delete snapshot s4. The diff of s4 should be combined to s3
+    hdfs.deleteSnapshot(sdir1, "s4");
+    // restart the cluster and check fsimage
+    restartClusterAndCheckImage();
+    
+    Path bar_s3 = SnapshotTestHelper.getSnapshotPath(sdir1, "s3", "foo/bar");
+    assertFalse(hdfs.exists(bar_s3));
+    bar_s3 = SnapshotTestHelper.getSnapshotPath(sdir2, "s3", "foo/bar");
+    assertTrue(hdfs.exists(bar_s3));
+    Path bar2_s3 = SnapshotTestHelper.getSnapshotPath(sdir1, "s3", "foo/bar2");
+    assertFalse(hdfs.exists(bar2_s3));
+    bar2_s3 = SnapshotTestHelper.getSnapshotPath(sdir2, "s3", "foo/bar2");
+    assertFalse(hdfs.exists(bar2_s3));
+    
+    // delete snapshot s3
+    hdfs.deleteSnapshot(sdir2, "s3");
+    final Path bar_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2",
+        "foo/bar");
+    assertTrue(hdfs.exists(bar_s2));
+    
+    // check internal details
+    final Path foo_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2", "foo");
+    INodeReference fooRef = fsdir.getINode(foo_s2.toString()).asReference();
+    assertTrue(fooRef instanceof INodeReference.WithName);
+    INodeReference.WithCount fooWC = (WithCount) fooRef.getReferredINode();
+    assertEquals(1, fooWC.getReferenceCount());
+    INodeDirectoryWithSnapshot fooDir = (INodeDirectoryWithSnapshot) fooWC
+        .getReferredINode().asDirectory();
+    List<DirectoryDiff> diffs = fooDir.getDiffs().asList();
+    assertEquals(1, diffs.size());
+    assertEquals("s2", diffs.get(0).snapshot.getRoot().getLocalName());
+    
+    // restart the cluster and check fsimage
+    restartClusterAndCheckImage();
+    
+    // delete snapshot s2.
+    hdfs.deleteSnapshot(sdir2, "s2");
+    assertFalse(hdfs.exists(bar_s2));
+    restartClusterAndCheckImage();
+    // make sure the whole referred subtree has been destroyed
+    assertEquals(4, fsdir.getRoot().getNamespace());
+    assertEquals(0, fsdir.getRoot().getDiskspace());
+    
+    hdfs.deleteSnapshot(sdir1, "s1");
+    restartClusterAndCheckImage();
+    assertEquals(3, fsdir.getRoot().getNamespace());
+    assertEquals(0, fsdir.getRoot().getDiskspace());
+  }
+  
+  /**
+   * Rename a file and then append the same file. 
+   */
+  @Test
+  public void testRenameAndAppend() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    hdfs.mkdirs(sdir1);
+    hdfs.mkdirs(sdir2);
+    
+    final Path foo = new Path(sdir1, "foo");
+    DFSTestUtil.createFile(hdfs, foo, BLOCKSIZE, REPL, SEED);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, snap1);
+
+    final Path foo2 = new Path(sdir2, "foo");
+    hdfs.rename(foo, foo2);
+    
+    INode fooRef = fsdir.getINode4Write(foo2.toString());
+    assertTrue(fooRef instanceof INodeReference.DstReference);
+    
+    FSDataOutputStream out = hdfs.append(foo2);
+    try {
+      byte[] content = new byte[1024];
+      (new Random()).nextBytes(content);
+      out.write(content);
+      fooRef = fsdir.getINode4Write(foo2.toString());
+      assertTrue(fooRef instanceof INodeReference.DstReference);
+      INode fooNode = fooRef.asFile();
+      assertTrue(fooNode instanceof INodeFileUnderConstructionWithSnapshot);
+    } finally {
+      if (out != null) {
+        out.close();
+      }
+    }
+    
+    fooRef = fsdir.getINode4Write(foo2.toString());
+    assertTrue(fooRef instanceof INodeReference.DstReference);
+    INode fooNode = fooRef.asFile();
+    assertTrue(fooNode instanceof INodeFileWithSnapshot);
+    
+    restartClusterAndCheckImage();
+  }
+  
+  /**
+   * Test the undo section of rename. Before the rename, we create the renamed 
+   * file/dir before taking the snapshot.
+   */
+  @Test
+  public void testRenameUndo_1() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    hdfs.mkdirs(sdir1);
+    hdfs.mkdirs(sdir2);
+    final Path foo = new Path(sdir1, "foo");
+    final Path bar = new Path(foo, "bar");
+    DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
+    final Path dir2file = new Path(sdir2, "file");
+    DFSTestUtil.createFile(hdfs, dir2file, BLOCKSIZE, REPL, SEED);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
+    
+    INodeDirectory dir2 = fsdir.getINode4Write(sdir2.toString()).asDirectory();
+    INodeDirectory mockDir2 = spy(dir2);
+    doReturn(false).when(mockDir2).addChild((INode) anyObject(), anyBoolean(),
+            (Snapshot) anyObject(), (INodeMap) anyObject());
+    INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
+    root.replaceChild(dir2, mockDir2, fsdir.getINodeMap());
+    
+    final Path newfoo = new Path(sdir2, "foo");
+    boolean result = hdfs.rename(foo, newfoo);
+    assertFalse(result);
+    
+    // check the current internal details
+    INodeDirectorySnapshottable dir1Node = (INodeDirectorySnapshottable) fsdir
+        .getINode4Write(sdir1.toString());
+    ReadOnlyList<INode> dir1Children = dir1Node.getChildrenList(null);
+    assertEquals(1, dir1Children.size());
+    assertEquals(foo.getName(), dir1Children.get(0).getLocalName());
+    List<DirectoryDiff> dir1Diffs = dir1Node.getDiffs().asList();
+    assertEquals(1, dir1Diffs.size());
+    assertEquals("s1", dir1Diffs.get(0).snapshot.getRoot().getLocalName());
+    
+    // after the undo of rename, both the created and deleted list of sdir1
+    // should be empty
+    ChildrenDiff childrenDiff = dir1Diffs.get(0).getChildrenDiff();
+    assertEquals(0, childrenDiff.getList(ListType.DELETED).size());
+    assertEquals(0, childrenDiff.getList(ListType.CREATED).size());
+    
+    INode fooNode = fsdir.getINode4Write(foo.toString());
+    assertTrue(fooNode instanceof INodeDirectoryWithSnapshot);
+    List<DirectoryDiff> fooDiffs = ((INodeDirectoryWithSnapshot) fooNode)
+        .getDiffs().asList();
+    assertEquals(1, fooDiffs.size());
+    assertEquals("s1", fooDiffs.get(0).snapshot.getRoot().getLocalName());
+    
+    final Path foo_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1", "foo");
+    INode fooNode_s1 = fsdir.getINode(foo_s1.toString());
+    assertTrue(fooNode_s1 == fooNode);
+    
+    // check sdir2
+    assertFalse(hdfs.exists(newfoo));
+    INodeDirectory dir2Node = fsdir.getINode4Write(sdir2.toString())
+        .asDirectory();
+    assertFalse(dir2Node instanceof INodeDirectoryWithSnapshot);
+    ReadOnlyList<INode> dir2Children = dir2Node.getChildrenList(null);
+    assertEquals(1, dir2Children.size());
+    assertEquals(dir2file.getName(), dir2Children.get(0).getLocalName());
+  }
+
+  /**
+   * Test the undo section of rename. Before the rename, we create the renamed 
+   * file/dir after taking the snapshot.
+   */
+  @Test
+  public void testRenameUndo_2() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    hdfs.mkdirs(sdir1);
+    hdfs.mkdirs(sdir2);
+    final Path dir2file = new Path(sdir2, "file");
+    DFSTestUtil.createFile(hdfs, dir2file, BLOCKSIZE, REPL, SEED);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
+    
+    // create foo after taking snapshot
+    final Path foo = new Path(sdir1, "foo");
+    final Path bar = new Path(foo, "bar");
+    DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
+    
+    INodeDirectory dir2 = fsdir.getINode4Write(sdir2.toString()).asDirectory();
+    INodeDirectory mockDir2 = spy(dir2);
+    doReturn(false).when(mockDir2).addChild((INode) anyObject(), anyBoolean(),
+            (Snapshot) anyObject(), (INodeMap) anyObject());
+    INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
+    root.replaceChild(dir2, mockDir2, fsdir.getINodeMap());
+    
+    final Path newfoo = new Path(sdir2, "foo");
+    boolean result = hdfs.rename(foo, newfoo);
+    assertFalse(result);
+    
+    // check the current internal details
+    INodeDirectorySnapshottable dir1Node = (INodeDirectorySnapshottable) fsdir
+        .getINode4Write(sdir1.toString());
+    ReadOnlyList<INode> dir1Children = dir1Node.getChildrenList(null);
+    assertEquals(1, dir1Children.size());
+    assertEquals(foo.getName(), dir1Children.get(0).getLocalName());
+    List<DirectoryDiff> dir1Diffs = dir1Node.getDiffs().asList();
+    assertEquals(1, dir1Diffs.size());
+    assertEquals("s1", dir1Diffs.get(0).snapshot.getRoot().getLocalName());
+    
+    // after the undo of rename, the created list of sdir1 should contain 
+    // 1 element
+    ChildrenDiff childrenDiff = dir1Diffs.get(0).getChildrenDiff();
+    assertEquals(0, childrenDiff.getList(ListType.DELETED).size());
+    assertEquals(1, childrenDiff.getList(ListType.CREATED).size());
+    
+    INode fooNode = fsdir.getINode4Write(foo.toString());
+    assertTrue(fooNode instanceof INodeDirectory);
+    assertTrue(childrenDiff.getList(ListType.CREATED).get(0) == fooNode);
+    
+    final Path foo_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1", "foo");
+    assertFalse(hdfs.exists(foo_s1));
+    
+    // check sdir2
+    assertFalse(hdfs.exists(newfoo));
+    INodeDirectory dir2Node = fsdir.getINode4Write(sdir2.toString())
+        .asDirectory();
+    assertFalse(dir2Node instanceof INodeDirectoryWithSnapshot);
+    ReadOnlyList<INode> dir2Children = dir2Node.getChildrenList(null);
+    assertEquals(1, dir2Children.size());
+    assertEquals(dir2file.getName(), dir2Children.get(0).getLocalName());
+  }
+  
+  /**
+   * Test the undo section of the second-time rename.
+   */
+  @Test
+  public void testRenameUndo_3() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    final Path sdir3 = new Path("/dir3");
+    hdfs.mkdirs(sdir1);
+    hdfs.mkdirs(sdir2);
+    hdfs.mkdirs(sdir3);
+    final Path foo = new Path(sdir1, "foo");
+    final Path bar = new Path(foo, "bar");
+    DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
+    
+    INodeDirectory dir3 = fsdir.getINode4Write(sdir3.toString()).asDirectory();
+    INodeDirectory mockDir3 = spy(dir3);
+    doReturn(false).when(mockDir3).addChild((INode) anyObject(), anyBoolean(),
+            (Snapshot) anyObject(), (INodeMap) anyObject());
+    INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
+    root.replaceChild(dir3, mockDir3, fsdir.getINodeMap());
+    
+    final Path foo_dir2 = new Path(sdir2, "foo");
+    final Path foo_dir3 = new Path(sdir3, "foo");
+    hdfs.rename(foo, foo_dir2);
+    boolean result = hdfs.rename(foo_dir2, foo_dir3);
+    assertFalse(result);
+    
+    // check the current internal details
+    INodeDirectorySnapshottable dir2Node = (INodeDirectorySnapshottable) fsdir
+        .getINode4Write(sdir2.toString());
+    ReadOnlyList<INode> dir2Children = dir2Node.getChildrenList(null);
+    assertEquals(1, dir2Children.size());
+    List<DirectoryDiff> dir2Diffs = dir2Node.getDiffs().asList();
+    assertEquals(1, dir2Diffs.size());
+    assertEquals("s2", Snapshot.getSnapshotName(dir2Diffs.get(0).snapshot));
+    ChildrenDiff childrenDiff = dir2Diffs.get(0).getChildrenDiff();
+    assertEquals(0, childrenDiff.getList(ListType.DELETED).size());
+    assertEquals(1, childrenDiff.getList(ListType.CREATED).size());
+    final Path foo_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2", "foo");
+    assertFalse(hdfs.exists(foo_s2));
+    
+    INode fooNode = fsdir.getINode4Write(foo_dir2.toString());
+    assertTrue(childrenDiff.getList(ListType.CREATED).get(0) == fooNode);
+    assertTrue(fooNode instanceof INodeReference.DstReference);
+    List<DirectoryDiff> fooDiffs = ((INodeDirectoryWithSnapshot) fooNode
+        .asDirectory()).getDiffs().asList();
+    assertEquals(1, fooDiffs.size());
+    assertEquals("s1", fooDiffs.get(0).snapshot.getRoot().getLocalName());
+    
+    // create snapshot on sdir2 and rename again
+    hdfs.createSnapshot(sdir2, "s3");
+    result = hdfs.rename(foo_dir2, foo_dir3);
+    assertFalse(result);
+
+    // check internal details again
+    dir2Node = (INodeDirectorySnapshottable) fsdir.getINode4Write(sdir2
+        .toString());
+    fooNode = fsdir.getINode4Write(foo_dir2.toString());
+    dir2Children = dir2Node.getChildrenList(null);
+    assertEquals(1, dir2Children.size());
+    dir2Diffs = dir2Node.getDiffs().asList();
+    assertEquals(2, dir2Diffs.size());
+    assertEquals("s2", Snapshot.getSnapshotName(dir2Diffs.get(0).snapshot));
+    assertEquals("s3", Snapshot.getSnapshotName(dir2Diffs.get(1).snapshot));
+    
+    childrenDiff = dir2Diffs.get(0).getChildrenDiff();
+    assertEquals(0, childrenDiff.getList(ListType.DELETED).size());
+    assertEquals(1, childrenDiff.getList(ListType.CREATED).size());
+    assertTrue(childrenDiff.getList(ListType.CREATED).get(0) == fooNode);
+    
+    childrenDiff = dir2Diffs.get(1).getChildrenDiff();
+    assertEquals(0, childrenDiff.getList(ListType.DELETED).size());
+    assertEquals(0, childrenDiff.getList(ListType.CREATED).size());
+    
+    final Path foo_s3 = SnapshotTestHelper.getSnapshotPath(sdir2, "s3", "foo");
+    assertFalse(hdfs.exists(foo_s2));
+    assertTrue(hdfs.exists(foo_s3));
+    
+    assertTrue(fooNode instanceof INodeReference.DstReference);
+    fooDiffs = ((INodeDirectoryWithSnapshot) fooNode.asDirectory()).getDiffs()
+        .asList();
+    assertEquals(2, fooDiffs.size());
+    assertEquals("s1", fooDiffs.get(0).snapshot.getRoot().getLocalName());
+    assertEquals("s3", fooDiffs.get(1).snapshot.getRoot().getLocalName());
+  }
+  
+  /**
+   * Test undo where dst node being overwritten is a reference node
+   */
+  @Test
+  public void testRenameUndo_4() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    final Path sdir3 = new Path("/dir3");
+    hdfs.mkdirs(sdir1);
+    hdfs.mkdirs(sdir2);
+    hdfs.mkdirs(sdir3);
+    
+    final Path foo = new Path(sdir1, "foo");
+    final Path bar = new Path(foo, "bar");
+    DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
+    
+    final Path foo2 = new Path(sdir2, "foo");
+    hdfs.mkdirs(foo2);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
+    
+    // rename foo2 to foo3, so that foo3 will be a reference node
+    final Path foo3 = new Path(sdir3, "foo");
+    hdfs.rename(foo2, foo3);
+    
+    INode foo3Node = fsdir.getINode4Write(foo3.toString());
+    assertTrue(foo3Node.isReference());
+    
+    INodeDirectory dir3 = fsdir.getINode4Write(sdir3.toString()).asDirectory();
+    INodeDirectory mockDir3 = spy(dir3);
+    // fail the rename but succeed in undo
+    doReturn(false).when(mockDir3).addChild((INode) Mockito.isNull(),
+        anyBoolean(), (Snapshot) anyObject(), (INodeMap) anyObject());
+    Mockito.when(mockDir3.addChild((INode) Mockito.isNotNull(), 
+        anyBoolean(), (Snapshot) anyObject(), 
+        (INodeMap) anyObject())).thenReturn(false).thenCallRealMethod();
+    INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
+    root.replaceChild(dir3, mockDir3, fsdir.getINodeMap());
+    foo3Node.setParent(mockDir3);
+    
+    try {
+      hdfs.rename(foo, foo3, Rename.OVERWRITE);
+      fail("the rename from " + foo + " to " + foo3 + " should fail");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("rename from " + foo + " to "
+          + foo3 + " failed.", e);
+    }
+    
+    // make sure the undo is correct
+    final INode foo3Node_undo = fsdir.getINode4Write(foo3.toString());
+    assertSame(foo3Node, foo3Node_undo);
+    INodeReference.WithCount foo3_wc = (WithCount) foo3Node.asReference()
+        .getReferredINode();
+    assertEquals(2, foo3_wc.getReferenceCount());
+    assertSame(foo3Node, foo3_wc.getParentReference());
+  }
+  
+  /**
+   * Test rename while the rename operation will exceed the quota in the dst
+   * tree.
+   */
+  @Test
+  public void testRenameUndo_5() throws Exception {
+    final Path test = new Path("/test");
+    final Path dir1 = new Path(test, "dir1");
+    final Path dir2 = new Path(test, "dir2");
+    final Path subdir2 = new Path(dir2, "subdir2");
+    hdfs.mkdirs(dir1);
+    hdfs.mkdirs(subdir2);
+    
+    final Path foo = new Path(dir1, "foo");
+    final Path bar = new Path(foo, "bar");
+    DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, dir2, "s2");
+    
+    // set ns quota of dir2 to 5, so the current remaining is 2 (already has
+    // dir2, subdir2, and s2)
+    hdfs.setQuota(dir2, 5, Long.MAX_VALUE - 1);
+    
+    final Path foo2 = new Path(subdir2, foo.getName());
+    // rename /test/dir1/foo to /test/dir2/subdir2/foo. 
+    // FSDirectory#verifyQuota4Rename will pass since foo/bar only be counted 
+    // as 2 in NS quota. However, the rename operation will fail when adding
+    // foo to subdir2, since we will create a snapshot diff for subdir2. 
+    boolean rename = hdfs.rename(foo, foo2);
+    assertFalse(rename);
+    
+    // check the undo
+    assertTrue(hdfs.exists(foo));
+    assertTrue(hdfs.exists(bar));
+    INodeDirectory dir1Node = fsdir.getINode4Write(dir1.toString())
+        .asDirectory();
+    List<INode> childrenList = ReadOnlyList.Util.asList(dir1Node
+        .getChildrenList(null));
+    assertEquals(1, childrenList.size());
+    INode fooNode = childrenList.get(0);
+    assertTrue(fooNode.getClass() == INodeDirectoryWithSnapshot.class);
+    INode barNode = fsdir.getINode4Write(bar.toString());
+    assertTrue(barNode.getClass() == INodeFile.class);
+    assertSame(fooNode, barNode.getParent());
+    List<DirectoryDiff> diffList = ((INodeDirectorySnapshottable) dir1Node)
+        .getDiffs().asList();
+    assertEquals(1, diffList.size());
+    DirectoryDiff diff = diffList.get(0);
+    assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
+    assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
+    
+    // check dir2
+    INode dir2Node = fsdir.getINode4Write(dir2.toString());
+    assertTrue(dir2Node.getClass() == INodeDirectorySnapshottable.class);
+    Quota.Counts counts = dir2Node.computeQuotaUsage();
+    assertEquals(3, counts.get(Quota.NAMESPACE));
+    assertEquals(0, counts.get(Quota.DISKSPACE));
+    childrenList = ReadOnlyList.Util.asList(dir2Node.asDirectory()
+        .getChildrenList(null));
+    assertEquals(1, childrenList.size());
+    INode subdir2Node = childrenList.get(0);
+    assertSame(dir2Node, subdir2Node.getParent());
+    assertSame(subdir2Node, fsdir.getINode4Write(subdir2.toString()));
+    diffList = ((INodeDirectorySnapshottable) dir2Node)
+        .getDiffs().asList();
+    assertEquals(1, diffList.size());
+    diff = diffList.get(0);
+    assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
+    assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
+  }
+  
+  /**
+   * Test the rename undo when removing dst node fails
+   */
+  @Test
+  public void testRenameUndo_6() throws Exception {
+    final Path test = new Path("/test");
+    final Path dir1 = new Path(test, "dir1");
+    final Path dir2 = new Path(test, "dir2");
+    final Path sub_dir2 = new Path(dir2, "subdir");
+    final Path subsub_dir2 = new Path(sub_dir2, "subdir");
+    hdfs.mkdirs(dir1);
+    hdfs.mkdirs(subsub_dir2);
+    
+    final Path foo = new Path(dir1, "foo");
+    hdfs.mkdirs(foo);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, dir2, "s2");
+    
+    // set ns quota of dir2 to 4, so the current remaining is 0 (already has
+    // dir2, sub_dir2, subsub_dir2, and s2)
+    hdfs.setQuota(dir2, 4, Long.MAX_VALUE - 1);
+    
+    // rename /test/dir1/foo to /test/dir2/sub_dir2/subsub_dir2. 
+    // FSDirectory#verifyQuota4Rename will pass since foo only be counted 
+    // as 1 in NS quota. However, the rename operation will fail when removing
+    // subsub_dir2 since this step tries to add a snapshot diff in sub_dir2.
+    try {
+      hdfs.rename(foo, subsub_dir2, Rename.OVERWRITE);
+      fail("Expect QuotaExceedException");
+    } catch (QuotaExceededException e) {
+      String msg = "Failed to record modification for snapshot: "
+          + "The NameSpace quota (directories and files)"
+          + " is exceeded: quota=4 file count=5"; 
+      GenericTestUtils.assertExceptionContains(msg, e);
+    }
+    
+    // check the undo
+    assertTrue(hdfs.exists(foo));
+    INodeDirectory dir1Node = fsdir.getINode4Write(dir1.toString())
+        .asDirectory();
+    List<INode> childrenList = ReadOnlyList.Util.asList(dir1Node
+        .getChildrenList(null));
+    assertEquals(1, childrenList.size());
+    INode fooNode = childrenList.get(0);
+    assertTrue(fooNode.getClass() == INodeDirectoryWithSnapshot.class);
+    assertSame(dir1Node, fooNode.getParent());
+    List<DirectoryDiff> diffList = ((INodeDirectorySnapshottable) dir1Node)
+        .getDiffs().asList();
+    assertEquals(1, diffList.size());
+    DirectoryDiff diff = diffList.get(0);
+    assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
+    assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
+    
+    // check dir2
+    INode dir2Node = fsdir.getINode4Write(dir2.toString());
+    assertTrue(dir2Node.getClass() == INodeDirectorySnapshottable.class);
+    Quota.Counts counts = dir2Node.computeQuotaUsage();
+    assertEquals(4, counts.get(Quota.NAMESPACE));
+    assertEquals(0, counts.get(Quota.DISKSPACE));
+    childrenList = ReadOnlyList.Util.asList(dir2Node.asDirectory()
+        .getChildrenList(null));
+    assertEquals(1, childrenList.size());
+    INode subdir2Node = childrenList.get(0);
+    assertTrue(subdir2Node.getClass() == INodeDirectoryWithSnapshot.class);
+    assertSame(dir2Node, subdir2Node.getParent());
+    assertSame(subdir2Node, fsdir.getINode4Write(sub_dir2.toString()));
+    INode subsubdir2Node = fsdir.getINode4Write(subsub_dir2.toString());
+    assertTrue(subsubdir2Node.getClass() == INodeDirectory.class);
+    assertSame(subdir2Node, subsubdir2Node.getParent());
+    
+    diffList = ((INodeDirectorySnapshottable) dir2Node).getDiffs().asList();
+    assertEquals(1, diffList.size());
+    diff = diffList.get(0);
+    assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
+    assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
+    
+    diffList = ((INodeDirectoryWithSnapshot) subdir2Node).getDiffs().asList();
+    assertEquals(0, diffList.size());
+  }
+  
+  /**
+   * Test the rename undo when quota of dst tree is exceeded after rename.
+   */
+  @Test
+  public void testRenameExceedQuota() throws Exception {
+    final Path test = new Path("/test");
+    final Path dir1 = new Path(test, "dir1");
+    final Path dir2 = new Path(test, "dir2");
+    final Path sub_dir2 = new Path(dir2, "subdir");
+    final Path subfile_dir2 = new Path(sub_dir2, "subfile");
+    hdfs.mkdirs(dir1);
+    DFSTestUtil.createFile(hdfs, subfile_dir2, BLOCKSIZE, REPL, SEED);
+    
+    final Path foo = new Path(dir1, "foo");
+    DFSTestUtil.createFile(hdfs, foo, BLOCKSIZE, REPL, SEED);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, dir2, "s2");
+    
+    // set ns quota of dir2 to 4, so the current remaining is 1 (already has
+    // dir2, sub_dir2, subfile_dir2, and s2)
+    hdfs.setQuota(dir2, 5, Long.MAX_VALUE - 1);
+    
+    // rename /test/dir1/foo to /test/dir2/sub_dir2/subfile_dir2. 
+    // FSDirectory#verifyQuota4Rename will pass since foo only be counted 
+    // as 1 in NS quota. The rename operation will succeed while the real quota 
+    // of dir2 will become 7 (dir2, s2 in dir2, sub_dir2, s2 in sub_dir2,
+    // subfile_dir2 in deleted list, new subfile, s1 in new subfile).
+    hdfs.rename(foo, subfile_dir2, Rename.OVERWRITE);
+    
+    // check dir2
+    INode dir2Node = fsdir.getINode4Write(dir2.toString());
+    assertTrue(dir2Node.getClass() == INodeDirectorySnapshottable.class);
+    Quota.Counts counts = dir2Node.computeQuotaUsage();
+    assertEquals(7, counts.get(Quota.NAMESPACE));
+    assertEquals(BLOCKSIZE * REPL * 2, counts.get(Quota.DISKSPACE));
+  }
+  
+  @Test
+  public void testRename2PreDescendant() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    final Path foo = new Path(sdir1, "foo");
+    final Path bar = new Path(foo, "bar");
+    hdfs.mkdirs(bar);
+    hdfs.mkdirs(sdir2);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, snap1);
+    
+    // /dir1/foo/bar -> /dir2/bar
+    final Path bar2 = new Path(sdir2, "bar");
+    hdfs.rename(bar, bar2);
+    
+    // /dir1/foo -> /dir2/bar/foo
+    final Path foo2 = new Path(bar2, "foo");
+    hdfs.rename(foo, foo2);
+    
+    restartClusterAndCheckImage();
+  }
+  
+  /**
+   * After the following operations:
+   * Rename a dir -> create a snapshot s on dst tree -> delete the renamed dir
+   * -> delete snapshot s on dst tree
+   * 
+   * Make sure we destroy everything created after the rename under the renamed
+   * dir.
+   */
+  @Test
+  public void testRenameDirAndDeleteSnapshot_3() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    final Path foo = new Path(sdir1, "foo");
+    final Path bar = new Path(foo, "bar");
+    DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
+    hdfs.mkdirs(sdir2);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
+    
+    final Path foo2 = new Path(sdir2, "foo");
+    hdfs.rename(foo, foo2);
+    
+    // create two new files under foo2
+    final Path bar2 = new Path(foo2, "bar2");
+    DFSTestUtil.createFile(hdfs, bar2, BLOCKSIZE, REPL, SEED);
+    final Path bar3 = new Path(foo2, "bar3");
+    DFSTestUtil.createFile(hdfs, bar3, BLOCKSIZE, REPL, SEED);
+    
+    // create a new snapshot on sdir2
+    hdfs.createSnapshot(sdir2, "s3");
+    
+    // delete foo2
+    hdfs.delete(foo2, true);
+    // delete s3
+    hdfs.deleteSnapshot(sdir2, "s3");
+    
+    // check
+    final INodeDirectorySnapshottable dir1Node = 
+        (INodeDirectorySnapshottable) fsdir.getINode4Write(sdir1.toString());
+    assertEquals(4, dir1Node.getNamespace());
+    final INodeDirectorySnapshottable dir2Node = 
+        (INodeDirectorySnapshottable) fsdir.getINode4Write(sdir2.toString());
+    assertEquals(2, dir2Node.getNamespace());
+    
+    final Path foo_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1",
+        foo.getName());
+    INode fooRef = fsdir.getINode(foo_s1.toString());
+    assertTrue(fooRef instanceof INodeReference.WithName);
+    INodeReference.WithCount wc = 
+        (WithCount) fooRef.asReference().getReferredINode();
+    assertEquals(1, wc.getReferenceCount());
+    INodeDirectoryWithSnapshot fooNode = 
+        (INodeDirectoryWithSnapshot) wc.getReferredINode().asDirectory();
+    ReadOnlyList<INode> children = fooNode.getChildrenList(null);
+    assertEquals(1, children.size());
+    assertEquals(bar.getName(), children.get(0).getLocalName());
+    List<DirectoryDiff> diffList = fooNode.getDiffs().asList();
+    assertEquals(1, diffList.size());
+    assertEquals("s1", Snapshot.getSnapshotName(diffList.get(0).snapshot));
+    ChildrenDiff diff = diffList.get(0).getChildrenDiff();
+    assertEquals(0, diff.getList(ListType.CREATED).size());
+    assertEquals(0, diff.getList(ListType.DELETED).size());
+    
+    restartClusterAndCheckImage();
+  }
+  
+  /**
+   * After the following operations:
+   * Rename a dir -> create a snapshot s on dst tree -> rename the renamed dir
+   * again -> delete snapshot s on dst tree
+   * 
+   * Make sure we only delete the snapshot s under the renamed dir.
+   */
+  @Test
+  public void testRenameDirAndDeleteSnapshot_4() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    final Path foo = new Path(sdir1, "foo");
+    final Path bar = new Path(foo, "bar");
+    DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
+    hdfs.mkdirs(sdir2);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
+    SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
+    
+    final Path foo2 = new Path(sdir2, "foo");
+    hdfs.rename(foo, foo2);
+    
+    // create two new files under foo2
+    final Path bar2 = new Path(foo2, "bar2");
+    DFSTestUtil.createFile(hdfs, bar2, BLOCKSIZE, REPL, SEED);
+    final Path bar3 = new Path(foo2, "bar3");
+    DFSTestUtil.createFile(hdfs, bar3, BLOCKSIZE, REPL, SEED);
+    
+    // create a new snapshot on sdir2
+    hdfs.createSnapshot(sdir2, "s3");
+    
+    // rename foo2 again
+    hdfs.rename(foo2, foo);
+    // delete snapshot s3
+    hdfs.deleteSnapshot(sdir2, "s3");
+    
+    // check
+    final INodeDirectorySnapshottable dir1Node = 
+        (INodeDirectorySnapshottable) fsdir.getINode4Write(sdir1.toString());
+    // sdir1 + s1 + foo_s1 (foo) + foo (foo + s1 + bar~bar3)
+    assertEquals(9, dir1Node.getNamespace());
+    final INodeDirectorySnapshottable dir2Node = 
+        (INodeDirectorySnapshottable) fsdir.getINode4Write(sdir2.toString());
+    assertEquals(2, dir2Node.getNamespace());
+    
+    final Path foo_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1",
+        foo.getName());
+    final INode fooRef = fsdir.getINode(foo_s1.toString());
+    assertTrue(fooRef instanceof INodeReference.WithName);
+    INodeReference.WithCount wc = 
+        (WithCount) fooRef.asReference().getReferredINode();
+    assertEquals(2, wc.getReferenceCount());
+    INodeDirectoryWithSnapshot fooNode = 
+        (INodeDirectoryWithSnapshot) wc.getReferredINode().asDirectory();
+    ReadOnlyList<INode> children = fooNode.getChildrenList(null);
+    assertEquals(3, children.size());
+    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();
+    assertEquals(1, diffList.size());
+    assertEquals("s1", Snapshot.getSnapshotName(diffList.get(0).snapshot));
+    ChildrenDiff diff = diffList.get(0).getChildrenDiff();
+    // bar2 and bar3 in the created list
+    assertEquals(2, diff.getList(ListType.CREATED).size());
+    assertEquals(0, diff.getList(ListType.DELETED).size());
+    
+    final INode fooRef2 = fsdir.getINode4Write(foo.toString());
+    assertTrue(fooRef2 instanceof INodeReference.DstReference);
+    INodeReference.WithCount wc2 = 
+        (WithCount) fooRef2.asReference().getReferredINode();
+    assertSame(wc, wc2);
+    assertSame(fooRef2, wc.getParentReference());
+    
+    restartClusterAndCheckImage();
+  }
+  
+  /**
+   * This test demonstrates that 
+   * {@link INodeDirectoryWithSnapshot#removeChild(INode, Snapshot, INodeMap)}
+   * and 
+   * {@link INodeDirectoryWithSnapshot#addChild(INode, boolean, Snapshot, INodeMap)}
+   * should use {@link INode#isInLatestSnapshot(Snapshot)} to check if the 
+   * added/removed child should be recorded in snapshots.
+   */
+  @Test
+  public void testRenameDirAndDeleteSnapshot_5() throws Exception {
+    final Path dir1 = new Path("/dir1");
+    final Path dir2 = new Path("/dir2");
+    final Path dir3 = new Path("/dir3");
+    hdfs.mkdirs(dir1);
+    hdfs.mkdirs(dir2);
+    hdfs.mkdirs(dir3);
+    
+    final Path foo = new Path(dir1, "foo");
+    hdfs.mkdirs(foo);
+    SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
+    final Path bar = new Path(foo, "bar");
+    // create file bar, and foo will become an INodeDirectoryWithSnapshot
+    DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
+    // delete snapshot s1. now foo is not in any snapshot
+    hdfs.deleteSnapshot(dir1, "s1");
+    
+    SnapshotTestHelper.createSnapshot(hdfs, dir2, "s2");
+    // rename /dir1/foo to /dir2/foo
+    final Path foo2 = new Path(dir2, foo.getName());
+    hdfs.rename(foo, foo2);
+    // rename /dir2/foo/bar to /dir3/foo/bar
+    final Path bar2 = new Path(dir2, "foo/bar");
+    final Path bar3 = new Path(dir3, "bar");
+    hdfs.rename(bar2, bar3);
+    
+    // delete /dir2/foo. Since it is not in any snapshot, we will call its 
+    // destroy function. If we do not use isInLatestSnapshot in removeChild and
+    // addChild methods in INodeDirectoryWithSnapshot, the file bar will be 
+    // stored in the deleted list of foo, and will be destroyed.
+    hdfs.delete(foo2, true);
+    
+    // check if /dir3/bar still exists
+    assertTrue(hdfs.exists(bar3));
+    INodeFile barNode = (INodeFile) fsdir.getINode4Write(bar3.toString());
+    assertSame(fsdir.getINode4Write(dir3.toString()), barNode.getParent());
+  }
+}

+ 161 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSetQuotaWithSnapshot.java

@@ -0,0 +1,161 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
+import org.apache.hadoop.hdfs.util.Diff.ListType;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class TestSetQuotaWithSnapshot {
+  protected static final long seed = 0;
+  protected static final short REPLICATION = 3;
+  protected static final long BLOCKSIZE = 1024;
+  
+  protected Configuration conf;
+  protected MiniDFSCluster cluster;
+  protected FSNamesystem fsn;
+  protected FSDirectory fsdir;
+  protected DistributedFileSystem hdfs;
+  
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
+        .format(true).build();
+    cluster.waitActive();
+
+    fsn = cluster.getNamesystem();
+    fsdir = fsn.getFSDirectory();
+    hdfs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+  
+  @Test (timeout=60000)
+  public void testSetQuota() throws Exception {
+    final Path dir = new Path("/TestSnapshot");
+    hdfs.mkdirs(dir);
+    // allow snapshot on dir and create snapshot s1
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s1");
+    
+    Path sub = new Path(dir, "sub");
+    hdfs.mkdirs(sub);
+    Path fileInSub = new Path(sub, "file");
+    DFSTestUtil.createFile(hdfs, fileInSub, BLOCKSIZE, REPLICATION, seed);
+    INodeDirectory subNode = INodeDirectory.valueOf(
+        fsdir.getINode(sub.toString()), sub);
+    // subNode should be a INodeDirectory, but not an INodeDirectoryWithSnapshot
+    assertFalse(subNode instanceof INodeDirectoryWithSnapshot);
+    
+    hdfs.setQuota(sub, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
+    subNode = INodeDirectory.valueOf(fsdir.getINode(sub.toString()), sub);
+    assertTrue(subNode.isQuotaSet());
+    assertFalse(subNode instanceof INodeDirectoryWithSnapshot);
+  }
+  
+  /**
+   * Test clear quota of a snapshottable dir or a dir with snapshot.
+   */
+  @Test
+  public void testClearQuota() throws Exception {
+    final Path dir = new Path("/TestSnapshot");
+    hdfs.mkdirs(dir);
+    
+    hdfs.allowSnapshot(dir);
+    hdfs.setQuota(dir, HdfsConstants.QUOTA_DONT_SET,
+        HdfsConstants.QUOTA_DONT_SET);
+    INode dirNode = fsdir.getINode4Write(dir.toString());
+    assertTrue(dirNode instanceof INodeDirectorySnapshottable);
+    assertEquals(0, ((INodeDirectorySnapshottable) dirNode).getDiffs().asList()
+        .size());
+    
+    hdfs.setQuota(dir, HdfsConstants.QUOTA_DONT_SET - 1,
+        HdfsConstants.QUOTA_DONT_SET - 1);
+    dirNode = fsdir.getINode4Write(dir.toString());
+    assertTrue(dirNode instanceof INodeDirectorySnapshottable);
+    assertEquals(0, ((INodeDirectorySnapshottable) dirNode).getDiffs().asList()
+        .size());
+    
+    hdfs.setQuota(dir, HdfsConstants.QUOTA_RESET, HdfsConstants.QUOTA_RESET);
+    dirNode = fsdir.getINode4Write(dir.toString());
+    assertTrue(dirNode instanceof INodeDirectorySnapshottable);
+    assertEquals(0, ((INodeDirectorySnapshottable) dirNode).getDiffs().asList()
+        .size());
+    
+    // allow snapshot on dir and create snapshot s1
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s1");
+    
+    // clear quota of dir
+    hdfs.setQuota(dir, HdfsConstants.QUOTA_RESET, HdfsConstants.QUOTA_RESET);
+    // dir should still be a snapshottable directory
+    dirNode = fsdir.getINode4Write(dir.toString());
+    assertTrue(dirNode instanceof INodeDirectorySnapshottable);
+    assertEquals(1, ((INodeDirectorySnapshottable) dirNode).getDiffs().asList()
+        .size());
+    SnapshottableDirectoryStatus[] status = hdfs.getSnapshottableDirListing();
+    assertEquals(1, status.length);
+    assertEquals(dir, status[0].getFullPath());
+    
+    final Path subDir = new Path(dir, "sub");
+    hdfs.mkdirs(subDir);
+    hdfs.createSnapshot(dir, "s2");
+    final Path file = new Path(subDir, "file");
+    DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, seed);
+    hdfs.setQuota(dir, HdfsConstants.QUOTA_RESET, HdfsConstants.QUOTA_RESET);
+    INode subNode = fsdir.getINode4Write(subDir.toString());
+    assertTrue(subNode instanceof INodeDirectoryWithSnapshot);
+    List<DirectoryDiff> diffList = ((INodeDirectoryWithSnapshot) subNode).getDiffs().asList();
+    assertEquals(1, diffList.size());
+    assertEquals("s2", Snapshot.getSnapshotName(diffList.get(0).snapshot));
+    List<INode> createdList = diffList.get(0).getChildrenDiff().getList(ListType.CREATED);
+    assertEquals(1, createdList.size());
+    assertSame(fsdir.getINode4Write(file.toString()), createdList.get(0));
+  }
+}

+ 940 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java

@@ -0,0 +1,940 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Random;
+
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper.TestDirectoryTree;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper.TestDirectoryTree.Node;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Time;
+import org.apache.log4j.Level;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+/**
+ * This class tests snapshot functionality. One or multiple snapshots are
+ * created. The snapshotted directory is changed and verification is done to
+ * ensure snapshots remain unchanges.
+ */
+public class TestSnapshot {
+  {
+    ((Log4JLogger)INode.LOG).getLogger().setLevel(Level.ALL);
+    SnapshotTestHelper.disableLogs();
+  }
+
+  private static final long seed = Time.now();
+  protected static final short REPLICATION = 3;
+  protected static final int BLOCKSIZE = 1024;
+  /** The number of times snapshots are created for a snapshottable directory */
+  public static final int SNAPSHOT_ITERATION_NUMBER = 20;
+  /** Height of directory tree used for testing */
+  public static final int DIRECTORY_TREE_LEVEL = 5;
+  
+  protected Configuration conf;
+  protected static MiniDFSCluster cluster;
+  protected static FSNamesystem fsn;
+  protected static FSDirectory fsdir;
+  protected DistributedFileSystem hdfs;
+
+  private static Random random = new Random(seed);
+  
+  private static String testDir =
+      System.getProperty("test.build.data", "build/test/data");
+  
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+  
+  /**
+   * The list recording all previous snapshots. Each element in the array
+   * records a snapshot root.
+   */
+  protected static ArrayList<Path> snapshotList = new ArrayList<Path>();
+  /**
+   * Check {@link SnapshotTestHelper.TestDirectoryTree}
+   */
+  private TestDirectoryTree dirTree;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
+        .build();
+    cluster.waitActive();
+
+    fsn = cluster.getNamesystem();
+    fsdir = fsn.getFSDirectory();
+    hdfs = cluster.getFileSystem();
+    dirTree = new TestDirectoryTree(DIRECTORY_TREE_LEVEL, hdfs);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  static int modificationCount = 0;
+  /**
+   * Make changes (modification, deletion, creation) to the current files/dir.
+   * Then check if the previous snapshots are still correct.
+   * 
+   * @param modifications Modifications that to be applied to the current dir.
+   */
+  private void modifyCurrentDirAndCheckSnapshots(Modification[] modifications)
+      throws Exception {
+    for (Modification modification : modifications) {
+      System.out.println(++modificationCount + ") " + modification);
+      modification.loadSnapshots();
+      modification.modify();
+      modification.checkSnapshots();
+    }
+  }
+
+  /**
+   * Create two snapshots in each iteration. Each time we will create a snapshot
+   * for the top node, then randomly pick a dir in the tree and create
+   * snapshot for it.
+   * 
+   * Finally check the snapshots are created correctly.
+   */
+  protected TestDirectoryTree.Node[] createSnapshots() throws Exception {
+    TestDirectoryTree.Node[] nodes = new TestDirectoryTree.Node[2];
+    // Each time we will create a snapshot for the top level dir
+    Path root = SnapshotTestHelper.createSnapshot(hdfs,
+        dirTree.topNode.nodePath, nextSnapshotName());
+    snapshotList.add(root);
+    nodes[0] = dirTree.topNode; 
+    SnapshotTestHelper.checkSnapshotCreation(hdfs, root, nodes[0].nodePath);
+    
+    // Then randomly pick one dir from the tree (cannot be the top node) and
+    // create snapshot for it
+    ArrayList<TestDirectoryTree.Node> excludedList = 
+        new ArrayList<TestDirectoryTree.Node>();
+    excludedList.add(nodes[0]);
+    nodes[1] = dirTree.getRandomDirNode(random, excludedList);
+
+    root = SnapshotTestHelper.createSnapshot(hdfs, nodes[1].nodePath,
+        nextSnapshotName());
+
+    snapshotList.add(root);
+    SnapshotTestHelper.checkSnapshotCreation(hdfs, root, nodes[1].nodePath);
+    return nodes;
+  }
+
+  private File getDumpTreeFile(String dir, String suffix) {
+    return new File(dir, String.format("dumptree_%s", suffix));
+  }
+
+  /**
+   * Restart the cluster to check edit log applying and fsimage saving/loading
+   */
+  private void checkFSImage() throws Exception {
+    File fsnBefore = getDumpTreeFile(testDir, "before");
+    File fsnMiddle = getDumpTreeFile(testDir, "middle");
+    File fsnAfter = getDumpTreeFile(testDir, "after");
+    
+    SnapshotTestHelper.dumpTree2File(fsdir, fsnBefore);
+    
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).format(false)
+        .numDataNodes(REPLICATION).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+    // later check fsnMiddle to see if the edit log is applied correctly 
+    SnapshotTestHelper.dumpTree2File(fsdir, fsnMiddle);
+   
+    // save namespace and restart cluster
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.saveNamespace();
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).format(false)
+        .numDataNodes(REPLICATION).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+    // dump the namespace loaded from fsimage
+    SnapshotTestHelper.dumpTree2File(fsdir, fsnAfter);
+    
+    SnapshotTestHelper.compareDumpedTreeInFile(fsnBefore, fsnMiddle, true);
+    SnapshotTestHelper.compareDumpedTreeInFile(fsnBefore, fsnAfter, true);
+  }
+  
+  /**
+   * Main test, where we will go in the following loop:
+   * <pre>
+   *    Create snapshot and check the creation <--+  
+   * -> Change the current/live files/dir         | 
+   * -> Check previous snapshots -----------------+
+   * </pre>
+   */
+  @Test
+  public void testSnapshot() throws Throwable {
+    try {
+      runTestSnapshot();
+    } catch(Throwable t) {
+      SnapshotTestHelper.LOG.info("FAILED", t);
+      SnapshotTestHelper.dumpTree("FAILED", cluster);
+      throw t;
+    }
+  }
+
+  private void runTestSnapshot() throws Exception {
+    for (int i = 0; i < SNAPSHOT_ITERATION_NUMBER; i++) {
+      // create snapshot and check the creation
+      cluster.getNamesystem().getSnapshotManager().setAllowNestedSnapshots(true);
+      TestDirectoryTree.Node[] ssNodes = createSnapshots();
+      
+      // prepare the modifications for the snapshotted dirs
+      // we cover the following directories: top, new, and a random
+      ArrayList<TestDirectoryTree.Node> excludedList = 
+          new ArrayList<TestDirectoryTree.Node>();
+      TestDirectoryTree.Node[] modNodes = 
+          new TestDirectoryTree.Node[ssNodes.length + 1];
+      for (int n = 0; n < ssNodes.length; n++) {
+        modNodes[n] = ssNodes[n];
+        excludedList.add(ssNodes[n]);
+      }
+      modNodes[modNodes.length - 1] = dirTree.getRandomDirNode(random,
+          excludedList);
+      Modification[] mods = prepareModifications(modNodes);
+      // make changes to the directories/files
+      modifyCurrentDirAndCheckSnapshots(mods);
+      
+      // also update the metadata of directories
+      TestDirectoryTree.Node chmodDir = dirTree.getRandomDirNode(random, null);
+      Modification chmod = new FileChangePermission(chmodDir.nodePath, hdfs,
+          genRandomPermission());
+      String[] userGroup = genRandomOwner();
+      TestDirectoryTree.Node chownDir = dirTree.getRandomDirNode(random,
+          Arrays.asList(chmodDir));
+      Modification chown = new FileChown(chownDir.nodePath, hdfs, userGroup[0],
+          userGroup[1]);
+      modifyCurrentDirAndCheckSnapshots(new Modification[]{chmod, chown});
+      
+      // check fsimage saving/loading
+      checkFSImage();
+    }
+  }
+
+  /**
+   * A simple test that updates a sub-directory of a snapshottable directory
+   * with snapshots
+   */
+  @Test (timeout=60000)
+  public void testUpdateDirectory() throws Exception {
+    Path dir = new Path("/dir");
+    Path sub = new Path(dir, "sub");
+    Path subFile = new Path(sub, "file");
+    DFSTestUtil.createFile(hdfs, subFile, BLOCKSIZE, REPLICATION, seed);
+
+    FileStatus oldStatus = hdfs.getFileStatus(sub);
+
+    hdfs.allowSnapshot(dir);
+    hdfs.createSnapshot(dir, "s1");
+    hdfs.setTimes(sub, 100L, 100L);
+
+    Path snapshotPath = SnapshotTestHelper.getSnapshotPath(dir, "s1", "sub");
+    FileStatus snapshotStatus = hdfs.getFileStatus(snapshotPath);
+    assertEquals(oldStatus.getModificationTime(),
+        snapshotStatus.getModificationTime());
+    assertEquals(oldStatus.getAccessTime(), snapshotStatus.getAccessTime());
+  }
+  
+  /**
+   * Creating snapshots for a directory that is not snapshottable must fail.
+   */
+  @Test (timeout=60000)
+  public void testSnapshottableDirectory() throws Exception {
+    Path dir = new Path("/TestSnapshot/sub");
+    Path file0 = new Path(dir, "file0");
+    Path file1 = new Path(dir, "file1");
+    DFSTestUtil.createFile(hdfs, file0, BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, file1, BLOCKSIZE, REPLICATION, seed);
+
+    try {
+      hdfs.createSnapshot(dir, "s1");
+      fail("Exception expected: " + dir + " is not snapshottable");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains(
+          "Directory is not a snapshottable directory: " + dir, e);
+    }
+
+    try {
+      hdfs.deleteSnapshot(dir, "s1");
+      fail("Exception expected: " + dir + " is not a snapshottale dir");
+    } catch (Exception e) {
+      GenericTestUtils.assertExceptionContains(
+          "Directory is not a snapshottable directory: " + dir, e);
+    }
+
+    try {
+      hdfs.renameSnapshot(dir, "s1", "s2");
+      fail("Exception expected: " + dir + " is not a snapshottale dir");
+    } catch (Exception e) {
+      GenericTestUtils.assertExceptionContains(
+          "Directory is not a snapshottable directory: " + dir, e);
+    }
+  }
+
+  /**
+   * Prepare a list of modifications. A modification may be a file creation,
+   * file deletion, or a modification operation such as appending to an existing
+   * file.
+   */
+  private Modification[] prepareModifications(TestDirectoryTree.Node[] nodes)
+      throws Exception {
+    ArrayList<Modification> mList = new ArrayList<Modification>();
+    for (TestDirectoryTree.Node node : nodes) {
+      // If the node does not have files in it, create files
+      if (node.fileList == null) {
+        node.initFileList(hdfs, node.nodePath.getName(), BLOCKSIZE,
+            REPLICATION, seed, 6);
+      }
+      
+      //
+      // Modification iterations are as follows:
+      // Iteration 0 - create:fileList[5], delete:fileList[0],
+      //               append:fileList[1], chmod:fileList[2], 
+      //               chown:fileList[3],  change_replication:fileList[4].
+      //               Set nullFileIndex to 0
+      //
+      // Iteration 1 - create:fileList[0], delete:fileList[1],
+      //               append:fileList[2], chmod:fileList[3], 
+      //               chown:fileList[4],  change_replication:fileList[5]
+      //               Set nullFileIndex to 1
+      // 
+      // Iteration 2 - create:fileList[1], delete:fileList[2],
+      //               append:fileList[3], chmod:fileList[4], 
+      //               chown:fileList[5],  change_replication:fileList[6]
+      //               Set nullFileIndex to 2
+      // ...
+      //
+      Modification create = new FileCreation(
+          node.fileList.get(node.nullFileIndex), hdfs, (int) BLOCKSIZE);
+      Modification delete = new FileDeletion(
+          node.fileList.get((node.nullFileIndex + 1) % node.fileList.size()),
+          hdfs);
+
+      Path f = node.fileList.get((node.nullFileIndex + 2) % node.fileList.size());
+      Modification append = new FileAppend(f, hdfs, BLOCKSIZE);
+      FileAppendNotClose appendNotClose = new FileAppendNotClose(f, hdfs, BLOCKSIZE);
+      Modification appendClose = new FileAppendClose(f, hdfs, BLOCKSIZE, appendNotClose);
+
+      Modification chmod = new FileChangePermission(
+          node.fileList.get((node.nullFileIndex + 3) % node.fileList.size()),
+          hdfs, genRandomPermission());
+      String[] userGroup = genRandomOwner();
+      Modification chown = new FileChown(
+          node.fileList.get((node.nullFileIndex + 4) % node.fileList.size()),
+          hdfs, userGroup[0], userGroup[1]);
+      Modification replication = new FileChangeReplication(
+          node.fileList.get((node.nullFileIndex + 5) % node.fileList.size()),
+          hdfs, (short) (random.nextInt(REPLICATION) + 1));
+      node.nullFileIndex = (node.nullFileIndex + 1) % node.fileList.size();
+      Modification dirChange = new DirCreationOrDeletion(node.nodePath, hdfs,
+          node, random.nextBoolean());
+      // dir rename
+      Node dstParent = dirTree.getRandomDirNode(random, Arrays.asList(nodes));
+      Modification dirRename = new DirRename(node.nodePath, hdfs, node,
+          dstParent);
+      
+      mList.add(create);
+      mList.add(delete);
+      mList.add(append); 
+      mList.add(appendNotClose); 
+      mList.add(appendClose); 
+      mList.add(chmod);
+      mList.add(chown);
+      mList.add(replication);
+      mList.add(dirChange);
+      mList.add(dirRename);
+    }
+    return mList.toArray(new Modification[mList.size()]);
+  }
+  
+  /**
+   * @return A random FsPermission
+   */
+  private FsPermission genRandomPermission() {
+    // randomly select between "rwx" and "rw-"
+    FsAction u = random.nextBoolean() ? FsAction.ALL : FsAction.READ_WRITE;
+    FsAction g = random.nextBoolean() ? FsAction.ALL : FsAction.READ_WRITE;
+    FsAction o = random.nextBoolean() ? FsAction.ALL : FsAction.READ_WRITE;
+    return new FsPermission(u, g, o);
+  }
+  
+  /**
+   * @return A string array containing two string: the first string indicates
+   *         the owner, and the other indicates the group
+   */
+  private String[] genRandomOwner() {
+    String[] userGroup = new String[]{"dr.who", "unknown"};
+    return userGroup;
+  }
+  
+  
+  private static int snapshotCount = 0;
+
+  /** @return The next snapshot name */
+  static String nextSnapshotName() {
+    return String.format("s-%d", ++snapshotCount);
+  }
+
+  /**
+   * Base class to present changes applied to current file/dir. A modification
+   * can be file creation, deletion, or other modifications such as appending on
+   * an existing file. Three abstract methods need to be implemented by
+   * subclasses: loadSnapshots() captures the states of snapshots before the
+   * modification, modify() applies the modification to the current directory,
+   * and checkSnapshots() verifies the snapshots do not change after the
+   * modification.
+   */
+  static abstract class Modification {
+    protected final Path file;
+    protected final FileSystem fs;
+    final String type;
+
+    Modification(Path file, FileSystem fs, String type) {
+      this.file = file;
+      this.fs = fs;
+      this.type = type;
+    }
+
+    abstract void loadSnapshots() throws Exception;
+
+    abstract void modify() throws Exception;
+
+    abstract void checkSnapshots() throws Exception;
+    
+    @Override
+    public String toString() {
+      return getClass().getSimpleName() + ":" + type + ":" + file;
+    }
+  }
+
+  /**
+   * Modifications that change the file status. We check the FileStatus of
+   * snapshot files before/after the modification.
+   */
+  static abstract class FileStatusChange extends Modification {
+    protected final HashMap<Path, FileStatus> statusMap;
+    
+    FileStatusChange(Path file, FileSystem fs, String type) {
+      super(file, fs, type);
+      statusMap = new HashMap<Path, FileStatus>();
+    }
+     
+    @Override
+    void loadSnapshots() throws Exception {
+      for (Path snapshotRoot : snapshotList) {
+        Path snapshotFile = SnapshotTestHelper.getSnapshotFile(
+            snapshotRoot, file);
+        if (snapshotFile != null) {
+          if (fs.exists(snapshotFile)) {
+            FileStatus status = fs.getFileStatus(snapshotFile);
+            statusMap.put(snapshotFile, status);
+          } else {
+            statusMap.put(snapshotFile, null);
+          }
+        }
+      }
+    }
+    
+    @Override
+    void checkSnapshots() throws Exception {
+      for (Path snapshotFile : statusMap.keySet()) {
+        FileStatus currentStatus = fs.exists(snapshotFile) ? fs
+            .getFileStatus(snapshotFile) : null;
+        FileStatus originalStatus = statusMap.get(snapshotFile);
+        assertEquals(currentStatus, originalStatus);
+        if (currentStatus != null) {
+          String s = null;
+          if (!currentStatus.toString().equals(originalStatus.toString())) {
+            s = "FAILED: " + getClass().getSimpleName()
+                + ": file="  + file + ", snapshotFile" + snapshotFile
+                + "\n\n currentStatus = " + currentStatus
+                +   "\noriginalStatus = " + originalStatus
+                + "\n\nfile        : " + fsdir.getINode(file.toString()).toDetailString()
+                + "\n\nsnapshotFile: " + fsdir.getINode(snapshotFile.toString()).toDetailString();
+            
+            SnapshotTestHelper.dumpTree(s, cluster);
+          }
+          assertEquals(s, currentStatus.toString(), originalStatus.toString());
+        }
+      }
+    }
+  }
+  
+  /**
+   * Change the file permission
+   */
+  static class FileChangePermission extends FileStatusChange {
+    private final FsPermission newPermission;
+    
+    FileChangePermission(Path file, FileSystem fs, FsPermission newPermission) {
+      super(file, fs, "chmod");
+      this.newPermission = newPermission;
+    }
+        
+    @Override
+    void modify() throws Exception {
+      assertTrue(fs.exists(file));
+      fs.setPermission(file, newPermission);
+    }
+  }
+  
+  /**
+   * Change the replication factor of file
+   */
+  static class FileChangeReplication extends FileStatusChange {
+    private final short newReplication;
+    
+    FileChangeReplication(Path file, FileSystem fs, short replication) {
+      super(file, fs, "replication");
+      this.newReplication = replication;
+    }
+    
+    @Override
+    void modify() throws Exception {
+      assertTrue(fs.exists(file));
+      fs.setReplication(file, newReplication);
+    }
+  }
+  
+  /**
+   * Change the owner:group of a file
+   */
+  static class FileChown extends FileStatusChange {
+    private final String newUser;
+    private final String newGroup;
+    
+    FileChown(Path file, FileSystem fs, String user, String group) {
+      super(file, fs, "chown");
+      this.newUser = user;
+      this.newGroup = group;
+    }
+
+    @Override
+    void modify() throws Exception {
+      assertTrue(fs.exists(file));
+      fs.setOwner(file, newUser, newGroup);
+    }
+  }
+  
+  /**
+   * Appending a specified length to an existing file
+   */
+  static class FileAppend extends Modification {
+    final int appendLen;
+    private final HashMap<Path, Long> snapshotFileLengthMap;
+
+    FileAppend(Path file, FileSystem fs, int len) {
+      super(file, fs, "append");
+      this.appendLen = len;
+      this.snapshotFileLengthMap = new HashMap<Path, Long>();
+    }
+
+    @Override
+    void loadSnapshots() throws Exception {
+      for (Path snapshotRoot : snapshotList) {
+        Path snapshotFile = SnapshotTestHelper.getSnapshotFile(
+            snapshotRoot, file);
+        if (snapshotFile != null) {
+          long snapshotFileLen = fs.exists(snapshotFile) ? fs.getFileStatus(
+              snapshotFile).getLen() : -1L;
+          snapshotFileLengthMap.put(snapshotFile, snapshotFileLen);
+        }
+      }
+    }
+
+    @Override
+    void modify() throws Exception {
+      assertTrue(fs.exists(file));
+      DFSTestUtil.appendFile(fs, file, appendLen);
+    }
+
+    @Override
+    void checkSnapshots() throws Exception {
+      byte[] buffer = new byte[32];
+      for (Path snapshotFile : snapshotFileLengthMap.keySet()) {        
+        long currentSnapshotFileLen = fs.exists(snapshotFile) ? fs
+            .getFileStatus(snapshotFile).getLen() : -1L;
+        long originalSnapshotFileLen = snapshotFileLengthMap.get(snapshotFile);
+        String s = null;
+        if (currentSnapshotFileLen != originalSnapshotFileLen) {
+          s = "FAILED: " + getClass().getSimpleName()
+              + ": file="  + file + ", snapshotFile" + snapshotFile
+              + "\n\n currentSnapshotFileLen = " + currentSnapshotFileLen
+              +   "\noriginalSnapshotFileLen = " + originalSnapshotFileLen
+              + "\n\nfile        : " + fsdir.getINode(file.toString()).toDetailString()
+              + "\n\nsnapshotFile: " + fsdir.getINode(snapshotFile.toString()).toDetailString();
+          SnapshotTestHelper.dumpTree(s, cluster);
+        }
+        assertEquals(s, originalSnapshotFileLen, currentSnapshotFileLen);
+        // Read the snapshot file out of the boundary
+        if (currentSnapshotFileLen != -1L
+            && !(this instanceof FileAppendNotClose)) {
+          FSDataInputStream input = fs.open(snapshotFile);
+          int readLen = input.read(currentSnapshotFileLen, buffer, 0, 1);
+          if (readLen != -1) {
+            s = "FAILED: " + getClass().getSimpleName()
+                + ": file="  + file + ", snapshotFile" + snapshotFile
+                + "\n\n currentSnapshotFileLen = " + currentSnapshotFileLen
+                +   "\n                readLen = " + readLen
+                + "\n\nfile        : " + fsdir.getINode(file.toString()).toDetailString()
+                + "\n\nsnapshotFile: " + fsdir.getINode(snapshotFile.toString()).toDetailString();
+            SnapshotTestHelper.dumpTree(s, cluster);
+          }
+          assertEquals(s, -1, readLen);
+          input.close();
+        }
+      }
+    }
+  }
+
+  /**
+   * Appending a specified length to an existing file but not close the file
+   */
+  static class FileAppendNotClose extends FileAppend {
+    HdfsDataOutputStream out;
+
+    FileAppendNotClose(Path file, FileSystem fs, int len) {
+      super(file, fs, len);
+    }
+
+    @Override
+    void modify() throws Exception {
+      assertTrue(fs.exists(file));
+      byte[] toAppend = new byte[appendLen];
+      random.nextBytes(toAppend);
+
+      out = (HdfsDataOutputStream)fs.append(file);
+      out.write(toAppend);
+      out.hsync(EnumSet.of(HdfsDataOutputStream.SyncFlag.UPDATE_LENGTH));
+    }
+  }
+
+  /**
+   * Appending a specified length to an existing file
+   */
+  static class FileAppendClose extends FileAppend {
+    final FileAppendNotClose fileAppendNotClose;
+
+    FileAppendClose(Path file, FileSystem fs, int len,
+        FileAppendNotClose fileAppendNotClose) {
+      super(file, fs, len);
+      this.fileAppendNotClose = fileAppendNotClose;
+    }
+
+    @Override
+    void modify() throws Exception {
+      assertTrue(fs.exists(file));
+      byte[] toAppend = new byte[appendLen];
+      random.nextBytes(toAppend);
+
+      fileAppendNotClose.out.write(toAppend);
+      fileAppendNotClose.out.close();
+    }
+  }
+
+  /**
+   * New file creation
+   */
+  static class FileCreation extends Modification {
+    final int fileLen;
+    private final HashMap<Path, FileStatus> fileStatusMap;
+
+    FileCreation(Path file, FileSystem fs, int len) {
+      super(file, fs, "creation");
+      assert len >= 0;
+      this.fileLen = len;
+      fileStatusMap = new HashMap<Path, FileStatus>();
+    }
+
+    @Override
+    void loadSnapshots() throws Exception {
+      for (Path snapshotRoot : snapshotList) {
+        Path snapshotFile = SnapshotTestHelper.getSnapshotFile(
+            snapshotRoot, file);
+        if (snapshotFile != null) {
+          FileStatus status = 
+              fs.exists(snapshotFile) ? fs.getFileStatus(snapshotFile) : null; 
+          fileStatusMap.put(snapshotFile, status);
+        }
+      }
+    }
+
+    @Override
+    void modify() throws Exception {
+      DFSTestUtil.createFile(fs, file, fileLen,
+          REPLICATION, seed);
+    }
+
+    @Override
+    void checkSnapshots() throws Exception {
+      for (Path snapshotRoot : snapshotList) {
+        Path snapshotFile = SnapshotTestHelper.getSnapshotFile(
+            snapshotRoot, file);
+        if (snapshotFile != null) {
+          boolean computed = fs.exists(snapshotFile);
+          boolean expected = fileStatusMap.get(snapshotFile) != null;
+          assertEquals(expected, computed);
+          if (computed) {
+            FileStatus currentSnapshotStatus = fs.getFileStatus(snapshotFile);
+            FileStatus originalStatus = fileStatusMap.get(snapshotFile);
+            // We compare the string because it contains all the information,
+            // while FileStatus#equals only compares the path
+            assertEquals(currentSnapshotStatus.toString(),
+                originalStatus.toString());
+          }
+        }
+      }
+    }
+  }
+  
+  /**
+   * File deletion
+   */
+  static class FileDeletion extends Modification {
+    private final HashMap<Path, Boolean> snapshotFileExistenceMap;
+
+    FileDeletion(Path file, FileSystem fs) {
+      super(file, fs, "deletion");
+      snapshotFileExistenceMap = new HashMap<Path, Boolean>();
+    }
+
+    @Override
+    void loadSnapshots() throws Exception {
+      for (Path snapshotRoot : snapshotList) {
+        boolean existence = SnapshotTestHelper.getSnapshotFile(
+            snapshotRoot, file) != null;
+        snapshotFileExistenceMap.put(snapshotRoot, existence);
+      }
+    }
+
+    @Override
+    void modify() throws Exception {
+      fs.delete(file, true);
+    }
+
+    @Override
+    void checkSnapshots() throws Exception {
+      for (Path snapshotRoot : snapshotList) {
+        boolean currentSnapshotFileExist = SnapshotTestHelper.getSnapshotFile(
+            snapshotRoot, file) != null;
+        boolean originalSnapshotFileExist = snapshotFileExistenceMap
+            .get(snapshotRoot);
+        assertEquals(currentSnapshotFileExist, originalSnapshotFileExist);
+      }
+    }
+  }
+  
+  /**
+   * Directory creation or deletion.
+   */
+  class DirCreationOrDeletion extends Modification {
+    private final TestDirectoryTree.Node node;
+    private final boolean isCreation;
+    private final Path changedPath;
+    private final HashMap<Path, FileStatus> statusMap;
+    
+    DirCreationOrDeletion(Path file, FileSystem fs, TestDirectoryTree.Node node,
+        boolean isCreation) {
+      super(file, fs, "dircreation");
+      this.node = node;
+      // If the node's nonSnapshotChildren is empty, we still need to create
+      // sub-directories
+      this.isCreation = isCreation || node.nonSnapshotChildren.isEmpty();
+      if (this.isCreation) {
+        // Generate the path for the dir to be created
+        changedPath = new Path(node.nodePath, "sub"
+            + node.nonSnapshotChildren.size());
+      } else {
+        // If deletion, we delete the current last dir in nonSnapshotChildren
+        changedPath = node.nonSnapshotChildren.get(node.nonSnapshotChildren
+            .size() - 1).nodePath;
+      }
+      this.statusMap = new HashMap<Path, FileStatus>();
+    }
+    
+    @Override
+    void loadSnapshots() throws Exception {
+      for (Path snapshotRoot : snapshotList) {
+        Path snapshotDir = SnapshotTestHelper.getSnapshotFile(snapshotRoot,
+            changedPath);
+        if (snapshotDir != null) {
+          FileStatus status = fs.exists(snapshotDir) ? fs
+              .getFileStatus(snapshotDir) : null;
+          statusMap.put(snapshotDir, status);
+          // In each non-snapshottable directory, we also create a file. Thus
+          // here we also need to check the file's status before/after taking
+          // snapshots
+          Path snapshotFile = new Path(snapshotDir, "file0");
+          status = fs.exists(snapshotFile) ? fs.getFileStatus(snapshotFile)
+              : null;
+          statusMap.put(snapshotFile, status);
+        }
+      }
+    }
+
+    @Override
+    void modify() throws Exception {
+      if (isCreation) {
+        // creation
+        TestDirectoryTree.Node newChild = new TestDirectoryTree.Node(
+            changedPath, node.level + 1, node, hdfs);
+        // create file under the new non-snapshottable directory
+        newChild.initFileList(hdfs, node.nodePath.getName(), BLOCKSIZE,
+            REPLICATION, seed, 2);
+        node.nonSnapshotChildren.add(newChild);
+      } else {
+        // deletion
+        TestDirectoryTree.Node childToDelete = node.nonSnapshotChildren
+            .remove(node.nonSnapshotChildren.size() - 1);
+        hdfs.delete(childToDelete.nodePath, true);
+      }
+    }
+
+    @Override
+    void checkSnapshots() throws Exception {
+      for (Path snapshot : statusMap.keySet()) {
+        FileStatus currentStatus = fs.exists(snapshot) ? fs
+            .getFileStatus(snapshot) : null;
+        FileStatus originalStatus = statusMap.get(snapshot);
+        assertEquals(currentStatus, originalStatus);
+        if (currentStatus != null) {
+          assertEquals(currentStatus.toString(), originalStatus.toString());
+        }
+      }
+    } 
+  }
+  
+  /**
+   * Directory creation or deletion.
+   */
+  class DirRename extends Modification {
+    private final TestDirectoryTree.Node srcParent;
+    private final TestDirectoryTree.Node dstParent;
+    private final Path srcPath;
+    private final Path dstPath;
+    private final HashMap<Path, FileStatus> statusMap;
+    
+    DirRename(Path file, FileSystem fs, TestDirectoryTree.Node src,
+        TestDirectoryTree.Node dst) throws Exception {
+      super(file, fs, "dirrename");
+      this.srcParent = src;
+      this.dstParent = dst;
+      dstPath = new Path(dstParent.nodePath, "sub"
+          + dstParent.nonSnapshotChildren.size());
+      
+      // If the srcParent's nonSnapshotChildren is empty, we need to create
+      // sub-directories
+      if (srcParent.nonSnapshotChildren.isEmpty()) {
+        srcPath = new Path(srcParent.nodePath, "sub"
+            + srcParent.nonSnapshotChildren.size());
+        // creation
+        TestDirectoryTree.Node newChild = new TestDirectoryTree.Node(
+            srcPath, srcParent.level + 1, srcParent, hdfs);
+        // create file under the new non-snapshottable directory
+        newChild.initFileList(hdfs, srcParent.nodePath.getName(), BLOCKSIZE,
+            REPLICATION, seed, 2);
+        srcParent.nonSnapshotChildren.add(newChild);
+      } else {
+        srcPath = new Path(srcParent.nodePath, "sub"
+            + (srcParent.nonSnapshotChildren.size() - 1));
+      }
+      this.statusMap = new HashMap<Path, FileStatus>();
+    }
+    
+    @Override
+    void loadSnapshots() throws Exception {
+      for (Path snapshotRoot : snapshotList) {
+        Path snapshotDir = SnapshotTestHelper.getSnapshotFile(snapshotRoot,
+            srcPath);
+        if (snapshotDir != null) {
+          FileStatus status = fs.exists(snapshotDir) ? fs
+              .getFileStatus(snapshotDir) : null;
+          statusMap.put(snapshotDir, status);
+          // In each non-snapshottable directory, we also create a file. Thus
+          // here we also need to check the file's status before/after taking
+          // snapshots
+          Path snapshotFile = new Path(snapshotDir, "file0");
+          status = fs.exists(snapshotFile) ? fs.getFileStatus(snapshotFile)
+              : null;
+          statusMap.put(snapshotFile, status);
+        }
+      }
+    }
+
+    @Override
+    void modify() throws Exception {
+      hdfs.rename(srcPath, dstPath);
+      TestDirectoryTree.Node newDstChild = new TestDirectoryTree.Node(
+          dstPath, dstParent.level + 1, dstParent, hdfs);
+      dstParent.nonSnapshotChildren.add(newDstChild);
+    }
+
+    @Override
+    void checkSnapshots() throws Exception {
+      for (Path snapshot : statusMap.keySet()) {
+        FileStatus currentStatus = fs.exists(snapshot) ? fs
+            .getFileStatus(snapshot) : null;
+        FileStatus originalStatus = statusMap.get(snapshot);
+        assertEquals(currentStatus, originalStatus);
+        if (currentStatus != null) {
+          assertEquals(currentStatus.toString(), originalStatus.toString());
+        }
+      }
+    } 
+  }
+}

+ 208 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java

@@ -0,0 +1,208 @@
+/**
+ * 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 static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test cases for snapshot-related information in blocksMap.
+ */
+public class TestSnapshotBlocksMap {
+  private static final long seed = 0;
+  private static final short REPLICATION = 3;
+  private static final int BLOCKSIZE = 1024;
+  
+  private final Path dir = new Path("/TestSnapshot");
+  private final Path sub1 = new Path(dir, "sub1");
+  
+  protected Configuration conf;
+  protected MiniDFSCluster cluster;
+  protected FSNamesystem fsn;
+  FSDirectory fsdir;
+  BlockManager blockmanager;
+  protected DistributedFileSystem hdfs;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
+        .build();
+    cluster.waitActive();
+
+    fsn = cluster.getNamesystem();
+    fsdir = fsn.getFSDirectory();
+    blockmanager = fsn.getBlockManager();
+    hdfs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  void assertAllNull(INodeFile inode, Path path, String[] snapshots) throws Exception { 
+    Assert.assertNull(inode.getBlocks());
+    assertINodeNull(path.toString());
+    assertINodeNullInSnapshots(path, snapshots);
+  }
+
+  void assertINodeNull(String path) throws Exception {
+    Assert.assertNull(fsdir.getINode(path));
+  }
+
+  void assertINodeNullInSnapshots(Path path, String... snapshots) throws Exception {
+    for(String s : snapshots) {
+      assertINodeNull(SnapshotTestHelper.getSnapshotPath(
+          path.getParent(), s, path.getName()).toString());
+    }
+  }
+
+  static INodeFile assertBlockCollection(String path, int numBlocks,
+     final FSDirectory dir, final BlockManager blkManager) throws Exception {
+    final INodeFile file = INodeFile.valueOf(dir.getINode(path), path);
+    assertEquals(numBlocks, file.getBlocks().length);
+    for(BlockInfo b : file.getBlocks()) {
+      assertBlockCollection(blkManager, file, b);
+    }
+    return file;
+  }
+
+  static void assertBlockCollection(final BlockManager blkManager,
+      final INodeFile file, final BlockInfo b) { 
+    Assert.assertSame(b, blkManager.getStoredBlock(b));
+    Assert.assertSame(file, blkManager.getBlockCollection(b));
+    Assert.assertSame(file, b.getBlockCollection());
+  }
+
+  /**
+   * Test deleting a file with snapshots. Need to check the blocksMap to make
+   * sure the corresponding record is updated correctly.
+   */
+  @Test (timeout=60000)
+  public void testDeletionWithSnapshots() throws Exception {
+    Path file0 = new Path(sub1, "file0");
+    Path file1 = new Path(sub1, "file1");
+    
+    Path sub2 = new Path(sub1, "sub2");
+    Path file2 = new Path(sub2, "file2");
+
+    Path file3 = new Path(sub1, "file3");
+    Path file4 = new Path(sub1, "file4");
+    Path file5 = new Path(sub1, "file5");
+    
+    // Create file under sub1
+    DFSTestUtil.createFile(hdfs, file0, 4*BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, file1, 2*BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, file2, 3*BLOCKSIZE, REPLICATION, seed);
+    
+    // Normal deletion
+    {
+      final INodeFile f2 = assertBlockCollection(file2.toString(), 3, fsdir,
+          blockmanager);
+      BlockInfo[] blocks = f2.getBlocks();
+      hdfs.delete(sub2, true);
+      // The INode should have been removed from the blocksMap
+      for(BlockInfo b : blocks) {
+        assertNull(blockmanager.getBlockCollection(b));
+      }
+    }
+    
+    // Create snapshots for sub1
+    final String[] snapshots = {"s0", "s1", "s2"};
+    DFSTestUtil.createFile(hdfs, file3, 5*BLOCKSIZE, REPLICATION, seed);
+    SnapshotTestHelper.createSnapshot(hdfs, sub1, snapshots[0]);
+    DFSTestUtil.createFile(hdfs, file4, 1*BLOCKSIZE, REPLICATION, seed);
+    SnapshotTestHelper.createSnapshot(hdfs, sub1, snapshots[1]);
+    DFSTestUtil.createFile(hdfs, file5, 7*BLOCKSIZE, REPLICATION, seed);
+    SnapshotTestHelper.createSnapshot(hdfs, sub1, snapshots[2]);
+
+    // set replication so that the inode should be replaced for snapshots
+    {
+      INodeFile f1 = assertBlockCollection(file1.toString(), 2, fsdir,
+          blockmanager);
+      Assert.assertSame(INodeFile.class, f1.getClass());
+      hdfs.setReplication(file1, (short)2);
+      f1 = assertBlockCollection(file1.toString(), 2, fsdir, blockmanager);
+      Assert.assertSame(INodeFileWithSnapshot.class, f1.getClass());
+    }
+    
+    // Check the block information for file0
+    final INodeFile f0 = assertBlockCollection(file0.toString(), 4, fsdir,
+        blockmanager);
+    BlockInfo[] blocks0 = f0.getBlocks();
+    
+    // Also check the block information for snapshot of file0
+    Path snapshotFile0 = SnapshotTestHelper.getSnapshotPath(sub1, "s0",
+        file0.getName());
+    assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager);
+    
+    // Delete file0
+    hdfs.delete(file0, true);
+    // Make sure the blocks of file0 is still in blocksMap
+    for(BlockInfo b : blocks0) {
+      assertNotNull(blockmanager.getBlockCollection(b));
+    }
+    assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager);
+    
+    // Compare the INode in the blocksMap with INodes for snapshots
+    String s1f0 = SnapshotTestHelper.getSnapshotPath(sub1, "s1",
+        file0.getName()).toString();
+    assertBlockCollection(s1f0, 4, fsdir, blockmanager);
+    
+    // Delete snapshot s1
+    hdfs.deleteSnapshot(sub1, "s1");
+
+    // Make sure the first block of file0 is still in blocksMap
+    for(BlockInfo b : blocks0) {
+      assertNotNull(blockmanager.getBlockCollection(b));
+    }
+    assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager);
+
+    try {
+      INodeFile.valueOf(fsdir.getINode(s1f0), s1f0);
+      fail("Expect FileNotFoundException when identifying the INode in a deleted Snapshot");
+    } catch (IOException e) {
+      assertExceptionContains("File does not exist: " + s1f0, e);
+    }
+  }
+}

+ 887 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java

@@ -0,0 +1,887 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryWithQuota;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiffList;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+/**
+ * Tests snapshot deletion.
+ */
+public class TestSnapshotDeletion {
+  protected static final long seed = 0;
+  protected static final short REPLICATION = 3;
+  protected static final short REPLICATION_1 = 2;
+  protected static final long BLOCKSIZE = 1024;
+  
+  private final Path dir = new Path("/TestSnapshot");
+  private final Path sub = new Path(dir, "sub1");
+  private final Path subsub = new Path(sub, "subsub1");
+  
+  protected Configuration conf;
+  protected MiniDFSCluster cluster;
+  protected FSNamesystem fsn;
+  protected FSDirectory fsdir;
+  protected BlockManager blockmanager;
+  protected DistributedFileSystem hdfs;
+  
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new Configuration();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
+        .format(true).build();
+    cluster.waitActive();
+
+    fsn = cluster.getNamesystem();
+    fsdir = fsn.getFSDirectory();
+    blockmanager = fsn.getBlockManager();
+    hdfs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+    
+  /**
+   * Deleting snapshottable directory with snapshots must fail.
+   */
+  @Test (timeout=300000)
+  public void testDeleteDirectoryWithSnapshot() throws Exception {
+    Path file0 = new Path(sub, "file0");
+    Path file1 = new Path(sub, "file1");
+    DFSTestUtil.createFile(hdfs, file0, BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, file1, BLOCKSIZE, REPLICATION, seed);
+
+    // Allow snapshot for sub1, and create snapshot for it
+    hdfs.allowSnapshot(sub);
+    hdfs.createSnapshot(sub, "s1");
+
+    // Deleting a snapshottable dir with snapshots should fail
+    exception.expect(RemoteException.class);
+    String error = "The direcotry " + sub.toString()
+        + " cannot be deleted since " + sub.toString()
+        + " is snapshottable and already has snapshots";
+    exception.expectMessage(error);
+    hdfs.delete(sub, true);
+  }
+  
+  /**
+   * Deleting directory with snapshottable descendant with snapshots must fail.
+   */
+  @Test (timeout=300000)
+  public void testDeleteDirectoryWithSnapshot2() throws Exception {
+    Path file0 = new Path(sub, "file0");
+    Path file1 = new Path(sub, "file1");
+    DFSTestUtil.createFile(hdfs, file0, BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, file1, BLOCKSIZE, REPLICATION, seed);
+    
+    Path subfile1 = new Path(subsub, "file0");
+    Path subfile2 = new Path(subsub, "file1");
+    DFSTestUtil.createFile(hdfs, subfile1, BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, subfile2, BLOCKSIZE, REPLICATION, seed);
+
+    // Allow snapshot for subsub1, and create snapshot for it
+    hdfs.allowSnapshot(subsub);
+    hdfs.createSnapshot(subsub, "s1");
+
+    // Deleting dir while its descedant subsub1 having snapshots should fail
+    exception.expect(RemoteException.class);
+    String error = subsub.toString()
+        + " is snapshottable and already has snapshots";
+    exception.expectMessage(error);
+    hdfs.delete(dir, true);
+  }
+  
+  private void checkQuotaUsageComputation(final Path dirPath,
+      final long expectedNs, final long expectedDs) throws IOException {
+    INode node = fsdir.getINode(dirPath.toString());
+    assertTrue(node.isDirectory() && node.isQuotaSet());
+    INodeDirectoryWithQuota dirNode = (INodeDirectoryWithQuota) node;
+    assertEquals(dirNode.dumpTreeRecursively().toString(), expectedNs,
+        dirNode.getNamespace());
+    assertEquals(dirNode.dumpTreeRecursively().toString(), expectedDs,
+        dirNode.getDiskspace());
+    Quota.Counts counts = Quota.Counts.newInstance();
+    dirNode.computeQuotaUsage(counts, false);
+    assertEquals(dirNode.dumpTreeRecursively().toString(), expectedNs,
+        counts.get(Quota.NAMESPACE));
+    assertEquals(dirNode.dumpTreeRecursively().toString(), expectedDs,
+        counts.get(Quota.DISKSPACE));
+  }
+  
+  /**
+   * Test deleting a directory which is a descendant of a snapshottable
+   * directory. In the test we need to cover the following cases:
+   * 
+   * <pre>
+   * 1. Delete current INodeFile/INodeDirectory without taking any snapshot.
+   * 2. Delete current INodeFile/INodeDirectory while snapshots have been taken 
+   *    on ancestor(s).
+   * 3. Delete current INodeFileWithSnapshot.
+   * 4. Delete current INodeDirectoryWithSnapshot.
+   * </pre>
+   */
+  @Test (timeout=300000)
+  public void testDeleteCurrentFileDirectory() throws Exception {
+    // create a folder which will be deleted before taking snapshots
+    Path deleteDir = new Path(subsub, "deleteDir");
+    Path deleteFile = new Path(deleteDir, "deleteFile");
+    // create a directory that we will not change during the whole process.
+    Path noChangeDirParent = new Path(sub, "noChangeDirParent");
+    Path noChangeDir = new Path(noChangeDirParent, "noChangeDir");
+    // create a file that we will not change in the future
+    Path noChangeFile = new Path(noChangeDir, "noChangeFile");
+    DFSTestUtil.createFile(hdfs, deleteFile, BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, noChangeFile, BLOCKSIZE, REPLICATION, seed);
+    // we will change this file's metadata in the future
+    Path metaChangeFile1 = new Path(subsub, "metaChangeFile1");
+    DFSTestUtil.createFile(hdfs, metaChangeFile1, BLOCKSIZE, REPLICATION, seed);
+    // another file, created under noChangeDir, whose metadata will be changed
+    Path metaChangeFile2 = new Path(noChangeDir, "metaChangeFile2");
+    DFSTestUtil.createFile(hdfs, metaChangeFile2, BLOCKSIZE, REPLICATION, seed);
+    
+    // Case 1: delete deleteDir before taking snapshots
+    hdfs.delete(deleteDir, true);
+    
+    // create snapshot s0
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
+    
+    // after creating snapshot s0, create a directory tempdir under dir and then
+    // delete dir immediately
+    Path tempDir = new Path(dir, "tempdir");
+    Path tempFile = new Path(tempDir, "tempfile");
+    DFSTestUtil.createFile(hdfs, tempFile, BLOCKSIZE, REPLICATION, seed);
+    final INodeFile temp = TestSnapshotBlocksMap.assertBlockCollection(
+        tempFile.toString(), 1, fsdir, blockmanager);
+    BlockInfo[] blocks = temp.getBlocks();
+    hdfs.delete(tempDir, true);
+    // check dir's quota usage
+    checkQuotaUsageComputation(dir, 9L, BLOCKSIZE * REPLICATION * 3);
+    // check blocks of tempFile
+    for (BlockInfo b : blocks) {
+      assertNull(blockmanager.getBlockCollection(b));
+    }
+    
+    // make a change: create a new file under subsub
+    Path newFileAfterS0 = new Path(subsub, "newFile");
+    DFSTestUtil.createFile(hdfs, newFileAfterS0, BLOCKSIZE, REPLICATION, seed);
+    // further change: change the replicator factor of metaChangeFile
+    hdfs.setReplication(metaChangeFile1, REPLICATION_1);
+    hdfs.setReplication(metaChangeFile2, REPLICATION_1);
+    
+    // create snapshot s1
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s1");
+    // check dir's quota usage
+    checkQuotaUsageComputation(dir, 14L, BLOCKSIZE * REPLICATION * 4);
+    
+    // get two snapshots for later use
+    Snapshot snapshot0 = ((INodeDirectorySnapshottable) fsdir.getINode(dir
+        .toString())).getSnapshot(DFSUtil.string2Bytes("s0"));
+    Snapshot snapshot1 = ((INodeDirectorySnapshottable) fsdir.getINode(dir
+        .toString())).getSnapshot(DFSUtil.string2Bytes("s1"));
+    
+    // Case 2 + Case 3: delete noChangeDirParent, noChangeFile, and
+    // metaChangeFile2. Note that when we directly delete a directory, the 
+    // directory will be converted to an INodeDirectoryWithSnapshot. To make
+    // sure the deletion goes through an INodeDirectory, we delete the parent
+    // of noChangeDir
+    hdfs.delete(noChangeDirParent, true);
+    // while deletion, we add a diff for metaChangeFile2 as its snapshot copy
+    // for s1, we also add diffs for both sub and noChangeDirParent
+    checkQuotaUsageComputation(dir, 17L, BLOCKSIZE * REPLICATION * 4);
+    
+    // check the snapshot copy of noChangeDir 
+    Path snapshotNoChangeDir = SnapshotTestHelper.getSnapshotPath(dir, "s1",
+        sub.getName() + "/" + noChangeDirParent.getName() + "/"
+            + noChangeDir.getName());
+    INodeDirectory snapshotNode = 
+        (INodeDirectory) fsdir.getINode(snapshotNoChangeDir.toString());
+    // should still be an INodeDirectory
+    assertEquals(INodeDirectory.class, snapshotNode.getClass());
+    ReadOnlyList<INode> children = snapshotNode.getChildrenList(null);
+    // check 2 children: noChangeFile and metaChangeFile2
+    assertEquals(2, children.size());
+    INode noChangeFileSCopy = children.get(1);
+    assertEquals(noChangeFile.getName(), noChangeFileSCopy.getLocalName());
+    assertEquals(INodeFile.class, noChangeFileSCopy.getClass());
+    TestSnapshotBlocksMap.assertBlockCollection(new Path(snapshotNoChangeDir,
+        noChangeFileSCopy.getLocalName()).toString(), 1, fsdir, blockmanager);
+    
+    INodeFileWithSnapshot metaChangeFile2SCopy = 
+        (INodeFileWithSnapshot) children.get(0);
+    assertEquals(metaChangeFile2.getName(), metaChangeFile2SCopy.getLocalName());
+    assertEquals(INodeFileWithSnapshot.class, metaChangeFile2SCopy.getClass());
+    TestSnapshotBlocksMap.assertBlockCollection(new Path(snapshotNoChangeDir,
+        metaChangeFile2SCopy.getLocalName()).toString(), 1, fsdir, blockmanager);
+    
+    // check the replication factor of metaChangeFile2SCopy
+    assertEquals(REPLICATION_1,
+        metaChangeFile2SCopy.getFileReplication(null));
+    assertEquals(REPLICATION_1,
+        metaChangeFile2SCopy.getFileReplication(snapshot1));
+    assertEquals(REPLICATION,
+        metaChangeFile2SCopy.getFileReplication(snapshot0));
+    
+    // Case 4: delete directory sub
+    // before deleting sub, we first create a new file under sub
+    Path newFile = new Path(sub, "newFile");
+    DFSTestUtil.createFile(hdfs, newFile, BLOCKSIZE, REPLICATION, seed);
+    final INodeFile newFileNode = TestSnapshotBlocksMap.assertBlockCollection(
+        newFile.toString(), 1, fsdir, blockmanager);
+    blocks = newFileNode.getBlocks();
+    checkQuotaUsageComputation(dir, 18L, BLOCKSIZE * REPLICATION * 5);
+    hdfs.delete(sub, true);
+    // while deletion, we add diff for subsub and metaChangeFile1, and remove
+    // newFile
+    checkQuotaUsageComputation(dir, 19L, BLOCKSIZE * REPLICATION * 4);
+    for (BlockInfo b : blocks) {
+      assertNull(blockmanager.getBlockCollection(b));
+    }
+    
+    // make sure the whole subtree of sub is stored correctly in snapshot
+    Path snapshotSub = SnapshotTestHelper.getSnapshotPath(dir, "s1",
+        sub.getName());
+    INodeDirectoryWithSnapshot snapshotNode4Sub = 
+        (INodeDirectoryWithSnapshot) fsdir.getINode(snapshotSub.toString());
+    assertEquals(INodeDirectoryWithSnapshot.class, snapshotNode4Sub.getClass());
+    // the snapshot copy of sub has only one child subsub.
+    // newFile should have been destroyed
+    assertEquals(1, snapshotNode4Sub.getChildrenList(null).size());
+    // but should have two children, subsub and noChangeDir, when s1 was taken  
+    assertEquals(2, snapshotNode4Sub.getChildrenList(snapshot1).size());
+    
+    // check the snapshot copy of subsub, which is contained in the subtree of
+    // sub's snapshot copy
+    INode snapshotNode4Subsub = snapshotNode4Sub.getChildrenList(null).get(0);
+    assertEquals(INodeDirectoryWithSnapshot.class,
+        snapshotNode4Subsub.getClass());
+    assertTrue(snapshotNode4Sub == snapshotNode4Subsub.getParent());
+    // check the children of subsub
+    INodeDirectory snapshotSubsubDir = (INodeDirectory) snapshotNode4Subsub;
+    children = snapshotSubsubDir.getChildrenList(null);
+    assertEquals(2, children.size());
+    assertEquals(children.get(0).getLocalName(), metaChangeFile1.getName());
+    assertEquals(children.get(1).getLocalName(), newFileAfterS0.getName());
+    // only one child before snapshot s0 
+    children = snapshotSubsubDir.getChildrenList(snapshot0);
+    assertEquals(1, children.size());
+    INode child = children.get(0);
+    assertEquals(child.getLocalName(), metaChangeFile1.getName());
+    // check snapshot copy of metaChangeFile1
+    assertEquals(INodeFileWithSnapshot.class, child.getClass());
+    INodeFileWithSnapshot metaChangeFile1SCopy = (INodeFileWithSnapshot) child;
+    assertEquals(REPLICATION_1,
+        metaChangeFile1SCopy.getFileReplication(null));
+    assertEquals(REPLICATION_1,
+        metaChangeFile1SCopy.getFileReplication(snapshot1));
+    assertEquals(REPLICATION,
+        metaChangeFile1SCopy.getFileReplication(snapshot0));
+  }
+  
+  /**
+   * Test deleting the earliest (first) snapshot. In this simplest scenario, the 
+   * snapshots are taken on the same directory, and we do not need to combine
+   * snapshot diffs.
+   */
+  @Test (timeout=300000)
+  public void testDeleteEarliestSnapshot1() throws Exception {
+    // create files under sub
+    Path file0 = new Path(sub, "file0");
+    Path file1 = new Path(sub, "file1");
+    DFSTestUtil.createFile(hdfs, file0, BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, file1, BLOCKSIZE, REPLICATION, seed);
+    
+    String snapshotName = "s1";
+    try {
+      hdfs.deleteSnapshot(sub, snapshotName);
+      fail("SnapshotException expected: " + sub.toString()
+          + " is not snapshottable yet");
+    } catch (Exception e) {
+      GenericTestUtils.assertExceptionContains(
+          "Directory is not a snapshottable directory: " + sub, e);
+    }
+    
+    // make sub snapshottable
+    hdfs.allowSnapshot(sub);
+    try {
+      hdfs.deleteSnapshot(sub, snapshotName);
+      fail("SnapshotException expected: snapshot " + snapshotName
+          + " does not exist for " + sub.toString());
+    } catch (Exception e) {
+      GenericTestUtils.assertExceptionContains("Cannot delete snapshot "
+          + snapshotName + " from path " + sub.toString()
+          + ": the snapshot does not exist.", e);
+    }
+    
+    // create snapshot s1 for sub
+    SnapshotTestHelper.createSnapshot(hdfs, sub, snapshotName);
+    // check quota usage computation
+    checkQuotaUsageComputation(sub, 4, BLOCKSIZE * REPLICATION * 2);
+    // delete s1
+    hdfs.deleteSnapshot(sub, snapshotName);
+    checkQuotaUsageComputation(sub, 3, BLOCKSIZE * REPLICATION * 2);
+    // now we can create a snapshot with the same name
+    hdfs.createSnapshot(sub, snapshotName);
+    checkQuotaUsageComputation(sub, 4, BLOCKSIZE * REPLICATION * 2);
+    
+    // create a new file under sub
+    Path newFile = new Path(sub, "newFile");
+    DFSTestUtil.createFile(hdfs, newFile, BLOCKSIZE, REPLICATION, seed);
+    // create another snapshot s2
+    String snapshotName2 = "s2";
+    hdfs.createSnapshot(sub, snapshotName2);
+    checkQuotaUsageComputation(sub, 6, BLOCKSIZE * REPLICATION * 3);
+    // Get the filestatus of sub under snapshot s2
+    Path ss = SnapshotTestHelper
+        .getSnapshotPath(sub, snapshotName2, "newFile");
+    FileStatus statusBeforeDeletion = hdfs.getFileStatus(ss);
+    // delete s1
+    hdfs.deleteSnapshot(sub, snapshotName);
+    checkQuotaUsageComputation(sub, 5, BLOCKSIZE * REPLICATION * 3);
+    FileStatus statusAfterDeletion = hdfs.getFileStatus(ss);
+    System.out.println("Before deletion: " + statusBeforeDeletion.toString()
+        + "\n" + "After deletion: " + statusAfterDeletion.toString());
+    assertEquals(statusBeforeDeletion.toString(),
+        statusAfterDeletion.toString());
+  }
+  
+  /**
+   * Test deleting the earliest (first) snapshot. In this more complicated 
+   * scenario, the snapshots are taken across directories.
+   * <pre>
+   * The test covers the following scenarios:
+   * 1. delete the first diff in the diff list of a directory
+   * 2. delete the first diff in the diff list of a file
+   * </pre>
+   * Also, the recursive cleanTree process should cover both INodeFile and 
+   * INodeDirectory.
+   */
+  @Test (timeout=300000)
+  public void testDeleteEarliestSnapshot2() throws Exception {
+    Path noChangeDir = new Path(sub, "noChangeDir");
+    Path noChangeFile = new Path(noChangeDir, "noChangeFile");
+    Path metaChangeFile = new Path(noChangeDir, "metaChangeFile");
+    Path metaChangeDir = new Path(noChangeDir, "metaChangeDir");
+    Path toDeleteFile = new Path(metaChangeDir, "toDeleteFile");
+    DFSTestUtil.createFile(hdfs, noChangeFile, BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, metaChangeFile, BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, toDeleteFile, BLOCKSIZE, REPLICATION, seed);
+    
+    final INodeFile toDeleteFileNode = TestSnapshotBlocksMap
+        .assertBlockCollection(toDeleteFile.toString(), 1, fsdir, blockmanager);
+    BlockInfo[] blocks = toDeleteFileNode.getBlocks();
+    
+    // create snapshot s0 on dir
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
+    checkQuotaUsageComputation(dir, 8, 3 * BLOCKSIZE * REPLICATION);
+    
+    // delete /TestSnapshot/sub/noChangeDir/metaChangeDir/toDeleteFile
+    hdfs.delete(toDeleteFile, true);
+    // the deletion adds diff of toDeleteFile and metaChangeDir
+    checkQuotaUsageComputation(dir, 10, 3 * BLOCKSIZE * REPLICATION);
+    // change metadata of /TestSnapshot/sub/noChangeDir/metaChangeDir and
+    // /TestSnapshot/sub/noChangeDir/metaChangeFile
+    hdfs.setReplication(metaChangeFile, REPLICATION_1);
+    hdfs.setOwner(metaChangeDir, "unknown", "unknown");
+    checkQuotaUsageComputation(dir, 11, 3 * BLOCKSIZE * REPLICATION);
+    
+    // create snapshot s1 on dir
+    hdfs.createSnapshot(dir, "s1");
+    checkQuotaUsageComputation(dir, 12, 3 * BLOCKSIZE * REPLICATION);
+    
+    // delete snapshot s0
+    hdfs.deleteSnapshot(dir, "s0");
+    // namespace: remove toDeleteFile and its diff, metaChangeFile's diff, 
+    // metaChangeDir's diff, dir's diff. diskspace: remove toDeleteFile, and 
+    // metaChangeFile's replication factor decreases
+    checkQuotaUsageComputation(dir, 7, 2 * BLOCKSIZE * REPLICATION - BLOCKSIZE);
+    for (BlockInfo b : blocks) {
+      assertNull(blockmanager.getBlockCollection(b));
+    }
+    
+    // check 1. there is no snapshot s0
+    final INodeDirectorySnapshottable dirNode = 
+        (INodeDirectorySnapshottable) fsdir.getINode(dir.toString());
+    Snapshot snapshot0 = dirNode.getSnapshot(DFSUtil.string2Bytes("s0"));
+    assertNull(snapshot0);
+    DirectoryDiffList diffList = dirNode.getDiffs();
+    assertEquals(1, diffList.asList().size());
+    assertEquals("s1", diffList.getLast().snapshot.getRoot().getLocalName());
+    diffList = ((INodeDirectoryWithSnapshot) fsdir.getINode(
+        metaChangeDir.toString())).getDiffs();
+    assertEquals(0, diffList.asList().size());
+    
+    // check 2. noChangeDir and noChangeFile are still there
+    final INodeDirectory noChangeDirNode = 
+        (INodeDirectory) fsdir.getINode(noChangeDir.toString());
+    assertEquals(INodeDirectory.class, noChangeDirNode.getClass());
+    final INodeFile noChangeFileNode = 
+        (INodeFile) fsdir.getINode(noChangeFile.toString());
+    assertEquals(INodeFile.class, noChangeFileNode.getClass());
+    TestSnapshotBlocksMap.assertBlockCollection(noChangeFile.toString(), 1,
+        fsdir, blockmanager);
+    
+    // check 3: current metadata of metaChangeFile and metaChangeDir
+    FileStatus status = hdfs.getFileStatus(metaChangeDir);
+    assertEquals("unknown", status.getOwner());
+    assertEquals("unknown", status.getGroup());
+    status = hdfs.getFileStatus(metaChangeFile);
+    assertEquals(REPLICATION_1, status.getReplication());
+    TestSnapshotBlocksMap.assertBlockCollection(metaChangeFile.toString(), 1,
+        fsdir, blockmanager);
+    
+    // check 4: no snapshot copy for toDeleteFile
+    try {
+      status = hdfs.getFileStatus(toDeleteFile);
+      fail("should throw FileNotFoundException");
+    } catch (FileNotFoundException e) {
+      GenericTestUtils.assertExceptionContains("File does not exist: "
+          + toDeleteFile.toString(), e);
+    }
+    
+    final Path toDeleteFileInSnapshot = SnapshotTestHelper.getSnapshotPath(dir,
+        "s0", toDeleteFile.toString().substring(dir.toString().length()));
+    try {
+      status = hdfs.getFileStatus(toDeleteFileInSnapshot);
+      fail("should throw FileNotFoundException");
+    } catch (FileNotFoundException e) {
+      GenericTestUtils.assertExceptionContains("File does not exist: "
+          + toDeleteFileInSnapshot.toString(), e);
+    }
+  }
+  
+  /**
+   * Test deleting snapshots in a more complicated scenario: need to combine
+   * snapshot diffs, but no need to handle diffs distributed in a dir tree
+   */
+  @Test (timeout=300000)
+  public void testCombineSnapshotDiff1() throws Exception {
+    testCombineSnapshotDiffImpl(sub, "", 1);
+  }
+  
+  /**
+   * Test deleting snapshots in more complicated scenarios (snapshot diffs are
+   * distributed in the directory sub-tree)
+   */
+  @Test (timeout=300000)
+  public void testCombineSnapshotDiff2() throws Exception {
+    testCombineSnapshotDiffImpl(sub, "subsub1/subsubsub1/", 3);
+  }
+  
+  /**
+   * When combine two snapshots, make sure files/directories created after the 
+   * prior snapshot get destroyed.
+   */
+  @Test (timeout=300000)
+  public void testCombineSnapshotDiff3() throws Exception {
+    // create initial dir and subdir
+    Path dir = new Path("/dir");
+    Path subDir1 = new Path(dir, "subdir1");
+    Path subDir2 = new Path(dir, "subdir2");
+    hdfs.mkdirs(subDir2);
+    Path subsubDir = new Path(subDir1, "subsubdir");
+    hdfs.mkdirs(subsubDir);
+    
+    // take snapshots on subdir and dir
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s1");
+    
+    // create new dir under initial dir
+    Path newDir = new Path(subsubDir, "newdir");
+    Path newFile = new Path(newDir, "newfile");
+    DFSTestUtil.createFile(hdfs, newFile, BLOCKSIZE, REPLICATION, seed);
+    Path newFile2 = new Path(subDir2, "newfile");
+    DFSTestUtil.createFile(hdfs, newFile2, BLOCKSIZE, REPLICATION, seed);
+    
+    // create another snapshot
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s2");
+    
+    checkQuotaUsageComputation(dir, 11, BLOCKSIZE * 2 * REPLICATION);
+    
+    // delete subsubdir and subDir2
+    hdfs.delete(subsubDir, true);
+    hdfs.delete(subDir2, true);
+    
+    // add diff of s2 to subDir1, subsubDir, and subDir2
+    checkQuotaUsageComputation(dir, 14, BLOCKSIZE * 2 * REPLICATION);
+    
+    // delete snapshot s2
+    hdfs.deleteSnapshot(dir, "s2");
+    
+    // delete s2 diff in dir, subDir2, and subsubDir. Delete newFile, newDir,
+    // and newFile2. Rename s2 diff to s1 for subDir1 
+    checkQuotaUsageComputation(dir, 8, 0);
+    // Check rename of snapshot diff in subDir1
+    Path subdir1_s1 = SnapshotTestHelper.getSnapshotPath(dir, "s1",
+        subDir1.getName());
+    Path subdir1_s2 = SnapshotTestHelper.getSnapshotPath(dir, "s2",
+        subDir1.getName());
+    assertTrue(hdfs.exists(subdir1_s1));
+    assertFalse(hdfs.exists(subdir1_s2));
+  }
+  
+  /**
+   * Test snapshot deletion
+   * @param snapshotRoot The dir where the snapshots are created
+   * @param modDirStr The snapshotRoot itself or one of its sub-directory, 
+   *        where the modifications happen. It is represented as a relative 
+   *        path to the snapshotRoot.
+   */
+  private void testCombineSnapshotDiffImpl(Path snapshotRoot, String modDirStr,
+      int dirNodeNum) throws Exception {
+    Path modDir = modDirStr.isEmpty() ? snapshotRoot : new Path(snapshotRoot,
+        modDirStr);
+    final int delta = modDirStr.isEmpty() ? 0 : 1;
+    Path file10 = new Path(modDir, "file10");
+    Path file11 = new Path(modDir, "file11");
+    Path file12 = new Path(modDir, "file12");
+    Path file13 = new Path(modDir, "file13");
+    Path file14 = new Path(modDir, "file14");
+    Path file15 = new Path(modDir, "file15");
+    DFSTestUtil.createFile(hdfs, file10, BLOCKSIZE, REPLICATION_1, seed);
+    DFSTestUtil.createFile(hdfs, file11, BLOCKSIZE, REPLICATION_1, seed);
+    DFSTestUtil.createFile(hdfs, file12, BLOCKSIZE, REPLICATION_1, seed);
+    DFSTestUtil.createFile(hdfs, file13, BLOCKSIZE, REPLICATION_1, seed);
+
+    // create snapshot s1 for snapshotRoot
+    SnapshotTestHelper.createSnapshot(hdfs, snapshotRoot, "s1");
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 5, 8 * BLOCKSIZE);
+    
+    // delete file11
+    hdfs.delete(file11, true);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 6 + delta,
+        8 * BLOCKSIZE);
+    
+    // modify file12
+    hdfs.setReplication(file12, REPLICATION);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 7 + delta,
+        9 * BLOCKSIZE);
+    
+    // modify file13
+    hdfs.setReplication(file13, REPLICATION);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 8 + delta,
+        10 * BLOCKSIZE);
+    
+    // create file14
+    DFSTestUtil.createFile(hdfs, file14, BLOCKSIZE, REPLICATION, seed);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 9 + delta,
+        13 * BLOCKSIZE);
+    
+    // create file15
+    DFSTestUtil.createFile(hdfs, file15, BLOCKSIZE, REPLICATION, seed);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 10 + delta,
+        16 * BLOCKSIZE);
+    
+    // create snapshot s2 for snapshotRoot
+    hdfs.createSnapshot(snapshotRoot, "s2");
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 11 + delta,
+        16 * BLOCKSIZE);
+    
+    // create file11 again: (0, d) + (c, 0)
+    DFSTestUtil.createFile(hdfs, file11, BLOCKSIZE, REPLICATION, seed);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 12 + delta * 2,
+        19 * BLOCKSIZE);
+    
+    // delete file12
+    hdfs.delete(file12, true);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 13 + delta * 2,
+        19 * BLOCKSIZE);
+    
+    // modify file13
+    hdfs.setReplication(file13, (short) (REPLICATION - 2));
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 14 + delta * 2,
+        19 * BLOCKSIZE);
+    
+    // delete file14: (c, 0) + (0, d)
+    hdfs.delete(file14, true);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 15 + delta * 2,
+        19 * BLOCKSIZE);
+    
+    // modify file15
+    hdfs.setReplication(file15, REPLICATION_1);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 16 + delta * 2,
+        19 * BLOCKSIZE);
+    
+    // create snapshot s3 for snapshotRoot
+    hdfs.createSnapshot(snapshotRoot, "s3");
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 17 + delta * 2,
+        19 * BLOCKSIZE);
+    
+    // modify file10, to check if the posterior diff was set correctly
+    hdfs.setReplication(file10, REPLICATION);
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 18 + delta * 2,
+        20 * BLOCKSIZE);
+    
+    Path file10_s1 = SnapshotTestHelper.getSnapshotPath(snapshotRoot, "s1",
+        modDirStr + "file10");
+    Path file11_s1 = SnapshotTestHelper.getSnapshotPath(snapshotRoot, "s1",
+        modDirStr + "file11");
+    Path file12_s1 = SnapshotTestHelper.getSnapshotPath(snapshotRoot, "s1",
+        modDirStr + "file12");
+    Path file13_s1 = SnapshotTestHelper.getSnapshotPath(snapshotRoot, "s1",
+        modDirStr + "file13");
+    Path file14_s2 = SnapshotTestHelper.getSnapshotPath(snapshotRoot, "s2",
+        modDirStr + "file14");
+    Path file15_s2 = SnapshotTestHelper.getSnapshotPath(snapshotRoot, "s2",
+        modDirStr + "file15");
+    FileStatus statusBeforeDeletion10 = hdfs.getFileStatus(file10_s1);
+    FileStatus statusBeforeDeletion11 = hdfs.getFileStatus(file11_s1);
+    FileStatus statusBeforeDeletion12 = hdfs.getFileStatus(file12_s1);
+    FileStatus statusBeforeDeletion13 = hdfs.getFileStatus(file13_s1);
+    INodeFile file14Node = TestSnapshotBlocksMap.assertBlockCollection(
+        file14_s2.toString(), 1, fsdir, blockmanager);
+    BlockInfo[] blocks_14 = file14Node.getBlocks();
+    TestSnapshotBlocksMap.assertBlockCollection(file15_s2.toString(), 1, fsdir,
+        blockmanager);
+    
+    // delete s2, in which process we need to combine the diff in s2 to s1
+    hdfs.deleteSnapshot(snapshotRoot, "s2");
+    checkQuotaUsageComputation(snapshotRoot, dirNodeNum + 12 + delta,
+        14 * BLOCKSIZE);
+    
+    // check the correctness of s1
+    FileStatus statusAfterDeletion10 = hdfs.getFileStatus(file10_s1);
+    FileStatus statusAfterDeletion11 = hdfs.getFileStatus(file11_s1);
+    FileStatus statusAfterDeletion12 = hdfs.getFileStatus(file12_s1);
+    FileStatus statusAfterDeletion13 = hdfs.getFileStatus(file13_s1);
+    assertEquals(statusBeforeDeletion10.toString(),
+        statusAfterDeletion10.toString());
+    assertEquals(statusBeforeDeletion11.toString(),
+        statusAfterDeletion11.toString());
+    assertEquals(statusBeforeDeletion12.toString(),
+        statusAfterDeletion12.toString());
+    assertEquals(statusBeforeDeletion13.toString(),
+        statusAfterDeletion13.toString());
+    TestSnapshotBlocksMap.assertBlockCollection(file10_s1.toString(), 1, fsdir,
+        blockmanager);
+    TestSnapshotBlocksMap.assertBlockCollection(file11_s1.toString(), 1, fsdir,
+        blockmanager);
+    TestSnapshotBlocksMap.assertBlockCollection(file12_s1.toString(), 1, fsdir,
+        blockmanager);
+    TestSnapshotBlocksMap.assertBlockCollection(file13_s1.toString(), 1, fsdir,
+        blockmanager);
+    
+    // make sure file14 and file15 are not included in s1
+    Path file14_s1 = SnapshotTestHelper.getSnapshotPath(snapshotRoot, "s1",
+        modDirStr + "file14");
+    Path file15_s1 = SnapshotTestHelper.getSnapshotPath(snapshotRoot, "s1",
+        modDirStr + "file15");
+    assertFalse(hdfs.exists(file14_s1));
+    assertFalse(hdfs.exists(file15_s1));
+    for (BlockInfo b : blocks_14) {
+      assertNull(blockmanager.getBlockCollection(b));
+    }
+    
+    INodeFile nodeFile13 = (INodeFile) fsdir.getINode(file13.toString());
+    assertEquals(REPLICATION_1, nodeFile13.getBlockReplication());
+    TestSnapshotBlocksMap.assertBlockCollection(file13.toString(), 1, fsdir,
+        blockmanager);
+    
+    INodeFile nodeFile12 = (INodeFile) fsdir.getINode(file12_s1.toString());
+    assertEquals(REPLICATION_1, nodeFile12.getBlockReplication());
+  }
+  
+  /** Test deleting snapshots with modification on the metadata of directory */ 
+  @Test (timeout=300000)
+  public void testDeleteSnapshotWithDirModification() throws Exception {
+    Path file = new Path(sub, "file");
+    DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, seed);
+    hdfs.setOwner(sub, "user1", "group1");
+    
+    // create snapshot s1 for sub1, and change the metadata of sub1
+    SnapshotTestHelper.createSnapshot(hdfs, sub, "s1");
+    checkQuotaUsageComputation(sub, 3, BLOCKSIZE * 3);
+    hdfs.setOwner(sub, "user2", "group2");
+    checkQuotaUsageComputation(sub, 3, BLOCKSIZE * 3);
+    
+    // create snapshot s2 for sub1, but do not modify sub1 afterwards
+    hdfs.createSnapshot(sub, "s2");
+    checkQuotaUsageComputation(sub, 4, BLOCKSIZE * 3);
+    
+    // create snapshot s3 for sub1, and change the metadata of sub1
+    hdfs.createSnapshot(sub, "s3");
+    checkQuotaUsageComputation(sub, 5, BLOCKSIZE * 3);
+    hdfs.setOwner(sub, "user3", "group3");
+    checkQuotaUsageComputation(sub, 5, BLOCKSIZE * 3);
+    
+    // delete snapshot s3
+    hdfs.deleteSnapshot(sub, "s3");
+    checkQuotaUsageComputation(sub, 4, BLOCKSIZE * 3);
+    
+    // check sub1's metadata in snapshot s2
+    FileStatus statusOfS2 = hdfs.getFileStatus(new Path(sub,
+        HdfsConstants.DOT_SNAPSHOT_DIR + "/s2"));
+    assertEquals("user2", statusOfS2.getOwner());
+    assertEquals("group2", statusOfS2.getGroup());
+    
+    // delete snapshot s2
+    hdfs.deleteSnapshot(sub, "s2");
+    checkQuotaUsageComputation(sub, 3, BLOCKSIZE * 3);
+    
+    // check sub1's metadata in snapshot s1
+    FileStatus statusOfS1 = hdfs.getFileStatus(new Path(sub,
+        HdfsConstants.DOT_SNAPSHOT_DIR + "/s1"));
+    assertEquals("user1", statusOfS1.getOwner());
+    assertEquals("group1", statusOfS1.getGroup());
+  }
+  
+  /** 
+   * A test covering the case where the snapshot diff to be deleted is renamed 
+   * to its previous snapshot. 
+   */
+  @Test (timeout=300000)
+  public void testRenameSnapshotDiff() throws Exception {
+    cluster.getNamesystem().getSnapshotManager().setAllowNestedSnapshots(true);
+
+    final Path subFile0 = new Path(sub, "file0");
+    final Path subsubFile0 = new Path(subsub, "file0");
+    DFSTestUtil.createFile(hdfs, subFile0, BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, subsubFile0, BLOCKSIZE, REPLICATION, seed);
+    hdfs.setOwner(subsub, "owner", "group");
+    
+    // create snapshot s0 on sub
+    SnapshotTestHelper.createSnapshot(hdfs, sub, "s0");
+    checkQuotaUsageComputation(sub, 5, BLOCKSIZE * 6);
+    // make some changes on both sub and subsub
+    final Path subFile1 = new Path(sub, "file1");
+    final Path subsubFile1 = new Path(subsub, "file1");
+    DFSTestUtil.createFile(hdfs, subFile1, BLOCKSIZE, REPLICATION_1, seed);
+    DFSTestUtil.createFile(hdfs, subsubFile1, BLOCKSIZE, REPLICATION, seed);
+    checkQuotaUsageComputation(sub, 8, BLOCKSIZE * 11);
+    
+    // create snapshot s1 on sub
+    SnapshotTestHelper.createSnapshot(hdfs, sub, "s1");
+    checkQuotaUsageComputation(sub, 9, BLOCKSIZE * 11);
+    
+    // create snapshot s2 on dir
+    SnapshotTestHelper.createSnapshot(hdfs, dir, "s2");
+    checkQuotaUsageComputation(dir, 11, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(sub, 9, BLOCKSIZE * 11);
+    
+    // make changes on subsub and subsubFile1
+    hdfs.setOwner(subsub, "unknown", "unknown");
+    hdfs.setReplication(subsubFile1, REPLICATION_1);
+    checkQuotaUsageComputation(dir, 13, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(sub, 11, BLOCKSIZE * 11);
+    
+    // make changes on sub
+    hdfs.delete(subFile1, true);
+    checkQuotaUsageComputation(new Path("/"), 16, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(dir, 15, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(sub, 13, BLOCKSIZE * 11);
+    
+    Path subsubSnapshotCopy = SnapshotTestHelper.getSnapshotPath(dir, "s2",
+        sub.getName() + Path.SEPARATOR + subsub.getName());
+    Path subsubFile1SCopy = SnapshotTestHelper.getSnapshotPath(dir, "s2",
+        sub.getName() + Path.SEPARATOR + subsub.getName() + Path.SEPARATOR
+            + subsubFile1.getName());
+    Path subFile1SCopy = SnapshotTestHelper.getSnapshotPath(dir, "s2",
+        sub.getName() + Path.SEPARATOR + subFile1.getName());
+    FileStatus subsubStatus = hdfs.getFileStatus(subsubSnapshotCopy);
+    assertEquals("owner", subsubStatus.getOwner());
+    assertEquals("group", subsubStatus.getGroup());
+    FileStatus subsubFile1Status = hdfs.getFileStatus(subsubFile1SCopy);
+    assertEquals(REPLICATION, subsubFile1Status.getReplication());
+    FileStatus subFile1Status = hdfs.getFileStatus(subFile1SCopy);
+    assertEquals(REPLICATION_1, subFile1Status.getReplication());
+    
+    // delete snapshot s2
+    hdfs.deleteSnapshot(dir, "s2");
+    checkQuotaUsageComputation(new Path("/"), 14, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(dir, 13, BLOCKSIZE * 11);
+    checkQuotaUsageComputation(sub, 12, BLOCKSIZE * 11);
+    
+    // no snapshot copy for s2
+    try {
+      hdfs.getFileStatus(subsubSnapshotCopy);
+      fail("should throw FileNotFoundException");
+    } catch (FileNotFoundException e) {
+      GenericTestUtils.assertExceptionContains("File does not exist: "
+          + subsubSnapshotCopy.toString(), e);
+    }
+    try {
+      hdfs.getFileStatus(subsubFile1SCopy);
+      fail("should throw FileNotFoundException");
+    } catch (FileNotFoundException e) {
+      GenericTestUtils.assertExceptionContains("File does not exist: "
+          + subsubFile1SCopy.toString(), e);
+    }
+    try {
+      hdfs.getFileStatus(subFile1SCopy);
+      fail("should throw FileNotFoundException");
+    } catch (FileNotFoundException e) {
+      GenericTestUtils.assertExceptionContains("File does not exist: "
+          + subFile1SCopy.toString(), e);
+    }
+    
+    // the snapshot copy of s2 should now be renamed to s1 under sub
+    subsubSnapshotCopy = SnapshotTestHelper.getSnapshotPath(sub, "s1",
+        subsub.getName());
+    subsubFile1SCopy = SnapshotTestHelper.getSnapshotPath(sub, "s1",
+        subsub.getName() + Path.SEPARATOR + subsubFile1.getName());
+    subFile1SCopy = SnapshotTestHelper.getSnapshotPath(sub, "s1",
+        subFile1.getName());
+    subsubStatus = hdfs.getFileStatus(subsubSnapshotCopy);
+    assertEquals("owner", subsubStatus.getOwner());
+    assertEquals("group", subsubStatus.getGroup());
+    subsubFile1Status = hdfs.getFileStatus(subsubFile1SCopy);
+    assertEquals(REPLICATION, subsubFile1Status.getReplication());
+    // also subFile1's snapshot copy should have been moved to diff of s1 as 
+    // combination
+    subFile1Status = hdfs.getFileStatus(subFile1SCopy);
+    assertEquals(REPLICATION_1, subFile1Status.getReplication());
+  }
+}

+ 305 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java

@@ -0,0 +1,305 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.HashMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Tests snapshot deletion.
+ */
+public class TestSnapshotDiffReport {
+  protected static final long seed = 0;
+  protected static final short REPLICATION = 3;
+  protected static final short REPLICATION_1 = 2;
+  protected static final long BLOCKSIZE = 1024;
+  public static final int SNAPSHOTNUMBER = 10;
+  
+  private final Path dir = new Path("/TestSnapshot");
+  private final Path sub1 = new Path(dir, "sub1");
+  
+  protected Configuration conf;
+  protected MiniDFSCluster cluster;
+  protected DistributedFileSystem hdfs;
+  
+  private HashMap<Path, Integer> snapshotNumberMap = new HashMap<Path, Integer>();
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new Configuration();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
+        .format(true).build();
+    cluster.waitActive();
+    hdfs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+  
+  private String genSnapshotName(Path snapshotDir) {
+    int sNum = -1;
+    if (snapshotNumberMap.containsKey(snapshotDir)) {
+      sNum = snapshotNumberMap.get(snapshotDir);
+    }
+    snapshotNumberMap.put(snapshotDir, ++sNum);
+    return "s" + sNum;
+  }
+  
+  /**
+   * Create/modify/delete files under a given directory, also create snapshots
+   * of directories.
+   */ 
+  private void modifyAndCreateSnapshot(Path modifyDir, Path[] snapshotDirs)
+      throws Exception {
+    Path file10 = new Path(modifyDir, "file10");
+    Path file11 = new Path(modifyDir, "file11");
+    Path file12 = new Path(modifyDir, "file12");
+    Path file13 = new Path(modifyDir, "file13");
+    Path file14 = new Path(modifyDir, "file14");
+    Path file15 = new Path(modifyDir, "file15");
+    DFSTestUtil.createFile(hdfs, file10, BLOCKSIZE, REPLICATION_1, seed);
+    DFSTestUtil.createFile(hdfs, file11, BLOCKSIZE, REPLICATION_1, seed);
+    DFSTestUtil.createFile(hdfs, file12, BLOCKSIZE, REPLICATION_1, seed);
+    DFSTestUtil.createFile(hdfs, file13, BLOCKSIZE, REPLICATION_1, seed);
+    // create snapshot
+    for (Path snapshotDir : snapshotDirs) {
+      hdfs.allowSnapshot(snapshotDir);
+      hdfs.createSnapshot(snapshotDir, genSnapshotName(snapshotDir));
+    }
+    
+    // delete file11
+    hdfs.delete(file11, true);
+    // modify file12
+    hdfs.setReplication(file12, REPLICATION);
+    // modify file13
+    hdfs.setReplication(file13, REPLICATION);
+    // create file14
+    DFSTestUtil.createFile(hdfs, file14, BLOCKSIZE, REPLICATION, seed);
+    // create file15
+    DFSTestUtil.createFile(hdfs, file15, BLOCKSIZE, REPLICATION, seed);
+    
+    // create snapshot
+    for (Path snapshotDir : snapshotDirs) {
+      hdfs.createSnapshot(snapshotDir, genSnapshotName(snapshotDir));
+    }
+    
+    // create file11 again
+    DFSTestUtil.createFile(hdfs, file11, BLOCKSIZE, REPLICATION, seed);
+    // delete file12
+    hdfs.delete(file12, true);
+    // modify file13
+    hdfs.setReplication(file13, (short) (REPLICATION - 2));
+    // delete file14
+    hdfs.delete(file14, true);
+    // modify file15
+    hdfs.setReplication(file15, (short) (REPLICATION - 1));
+    
+    // create snapshot
+    for (Path snapshotDir : snapshotDirs) {
+      hdfs.createSnapshot(snapshotDir, genSnapshotName(snapshotDir));
+    }
+    // modify file10
+    hdfs.setReplication(file10, (short) (REPLICATION - 1));
+  }
+  
+  /** check the correctness of the diff reports */
+  private void verifyDiffReport(Path dir, String from, String to,
+      DiffReportEntry... entries) throws IOException {
+    SnapshotDiffReport report = hdfs.getSnapshotDiffReport(dir, from, to);
+    // reverse the order of from and to
+    SnapshotDiffReport inverseReport = hdfs
+        .getSnapshotDiffReport(dir, to, from);
+    System.out.println(report.toString());
+    System.out.println(inverseReport.toString() + "\n");
+    
+    assertEquals(entries.length, report.getDiffList().size());
+    assertEquals(entries.length, inverseReport.getDiffList().size());
+    
+    for (DiffReportEntry entry : entries) {
+      if (entry.getType() == DiffType.MODIFY) {
+        assertTrue(report.getDiffList().contains(entry));
+        assertTrue(inverseReport.getDiffList().contains(entry));
+      } else if (entry.getType() == DiffType.DELETE) {
+        assertTrue(report.getDiffList().contains(entry));
+        assertTrue(inverseReport.getDiffList().contains(
+            new DiffReportEntry(DiffType.CREATE, entry.getRelativePath())));
+      } else if (entry.getType() == DiffType.CREATE) {
+        assertTrue(report.getDiffList().contains(entry));
+        assertTrue(inverseReport.getDiffList().contains(
+            new DiffReportEntry(DiffType.DELETE, entry.getRelativePath())));
+      }
+    }
+  }
+  
+  /** Test the computation and representation of diff between snapshots */
+  @Test (timeout=60000)
+  public void testDiffReport() throws Exception {
+    cluster.getNamesystem().getSnapshotManager().setAllowNestedSnapshots(true);
+
+    Path subsub1 = new Path(sub1, "subsub1");
+    Path subsubsub1 = new Path(subsub1, "subsubsub1");
+    hdfs.mkdirs(subsubsub1);
+    modifyAndCreateSnapshot(sub1, new Path[]{sub1, subsubsub1});
+    modifyAndCreateSnapshot(subsubsub1, new Path[]{sub1, subsubsub1});
+    
+    try {
+      hdfs.getSnapshotDiffReport(subsub1, "s1", "s2");
+      fail("Expect exception when getting snapshot diff report: " + subsub1
+          + " is not a snapshottable directory.");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains(
+          "Directory is not a snapshottable directory: " + subsub1, e);
+    }
+    
+    final String invalidName = "invalid";
+    try {
+      hdfs.getSnapshotDiffReport(sub1, invalidName, invalidName);
+      fail("Expect exception when providing invalid snapshot name for diff report");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains(
+          "Cannot find the snapshot of directory " + sub1 + " with name "
+              + invalidName, e);
+    }
+    
+    // diff between the same snapshot
+    SnapshotDiffReport report = hdfs.getSnapshotDiffReport(sub1, "s0", "s0");
+    System.out.println(report);
+    assertEquals(0, report.getDiffList().size());
+    
+    report = hdfs.getSnapshotDiffReport(sub1, "", "");
+    System.out.println(report);
+    assertEquals(0, report.getDiffList().size());
+    
+    report = hdfs.getSnapshotDiffReport(subsubsub1, "s0", "s2");
+    System.out.println(report);
+    assertEquals(0, report.getDiffList().size());
+    
+    verifyDiffReport(sub1, "s0", "s2", 
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("file15")),
+        new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("file12")),
+        new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("file11")),
+        new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("file11")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("file13")));
+
+    verifyDiffReport(sub1, "s0", "s5", 
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("file15")),
+        new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("file12")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("file10")),
+        new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("file11")),
+        new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("file11")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("file13")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1/subsubsub1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file10")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file11")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file15")));
+    
+    verifyDiffReport(sub1, "s2", "s5",
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("file10")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1/subsubsub1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file10")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file11")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file15")));
+    
+    verifyDiffReport(sub1, "s3", "",
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1/subsubsub1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file15")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file12")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file10")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file11")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file11")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file13")));
+  }
+  
+  /**
+   * Make changes under a sub-directory, then delete the sub-directory. Make
+   * sure the diff report computation correctly retrieve the diff from the
+   * deleted sub-directory.
+   */
+  @Test (timeout=60000)
+  public void testDiffReport2() throws Exception {
+    Path subsub1 = new Path(sub1, "subsub1");
+    Path subsubsub1 = new Path(subsub1, "subsubsub1");
+    hdfs.mkdirs(subsubsub1);
+    modifyAndCreateSnapshot(subsubsub1, new Path[]{sub1});
+    
+    // delete subsub1
+    hdfs.delete(subsub1, true);
+    // check diff report between s0 and s2
+    verifyDiffReport(sub1, "s0", "s2", 
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1/subsubsub1")), 
+        new DiffReportEntry(DiffType.CREATE, 
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file15")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file12")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file11")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file11")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file13")));
+    // check diff report between s0 and the current status
+    verifyDiffReport(sub1, "s0", "", 
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("subsub1")));
+  }
+  
+}

+ 120 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotListing.java

@@ -0,0 +1,120 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestSnapshotListing {
+
+  static final long seed = 0;
+  static final short REPLICATION = 3;
+  static final long BLOCKSIZE = 1024;
+
+  private final Path dir = new Path("/test.snapshot/dir");
+  
+  Configuration conf;
+  MiniDFSCluster cluster;
+  FSNamesystem fsn;
+  DistributedFileSystem hdfs;
+  
+  @Before
+  public void setUp() throws Exception {
+    conf = new Configuration();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
+        .build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+    hdfs.mkdirs(dir);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+  
+  /**
+   * Test listing snapshots under a snapshottable directory
+   */
+  @Test (timeout=15000)
+  public void testListSnapshots() throws Exception {
+    final Path snapshotsPath = new Path(dir, ".snapshot");
+    FileStatus[] stats = null;
+    
+    // special case: snapshots of root
+    stats = hdfs.listStatus(new Path("/.snapshot"));
+    // should be 0 since root's snapshot quota is 0
+    assertEquals(0, stats.length);
+    
+    // list before set dir as snapshottable
+    try {
+      stats = hdfs.listStatus(snapshotsPath);
+      fail("expect SnapshotException");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains(
+          "Directory is not a snapshottable directory: " + dir.toString(), e);
+    }
+    
+    // list before creating snapshots
+    hdfs.allowSnapshot(dir);
+    stats = hdfs.listStatus(snapshotsPath);
+    assertEquals(0, stats.length);
+    
+    // list while creating snapshots
+    final int snapshotNum = 5;
+    for (int sNum = 0; sNum < snapshotNum; sNum++) {
+      hdfs.createSnapshot(dir, "s_" + sNum);
+      stats = hdfs.listStatus(snapshotsPath);
+      assertEquals(sNum + 1, stats.length);
+      for (int i = 0; i <= sNum; i++) {
+        assertEquals("s_" + i, stats[i].getPath().getName());
+      }
+    }
+    
+    // list while deleting snapshots
+    for (int sNum = snapshotNum - 1; sNum > 0; sNum--) {
+      hdfs.deleteSnapshot(dir, "s_" + sNum);
+      stats = hdfs.listStatus(snapshotsPath);
+      assertEquals(sNum, stats.length);
+      for (int i = 0; i < sNum; i++) {
+        assertEquals("s_" + i, stats[i].getPath().getName());
+      }
+    }
+    
+    // remove the last snapshot
+    hdfs.deleteSnapshot(dir, "s_0");
+    stats = hdfs.listStatus(snapshotsPath);
+    assertEquals(0, stats.length);
+  }
+}

+ 80 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotManager.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 org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.junit.*;
+import static org.mockito.Mockito.*;
+
+
+/**
+ * Testing snapshot manager functionality.
+ */
+public class TestSnapshotManager {
+  private static final int testMaxSnapshotLimit = 7;
+
+  /**
+   * Test that the global limit on snapshots is honored.
+   */
+  @Test (timeout=10000)
+  public void testSnapshotLimits() throws Exception {
+    // Setup mock objects for SnapshotManager.createSnapshot.
+    //
+    INodeDirectorySnapshottable ids = mock(INodeDirectorySnapshottable.class);
+    FSDirectory fsdir = mock(FSDirectory.class);
+
+    SnapshotManager sm = spy(new SnapshotManager(fsdir));
+    doReturn(ids).when(sm).getSnapshottableRoot(anyString());
+    doReturn(testMaxSnapshotLimit).when(sm).getMaxSnapshotID();
+
+    // Create testMaxSnapshotLimit snapshots. These should all succeed.
+    //
+    for (Integer i = 0; i < testMaxSnapshotLimit; ++i) {
+      sm.createSnapshot("dummy", i.toString());
+    }
+
+    // Attempt to create one more snapshot. This should fail due to snapshot
+    // ID rollover.
+    //
+    try {
+      sm.createSnapshot("dummy", "shouldFailSnapshot");
+      Assert.fail("Expected SnapshotException not thrown");
+    } catch (SnapshotException se) {
+      Assert.assertTrue(
+          se.getMessage().toLowerCase().contains("rollover"));
+    }
+
+    // Delete a snapshot to free up a slot.
+    //
+    sm.deleteSnapshot("", "", mock(INode.BlocksMapUpdateInfo.class), new ArrayList<INode>());
+
+    // Attempt to create a snapshot again. It should still fail due
+    // to snapshot ID rollover.
+    //
+    try {
+      sm.createSnapshot("dummy", "shouldFailSnapshot2");
+      Assert.fail("Expected SnapshotException not thrown");
+    } catch (SnapshotException se) {
+      Assert.assertTrue(
+          se.getMessage().toLowerCase().contains("rollover"));
+    }
+  }
+}

Неке датотеке нису приказане због велике количине промена