Browse Source

HDFS-4414. Add support for getting snapshot diff from DistributedFileSystem. Contributed by Jing Zhao.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1441808 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 12 years ago
parent
commit
24d96cbcdb
19 changed files with 668 additions and 182 deletions
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt
  2. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  3. 15 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  4. 34 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  5. 173 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java
  6. 23 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  7. 19 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  8. 75 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  9. 12 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  10. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  11. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java
  12. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
  13. 36 24
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectorySnapshottable.java
  14. 35 35
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java
  15. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
  16. 12 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  17. 19 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
  18. 36 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
  19. 141 82
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java

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

@@ -146,3 +146,6 @@ Branch-2802 Snapshot (Unreleased)
 
   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)

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

@@ -111,6 +111,7 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 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.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -2065,6 +2066,18 @@ public class DFSClient implements java.io.Closeable {
     namenode.disallowSnapshot(snapshotRoot);
   }
   
+  /**
+   * 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();
+    return namenode.getSnapshotDiffReport(snapshotDir.toString(), fromSnapshot,
+        toSnapshot);
+  }
+  
   /**
    * Save namespace image.
    * 

+ 15 - 4
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.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.BlockStorageLocation;
 import org.apache.hadoop.fs.ContentSummary;
 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.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -46,6 +45,7 @@ import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.VolumeId;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
@@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 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.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -917,9 +918,8 @@ public class DistributedFileSystem extends FileSystem {
   
   /**
    * Disallow snapshot on a directory.
-   * 
    * @param path the snapshottable directory.
-   * @throws IOException
+   * @throws IOException on error
    */
   public void disallowSnapshot(String path) throws IOException {
     dfs.disallowSnapshot(path);
@@ -957,4 +957,15 @@ public class DistributedFileSystem extends FileSystem {
       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);
+  }
 }

+ 34 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -24,15 +24,18 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.InvalidPathException;
 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.InvalidPathException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.FsPermission;
 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.SafeModeException;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotAccessControlException;
@@ -43,9 +46,6 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.token.Token;
 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 
