Bläddra i källkod

HDFS-4131. Add capability to namenode to get snapshot diff. Contributed by Jing Zhao.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1440152 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 12 år sedan
förälder
incheckning
66ea187228

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

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

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

@@ -138,7 +138,6 @@ 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.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -146,6 +145,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 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.AccessMode;
@@ -178,6 +178,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.INodeFileUnderConstructionWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
@@ -5814,6 +5815,37 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
   }
   
+  /**
+   * Compute 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 The difference between {@code fromSnapshot} and {@code toSnapshot},
+   *         i.e., applying difference to source will get target.
+   * @throws IOException
+   */
+  public SnapshotDiffReport getSnapshotDiffReport(String path,
+      String fromSnapshot, String toSnapshot) throws IOException {
+    SnapshotDiffReport diffs = null;
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      diffs = snapshotManager.diff(path, fromSnapshot, toSnapshot);
+    } finally {
+      readUnlock();
+    }
+    
+    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      logAuditEvent(UserGroupInformation.getCurrentUser(), getRemoteIp(),
+            "computeSnapshotDiff", null, null, null);
+    }
+    return diffs;
+  }
+  
   /**
    * Delete a snapshot of a snapshottable directory
    * @param snapshotRoot The snapshottable directory

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

@@ -622,6 +622,17 @@ public class INodeDirectory extends INode {
     return total;
   }
   
+  /**
+   * Compare the metadata with another INodeDirectory
+   */
+  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());
+  }
+  
   /**
    * Used by
    * {@link INodeDirectory#getExistingPathINodes(byte[][], int, boolean)}.

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

@@ -21,14 +21,19 @@ import java.io.IOException;
 import java.io.PrintWriter;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Comparator;
 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.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.diff.Diff;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.util.Time;
 
@@ -55,6 +60,76 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
     }
     return (INodeDirectorySnapshottable)dir;
   }
+  
+  /**
+   * A class describing the difference between snapshots of a snapshottable
+   * directory.
+   */
+  public static class SnapshotDiffReport {
+    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 {
+          return right == null ? 1 : left.compareTo(right.getLocalNameBytes());
+        }
+      }
+    };
+    
+    /** 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 capturing the detailed difference. Each key indicates a directory
+     * whose metadata or children have been changed between the two snapshots,
+     * while its associated value is a {@link Diff} storing the changes happened
+     * to the children (files).
+     */
+    private final SortedMap<INodeDirectoryWithSnapshot, ChildrenDiff> diffMap;
+    
+    public SnapshotDiffReport(INodeDirectorySnapshottable snapshotRoot,
+        Snapshot start, Snapshot end) {
+      this.snapshotRoot = snapshotRoot;
+      this.from = start;
+      this.to = end;
+      this.diffMap = new TreeMap<INodeDirectoryWithSnapshot, ChildrenDiff>(
+          INODE_COMPARATOR);
+    }
+    
+    /** Add a dir-diff pair into {@link #diffMap} */
+    public void addDiff(INodeDirectoryWithSnapshot dir, ChildrenDiff diff) {
+      diffMap.put(dir, diff);
+    }
+    
+    /**
+     * dump the diff
+     */
+    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));
+        }
+      }
+      return strBuffer.toString();
+    }
+  }
 
   /**
    * Snapshots of this directory in ascending order of snapshot names.
@@ -229,6 +304,80 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
     }
   }
 
+  /**
+   * 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.
+   */
+  SnapshotDiffReport computeDiff(final String from, final String to)
+      throws SnapshotException {
+    Snapshot fromSnapshot = getSnapshotByName(from);
+    Snapshot toSnapshot = getSnapshotByName(to); 
+    SnapshotDiffReport diffs = new SnapshotDiffReport(this, fromSnapshot,
+        toSnapshot);
+    computeDiffInDir(this, 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.
+   * @param dir The directory under which the diff is computed.
+   * @param diffReport data structure used to store the diff.
+   */
+  private void computeDiffInDir(INodeDirectory dir,
+      SnapshotDiffReport diffReport) {
+    ChildrenDiff diff = new ChildrenDiff();
+    if (dir instanceof INodeDirectoryWithSnapshot) {
+      boolean change = ((INodeDirectoryWithSnapshot) dir)
+          .computeDiffBetweenSnapshots(diffReport.from,
+              diffReport.to, diff);
+      if (change) {
+        diffReport.addDiff((INodeDirectoryWithSnapshot) dir,
+            diff); 
+      }
+    }
+    ReadOnlyList<INode> children = dir.getChildrenList(null);
+    for (INode child : children) {
+      if (child instanceof INodeDirectory
+          && diff.searchCreated(child.getLocalNameBytes()) == null) {
+        // Compute diff recursively for children that are directories. We do not
+        // need to compute diff for those contained in the created list since 
+        // directory contained in the created list must be new created.
+        computeDiffInDir((INodeDirectory) child, diffReport);
+      }
+    }
+  }
+  
   /**
    * Replace itself with {@link INodeDirectoryWithSnapshot} or
    * {@link INodeDirectory} depending on the latest snapshot.

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

@@ -25,6 +25,7 @@ import java.util.Iterator;
 import java.util.List;
 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.server.namenode.FSImageSerialization;
@@ -50,7 +51,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
    * of the children list of an INodeDirectory.
    */
   public static class ChildrenDiff extends Diff<byte[], INode> {
-    private ChildrenDiff() {}
+    ChildrenDiff() {}
     
     private ChildrenDiff(final List<INode> created, final List<INode> deleted) {
       super(created, deleted);
@@ -116,6 +117,59 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
       }
       return dirList;
     }
+    
+    /**
+     * 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.             
+     */
+    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;
+      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");
+          c++;
+          d++;
+        } else if (cnode.compareTo(dnode.getLocalNameBytes()) < 0) {
+          cBuffer.append(cStr + parent.getFullPathName() + Path.SEPARATOR
+              + cnode.getLocalName() + "\n");
+          c++;
+        } else {
+          dBuffer.append(dStr + parent.getFullPathName() + Path.SEPARATOR
+              + dnode.getLocalName() + "\n");
+          d++;
+        }
+      }   
+      for (; d < deleted.size(); d++) {
+        dBuffer.append(dStr + parent.getFullPathName() + Path.SEPARATOR
+            + deleted.get(d).getLocalName() + "\n");
+      }    
+      for (; c < created.size(); c++) {
+        cBuffer.append(cStr + parent.getFullPathName() + Path.SEPARATOR
+            + created.get(c).getLocalName() + "\n");
+      }
+      
+      str.append(cBuffer);
+      str.append(dBuffer);
+      str.append(mBuffer);
+    }
   }
   
   /**
@@ -324,6 +378,79 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     }
     return withSnapshot;
   }
+  
+  /**
+   * 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 earlierSnapshot = fromSnapshot;
+    Snapshot laterSnapshot = toSnapshot;
+    if (fromSnapshot == null
+        || (toSnapshot != null && Snapshot.ID_COMPARATOR.compare(fromSnapshot,
+            toSnapshot) > 0)) {
+      earlierSnapshot = toSnapshot;
+      laterSnapshot = fromSnapshot;
+    }
+    
+    int earlierDiffIndex = Collections.binarySearch(diffs, earlierSnapshot);
+    if (earlierDiffIndex < 0 && (-earlierDiffIndex - 1) == diffs.size()) {
+      // if the earlierSnapshot is after the latest SnapshotDiff stored in diffs,
+      // no modification happened after the earlierSnapshot
+      return false;
+    }
+    int laterDiffIndex = diffs.size();
+    if (laterSnapshot != null) {
+      laterDiffIndex = Collections.binarySearch(diffs, laterSnapshot);
+      if (laterDiffIndex == -1 || laterDiffIndex == 0) {
+        // if the endSnapshot is the earliest SnapshotDiff stored in
+        // diffs, or before it, no modification happened before the endSnapshot
+        return false;
+      }
+    }
+    
+    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++) {
+      SnapshotDiff sdiff = diffs.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 < diffs.size(); i++) {
+        if (diffs.get(i).snapshotINode != null
+            && !dirCopy.metadataEquals(diffs.get(i).snapshotINode)) {
+          return true;
+        }
+      }
+      return !dirCopy.metadataEquals(this);
+    }
+    return false;
+  }
 
   /** Diff list sorted by snapshot IDs, i.e. in chronological order. */
   private final List<SnapshotDiff> diffs;

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

