Ver Fonte

HDFS-12544. SnapshotDiff - support diff generation on any snapshot root descendant directory.

Manoj Govindassamy há 7 anos atrás
pai
commit
075dd45a24

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -374,6 +374,13 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String DFS_NAMENODE_SNAPSHOT_SKIP_CAPTURE_ACCESSTIME_ONLY_CHANGE = "dfs.namenode.snapshot.skip.capture.accesstime-only-change";
   public static final boolean DFS_NAMENODE_SNAPSHOT_SKIP_CAPTURE_ACCESSTIME_ONLY_CHANGE_DEFAULT = false;
 
+  public static final String
+      DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT =
+      "dfs.namenode.snapshotdiff.allow.snap-root-descendant";
+  public static final boolean
+      DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT_DEFAULT =
+      true;
+
   // Whether to enable datanode's stale state detection and usage for reads
   public static final String DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY = "dfs.namenode.avoid.read.stale.datanode";
   public static final boolean DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_DEFAULT = false;

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

@@ -253,6 +253,24 @@ public class INodeDirectory extends INodeWithAdditionalFields
     return getDirectorySnapshottableFeature() != null;
   }
 
+  /**
+   * Check if this directory is a descendant directory
+   * of a snapshot root directory.
+   * @param snapshotRootDir the snapshot root directory
+   * @return true if this directory is a descendant of snapshot root
+   */
+  public boolean isDescendantOfSnapshotRoot(INodeDirectory snapshotRootDir) {
+    Preconditions.checkArgument(snapshotRootDir.isSnapshottable());
+    INodeDirectory dir = this;
+    while(dir != null) {
+      if (dir.equals(snapshotRootDir)) {
+        return true;
+      }
+      dir = dir.getParent();
+    }
+    return false;
+  }
+
   public Snapshot getSnapshot(byte[] snapshotName) {
     return getDirectorySnapshottableFeature().getSnapshot(snapshotName);
   }

+ 25 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java

@@ -249,8 +249,12 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
 
   /**
    * Compute the difference between two snapshots (or a snapshot and the current
-   * directory) of the directory.
+   * directory) of the directory. The diff calculation can be scoped to either
+   * the snapshot root or any descendant directory under the snapshot root.
    *
+   * @param snapshotRootDir the snapshot root directory
+   * @param snapshotDiffScopeDir the descendant directory under snapshot root
+   *          to scope the diff calculation to.
    * @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.
@@ -259,18 +263,24 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
    *           point, or if endSnapshotName is not null but cannot be identified
    *           as a previous snapshot.
    */