@@ -1006,20 +1006,38 @@ public interface ClientProtocol {
   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
-     */
+  /**
+   * 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
-     */
+  /**
+   * 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;
 }
 

+ 173 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.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.hdfs.protocol;
+
+import java.util.Collections;
+import java.util.List;
+
+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 full path of the file/directory where changes have happened */
+    private final String fullPath;
+
+    public DiffReportEntry(DiffType type, String fullPath) {
+      this.type = type;
+      this.fullPath = fullPath;
+    }
+    
+    @Override
+    public String toString() {
+      return type.getLabel() + "\t" + fullPath;
+    }
+    
+    public DiffType getType() {
+      return type;
+    }
+
+    public String getFullPath() {
+      return fullPath;
+    }
+
+    @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())
+            && fullPath.equals(entry.getFullPath());
+      }
+      return false;
+    }
+    
+    @Override
+    public int hashCode() {
+      return fullPath.hashCode();
+    }
+  }
+  
+  /** 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("Diffence between snapshot " + from + " and " + to
+        + " under directory " + snapshotRoot + ":" + LINE_SEPARATOR);
+    for (DiffReportEntry entry : diffList) {
+      str.append(entry.toString() + LINE_SEPARATOR);
+    }
+    return str.toString();
+  }
+}

+ 23 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 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.AbandonBlockResponseProto;
@@ -62,13 +63,13 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncR
 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.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.GetBlockLocationsResponseProto;
 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.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.GetDatanodeReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
@@ -85,6 +86,8 @@ 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.GetServerDefaultsRequestProto;
 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.ListCorruptFileBlocksRequestProto;
@@ -170,6 +173,9 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       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 = 
   CreateResponseProto.newBuilder().build();
@@ -940,8 +946,6 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     }
   }
 
-  static final GetSnapshottableDirListingResponseProto NULL_GET_SNAPSHOTTABLE_DIR_LISTING_RESPONSE 
-             = GetSnapshottableDirListingResponseProto.newBuilder().build();
   @Override
   public GetSnapshottableDirListingResponseProto getSnapshottableDirListing(
       RpcController controller, GetSnapshottableDirListingRequestProto request)
@@ -959,4 +963,19 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       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);
+    }
+  }
 }

+ 19 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -80,6 +81,8 @@ 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.GetPreferredBlockSizeRequestProto;
 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.ListCorruptFileBlocksRequestProto;
@@ -936,4 +939,20 @@ public class ClientNamenodeProtocolTranslatorPB implements
       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);
+    }
+  }
 }

+ 75 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -30,24 +30,25 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
-import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 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.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 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.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 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.HdfsProtos;
 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.GetFsStatsResponseProto;
@@ -65,7 +66,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.RegisterCommandProto;
 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.BlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
@@ -74,6 +75,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.ContentSummaryProto;
 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.DatanodeInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState;
@@ -90,19 +92,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.NamenodeCommandProto;
 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.RecoveringBlockProto;
 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.NamenodeRegistrationProto.NamenodeRoleProto;
 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.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.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.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
@@ -124,15 +128,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.JournalInfo;
 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.NamenodeRegistration;
 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.BlockStatus;
 import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 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.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
@@ -1373,8 +1378,7 @@ public class PBHelper {
       SnapshottableDirectoryStatus[] result = 
           new SnapshottableDirectoryStatus[list.size()];
       for (int i = 0; i < list.size(); i++) {
-        result[i] = (SnapshottableDirectoryStatus) PBHelper
-            .convert(list.get(i));
+        result[i] = PBHelper.convert(list.get(i));
       }
       return result;
     }
@@ -1393,6 +1397,69 @@ public class PBHelper {
     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 ? new DiffReportEntry(type, entry.getFullpath())
+        : null;
+  }
+  
+  public static SnapshotDiffReportEntryProto convert(DiffReportEntry entry) {
+    if (entry == null) {
+      return null;
+    }
+    String fullPath = entry.getFullPath();
+    String modification = entry.getType().getLabel();
+    
+    SnapshotDiffReportEntryProto entryProto = SnapshotDiffReportEntryProto
+        .newBuilder().setFullpath(fullPath).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) {
     return DataChecksum.Type.valueOf(type.getNumber());

+ 12 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -138,8 +138,10 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 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.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -178,7 +180,7 @@ 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.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable.SnapshotDiffInfo;
 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;
@@ -5819,7 +5821,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
   
   /**
-   * Compute the difference between two snapshots (or between a snapshot and the
+   * 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.
@@ -5827,13 +5829,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    *          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 The difference between {@code fromSnapshot} and {@code toSnapshot},
-   *         i.e., applying difference to source will get target.
+   * @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
    */
   public SnapshotDiffReport getSnapshotDiffReport(String path,
       String fromSnapshot, String toSnapshot) throws IOException {
-    SnapshotDiffReport diffs = null;
+    SnapshotDiffInfo diffs = null;
     readLock();
     try {
       checkOperation(OperationCategory.READ);
@@ -5846,7 +5850,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       logAuditEvent(UserGroupInformation.getCurrentUser(), getRemoteIp(),
             "computeSnapshotDiff", null, null, null);
     }
-    return diffs;
+    return diffs != null ? diffs.generateReport() : new SnapshotDiffReport(
+        path, fromSnapshot, toSnapshot,
+        Collections.<DiffReportEntry> emptyList());
   }
   
   /**

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

@@ -57,6 +57,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 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.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -1143,4 +1144,13 @@ class NameNodeRpcServer implements NamenodeProtocols {
     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;
+  }
 }

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

@@ -69,6 +69,8 @@ public class NameNodeMetrics {
   MutableCounterLong renameSnapshotOps;
   @Metric("Number of listSnapshottableDirectory operations")
   MutableCounterLong listSnapshottableDirOps;
+  @Metric("Number of snapshotDiffReport operations")
+  MutableCounterLong snapshotDiffReportOps;
 
   @Metric("Journal transactions") MutableRate transactions;
   @Metric("Journal syncs") MutableRate syncs;
@@ -195,6 +197,10 @@ public class NameNodeMetrics {
     listSnapshottableDirOps.incr();
   }
   
+  public void incrSnapshotDiffReportOps() {
+    snapshotDiffReportOps.incr();
+  }
+  
   public void addTransaction(long latency) {
     transactions.add(latency);
   }

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

@@ -70,6 +70,10 @@ abstract class AbstractINodeDiffList<N extends INode,
       if (snapshotIndex > 0) {
         // combine the to-be-removed diff with its previous diff
         final AbstractINodeDiff<N, D> previous = diffs.get(snapshotIndex - 1);
+        if (previous.snapshotINode == null) {
+          // TODO: add a new testcase for this
+          previous.snapshotINode = removed.snapshotINode;
+        }
         previous.combinePosteriorAndCollectBlocks(removed, collectedBlocks);
         previous.setPosterior(removed.getPosterior());
       }

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

@@ -31,6 +31,9 @@ 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.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.diff.Diff;
@@ -65,7 +68,7 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
    * A class describing the difference between snapshots of a snapshottable
    * directory.
    */