@@ -31,6 +31,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 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;
 
 /**
  * Manage snapshottable directories and their snapshots.
@@ -233,4 +234,30 @@ public class SnapshotManager implements SnapshotStats {
       this.snapshottables.removeAll(toRemoveList);
     }
   }
+  
+  /**
+   * 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,
+      final String to) throws IOException {
+    if ((from == null || from.isEmpty())
+        && (to == null || to.isEmpty())) {
+      // both fromSnapshot and toSnapshot indicate the current tree
+      return null;
+    }
+    // if the start point is equal to the end point, return null
+    if (from.equals(to)) {
+      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.getMutableINodesInPath(path.toString());
+    final INodeDirectorySnapshottable snapshotRoot = INodeDirectorySnapshottable
+        .valueOf(inodesInPath.getLastINode(), path);
+    
+    return snapshotRoot.computeDiff(from, to);
+  }
+ 
 }

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

@@ -180,6 +180,12 @@ public class Diff<K, E extends Diff.Element<K>> {
     return d < 0 ? null : deleted.get(d);
   }
   
+  /** @return true if no changes contained in the diff */
+  public boolean isEmpty() {
+    return (created == null || created.isEmpty())
+        && (deleted == null || deleted.isEmpty());
+  }
+  
   /**
    * Insert the element to created.
    * @param i the insertion point defined

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

@@ -0,0 +1,198 @@
+/**
+ * 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.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+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.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 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 FSNamesystem fsn;
+  protected DistributedFileSystem hdfs;
+  
+  private int snapshotNum = 0;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new Configuration();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
+        .format(true).build();
+    cluster.waitActive();
+
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+  
+  /** 
+   * Create/modify/delete files and create snapshots under a given directory. 
+   */ 
+  private void modifyAndCreateSnapshot(Path modifyDir, Path snapshotDir)
+      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, (short) (REPLICATION - 1),
+        seed);
+    DFSTestUtil.createFile(hdfs, file11, BLOCKSIZE, (short) (REPLICATION - 1),
+        seed);
+    DFSTestUtil.createFile(hdfs, file12, BLOCKSIZE, (short) (REPLICATION - 1),
+        seed);
+    DFSTestUtil.createFile(hdfs, file13, BLOCKSIZE, (short) (REPLICATION - 1),
+        seed);
+    // create snapshot s1
+    hdfs.allowSnapshot(snapshotDir.toString());
+    hdfs.createSnapshot(snapshotDir, "s" + snapshotNum++);
+    
+    // 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 s2
+    hdfs.createSnapshot(snapshotDir, "s" + snapshotNum++);
+    
+    // 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 s3 for dir
+    hdfs.createSnapshot(snapshotDir, "s" + snapshotNum++);
+    // 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);
+    
+    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);
+    
+    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"));
+    
+    diffs = fsn.getSnapshotDiffReport(sub1.toString(), "s0", "");
+    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/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"));
+    
+    diffs = fsn.getSnapshotDiffReport(sub1.toString(), "s2", "s5");
+    diffStr = diffs.dump();
+    System.out.println(diffStr);
+    
+    assertTrue(diffStr.contains("M\t/TestSnapshot/sub1"));
+    assertTrue(diffStr.contains("+\t/TestSnapshot/sub1/subsub1"));
+    assertTrue(diffStr.contains("M\t/TestSnapshot/sub1/file10"));
+    
+    diffs = fsn.getSnapshotDiffReport(sub1.toString(), "s3", "");
+    diffStr = diffs.dump();
+    System.out.println(diffStr);
+    
+    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"));
+  }
+}