-  SnapshotDiffInfo computeDiff(final INodeDirectory snapshotRoot,
-      final String from, final String to) throws SnapshotException {
-    Snapshot fromSnapshot = getSnapshotByName(snapshotRoot, from);
-    Snapshot toSnapshot = getSnapshotByName(snapshotRoot, to);
+  SnapshotDiffInfo computeDiff(final INodeDirectory snapshotRootDir,
+      final INodeDirectory snapshotDiffScopeDir, final String from,
+      final String to) throws SnapshotException {
+    Preconditions.checkArgument(snapshotDiffScopeDir
+        .isDescendantOfSnapshotRoot(snapshotRootDir));
+    Snapshot fromSnapshot = getSnapshotByName(snapshotRootDir, from);
+    Snapshot toSnapshot = getSnapshotByName(snapshotRootDir, to);
     // if the start point is equal to the end point, return null
     if (from.equals(to)) {
       return null;
     }
-    SnapshotDiffInfo diffs = new SnapshotDiffInfo(snapshotRoot, fromSnapshot,
-        toSnapshot);
-    computeDiffRecursively(snapshotRoot, snapshotRoot, new ArrayList<byte[]>(),
-        diffs);
+    SnapshotDiffInfo diffs = new SnapshotDiffInfo(snapshotRootDir,
+        snapshotDiffScopeDir, fromSnapshot, toSnapshot);
+    // The snapshot diff scope dir is passed in as the snapshot dir
+    // so that the file paths in the diff report are relative to the
+    // snapshot scope dir.
+    computeDiffRecursively(snapshotDiffScopeDir, snapshotDiffScopeDir,
+        new ArrayList<>(), diffs);
     return diffs;
   }
 
@@ -300,13 +310,15 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
   /**
    * Recursively compute the difference between snapshots under a given
    * directory/file.
-   * @param snapshotRoot The directory where snapshots were taken.
+   * @param snapshotDir The directory where snapshots were taken. Can be a
+   *                    snapshot root directory or any descendant directory
+   *                    under snapshot root directory.
    * @param node The directory/file under which the diff is computed.
    * @param parentPath Relative path (corresponding to the snapshot root) of
    *                   the node's parent.
    * @param diffReport data structure used to store the diff.
    */
-  private void computeDiffRecursively(final INodeDirectory snapshotRoot,
+  private void computeDiffRecursively(final INodeDirectory snapshotDir,
       INode node, List<byte[]> parentPath, SnapshotDiffInfo diffReport) {
     final Snapshot earlierSnapshot = diffReport.isFromEarlier() ?
         diffReport.getFrom() : diffReport.getTo();
@@ -331,7 +343,7 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
         boolean toProcess = diff.searchIndex(ListType.DELETED, name) < 0;
         if (!toProcess && child instanceof INodeReference.WithName) {
           byte[][] renameTargetPath = findRenameTargetPath(
-              snapshotRoot, (WithName) child,
+              snapshotDir, (WithName) child,
               laterSnapshot == null ? Snapshot.CURRENT_STATE_ID :
                 laterSnapshot.getId());
           if (renameTargetPath != null) {
@@ -341,7 +353,7 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
         }
         if (toProcess) {
           parentPath.add(name);
-          computeDiffRecursively(snapshotRoot, child, parentPath, diffReport);
+          computeDiffRecursively(snapshotDir, child, parentPath, diffReport);
           parentPath.remove(parentPath.size() - 1);
         }
       }

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

@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
-import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
@@ -101,6 +100,10 @@ class SnapshotDiffInfo {
 
   /** The root directory of the snapshots */
   private final INodeDirectory snapshotRoot;
+  /**
+   *  The scope directory under which snapshot diff is calculated.
+   */
+  private final INodeDirectory snapshotDiffScopeDir;
   /** The starting point of the difference */
   private final Snapshot from;
   /** The end point of the difference */
@@ -123,9 +126,12 @@ class SnapshotDiffInfo {
   private final Map<Long, RenameEntry> renameMap =
       new HashMap<Long, RenameEntry>();
 
-  SnapshotDiffInfo(INodeDirectory snapshotRoot, Snapshot start, Snapshot end) {
-    Preconditions.checkArgument(snapshotRoot.isSnapshottable());
-    this.snapshotRoot = snapshotRoot;
+  SnapshotDiffInfo(INodeDirectory snapshotRootDir,
+      INodeDirectory snapshotDiffScopeDir, Snapshot start, Snapshot end) {
+    Preconditions.checkArgument(snapshotRootDir.isSnapshottable() &&
+        snapshotDiffScopeDir.isDescendantOfSnapshotRoot(snapshotRootDir));
+    this.snapshotRoot = snapshotRootDir;
+    this.snapshotDiffScopeDir = snapshotDiffScopeDir;
     this.from = start;
     this.to = end;
   }

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

@@ -38,6 +38,7 @@ import javax.management.ObjectName;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -83,6 +84,13 @@ public class SnapshotManager implements SnapshotStatsMXBean {
    * together with the modification in next snapshot.
    */
   private boolean skipCaptureAccessTimeOnlyChange = false;
+  /**
+   * If snapshotDiffAllowSnapRootDescendant is set to true, snapshot diff
+   * operation can be run for any descendant directory under a snapshot root
+   * directory and the diff calculation will be scoped to the descendant
+   * directory.
+   */
+  private final boolean snapshotDiffAllowSnapRootDescendant;
 
   private final AtomicInteger numSnapshots = new AtomicInteger();
   private static final int SNAPSHOT_ID_BIT_WIDTH = 24;
@@ -102,9 +110,15 @@ public class SnapshotManager implements SnapshotStatsMXBean {
     this.skipCaptureAccessTimeOnlyChange = conf.getBoolean(
         DFS_NAMENODE_SNAPSHOT_SKIP_CAPTURE_ACCESSTIME_ONLY_CHANGE,
         DFS_NAMENODE_SNAPSHOT_SKIP_CAPTURE_ACCESSTIME_ONLY_CHANGE_DEFAULT);
+    this.snapshotDiffAllowSnapRootDescendant = conf.getBoolean(
+        DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT,
+        DFSConfigKeys.
+            DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT_DEFAULT);
     LOG.info("Loaded config captureOpenFiles: " + captureOpenFiles
-        + "skipCaptureAccessTimeOnlyChange: " +
-        skipCaptureAccessTimeOnlyChange);
+        + ", skipCaptureAccessTimeOnlyChange: "
+        + skipCaptureAccessTimeOnlyChange
+        + ", snapshotDiffAllowSnapRootDescendant: "
+        + snapshotDiffAllowSnapRootDescendant);
   }
 
   /**
@@ -228,6 +242,30 @@ public class SnapshotManager implements SnapshotStatsMXBean {
     return dir;
   }
 
+  /**
+   * Get the snapshot root directory for the given directory. The given
+   * directory must either be a snapshot root or a descendant of any
+   * snapshot root directories.
+   * @param iip INodesInPath for the directory to get snapshot root.
+   * @return the snapshot root INodeDirectory
+   */
+  public INodeDirectory getSnapshottableAncestorDir(final INodesInPath iip)
+      throws IOException {
+    final String path = iip.getPath();
+    final INodeDirectory dir = INodeDirectory.valueOf(iip.getLastINode(), path);
+    if (dir.isSnapshottable()) {
+      return dir;
+    } else {
+      for (INodeDirectory snapRoot : this.snapshottables.values()) {
+        if (dir.isAncestorDirectory(snapRoot)) {
+          return snapRoot;
+        }
+      }
+      throw new SnapshotException("Directory is neither snapshottable nor" +
+          " under a snap root!");
+    }
+  }
+
   /**
    * Create a snapshot of the given path.
    * It is assumed that the caller will perform synchronization.
@@ -396,22 +434,31 @@ public class SnapshotManager implements SnapshotStatsMXBean {
    * snapshot of the directory and its current tree.
    */
   public SnapshotDiffReport diff(final INodesInPath iip,
-      final String snapshotRootPath, final String from,
+      final String snapshotPath, final String from,
       final String to) throws IOException {
     // Find the source root directory path where the snapshots were taken.
     // All the check for path has been included in the valueOf method.
-    final INodeDirectory snapshotRoot = getSnapshottableRoot(iip);
+    INodeDirectory snapshotRootDir;
+    if (this.snapshotDiffAllowSnapRootDescendant) {
+      snapshotRootDir = getSnapshottableAncestorDir(iip);
+    } else {
+      snapshotRootDir = getSnapshottableRoot(iip);
+    }
+    Preconditions.checkNotNull(snapshotRootDir);
+    INodeDirectory snapshotDescendantDir = INodeDirectory.valueOf(
+        iip.getLastINode(), snapshotPath);
 
     if ((from == null || from.isEmpty())
         && (to == null || to.isEmpty())) {
       // both fromSnapshot and toSnapshot indicate the current tree
-      return new SnapshotDiffReport(snapshotRootPath, from, to,
+      return new SnapshotDiffReport(snapshotPath, from, to,
           Collections.<DiffReportEntry> emptyList());
     }
-    final SnapshotDiffInfo diffs = snapshotRoot
-        .getDirectorySnapshottableFeature().computeDiff(snapshotRoot, from, to);
+    final SnapshotDiffInfo diffs = snapshotRootDir
+        .getDirectorySnapshottableFeature().computeDiff(
+            snapshotRootDir, snapshotDescendantDir, from, to);
     return diffs != null ? diffs.generateReport() : new SnapshotDiffReport(
-        snapshotRootPath, from, to, Collections.<DiffReportEntry> emptyList());
+        snapshotPath, from, to, Collections.<DiffReportEntry> emptyList());
   }
   
   public void clearSnapshottableDirs() {

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -4298,6 +4298,17 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.namenode.snapshotdiff.allow.snap-root-descendant</name>
+  <value>true</value>
+  <description>
+    If enabled, snapshotDiff command can be run for any descendant directory
+    under a snapshot root directory and the diff calculation will be scoped
+    to the given descendant directory. Otherwise, snapshot diff command can
+    only be run for a snapshot root directory.
+  </description>
+</property>
+
 <property>
   <name>dfs.pipeline.ecn</name>
   <value>false</value>

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

@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.snapshot;
+
+import static org.junit.Assert.fail;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test snapshot diff report for the snapshot root descendant directory.
+ */
+public class TestSnapRootDescendantDiff extends TestSnapshotDiffReport {
+  @Before
+  public void setUp() throws Exception {
+    conf = new Configuration();
+    conf.setBoolean(
+        DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES, true);
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, 1);
+    conf.setBoolean(
+        DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_SKIP_CAPTURE_ACCESSTIME_ONLY_CHANGE,
+        true);
+    conf.setBoolean(
+        DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT,
+        false);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
+        .format(true).build();
+    cluster.waitActive();
+    hdfs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+      cluster = null;
+    }
+  }
+
+  @Test
+  public void testNonSnapRootDiffReport() throws Exception {
+    Path subsub1 = new Path(getSnapRootDir(), "subsub1");
+    Path subsubsub1 = new Path(subsub1, "subsubsub1");
+    hdfs.mkdirs(subsubsub1);
+    modifyAndCreateSnapshot(getSnapRootDir(), new Path[]{getSnapRootDir()});
+    modifyAndCreateSnapshot(subsubsub1, new Path[]{getSnapRootDir()});
+
+    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);
+    }
+  }
+
+}

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

@@ -43,12 +43,15 @@ import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
+import org.apache.hadoop.hdfs.protocol.SnapshotException;
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 import org.junit.After;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -60,7 +63,6 @@ import org.slf4j.LoggerFactory;
 public class TestSnapshotDiffReport {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestSnapshotDiffReport.class);
-
   private static final long SEED = 0;
   private static final short REPLICATION = 3;
   private static final short REPLICATION_1 = 2;
@@ -74,7 +76,6 @@ public class TestSnapshotDiffReport {
   protected Configuration conf;
   protected MiniDFSCluster cluster;
   protected DistributedFileSystem hdfs;
-  
   private final HashMap<Path, Integer> snapshotNumberMap = new HashMap<Path, Integer>();
 
   @Before
@@ -86,6 +87,9 @@ public class TestSnapshotDiffReport {
     conf.setBoolean(
         DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_SKIP_CAPTURE_ACCESSTIME_ONLY_CHANGE,
         true);
+    conf.setBoolean(
+        DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT,
+        true);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
         .format(true).build();
     cluster.waitActive();
@@ -99,7 +103,11 @@ public class TestSnapshotDiffReport {
       cluster = null;
     }
   }
-  
+
+  protected Path getSnapRootDir() {
+    return sub1;
+  }
+
   private String genSnapshotName(Path snapshotDir) {
     int sNum = -1;
     if (snapshotNumberMap.containsKey(snapshotDir)) {
@@ -108,12 +116,12 @@ public class TestSnapshotDiffReport {
     snapshotNumberMap.put(snapshotDir, ++sNum);
     return "s" + sNum;
   }
-  
+
   /**
    * Create/modify/delete files under a given directory, also create snapshots
    * of directories.
-   */ 
-  private void modifyAndCreateSnapshot(Path modifyDir, Path[] snapshotDirs)
+   */
+  protected void modifyAndCreateSnapshot(Path modifyDir, Path[] snapshotDirs)
       throws Exception {
     Path file10 = new Path(modifyDir, "file10");
     Path file11 = new Path(modifyDir, "file11");
@@ -133,7 +141,7 @@ public class TestSnapshotDiffReport {
       hdfs.allowSnapshot(snapshotDir);
       hdfs.createSnapshot(snapshotDir, genSnapshotName(snapshotDir));
     }
-    
+
     // delete file11
     hdfs.delete(file11, true);
     // modify file12
@@ -146,12 +154,12 @@ public class TestSnapshotDiffReport {
     DFSTestUtil.createFile(hdfs, file14, BLOCKSIZE, REPLICATION, SEED);
     // create file15
     DFSTestUtil.createFile(hdfs, file15, BLOCKSIZE, REPLICATION, SEED);
-    
+
     // create snapshot
     for (Path snapshotDir : snapshotDirs) {
       hdfs.createSnapshot(snapshotDir, genSnapshotName(snapshotDir));
     }
-    
+
     // create file11 again
     DFSTestUtil.createFile(hdfs, file11, BLOCKSIZE, REPLICATION, SEED);
     // delete file12
@@ -164,7 +172,7 @@ public class TestSnapshotDiffReport {
     hdfs.delete(file14, true);
     // modify file15
     hdfs.setReplication(file15, (short) (REPLICATION - 1));
-    
+
     // create snapshot
     for (Path snapshotDir : snapshotDirs) {
       hdfs.createSnapshot(snapshotDir, genSnapshotName(snapshotDir));
@@ -172,8 +180,10 @@ public class TestSnapshotDiffReport {
     // modify file10
     hdfs.setReplication(file10, (short) (REPLICATION + 1));
   }
-  
-  /** check the correctness of the diff reports */
+
+  /**
+   * 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);
@@ -182,10 +192,10 @@ public class TestSnapshotDiffReport {
         .getSnapshotDiffReport(dir, to, from);
     LOG.info(report.toString());
     LOG.info(inverseReport.toString() + "\n");
-    
+
     assertEquals(entries.length, report.getDiffList().size());
     assertEquals(entries.length, inverseReport.getDiffList().size());
-    
+
     for (DiffReportEntry entry : entries) {
       if (entry.getType() == DiffType.MODIFY) {
         assertTrue(report.getDiffList().contains(entry));
@@ -201,9 +211,11 @@ public class TestSnapshotDiffReport {
       }
     }
   }
-  
-  /** Test the computation and representation of diff between snapshots */
-  @Test (timeout=60000)
+
+  /**
+   * Test the computation and representation of diff between snapshots.
+   */
+  @Test(timeout = 60000)
   public void testDiffReport() throws Exception {
     cluster.getNamesystem().getSnapshotManager().setAllowNestedSnapshots(true);
 
@@ -212,45 +224,38 @@ public class TestSnapshotDiffReport {
     hdfs.mkdirs(subsubsub1);
     modifyAndCreateSnapshot(sub1, new Path[]{sub1, subsubsub1});
     modifyAndCreateSnapshot(subsubsub1, new Path[]{sub1, subsubsub1});
-    
-    try {
-      hdfs.getSnapshotDiffReport(subsub1, "s1", "s2");
-      fail("Expect exception when getting snapshot diff report: " + subsub1
-          + " is not a snapshottable directory.");
-    } catch (IOException e) {
-      GenericTestUtils.assertExceptionContains(
-          "Directory is not a snapshottable directory: " + subsub1, e);
-    }
-    
+
     final String invalidName = "invalid";
     try {
       hdfs.getSnapshotDiffReport(sub1, invalidName, invalidName);
-      fail("Expect exception when providing invalid snapshot name for diff report");
+      fail("Expect exception when providing invalid snapshot name " +
+          "for diff report");
     } catch (IOException e) {
       GenericTestUtils.assertExceptionContains(
           "Cannot find the snapshot of directory " + sub1 + " with name "
               + invalidName, e);
     }
-    
+
     // diff between the same snapshot
     SnapshotDiffReport report = hdfs.getSnapshotDiffReport(sub1, "s0", "s0");
     LOG.info(report.toString());
     assertEquals(0, report.getDiffList().size());
-    
+
     report = hdfs.getSnapshotDiffReport(sub1, "", "");
     LOG.info(report.toString());
     assertEquals(0, report.getDiffList().size());
-    
+
     report = hdfs.getSnapshotDiffReport(subsubsub1, "s0", "s2");
     LOG.info(report.toString());
     assertEquals(0, report.getDiffList().size());
 
     // test path with scheme also works
-    report = hdfs.getSnapshotDiffReport(hdfs.makeQualified(subsubsub1), "s0", "s2");
+    report = hdfs.getSnapshotDiffReport(hdfs.makeQualified(subsubsub1),
+        "s0", "s2");
     LOG.info(report.toString());
     assertEquals(0, report.getDiffList().size());
 
-    verifyDiffReport(sub1, "s0", "s2", 
+    verifyDiffReport(sub1, "s0", "s2",
         new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
         new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("file15")),
         new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("file12")),
@@ -260,7 +265,7 @@ public class TestSnapshotDiffReport {
         new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("link13")),
         new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("link13")));
 
-    verifyDiffReport(sub1, "s0", "s5", 
+    verifyDiffReport(sub1, "s0", "s5",
         new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
         new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("file15")),
         new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("file12")),
@@ -282,7 +287,7 @@ public class TestSnapshotDiffReport {
             DFSUtil.string2Bytes("subsub1/subsubsub1/link13")),
         new DiffReportEntry(DiffType.CREATE,
             DFSUtil.string2Bytes("subsub1/subsubsub1/file15")));
-    
+
     verifyDiffReport(sub1, "s2", "s5",
         new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("file10")),
         new DiffReportEntry(DiffType.MODIFY,
@@ -297,7 +302,7 @@ public class TestSnapshotDiffReport {
             DFSUtil.string2Bytes("subsub1/subsubsub1/link13")),
         new DiffReportEntry(DiffType.CREATE,
             DFSUtil.string2Bytes("subsub1/subsubsub1/file15")));
-    
+
     verifyDiffReport(sub1, "s3", "",
         new DiffReportEntry(DiffType.MODIFY,
             DFSUtil.string2Bytes("subsub1/subsubsub1")),
@@ -318,7 +323,467 @@ public class TestSnapshotDiffReport {
         new DiffReportEntry(DiffType.DELETE,
             DFSUtil.string2Bytes("subsub1/subsubsub1/link13")));
   }
-  
+
+  @Test(timeout = 60000)
+  public void testSnapRootDescendantDiffReport() throws Exception {
+    Assume.assumeTrue(conf.getBoolean(
+        DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT,
+        DFSConfigKeys.
+            DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT_DEFAULT));
+    Path subSub = new Path(sub1, "subsub1");
+    Path subSubSub = new Path(subSub, "subsubsub1");
+    Path nonSnapDir = new Path(dir, "non_snap");
+    hdfs.mkdirs(subSubSub);
+    hdfs.mkdirs(nonSnapDir);
+
+    modifyAndCreateSnapshot(sub1, new Path[]{sub1});
+    modifyAndCreateSnapshot(subSub, new Path[]{sub1});
+    modifyAndCreateSnapshot(subSubSub, new Path[]{sub1});
+
+    try {
+      hdfs.getSnapshotDiffReport(subSub, "s1", "s2");
+      hdfs.getSnapshotDiffReport(subSubSub, "s1", "s2");
+    } catch (IOException e) {
+      fail("Unexpected exception when getting snapshot diff report " +
+          subSub + ": " + e);
+    }
+
+    try {
+      hdfs.getSnapshotDiffReport(nonSnapDir, "s1", "s2");
+      fail("Snapshot diff report on a non snapshot directory '"
+          + nonSnapDir.getName() + "'should fail!");
+    } catch (SnapshotException e) {
+      GenericTestUtils.assertExceptionContains(
+          "Directory is neither snapshottable nor under a snap root!", e);
+    }
+
+    final String invalidName = "invalid";
+    try {
+      hdfs.getSnapshotDiffReport(subSub, invalidName, invalidName);
+      fail("Expect exception when providing invalid snapshot name " +
+          "for diff report");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains(
+          "Cannot find the snapshot of directory " + sub1 + " with name "
+              + invalidName, e);
+    }
+
+    // diff between the same snapshot
+    SnapshotDiffReport report = hdfs.getSnapshotDiffReport(subSub, "s0", "s0");
+    assertEquals(0, report.getDiffList().size());
+
+    report = hdfs.getSnapshotDiffReport(subSub, "", "");
+    assertEquals(0, report.getDiffList().size());
+
+    report = hdfs.getSnapshotDiffReport(subSubSub, "s0", "s2");
+    assertEquals(0, report.getDiffList().size());
+
+    report = hdfs.getSnapshotDiffReport(
+        hdfs.makeQualified(subSubSub), "s0", "s2");
+    assertEquals(0, report.getDiffList().size());
+
+    verifyDescendantDiffReports(sub1, subSub, subSubSub);
+  }
+
+  private void verifyDescendantDiffReports(final Path snapDir,
+      final Path snapSubDir, final Path snapSubSubDir) throws
+      IOException {
+    verifyDiffReport(snapDir, "s0", "s2",
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("file15")),
+        new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("file12")),
+        new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("file11")),
+        new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("file11")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("file13")),
+        new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("link13")),
+        new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("link13")));
+    verifyDiffReport(snapSubDir, "s0", "s2", new DiffReportEntry[]{});
+    verifyDiffReport(snapSubSubDir, "s0", "s2", new DiffReportEntry[]{});
+
+    verifyDiffReport(snapDir, "s0", "s8",
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("file15")),
+        new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("file12")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("file10")),
+        new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("file11")),
+        new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("file11")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("file13")),
+        new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("link13")),
+        new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("link13")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/file10")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/file11")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/file13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/link13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/file15")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1/subsubsub1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file10")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file11")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/link13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file15")));
+
+    verifyDiffReport(snapSubDir, "s0", "s8",
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file10")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file11")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("link13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file15")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsubsub1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsubsub1/file10")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsubsub1/file11")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsubsub1/file13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsubsub1/link13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsubsub1/file15")));
+
+    verifyDiffReport(snapSubSubDir, "s0", "s8",
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file10")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file11")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("link13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file15")));
+
+    verifyDiffReport(snapDir, "s2", "s5",
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("file10")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/file10")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/file11")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/file13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/link13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/file15")));
+
+    verifyDiffReport(snapSubDir, "s2", "s5",
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file10")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file11")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("link13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file15")));
+    verifyDiffReport(snapSubSubDir, "s2", "s5",
+        new DiffReportEntry[]{});
+
+    verifyDiffReport(snapDir, "s3", "",
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/file15")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("subsub1/file12")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1/file10")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("subsub1/file11")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/file11")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1/file13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/link13")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("subsub1/link13")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1/subsubsub1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file10")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file11")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/link13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file15")));
+
+    verifyDiffReport(snapSubDir, "s3", "",
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file15")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("file12")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("file10")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("file11")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file11")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("file13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("link13")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("link13")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsubsub1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsubsub1/file10")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsubsub1/file11")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsubsub1/file13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsubsub1/link13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsubsub1/file15")));
+
+    verifyDiffReport(snapSubSubDir, "s3", "",
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file10")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file11")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("link13")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file15")));
+  }
+
+  @Test
+  public void testSnapRootDescendantDiffReportWithRename() throws Exception {
+    Assume.assumeTrue(conf.getBoolean(
+        DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT,
+        DFSConfigKeys.
+            DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT_DEFAULT));
+    Path subSub = new Path(sub1, "subsub1");
+    Path subSubSub = new Path(subSub, "subsubsub1");
+    Path nonSnapDir = new Path(dir, "non_snap");
+    hdfs.mkdirs(subSubSub);
+    hdfs.mkdirs(nonSnapDir);
+
+    hdfs.allowSnapshot(sub1);
+    hdfs.createSnapshot(sub1, genSnapshotName(sub1));
+    Path file20 = new Path(subSubSub, "file20");
+    DFSTestUtil.createFile(hdfs, file20, BLOCKSIZE, REPLICATION_1, SEED);
+    hdfs.createSnapshot(sub1, genSnapshotName(sub1));
+
+    // Case 1: Move a file away from a descendant dir, but within the snap root.
+    // mv <snaproot>/<subsub>/<subsubsub>/file20 <snaproot>/<subsub>/file20
+    hdfs.rename(file20, new Path(subSub, file20.getName()));
+    hdfs.createSnapshot(sub1, genSnapshotName(sub1));
+
+    // The snapshot diff for the snap root detects the change as file rename
+    // as the file move happened within the snap root.
+    verifyDiffReport(sub1, "s1", "s2",
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1/subsubsub1")),
+        new DiffReportEntry(DiffType.RENAME,
+            DFSUtil.string2Bytes("subsub1/subsubsub1/file20"),
+            DFSUtil.string2Bytes("subsub1/file20")));
+
+    // The snapshot diff for the descendant dir <subsub> still detects the
+    // change as file rename as the file move happened under the snap root
+    // descendant dir.
+    verifyDiffReport(subSub, "s1", "s2",
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsubsub1")),
+        new DiffReportEntry(DiffType.RENAME,
+            DFSUtil.string2Bytes("subsubsub1/file20"),
+            DFSUtil.string2Bytes("file20")));
+
+    // The snapshot diff for the descendant dir <subsubsub> detects the
+    // change as file delete as the file got moved from its scope.
+    verifyDiffReport(subSubSub, "s1", "s2",
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("file20")));
+
+    // Case 2: Move the file from the snap root descendant dir to any
+    // non snap root dir. mv <snaproot>/<subsub>/file20 <nonsnaproot>/file20.
+    hdfs.rename(new Path(subSub, file20.getName()),
+        new Path(dir, file20.getName()));
+    hdfs.createSnapshot(sub1, genSnapshotName(sub1));
+
+    // The snapshot diff for the snap root detects the change as file delete
+    // as the file got moved away from the snap root dir to some non snap
+    // root dir.
+    verifyDiffReport(sub1, "s2", "s3",
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("subsub1/file20")));
+
+    // The snapshot diff for the snap root descendant <subsub> detects the
+    // change as file delete as the file was previously under its scope and
+    // got moved away from its scope.
+    verifyDiffReport(subSub, "s2", "s3",
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("file20")));
+
+    // The file was already not under the descendant dir <subsubsub> scope.
+    // So, the snapshot diff report for the descendant dir doesn't
+    // show the file rename at all.
+    verifyDiffReport(subSubSub, "s2", "s3",
+        new DiffReportEntry[]{});
+
+    // Case 3: Move the file from the non-snap root dir to snap root dir
+    // mv <nonsnaproot>/file20 <snaproot>/file20
+    hdfs.rename(new Path(dir, file20.getName()),
+        new Path(sub1, file20.getName()));
+    hdfs.createSnapshot(sub1, genSnapshotName(sub1));
+
+    // Snap root directory should show the file moved in as a new file.
+    verifyDiffReport(sub1, "s3", "s4",
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file20")));
+
+    // Snap descendant directories don't have visibility to the moved in file.
+    verifyDiffReport(subSub, "s3", "s4",
+        new DiffReportEntry[]{});
+    verifyDiffReport(subSubSub, "s3", "s4",
+        new DiffReportEntry[]{});
+
+    hdfs.rename(new Path(sub1, file20.getName()),
+        new Path(subSub, file20.getName()));
+    hdfs.createSnapshot(sub1, genSnapshotName(sub1));
+
+    // Snap root directory now shows the rename as both source and
+    // destination paths are under the snap root.
+    verifyDiffReport(sub1, "s4", "s5",
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.RENAME,
+            DFSUtil.string2Bytes("file20"),
+            DFSUtil.string2Bytes("subsub1/file20")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1")));
+
+    // For the descendant directory under the snap root, the file
+    // moved in shows up as a new file created.
+    verifyDiffReport(subSub, "s4", "s5",
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("file20")));
+
+    verifyDiffReport(subSubSub, "s4", "s5",
+        new DiffReportEntry[]{});
+
+    // Case 4: Snapshot diff for the newly created descendant directory.
+    Path subSubSub2 = new Path(subSub, "subsubsub2");
+    hdfs.mkdirs(subSubSub2);
+    Path file30 = new Path(subSubSub2, "file30");
+    DFSTestUtil.createFile(hdfs, file30, BLOCKSIZE, REPLICATION_1, SEED);
+    hdfs.createFile(file30);
+    hdfs.createSnapshot(sub1, genSnapshotName(sub1));
+
+    verifyDiffReport(sub1, "s5", "s6",
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("subsub1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("subsub1/subsubsub2")));
+
+    verifyDiffReport(subSubSub2, "s5", "s6",
+        new DiffReportEntry[]{});
+
+    verifyDiffReport(subSubSub2, "s1", "s2",
+        new DiffReportEntry[]{});
+  }
+
+  @Test
+  public void testSnapshotDiffInfo() throws Exception {
+    Path snapshotRootDirPath = dir;
+    Path snapshotDirDescendantPath = new Path(snapshotRootDirPath, "desc");
+    Path snapshotDirNonDescendantPath = new Path("/dummy/non/snap/desc");
+    hdfs.mkdirs(snapshotDirDescendantPath);
+    hdfs.mkdirs(snapshotDirNonDescendantPath);
+
+    hdfs.allowSnapshot(snapshotRootDirPath);
+    hdfs.createSnapshot(snapshotRootDirPath, "s0");
+    hdfs.createSnapshot(snapshotRootDirPath, "s1");
+
+    INodeDirectory snapshotRootDir = cluster.getNameNode()
+        .getNamesystem().getFSDirectory().getINode(
+            snapshotRootDirPath.toUri().getPath())
+        .asDirectory();
+    INodeDirectory snapshotRootDescendantDir = cluster.getNameNode()
+        .getNamesystem().getFSDirectory().getINode(
+            snapshotDirDescendantPath.toUri().getPath())
+        .asDirectory();
+    INodeDirectory snapshotRootNonDescendantDir = cluster.getNameNode()
+        .getNamesystem().getFSDirectory().getINode(
+            snapshotDirNonDescendantPath.toUri().getPath())
+        .asDirectory();
+    try {
+      SnapshotDiffInfo sdi = new SnapshotDiffInfo(
+          snapshotRootDir,
+          snapshotRootDescendantDir,
+          new Snapshot(0, "s0", snapshotRootDescendantDir),
+          new Snapshot(0, "s1", snapshotRootDescendantDir));
+      LOG.info("SnapshotDiffInfo: " + sdi.getFrom() + " - " + sdi.getTo());
+    } catch (IllegalArgumentException iae){
+      fail("Unexpected exception when constructing SnapshotDiffInfo: " + iae);
+    }
+
+    try {
+      SnapshotDiffInfo sdi = new SnapshotDiffInfo(
+          snapshotRootDir,
+          snapshotRootNonDescendantDir,
+          new Snapshot(0, "s0", snapshotRootNonDescendantDir),
+          new Snapshot(0, "s1", snapshotRootNonDescendantDir));
+      LOG.info("SnapshotDiffInfo: " + sdi.getFrom() + " - " + sdi.getTo());
+      fail("SnapshotDiffInfo construction should fail for non snapshot root " +
+          "or non snapshot root descendant directories!");
+    } catch (IllegalArgumentException iae) {
+      // expected exception
+    }
+  }
+
   /**
    * Make changes under a sub-directory, then delete the sub-directory. Make
    * sure the diff report computation correctly retrieve the diff from the
@@ -828,4 +1293,4 @@ public class TestSnapshotDiffReport {
 
     assertAtimeNotEquals(filePostSS, root, "s2", "s3");
   }
-}
+}