-  public static class SnapshotDiffReport {
+  public static class SnapshotDiffInfo {
     public static final Comparator<INode> INODE_COMPARATOR = 
         new Comparator<INode>() {
       @Override
@@ -92,7 +95,7 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
      */
     private final SortedMap<INodeDirectoryWithSnapshot, ChildrenDiff> diffMap;
     
-    public SnapshotDiffReport(INodeDirectorySnapshottable snapshotRoot,
+    public SnapshotDiffInfo(INodeDirectorySnapshottable snapshotRoot,
         Snapshot start, Snapshot end) {
       this.snapshotRoot = snapshotRoot;
       this.from = start;
@@ -106,28 +109,37 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
       diffMap.put(dir, diff);
     }
     
+    /** 
+     * 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
+     */
+    private static String getSnapshotName(Snapshot s) {
+      return s != null ? s.getRoot().getLocalName() : "";
+    }
+    
+    /** @return True if {@link #from} is earlier than {@link #to} */
+    private boolean isFromEarlier() {
+      return to == null
+          || (from != null && Snapshot.ID_COMPARATOR.compare(from, to) < 0);
+    }
+    
     /**
-     * dump the diff
+     * Generate a {@link SnapshotDiffReport} based on detailed diff information.
+     * @return A {@link SnapshotDiffReport} describing the difference
      */
-    public String dump() {
-      StringBuilder strBuffer = new StringBuilder();
-      String fromStr = from == null ? "current directory" : "snapshot "
-          + from.getRoot().getLocalName();
-      String toStr = to == null ? "current directory" : "snapshot "
-          + to.getRoot().getLocalName();
-      strBuffer.append("Diffence between snapshot " + fromStr + " and " + toStr
-          + " under directory " + snapshotRoot.getFullPathName() + ":\n");
-      
-      if (!diffMap.isEmpty()) {
-        for (Map.Entry<INodeDirectoryWithSnapshot, ChildrenDiff> entry : diffMap
-            .entrySet()) {
-          strBuffer.append("M\t" + entry.getKey().getFullPathName() + "\n");
-          entry.getValue().printDiff(strBuffer, entry.getKey(),
-              from == null || 
-              (to != null && Snapshot.ID_COMPARATOR.compare(from, to) > 0));
-        }
+    public SnapshotDiffReport generateReport() {
+      List<DiffReportEntry> diffList = new ArrayList<DiffReportEntry>();
+      for (Map.Entry<INodeDirectoryWithSnapshot, ChildrenDiff> entry : diffMap
+          .entrySet()) {
+        diffList.add(new DiffReportEntry(DiffType.MODIFY, entry.getKey()
+            .getFullPathName()));
+        List<DiffReportEntry> subList = entry.getValue().generateReport(
+            entry.getKey(), isFromEarlier());
+        diffList.addAll(subList);
       }
-      return strBuffer.toString();
+      return new SnapshotDiffReport(snapshotRoot.getFullPathName(),
+          getSnapshotName(from), getSnapshotName(to), diffList);
     }
   }
 
@@ -297,11 +309,11 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
    *           point, or if endSnapshotName is not null but cannot be identified
    *           as a previous snapshot.
    */
-  SnapshotDiffReport computeDiff(final String from, final String to)
+  SnapshotDiffInfo computeDiff(final String from, final String to)
       throws SnapshotException {
     Snapshot fromSnapshot = getSnapshotByName(from);
     Snapshot toSnapshot = getSnapshotByName(to); 
-    SnapshotDiffReport diffs = new SnapshotDiffReport(this, fromSnapshot,
+    SnapshotDiffInfo diffs = new SnapshotDiffInfo(this, fromSnapshot,
         toSnapshot);
     computeDiffInDir(this, diffs);
     return diffs;
@@ -336,7 +348,7 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
    * @param diffReport data structure used to store the diff.
    */
   private void computeDiffInDir(INodeDirectory dir,
-      SnapshotDiffReport diffReport) {
+      SnapshotDiffInfo diffReport) {
     ChildrenDiff diff = new ChildrenDiff();
     if (dir instanceof INodeDirectoryWithSnapshot) {
       boolean change = ((INodeDirectoryWithSnapshot) dir)

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

@@ -28,6 +28,8 @@ import java.util.Map;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
@@ -119,56 +121,53 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     }
     
     /**
-     * Print out the content of the Diff. In general, "M"/"+"/"-" are used to 
-     * represent files/directories that were modified, created, and deleted, 
-     * respectively.
-     * 
-     * @param str A StringBuilder used to storing the description of the Diff.
-     * @param parent The directory that the Diff is about. Used to get the full 
-     *               path of the INodes contained in the Diff.             
+     * Interpret the diff and generate a list of {@link DiffReportEntry}.
+     * @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 void printDiff(StringBuilder str, INodeDirectoryWithSnapshot parent,
-        boolean reverse) {
-      final String mStr = "M\t";
-      final String cStr = reverse ? "-\t" : "+\t";
-      final String dStr = reverse ? "+\t" : "-\t";
-      StringBuilder cBuffer = new StringBuilder();
-      StringBuilder dBuffer = new StringBuilder();
-      StringBuilder mBuffer = new StringBuilder();
-      int c = 0;
-      int d = 0;
+    public List<DiffReportEntry> generateReport(
+        INodeDirectoryWithSnapshot parent, boolean fromEarlier) {
+      List<DiffReportEntry> mList = new ArrayList<DiffReportEntry>();
+      List<DiffReportEntry> cList = new ArrayList<DiffReportEntry>();
+      List<DiffReportEntry> dList = new ArrayList<DiffReportEntry>();
+      int c = 0, d = 0;
       List<INode> created = getCreatedList();
       List<INode> deleted = getDeletedList();
       for (; c < created.size() && d < deleted.size(); ) {
         INode cnode = created.get(c);
         INode dnode = deleted.get(d);
         if (cnode.equals(dnode)) {
-          mBuffer.append(mStr + parent.getFullPathName() + Path.SEPARATOR
-              + cnode.getLocalName() + "\n");
+          mList.add(new DiffReportEntry(DiffType.MODIFY, parent
+              .getFullPathName() + Path.SEPARATOR + cnode.getLocalName()));
           c++;
           d++;
         } else if (cnode.compareTo(dnode.getLocalNameBytes()) < 0) {
-          cBuffer.append(cStr + parent.getFullPathName() + Path.SEPARATOR
-              + cnode.getLocalName() + "\n");
+          cList.add(new DiffReportEntry(fromEarlier ? DiffType.CREATE
+              : DiffType.DELETE, parent.getFullPathName() + Path.SEPARATOR
+              + cnode.getLocalName()));
           c++;
         } else {
-          dBuffer.append(dStr + parent.getFullPathName() + Path.SEPARATOR
-              + dnode.getLocalName() + "\n");
+          dList.add(new DiffReportEntry(fromEarlier ? DiffType.DELETE
+              : DiffType.CREATE, parent.getFullPathName() + Path.SEPARATOR
+              + dnode.getLocalName()));
           d++;
         }
-      }   
+      }
       for (; d < deleted.size(); d++) {
-        dBuffer.append(dStr + parent.getFullPathName() + Path.SEPARATOR
-            + deleted.get(d).getLocalName() + "\n");
-      }    
+        dList.add(new DiffReportEntry(fromEarlier ? DiffType.DELETE
+            : DiffType.CREATE, parent.getFullPathName() + Path.SEPARATOR
+            + deleted.get(d).getLocalName()));
+      }
       for (; c < created.size(); c++) {
-        cBuffer.append(cStr + parent.getFullPathName() + Path.SEPARATOR
-            + created.get(c).getLocalName() + "\n");
+        cList.add(new DiffReportEntry(fromEarlier ? DiffType.CREATE
+            : DiffType.DELETE, parent.getFullPathName() + Path.SEPARATOR
+            + created.get(c).getLocalName()));
       }
-      
-      str.append(cBuffer);
-      str.append(dBuffer);
-      str.append(mBuffer);
+      cList.addAll(dList);
+      cList.addAll(mList);
+      return cList;
     }
   }
   
@@ -322,7 +321,8 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
   }
 
   /** A list of directory diffs. */
-  class DirectoryDiffList extends AbstractINodeDiffList<INodeDirectory, DirectoryDiff> {
+  class DirectoryDiffList extends
+      AbstractINodeDiffList<INodeDirectory, DirectoryDiff> {
     @Override
     INodeDirectoryWithSnapshot getCurrentINode() {
       return INodeDirectoryWithSnapshot.this;
@@ -530,7 +530,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     }
     return removed;
   }
-
+  
   @Override
   public ReadOnlyList<INode> getChildrenList(Snapshot snapshot) {
     final DirectoryDiff diff = diffs.getDiff(snapshot);

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

@@ -32,7 +32,7 @@ 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.INodeDirectory.INodesInPath;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable.SnapshotDiffInfo;
 
 /**
  * Manage snapshottable directories and their snapshots.
@@ -249,7 +249,7 @@ public class SnapshotManager implements SnapshotStats {
    * Compute the difference between two snapshots of a directory, or between a
    * snapshot of the directory and its current tree.
    */
-  public SnapshotDiffReport diff(final String path, final String from,
+  public SnapshotDiffInfo diff(final String path, final String from,
       final String to) throws IOException {
     if ((from == null || from.isEmpty())
         && (to == null || to.isEmpty())) {

+ 12 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -215,6 +215,15 @@ 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 {
   required string clientName = 1;
 }
@@ -552,5 +561,7 @@ service ClientNamenodeProtocol {
   rpc getSnapshottableDirListing(GetSnapshottableDirListingRequestProto)
       returns(GetSnapshottableDirListingResponseProto);
   rpc deleteSnapshot(DeleteSnapshotRequestProto)
-      returns(DeleteSnapshotResponseProto);     
+      returns(DeleteSnapshotResponseProto);
+  rpc getSnapshotDiffReport(GetSnapshotDiffReportRequestProto)
+      returns(GetSnapshotDiffReportResponseProto);
 }

+ 19 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto

@@ -227,6 +227,25 @@ message SnapshottableDirectoryListingProto {
   repeated SnapshottableDirectoryStatusProto snapshottableDirListing = 1;
 }
 
+/**
+ * Snapshot diff report entry
+ */
+message SnapshotDiffReportEntryProto {
+  required string 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
  */

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

@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path;
 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.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.ipc.RemoteException;
@@ -312,4 +313,39 @@ public class TestSnapshotDeletion {
     assertEquals(REPLICATION - 1, blockReplicationFile12);
   }
   
+  /** Test deleting snapshots with modification on the metadata of directory */ 
+  @Test
+  public void testDeleteSnapshotWithDirModification() throws Exception {
+    Path file = new Path(sub1, "file");
+    DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, seed);
+    hdfs.setOwner(sub1, "user1", "group1");
+    
+    // create snapshot s1 for sub1, and change the metadata of sub1
+    SnapshotTestHelper.createSnapshot(hdfs, sub1, "s1");
+    hdfs.setOwner(sub1, "user2", "group2");
+    
+    // create snapshot s2 for sub1, but do not modify sub1 afterwards
+    hdfs.createSnapshot(sub1, "s2");
+    
+    // create snapshot s3 for sub1, and change the metadata of sub1
+    hdfs.createSnapshot(sub1, "s3");
+    hdfs.setOwner(sub1, "user3", "group3");
+    
+    // delete snapshot s3
+    hdfs.deleteSnapshot(sub1, "s3");
+    // check sub1's metadata in snapshot s2
+    FileStatus statusOfS2 = hdfs.getFileStatus(new Path(sub1,
+        HdfsConstants.DOT_SNAPSHOT_DIR + "/s2"));
+    assertEquals("user2", statusOfS2.getOwner());
+    assertEquals("group2", statusOfS2.getGroup());
+    
+    // delete snapshot s2
+    hdfs.deleteSnapshot(sub1, "s2");
+    // check sub1's metadata in snapshot s1
+    FileStatus statusOfS1 = hdfs.getFileStatus(new Path(sub1,
+        HdfsConstants.DOT_SNAPSHOT_DIR + "/s1"));
+    assertEquals("user1", statusOfS1.getOwner());
+    assertEquals("group1", statusOfS1.getGroup());
+  }
+  
 }

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

@@ -17,16 +17,22 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
-import static org.junit.Assert.assertFalse;
+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.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable.SnapshotDiffReport;
+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;
@@ -45,10 +51,9 @@ public class TestSnapshotDiffReport {
   
   protected Configuration conf;
   protected MiniDFSCluster cluster;
-  protected FSNamesystem fsn;
   protected DistributedFileSystem hdfs;
   
-  private int snapshotNum = 0;
+  private HashMap<Path, Integer> snapshotNumberMap = new HashMap<Path, Integer>();
 
   @Before
   public void setUp() throws Exception {
@@ -56,8 +61,6 @@ public class TestSnapshotDiffReport {
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
         .format(true).build();
     cluster.waitActive();
-
-    fsn = cluster.getNamesystem();
     hdfs = cluster.getFileSystem();
   }
 
@@ -68,10 +71,20 @@ public class TestSnapshotDiffReport {
     }
   }
   
-  /** 
-   * Create/modify/delete files and create snapshots under a given directory. 
+  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 snapshotDir)
+  private void modifyAndCreateSnapshot(Path modifyDir, Path[] snapshotDirs)
       throws Exception {
     Path file10 = new Path(modifyDir, "file10");
     Path file11 = new Path(modifyDir, "file11");
@@ -87,9 +100,11 @@ public class TestSnapshotDiffReport {
         seed);
     DFSTestUtil.createFile(hdfs, file13, BLOCKSIZE, (short) (REPLICATION - 1),
         seed);
-    // create snapshot s1
-    hdfs.allowSnapshot(snapshotDir.toString());
-    hdfs.createSnapshot(snapshotDir, "s" + snapshotNum++);
+    // create snapshot
+    for (Path snapshotDir : snapshotDirs) {
+      hdfs.allowSnapshot(snapshotDir.toString());
+      hdfs.createSnapshot(snapshotDir, genSnapshotName(snapshotDir));
+    }
     
     // delete file11
     hdfs.delete(file11, true);
@@ -102,8 +117,10 @@ public class TestSnapshotDiffReport {
     // create file15
     DFSTestUtil.createFile(hdfs, file15, BLOCKSIZE, REPLICATION, seed);
     
-    // create snapshot s2
-    hdfs.createSnapshot(snapshotDir, "s" + snapshotNum++);
+    // create snapshot
+    for (Path snapshotDir : snapshotDirs) {
+      hdfs.createSnapshot(snapshotDir, genSnapshotName(snapshotDir));
+    }
     
     // create file11 again
     DFSTestUtil.createFile(hdfs, file11, BLOCKSIZE, REPLICATION, seed);
@@ -116,83 +133,125 @@ public class TestSnapshotDiffReport {
     // modify file15
     hdfs.setReplication(file15, (short) (REPLICATION - 1));
     
-    // create snapshot s3 for dir
-    hdfs.createSnapshot(snapshotDir, "s" + snapshotNum++);
+    // create snapshot
+    for (Path snapshotDir : snapshotDirs) {
+      hdfs.createSnapshot(snapshotDir, genSnapshotName(snapshotDir));
+    }
     // modify file10
     hdfs.setReplication(file10, (short) (REPLICATION - 1));
   }
   
-  /**
-   * Test the functionality of
-   * {@link FSNamesystem#getSnapshotDiffReport(String, String, String)}.
-   * TODO: without the definision of a DiffReport class, this test temporarily 
-   * check the output string of {@link SnapshotDiffReport#dump()} 
-   */
-  @Test
-  public void testDiff() throws Exception {
-    modifyAndCreateSnapshot(sub1, sub1);
-    modifyAndCreateSnapshot(new Path(sub1, "subsub1/subsubsub1"), sub1);
+  /** 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");
     
-    SnapshotDiffReport diffs = fsn.getSnapshotDiffReport(sub1.toString(), "s0", "s2");
-    String diffStr = diffs.dump();
-    System.out.println(diffStr);
-
-    assertTrue(diffStr.contains("M\t/TestSnapshot/sub1"));
-    assertTrue(diffStr.contains("+\t/TestSnapshot/sub1/file15"));
-    assertTrue(diffStr.contains("-\t/TestSnapshot/sub1/file12"));
-    assertTrue(diffStr.contains("M\t/TestSnapshot/sub1/file11"));
-    assertTrue(diffStr.contains("M\t/TestSnapshot/sub1/file13"));
-    assertFalse(diffStr.contains("file14"));
-
-    diffs = fsn.getSnapshotDiffReport(sub1.toString(), "s0", "s5");
-    diffStr = diffs.dump();
-    System.out.println(diffStr);
+    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.getFullPath())));
+      } else if (entry.getType() == DiffType.CREATE) {
+        assertTrue(report.getDiffList().contains(entry));
+        assertTrue(inverseReport.getDiffList().contains(
+            new DiffReportEntry(DiffType.DELETE, entry.getFullPath())));
+      }
+    }
+  }
+  
+  /** Test the computation and representation of diff between snapshots */
+  @Test
+  public void testDiffReport() throws Exception {
+    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});
     
-    assertTrue(diffStr.contains("M\t/TestSnapshot/sub1"));
-    assertTrue(diffStr.contains("+\t/TestSnapshot/sub1/file15"));
-    assertTrue(diffStr.contains("+\t/TestSnapshot/sub1/subsub1"));
-    assertTrue(diffStr.contains("-\t/TestSnapshot/sub1/file12"));
-    assertTrue(diffStr.contains("M\t/TestSnapshot/sub1/file10"));
-    assertTrue(diffStr.contains("M\t/TestSnapshot/sub1/file11"));
-    assertTrue(diffStr.contains("M\t/TestSnapshot/sub1/file13"));
-    assertFalse(diffStr.contains("file14"));
+    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);
+    }
     
-    diffs = fsn.getSnapshotDiffReport(sub1.toString(), "s0", "");
-    diffStr = diffs.dump();
-    System.out.println(diffStr);
+    // diff between the same snapshot
+    SnapshotDiffReport report = hdfs.getSnapshotDiffReport(sub1, "s0", "s0");
+    System.out.println(report);
+    assertEquals(0, report.getDiffList().size());
     
-    assertTrue(diffStr.contains("M\t/TestSnapshot/sub1"));
-    assertTrue(diffStr.contains("+\t/TestSnapshot/sub1/file15"));
-    assertTrue(diffStr.contains("+\t/TestSnapshot/sub1/subsub1"));
-    assertTrue(diffStr.contains("-\t/TestSnapshot/sub1/file12"));
-    assertTrue(diffStr.contains("M\t/TestSnapshot/sub1/file10"));
-    assertTrue(diffStr.contains("M\t/TestSnapshot/sub1/file11"));
-    assertTrue(diffStr.contains("M\t/TestSnapshot/sub1/file13"));
-    assertFalse(diffStr.contains("file14"));
+    report = hdfs.getSnapshotDiffReport(sub1, "", "");
+    System.out.println(report);
+    assertEquals(0, report.getDiffList().size());
     
-    diffs = fsn.getSnapshotDiffReport(sub1.toString(), "s2", "s5");
-    diffStr = diffs.dump();
-    System.out.println(diffStr);
+    report = hdfs.getSnapshotDiffReport(subsubsub1, "s0", "s2");
+    System.out.println(report);
+    assertEquals(0, report.getDiffList().size());
     
-    assertTrue(diffStr.contains("M\t/TestSnapshot/sub1"));
-    assertTrue(diffStr.contains("+\t/TestSnapshot/sub1/subsub1"));
-    assertTrue(diffStr.contains("M\t/TestSnapshot/sub1/file10"));
+    verifyDiffReport(sub1, "s0", "s2", 
+        new DiffReportEntry(DiffType.MODIFY, "/TestSnapshot/sub1"),
+        new DiffReportEntry(DiffType.CREATE, "/TestSnapshot/sub1/file15"),
+        new DiffReportEntry(DiffType.DELETE, "/TestSnapshot/sub1/file12"),
+        new DiffReportEntry(DiffType.MODIFY, "/TestSnapshot/sub1/file11"),
+        new DiffReportEntry(DiffType.MODIFY, "/TestSnapshot/sub1/file13"));
+
+    verifyDiffReport(sub1, "s0", "s5", 
+        new DiffReportEntry(DiffType.MODIFY, "/TestSnapshot/sub1"),
+        new DiffReportEntry(DiffType.CREATE, "/TestSnapshot/sub1/file15"),
+        new DiffReportEntry(DiffType.DELETE, "/TestSnapshot/sub1/file12"),
+        new DiffReportEntry(DiffType.MODIFY, "/TestSnapshot/sub1/file10"),
+        new DiffReportEntry(DiffType.MODIFY, "/TestSnapshot/sub1/file11"),
+        new DiffReportEntry(DiffType.MODIFY, "/TestSnapshot/sub1/file13"),
+        new DiffReportEntry(DiffType.MODIFY,
+            "/TestSnapshot/sub1/subsub1/subsubsub1"),
+        new DiffReportEntry(DiffType.CREATE,
+            "/TestSnapshot/sub1/subsub1/subsubsub1/file10"),
+        new DiffReportEntry(DiffType.CREATE,
+            "/TestSnapshot/sub1/subsub1/subsubsub1/file11"),
+        new DiffReportEntry(DiffType.CREATE,
+            "/TestSnapshot/sub1/subsub1/subsubsub1/file13"),
+        new DiffReportEntry(DiffType.CREATE,
+            "/TestSnapshot/sub1/subsub1/subsubsub1/file15"));
     
-    diffs = fsn.getSnapshotDiffReport(sub1.toString(), "s3", "");
-    diffStr = diffs.dump();
-    System.out.println(diffStr);
+    verifyDiffReport(sub1, "s2", "s5",
+        new DiffReportEntry(DiffType.MODIFY, "/TestSnapshot/sub1"),
+        new DiffReportEntry(DiffType.MODIFY, "/TestSnapshot/sub1/file10"),
+        new DiffReportEntry(DiffType.MODIFY,
+            "/TestSnapshot/sub1/subsub1/subsubsub1"),
+        new DiffReportEntry(DiffType.CREATE,
+            "/TestSnapshot/sub1/subsub1/subsubsub1/file10"),
+        new DiffReportEntry(DiffType.CREATE,
+            "/TestSnapshot/sub1/subsub1/subsubsub1/file11"),
+        new DiffReportEntry(DiffType.CREATE,
+            "/TestSnapshot/sub1/subsub1/subsubsub1/file13"),
+        new DiffReportEntry(DiffType.CREATE,
+            "/TestSnapshot/sub1/subsub1/subsubsub1/file15"));
     
-    assertTrue(diffStr.contains("M\t/TestSnapshot/sub1"));
-    assertTrue(diffStr.contains("M\t/TestSnapshot/sub1/subsub1/subsubsub1"));
-    assertTrue(diffStr
-        .contains("+\t/TestSnapshot/sub1/subsub1/subsubsub1/file15"));
-    assertTrue(diffStr
-        .contains("-\t/TestSnapshot/sub1/subsub1/subsubsub1/file12"));
-    assertTrue(diffStr
-        .contains("M\t/TestSnapshot/sub1/subsub1/subsubsub1/file10"));
-    assertTrue(diffStr
-        .contains("M\t/TestSnapshot/sub1/subsub1/subsubsub1/file11"));
-    assertTrue(diffStr
-        .contains("M\t/TestSnapshot/sub1/subsub1/subsubsub1/file13"));
+    verifyDiffReport(sub1, "s3", "",
+        new DiffReportEntry(DiffType.MODIFY,
+            "/TestSnapshot/sub1/subsub1/subsubsub1"),
+        new DiffReportEntry(DiffType.CREATE,
+            "/TestSnapshot/sub1/subsub1/subsubsub1/file15"),
+        new DiffReportEntry(DiffType.DELETE,
+            "/TestSnapshot/sub1/subsub1/subsubsub1/file12"),
+        new DiffReportEntry(DiffType.MODIFY,
+            "/TestSnapshot/sub1/subsub1/subsubsub1/file10"),
+        new DiffReportEntry(DiffType.MODIFY,
+            "/TestSnapshot/sub1/subsub1/subsubsub1/file11"),
+        new DiffReportEntry(DiffType.MODIFY,
+            "/TestSnapshot/sub1/subsub1/subsubsub1/file13"));
   }
-}
+}