Ver código fonte

HDFS-4667. Capture renamed files/directories in snapshot diff report. Contributed by Jing Zhao and Binglin Chang.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1604488 13f79535-47bb-0310-9956-ffa450edef68
Jing Zhao 11 anos atrás
pai
commit
8a83bb7ad6
18 arquivos alterados com 558 adições e 527 exclusões
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 41 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java
  3. 16 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  4. 1 48
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  5. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  6. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryAttributes.java
  7. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  8. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java
  9. 19 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
  10. 21 19
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
  11. 25 54
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
  12. 36 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
  13. 135 22
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectorySnapshottable.java
  14. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
  15. 28 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/xdoc/HdfsSnapshots.xml
  16. 0 295
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFullPathNameWithSnapshot.java
  17. 38 37
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
  18. 166 4
      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.txt

@@ -458,6 +458,9 @@ Release 2.5.0 - UNRELEASED
 
     HDFS-6557. Move the reference of fsimage to FSNamesystem. (wheat9)
 
+    HDFS-4667. Capture renamed files/directories in snapshot diff report. (jing9
+    and Binglin Chang via jing9)
+
   OPTIMIZATIONS
 
     HDFS-6214. Webhdfs has poor throughput for files >2GB (daryn)

+ 41 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java

@@ -25,6 +25,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable.SnapshotDiffInfo;
 
+import com.google.common.base.Objects;
+
 /**
  * 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
@@ -79,43 +81,64 @@ public class SnapshotDiffReport {
     /** The type of the difference. */
     private final DiffType type;
     /**
-     * The relative path (related to the snapshot root) of the file/directory
-     * where changes have happened
+     * The relative path (related to the snapshot root) of 1) the file/directory
+     * where changes have happened, or 2) the source file/dir of a rename op.
      */
-    private final byte[] relativePath;
+    private final byte[] sourcePath;
+    private final byte[] targetPath;
+
+    public DiffReportEntry(DiffType type, byte[] sourcePath) {
+      this(type, sourcePath, null);
+    }
+
+    public DiffReportEntry(DiffType type, byte[][] sourcePathComponents) {
+      this(type, sourcePathComponents, null);
+    }
 
-    public DiffReportEntry(DiffType type, byte[] path) {
+    public DiffReportEntry(DiffType type, byte[] sourcePath, byte[] targetPath) {
       this.type = type;
-      this.relativePath = path;
+      this.sourcePath = sourcePath;
+      this.targetPath = targetPath;
     }
     
-    public DiffReportEntry(DiffType type, byte[][] pathComponents) {
+    public DiffReportEntry(DiffType type, byte[][] sourcePathComponents,
+        byte[][] targetPathComponents) {
       this.type = type;
-      this.relativePath = DFSUtil.byteArray2bytes(pathComponents);
+      this.sourcePath = DFSUtil.byteArray2bytes(sourcePathComponents);
+      this.targetPath = targetPathComponents == null ? null : DFSUtil
+          .byteArray2bytes(targetPathComponents);
     }
     
     @Override
     public String toString() {
-      return type.getLabel() + "\t" + getRelativePathString();
+      String str = type.getLabel() + "\t" + getPathString(sourcePath);
+      if (type == DiffType.RENAME) {
+        str += " -> " + getPathString(targetPath);
+      }
+      return str;
     }
     
     public DiffType getType() {
       return type;
     }
 
-    public String getRelativePathString() {
-      String path = DFSUtil.bytes2String(relativePath);
-      if (path.isEmpty()) {
+    static String getPathString(byte[] path) {
+      String pathStr = DFSUtil.bytes2String(path);
+      if (pathStr.isEmpty()) {
         return Path.CUR_DIR;
       } else {
-        return Path.CUR_DIR + Path.SEPARATOR + path;
+        return Path.CUR_DIR + Path.SEPARATOR + pathStr;
       }
     }
 
-    public byte[] getRelativePath() {
-      return relativePath;
+    public byte[] getSourcePath() {
+      return sourcePath;
     }
-    
+
+    public byte[] getTargetPath() {
+      return targetPath;
+    }
+
     @Override
     public boolean equals(Object other) {
       if (this == other) {
@@ -124,14 +147,15 @@ public class SnapshotDiffReport {
       if (other != null && other instanceof DiffReportEntry) {
         DiffReportEntry entry = (DiffReportEntry) other;
         return type.equals(entry.getType())
-            && Arrays.equals(relativePath, entry.getRelativePath());
+            && Arrays.equals(sourcePath, entry.getSourcePath())
+            && Arrays.equals(targetPath, entry.getTargetPath());
       }
       return false;
     }
     
     @Override
     public int hashCode() {
-      return Arrays.hashCode(relativePath);
+      return Objects.hashCode(getSourcePath(), getTargetPath());
     }
   }
   

+ 16 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -44,7 +44,6 @@ import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.StorageType;
-import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -1737,24 +1736,29 @@ public class PBHelper {
     }
     DiffType type = DiffType.getTypeFromLabel(entry
         .getModificationLabel());
-    return type == null ? null : 
-      new DiffReportEntry(type, entry.getFullpath().toByteArray());
+    return type == null ? null : new DiffReportEntry(type, entry.getFullpath()
+        .toByteArray(), entry.hasTargetPath() ? entry.getTargetPath()
+        .toByteArray() : null);
   }
   
   public static SnapshotDiffReportEntryProto convert(DiffReportEntry entry) {
     if (entry == null) {
       return null;
     }
-    byte[] fullPath = entry.getRelativePath();
-    ByteString fullPathString = ByteString
-        .copyFrom(fullPath == null ? DFSUtil.EMPTY_BYTES : fullPath);
-    
+    ByteString sourcePath = ByteString
+        .copyFrom(entry.getSourcePath() == null ? DFSUtil.EMPTY_BYTES : entry
+            .getSourcePath());
     String modification = entry.getType().getLabel();
-    
-    SnapshotDiffReportEntryProto entryProto = SnapshotDiffReportEntryProto
-        .newBuilder().setFullpath(fullPathString)
-        .setModificationLabel(modification).build();
-    return entryProto;
+    SnapshotDiffReportEntryProto.Builder builder = SnapshotDiffReportEntryProto
+        .newBuilder().setFullpath(sourcePath)
+        .setModificationLabel(modification);
+    if (entry.getType() == DiffType.RENAME) {
+      ByteString targetPath = ByteString
+          .copyFrom(entry.getTargetPath() == null ? DFSUtil.EMPTY_BYTES : entry
+              .getTargetPath());
+      builder.setTargetPath(targetPath);
+    }
+    return builder.build();
   }
   
   public static SnapshotDiffReport convert(SnapshotDiffReportProto reportProto) {

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

@@ -2790,7 +2790,7 @@ public class FSDirectory implements Closeable {
    * Note that this method cannot handle scenarios where the inode is in a
    * snapshot.
    */
-  static byte[][] getPathComponents(INode inode) {
+  public static byte[][] getPathComponents(INode inode) {
     List<byte[]> components = new ArrayList<byte[]>();
     components.add(0, inode.getLocalNameBytes());
     while(inode.getParent() != null) {
@@ -2800,53 +2800,6 @@ public class FSDirectory implements Closeable {
     return components.toArray(new byte[components.size()][]);
   }
 
-  /**
-   * The same functionality with {@link #getPathComponents(INode)}, but can
-   * handle snapshots.
-   */
-  public static byte[][] getPathComponentsWithSnapshot(INode inode) {
-    List<byte[]> components = new ArrayList<byte[]>();
-    boolean inSnapshot = false;
-    int snapshotId = Snapshot.CURRENT_STATE_ID;
-    do {
-      if (inode instanceof INodeReference.WithCount) {
-        // identify the corresponding WithName or DstReference node
-        inode = ((WithCount) inode).getParentRef(snapshotId);
-      } else { // normal INode and WithName/DstReference
-        if (inode instanceof INodeDirectory
-            && inode.asDirectory().isSnapshottable() && inSnapshot
-            && snapshotId != Snapshot.CURRENT_STATE_ID) {
-          INodeDirectorySnapshottable sdir = (INodeDirectorySnapshottable) inode
-              .asDirectory();
-          Snapshot snapshot = sdir.getSnapshotById(snapshotId);
-          if (snapshot != null) {
-            components.add(0, snapshot.getRoot().getLocalNameBytes());
-            components.add(0, HdfsConstants.DOT_SNAPSHOT_DIR_BYTES);
-            // the snapshot has been found, thus no need to check snapshottable
-            // directory afterwards
-            inSnapshot = false;
-          }
-        }
-        INode parent = inode.getParentReference() != null ? inode
-            .getParentReference() : inode.getParent();
-        if (parent != null && parent instanceof INodeDirectory) {
-          int sid = parent.asDirectory().searchChild(inode);
-          Preconditions.checkState(sid != Snapshot.NO_SNAPSHOT_ID);
-          if (sid != Snapshot.CURRENT_STATE_ID
-              && snapshotId == Snapshot.CURRENT_STATE_ID) {
-            snapshotId = sid;
-            inSnapshot = true;
-          }
-          components.add(0, inode.getLocalNameBytes());
-        } else if (parent == null) { // root
-          components.add(0, inode.getLocalNameBytes());
-        }
-        inode = parent;
-      }
-    } while (inode != null);
-    return components.toArray(new byte[components.size()][]);
-  }
-
   /**
    * @return path components for reserved path, else null.
    */

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

@@ -26,11 +26,9 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.fs.PathIsNotDirectoryException;
-import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiffList;
@@ -365,7 +363,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
    * children list nor in any snapshot; otherwise the snapshot id of the
    * corresponding snapshot diff list.
    */
-  int searchChild(INode inode) {
+  public int searchChild(INode inode) {
     INode child = getChild(inode.getLocalNameBytes(), Snapshot.CURRENT_STATE_ID);
     if (child != inode) {
       // inode is not in parent's children list, thus inode must be in
@@ -764,7 +762,9 @@ public class INodeDirectory extends INodeWithAdditionalFields
   public boolean metadataEquals(INodeDirectoryAttributes other) {
     return other != null
         && getQuotaCounts().equals(other.getQuotaCounts())
-        && getPermissionLong() == other.getPermissionLong();
+        && getPermissionLong() == other.getPermissionLong()
+        && getAclFeature() == other.getAclFeature()
+        && getXAttrFeature() == other.getXAttrFeature();
   }
   
   /*

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryAttributes.java

@@ -53,8 +53,10 @@ public interface INodeDirectoryAttributes extends INodeAttributes {
     @Override
     public boolean metadataEquals(INodeDirectoryAttributes other) {
       return other != null
-          && this.getQuotaCounts().equals(other.getQuotaCounts())
-          && getPermissionLong() == other.getPermissionLong();
+          && getQuotaCounts().equals(other.getQuotaCounts())
+          && getPermissionLong() == other.getPermissionLong()
+          && getAclFeature() == other.getAclFeature()
+          && getXAttrFeature() == other.getXAttrFeature();
     }
   }
 

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -144,6 +144,15 @@ public class INodeFile extends INodeWithAdditionalFields
     return this;
   }
 
+  @Override
+  public boolean metadataEquals(INodeFileAttributes other) {
+    return other != null
+        && getHeaderLong()== other.getHeaderLong()
+        && getPermissionLong() == other.getPermissionLong()
+        && getAclFeature() == other.getAclFeature()
+        && getXAttrFeature() == other.getXAttrFeature();
+  }
+
   /* Start of Under-Construction Feature */
 
   /**

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

@@ -36,6 +36,8 @@ public interface INodeFileAttributes extends INodeAttributes {
   /** @return the header as a long. */
   public long getHeaderLong();
 
+  public boolean metadataEquals(INodeFileAttributes other);
+
   /** A copy of the inode file attributes */
   public static class SnapshotCopy extends INodeAttributes.SnapshotCopy
       implements INodeFileAttributes {
@@ -70,5 +72,14 @@ public interface INodeFileAttributes extends INodeAttributes {
     public long getHeaderLong() {
       return header;
     }
+
+    @Override
+    public boolean metadataEquals(INodeFileAttributes other) {
+      return other != null
+          && getHeaderLong()== other.getHeaderLong()
+          && getPermissionLong() == other.getPermissionLong()
+          && getAclFeature() == other.getAclFeature()
+          && getXAttrFeature() == other.getXAttrFeature();
+    }
   }
 }

+ 19 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java

@@ -435,21 +435,28 @@ public abstract class INodeReference extends INode {
       }
     }
 
+    /**
+     * @return the WithName/DstReference node contained in the given snapshot.
+     */
     public INodeReference getParentRef(int snapshotId) {
-      // when the given snapshotId is CURRENT_STATE_ID, it is possible that we
-      // do not know where the corresponding inode belongs, thus we simply
-      // return the last reference node
-      if (snapshotId == Snapshot.CURRENT_STATE_ID) {
-        return this.getParentReference() != null ? this.getParentReference()
-            : this.getLastWithName();
-      }
-      // otherwise we search the withNameList
-      for (int i = 0; i < withNameList.size(); i++) {
-        if (snapshotId <= withNameList.get(i).lastSnapshotId) {
-          return withNameList.get(i);
+      int start = 0;
+      int end = withNameList.size() - 1;
+      while (start < end) {
+        int mid = start + (end - start) / 2;
+        int sid = withNameList.get(mid).lastSnapshotId; 
+        if (sid == snapshotId) {
+          return withNameList.get(mid);
+        } else if (sid < snapshotId) {
+          start = mid + 1;
+        } else {
+          end = mid;
         }
       }
-      return this.getParentReference();
+      if (withNameList.get(start).lastSnapshotId >= snapshotId) {
+        return withNameList.get(start);
+      } else {
+        return this.getParentReference();
+      }
     }
   }
   

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

@@ -227,32 +227,34 @@ abstract class AbstractINodeDiffList<N extends INode,
     D diff = getDiffById(snapshotId);
     return diff == null ? Snapshot.CURRENT_STATE_ID : diff.getSnapshotId();
   }
-  
-  /**
-   * Check if changes have happened between two snapshots.
-   * @param earlier The snapshot taken earlier
-   * @param later The snapshot taken later
-   * @return Whether or not modifications (including diretory/file metadata
-   *         change, file creation/deletion under the directory) have happened
-   *         between snapshots.
-   */
-  final boolean changedBetweenSnapshots(Snapshot earlier, Snapshot later) {
+
+  final int[] changedBetweenSnapshots(Snapshot from, Snapshot to) {
+    Snapshot earlier = from;
+    Snapshot later = to;
+    if (Snapshot.ID_COMPARATOR.compare(from, to) > 0) {
+      earlier = to;
+      later = from;
+    }
+
     final int size = diffs.size();
     int earlierDiffIndex = Collections.binarySearch(diffs, earlier.getId());
+    int laterDiffIndex = later == null ? size : Collections
+        .binarySearch(diffs, later.getId());
     if (-earlierDiffIndex - 1 == size) {
       // if the earlierSnapshot is after the latest SnapshotDiff stored in
       // diffs, no modification happened after the earlierSnapshot
-      return false;
+      return null;
     }
-    if (later != null) {
-      int laterDiffIndex = Collections.binarySearch(diffs, later.getId());
-      if (laterDiffIndex == -1 || laterDiffIndex == 0) {
-        // if the laterSnapshot is the earliest SnapshotDiff stored in diffs, or
-        // before it, no modification happened before the laterSnapshot
-        return false;
-      }
+    if (laterDiffIndex == -1 || laterDiffIndex == 0) {
+      // if the laterSnapshot is the earliest SnapshotDiff stored in diffs, or
+      // before it, no modification happened before the laterSnapshot
+      return null;
     }
-    return true;
+    earlierDiffIndex = earlierDiffIndex < 0 ? (-earlierDiffIndex - 1)
+        : earlierDiffIndex;
+    laterDiffIndex = laterDiffIndex < 0 ? (-laterDiffIndex - 1)
+        : laterDiffIndex;
+    return new int[]{earlierDiffIndex, laterDiffIndex};
   }
 
   /**

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

@@ -21,7 +21,6 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.Deque;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -42,6 +41,7 @@ import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable.SnapshotDiffInfo.RenameEntry;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
 import org.apache.hadoop.hdfs.util.Diff;
 import org.apache.hadoop.hdfs.util.Diff.Container;
@@ -170,49 +170,33 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
      * @return A list of {@link DiffReportEntry} as the diff report.
      */
     public List<DiffReportEntry> generateReport(byte[][] parentPath,
-        boolean fromEarlier) {
-      List<DiffReportEntry> cList = new ArrayList<DiffReportEntry>();
-      List<DiffReportEntry> dList = new ArrayList<DiffReportEntry>();
-      int c = 0, d = 0;
+        boolean fromEarlier, Map<Long, RenameEntry> renameMap) {
+      List<DiffReportEntry> list = new ArrayList<DiffReportEntry>();
       List<INode> created = getList(ListType.CREATED);
       List<INode> deleted = getList(ListType.DELETED);
       byte[][] fullPath = new byte[parentPath.length + 1][];
       System.arraycopy(parentPath, 0, fullPath, 0, parentPath.length);
-      for (; c < created.size() && d < deleted.size(); ) {
-        INode cnode = created.get(c);
-        INode dnode = deleted.get(d);
-        if (cnode.compareTo(dnode.getLocalNameBytes()) == 0) {
+      for (INode cnode : created) {
+        RenameEntry entry = renameMap.get(cnode.getId());
+        if (entry == null || !entry.isRename()) {
           fullPath[fullPath.length - 1] = cnode.getLocalNameBytes();
-          // must be the case: delete first and then create an inode with the
-          // same name
-          cList.add(new DiffReportEntry(DiffType.CREATE, fullPath));
-          dList.add(new DiffReportEntry(DiffType.DELETE, fullPath));
-          c++;
-          d++;
-        } else if (cnode.compareTo(dnode.getLocalNameBytes()) < 0) {
-          fullPath[fullPath.length - 1] = cnode.getLocalNameBytes();
-          cList.add(new DiffReportEntry(fromEarlier ? DiffType.CREATE
+          list.add(new DiffReportEntry(fromEarlier ? DiffType.CREATE
               : DiffType.DELETE, fullPath));
-          c++;
+        }
+      }
+      for (INode dnode : deleted) {
+        RenameEntry entry = renameMap.get(dnode.getId());
+        if (entry != null && entry.isRename()) {
+          list.add(new DiffReportEntry(DiffType.RENAME,
+              fromEarlier ? entry.getSourcePath() : entry.getTargetPath(),
+              fromEarlier ? entry.getTargetPath() : entry.getSourcePath()));
         } else {
           fullPath[fullPath.length - 1] = dnode.getLocalNameBytes();
-          dList.add(new DiffReportEntry(fromEarlier ? DiffType.DELETE
+          list.add(new DiffReportEntry(fromEarlier ? DiffType.DELETE
               : DiffType.CREATE, fullPath));
-          d++;
         }
       }
-      for (; d < deleted.size(); d++) {
-        fullPath[fullPath.length - 1] = deleted.get(d).getLocalNameBytes();
-        dList.add(new DiffReportEntry(fromEarlier ? DiffType.DELETE
-            : DiffType.CREATE, fullPath));
-      }
-      for (; c < created.size(); c++) {
-        fullPath[fullPath.length - 1] = created.get(c).getLocalNameBytes();
-        cList.add(new DiffReportEntry(fromEarlier ? DiffType.CREATE
-            : DiffType.DELETE, fullPath));
-      }
-      dList.addAll(cList);
-      return dList;
+      return list;
     }
   }
 
@@ -724,34 +708,21 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
    */
   boolean computeDiffBetweenSnapshots(Snapshot fromSnapshot,
       Snapshot toSnapshot, ChildrenDiff diff, INodeDirectory currentINode) {
-    Snapshot earlier = fromSnapshot;
-    Snapshot later = toSnapshot;
-    if (Snapshot.ID_COMPARATOR.compare(fromSnapshot, toSnapshot) > 0) {
-      earlier = toSnapshot;
-      later = fromSnapshot;
-    }
-
-    boolean modified = diffs.changedBetweenSnapshots(earlier, later);
-    if (!modified) {
+    int[] diffIndexPair = diffs.changedBetweenSnapshots(fromSnapshot,
+        toSnapshot);
+    if (diffIndexPair == null) {
       return false;
     }
-
-    final List<DirectoryDiff> difflist = diffs.asList();
-    final int size = difflist.size();
-    int earlierDiffIndex = Collections.binarySearch(difflist, earlier.getId());
-    int laterDiffIndex = later == null ? size : Collections
-        .binarySearch(difflist, later.getId());
-    earlierDiffIndex = earlierDiffIndex < 0 ? (-earlierDiffIndex - 1)
-        : earlierDiffIndex;
-    laterDiffIndex = laterDiffIndex < 0 ? (-laterDiffIndex - 1)
-        : laterDiffIndex;
+    int earlierDiffIndex = diffIndexPair[0];
+    int laterDiffIndex = diffIndexPair[1];
 
     boolean dirMetadataChanged = false;
     INodeDirectoryAttributes dirCopy = null;
+    List<DirectoryDiff> difflist = diffs.asList();
     for (int i = earlierDiffIndex; i < laterDiffIndex; i++) {
       DirectoryDiff sdiff = difflist.get(i);
       diff.combinePosterior(sdiff.diff, null);
-      if (dirMetadataChanged == false && sdiff.snapshotINode != null) {
+      if (!dirMetadataChanged && sdiff.snapshotINode != null) {
         if (dirCopy == null) {
           dirCopy = sdiff.snapshotINode;
         } else if (!dirCopy.metadataEquals(sdiff.snapshotINode)) {
@@ -763,7 +734,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     if (!diff.isEmpty() || dirMetadataChanged) {
       return true;
     } else if (dirCopy != null) {
-      for (int i = laterDiffIndex; i < size; i++) {
+      for (int i = laterDiffIndex; i < difflist.size(); i++) {
         if (!dirCopy.metadataEquals(difflist.get(i).snapshotINode)) {
           return true;
         }

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

@@ -25,6 +25,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 
 /**
@@ -73,7 +74,41 @@ public class FileWithSnapshotFeature implements INode.Feature {
     }
     return max;
   }
-  
+
+  boolean changedBetweenSnapshots(INodeFile file, Snapshot from, Snapshot to) {
+    int[] diffIndexPair = diffs.changedBetweenSnapshots(from, to);
+    if (diffIndexPair == null) {
+      return false;
+    }
+    int earlierDiffIndex = diffIndexPair[0];
+    int laterDiffIndex = diffIndexPair[1];
+
+    final List<FileDiff> diffList = diffs.asList();
+    final long earlierLength = diffList.get(earlierDiffIndex).getFileSize();
+    final long laterLength = laterDiffIndex == diffList.size() ? file
+        .computeFileSize(true, false) : diffList.get(laterDiffIndex)
+        .getFileSize();
+    if (earlierLength != laterLength) { // file length has been changed
+      return true;
+    }
+
+    INodeFileAttributes earlierAttr = null; // check the metadata
+    for (int i = earlierDiffIndex; i < laterDiffIndex; i++) {
+      FileDiff diff = diffList.get(i);
+      if (diff.snapshotINode != null) {
+        earlierAttr = diff.snapshotINode;
+        break;
+      }
+    }
+    if (earlierAttr == null) { // no meta-change at all, return false
+      return false;
+    }
+    INodeFileAttributes laterAttr = diffs.getSnapshotINode(
+        Math.max(Snapshot.getSnapshotId(from), Snapshot.getSnapshotId(to)),
+        file);
+    return !earlierAttr.metadataEquals(laterAttr);
+  }
+
   public String getDetailedString() {
     return (isCurrentFileDeleted()? "(DELETED), ": ", ") + diffs;
   }

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

@@ -24,6 +24,7 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Iterator;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.SortedMap;
@@ -43,6 +44,9 @@ import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeMap;
+import org.apache.hadoop.hdfs.server.namenode.INodeReference;
+import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
+import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.ChildrenDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
@@ -51,6 +55,7 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.util.Time;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 import com.google.common.primitives.SignedBytes;
 
 /**
@@ -98,7 +103,43 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
         }
       }
     };
-    
+
+    static class RenameEntry {
+      private byte[][] sourcePath;
+      private byte[][] targetPath;
+
+      void setSource(INode source, byte[][] sourceParentPath) {
+        Preconditions.checkState(sourcePath == null);
+        sourcePath = new byte[sourceParentPath.length + 1][];
+        System.arraycopy(sourceParentPath, 0, sourcePath, 0,
+            sourceParentPath.length);
+        sourcePath[sourcePath.length - 1] = source.getLocalNameBytes();
+      }
+
+      void setTarget(INode target, byte[][] targetParentPath) {
+        targetPath = new byte[targetParentPath.length + 1][];
+        System.arraycopy(targetParentPath, 0, targetPath, 0,
+            targetParentPath.length);
+        targetPath[targetPath.length - 1] = target.getLocalNameBytes();
+      }
+
+      void setTarget(byte[][] targetPath) {
+        this.targetPath = targetPath;
+      }
+
+      boolean isRename() {
+        return sourcePath != null && targetPath != null;
+      }
+
+      byte[][] getSourcePath() {
+        return sourcePath;
+      }
+
+      byte[][] getTargetPath() {
+        return targetPath;
+      }
+    }
+
     /** The root directory of the snapshots */
     private final INodeDirectorySnapshottable snapshotRoot;
     /** The starting point of the difference */
@@ -109,7 +150,7 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
      * A map recording modified INodeFile and INodeDirectory and their relative
      * path corresponding to the snapshot root. Sorted based on their names.
      */ 
-    private final SortedMap<INode, byte[][]> diffMap = 
+    private final SortedMap<INode, byte[][]> diffMap =
         new TreeMap<INode, byte[][]>(INODE_COMPARATOR);
     /**
      * A map capturing the detailed difference about file creation/deletion.
@@ -119,7 +160,10 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
      */
     private final Map<INodeDirectory, ChildrenDiff> dirDiffMap = 
         new HashMap<INodeDirectory, ChildrenDiff>();
-    
+
+    private final Map<Long, RenameEntry> renameMap =
+        new HashMap<Long, RenameEntry>();
+
     SnapshotDiffInfo(INodeDirectorySnapshottable snapshotRoot, Snapshot start,
         Snapshot end) {
       this.snapshotRoot = snapshotRoot;
@@ -132,8 +176,36 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
         ChildrenDiff diff) {
       dirDiffMap.put(dir, diff);
       diffMap.put(dir, relativePath);
+      // detect rename
+      for (INode created : diff.getList(ListType.CREATED)) {
+        if (created.isReference()) {
+          RenameEntry entry = getEntry(created.getId());
+          if (entry.getTargetPath() == null) {
+            entry.setTarget(created, relativePath);
+          }
+        }
+      }
+      for (INode deleted : diff.getList(ListType.DELETED)) {
+        if (deleted instanceof INodeReference.WithName) {
+          RenameEntry entry = getEntry(deleted.getId());
+          entry.setSource(deleted, relativePath);
+        }
+      }
     }
-    
+
+    private RenameEntry getEntry(long inodeId) {
+      RenameEntry entry = renameMap.get(inodeId);
+      if (entry == null) {
+        entry = new RenameEntry();
+        renameMap.put(inodeId, entry);
+      }
+      return entry;
+    }
+
+    private void setRenameTarget(long inodeId, byte[][] path) {
+      getEntry(inodeId).setTarget(path);
+    }
+
     /** Add a modified file */ 
     private void addFileDiff(INodeFile file, byte[][] relativePath) {
       diffMap.put(file, relativePath);
@@ -152,11 +224,11 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
       List<DiffReportEntry> diffReportList = new ArrayList<DiffReportEntry>();
       for (INode node : diffMap.keySet()) {
         diffReportList.add(new DiffReportEntry(DiffType.MODIFY, diffMap
-            .get(node)));
+            .get(node), null));
         if (node.isDirectory()) {
           ChildrenDiff dirDiff = dirDiffMap.get(node);
           List<DiffReportEntry> subList = dirDiff.generateReport(
-              diffMap.get(node), isFromEarlier());
+              diffMap.get(node), isFromEarlier(), renameMap);
           diffReportList.addAll(subList);
         }
       }
@@ -423,25 +495,37 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
    */
   private void computeDiffRecursively(INode node, List<byte[]> parentPath,
       SnapshotDiffInfo diffReport) {
-    ChildrenDiff diff = new ChildrenDiff();
+    final Snapshot earlierSnapshot = diffReport.isFromEarlier() ?
+        diffReport.from : diffReport.to;
+    final Snapshot laterSnapshot = diffReport.isFromEarlier() ?
+        diffReport.to : diffReport.from;
     byte[][] relativePath = parentPath.toArray(new byte[parentPath.size()][]);
     if (node.isDirectory()) {
+      final ChildrenDiff diff = new ChildrenDiff();
       INodeDirectory dir = node.asDirectory();
       DirectoryWithSnapshotFeature sf = dir.getDirectoryWithSnapshotFeature();
       if (sf != null) {
-        boolean change = sf.computeDiffBetweenSnapshots(diffReport.from,
-            diffReport.to, diff, dir);
+        boolean change = sf.computeDiffBetweenSnapshots(earlierSnapshot,
+            laterSnapshot, diff, dir);
         if (change) {
           diffReport.addDirDiff(dir, relativePath, diff);
         }
       }
-      ReadOnlyList<INode> children = dir.getChildrenList(
-          diffReport.isFromEarlier() ? Snapshot.getSnapshotId(diffReport.to) : 
-            Snapshot.getSnapshotId(diffReport.from));
+      ReadOnlyList<INode> children = dir.getChildrenList(earlierSnapshot
+          .getId());
       for (INode child : children) {
         final byte[] name = child.getLocalNameBytes();
-        if (diff.searchIndex(ListType.CREATED, name) < 0
-            && diff.searchIndex(ListType.DELETED, name) < 0) {
+        boolean toProcess = diff.searchIndex(ListType.DELETED, name) < 0;
+        if (!toProcess && child instanceof INodeReference.WithName) {
+          byte[][] renameTargetPath = findRenameTargetPath((WithName) child,
+              laterSnapshot == null ? Snapshot.CURRENT_STATE_ID : 
+                                      laterSnapshot.getId());
+          if (renameTargetPath != null) {
+            toProcess = true;
+            diffReport.setRenameTarget(child.getId(), renameTargetPath);
+          }
+        }
+        if (toProcess) {
           parentPath.add(name);
           computeDiffRecursively(child, parentPath, diffReport);
           parentPath.remove(parentPath.size() - 1);
@@ -449,18 +533,47 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
       }
     } else if (node.isFile() && node.asFile().isWithSnapshot()) {
       INodeFile file = node.asFile();
-      Snapshot earlierSnapshot = diffReport.isFromEarlier() ? diffReport.from
-          : diffReport.to;
-      Snapshot laterSnapshot = diffReport.isFromEarlier() ? diffReport.to
-          : diffReport.from;
-      boolean change = file.getDiffs().changedBetweenSnapshots(earlierSnapshot,
-          laterSnapshot);
+      boolean change = file.getFileWithSnapshotFeature()
+          .changedBetweenSnapshots(file, earlierSnapshot, laterSnapshot);
       if (change) {
         diffReport.addFileDiff(file, relativePath);
       }
     }
   }
-  
+
+  /**
+   * We just found a deleted WithName node as the source of a rename operation.
+   * However, we should include it in our snapshot diff report as rename only
+   * if the rename target is also under the same snapshottable directory.
+   */
+  private byte[][] findRenameTargetPath(INodeReference.WithName wn,
+      final int snapshotId) {
+    INode inode = wn.getReferredINode();
+    final LinkedList<byte[]> ancestors = Lists.newLinkedList();
+    while (inode != null) {
+      if (inode == this) {
+        return ancestors.toArray(new byte[ancestors.size()][]);
+      }
+      if (inode instanceof INodeReference.WithCount) {
+        inode = ((WithCount) inode).getParentRef(snapshotId);
+      } else {
+        INode parent = inode.getParentReference() != null ? inode
+            .getParentReference() : inode.getParent();
+        if (parent != null && parent instanceof INodeDirectory) {
+          int sid = parent.asDirectory().searchChild(inode);
+          if (sid < snapshotId) {
+            return null;
+          }
+        }
+        if (!(parent instanceof WithCount)) {
+          ancestors.addFirst(inode.getLocalNameBytes());
+        }
+        inode = parent;
+      }
+    }
+    return null;
+  }
+
   /**
    * Replace itself with {@link INodeDirectoryWithSnapshot} or
    * {@link INodeDirectory} depending on the latest snapshot.
@@ -549,4 +662,4 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
       });
     }
   }
-}
+}

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

@@ -275,6 +275,7 @@ message SnapshottableDirectoryListingProto {
 message SnapshotDiffReportEntryProto {
   required bytes fullpath = 1;
   required string modificationLabel = 2;
+  optional bytes targetPath = 3;
 }
 
 /**

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

@@ -255,7 +255,35 @@
       <tr><td>fromSnapshot</td><td>The name of the starting snapshot.</td></tr>
       <tr><td>toSnapshot</td><td>The name of the ending snapshot.</td></tr>
     </table></li>
+    <li>Results:
+      <table>
+        <tr><td>+</td><td>The file/directory has been created.</td></tr>
+        <tr><td>-</td><td>The file/directory has been deleted.</td></tr>
+        <tr><td>M</td><td>The file/directory has been modified.</td></tr>
+        <tr><td>R</td><td>The file/directory has been renamed.</td></tr>
+      </table>
+    </li>
   </ul>
+  <p>
+    A <em>RENAME</em> entry indicates a file/directory has been renamed but
+    is still under the same snapshottable directory. A file/directory is
+    reported as deleted if it was renamed to outside of the snapshottble directory.
+    A file/directory renamed from outside of the snapshottble directory is
+    reported as newly created.
+  </p>
+  <p>
+    The snapshot difference report does not guarantee the same operation sequence.
+    For example, if we rename the directory <em>"/foo"</em> to <em>"/foo2"</em>, and
+    then append new data to the file <em>"/foo2/bar"</em>, the difference report will
+    be:
+    <source>
+    R. /foo -> /foo2
+    M. /foo/bar
+    </source>
+    I.e., the changes on the files/directories under a renamed directory is
+    reported using the original path before the rename (<em>"/foo/bar"</em> in
+    the above example).
+  </p>
   <p>
     See also the corresponding Java API
     <code>SnapshotDiffReport getSnapshotDiffReport(Path path, String fromSnapshot, String toSnapshot)</code>

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

@@ -1,295 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.namenode.snapshot;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
-import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.INodeReference;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Test getting the full path name of a given inode. The INode may be in
- * snapshot.
- */
-public class TestFullPathNameWithSnapshot {
-  private static final int BLOCKSIZE = 1024;
-  private static final short REPLICATION = 1;
-
-  private Configuration conf;
-  private MiniDFSCluster cluster;
-  private FSDirectory fsdir;
-  private DistributedFileSystem dfs;
-
-  @Before
-  public void setUp() throws Exception {
-    conf = new Configuration();
-    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
-        .build();
-    cluster.waitActive();
-
-    fsdir = cluster.getNamesystem().getFSDirectory();
-    dfs = cluster.getFileSystem();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  /**
-   * Normal case without snapshot involved
-   */
-  @Test
-  public void testNormalINode() throws Exception {
-    final Path bar = new Path("/foo/bar");
-    dfs.mkdirs(bar);
-    final Path file = new Path(bar, "file");
-    DFSTestUtil.createFile(dfs, file, BLOCKSIZE, REPLICATION, 0L);
-    INode fileNode = fsdir.getINode4Write(file.toString());
-    byte[][] pathComponents = FSDirectory
-        .getPathComponentsWithSnapshot(fileNode);
-    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(file.toString()),
-        pathComponents);
-  }
-
-  /**
-   * INode in deleted list
-   */
-  @Test
-  public void testDeletedINode() throws Exception {
-    final Path foo = new Path("/foo");
-    final Path f1 = new Path(foo, "f1");
-    DFSTestUtil.createFile(dfs, f1, BLOCKSIZE, REPLICATION, 0L);
-    final Path bar = new Path(foo, "bar");
-    dfs.mkdirs(bar);
-    final Path f2 = new Path(bar, "f2");
-    DFSTestUtil.createFile(dfs, f2, BLOCKSIZE, REPLICATION, 0L);
-
-    INode f1Node = fsdir.getINode4Write(f1.toString());
-    INode f2Node = fsdir.getINode4Write(f2.toString());
-
-    SnapshotTestHelper.createSnapshot(dfs, foo, "s1");
-    dfs.delete(bar, true);
-    SnapshotTestHelper.createSnapshot(dfs, foo, "s2");
-    dfs.delete(f1, true);
-
-    byte[][] f1Components = FSDirectory.getPathComponentsWithSnapshot(f1Node);
-    byte[][] f2Components = FSDirectory.getPathComponentsWithSnapshot(f2Node);
-    // expected: /foo/.snapshot/s2/f1
-    String f1Snapshot = SnapshotTestHelper.getSnapshotPath(foo, "s2",
-        f1.getName()).toString();
-    // expected: /foo/.snapshot/s1/bar/f2
-    String f2Snapshot = SnapshotTestHelper.getSnapshotPath(foo, "s1", "bar/f2")
-        .toString();
-    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(f1Snapshot),
-        f1Components);
-    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(f2Snapshot),
-        f2Components);
-
-    // delete snapshot s2
-    dfs.deleteSnapshot(foo, "s2");
-    // expected: /foo/.snapshot/s1/f1
-    f1Snapshot = SnapshotTestHelper.getSnapshotPath(foo, "s1", f1.getName())
-        .toString();
-    f1Components = FSDirectory.getPathComponentsWithSnapshot(f1Node);
-    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(f1Snapshot),
-        f1Components);
-  }
-
-  /**
-   * INode after renaming
-   */
-  @Test
-  public void testRenamedINode() throws Exception {
-    final Path foo = new Path("/foo");
-    final Path bar = new Path(foo, "bar");
-    final Path f1 = new Path(bar, "f1");
-    final Path f2 = new Path(bar, "f2");
-    DFSTestUtil.createFile(dfs, f1, BLOCKSIZE, REPLICATION, 0L);
-    DFSTestUtil.createFile(dfs, f2, BLOCKSIZE, REPLICATION, 0L);
-
-    // create snapshot s1
-    SnapshotTestHelper.createSnapshot(dfs, foo, "s1");
-    INode f2Node = fsdir.getINode4Write(f2.toString());
-    // delete /foo/bar/f2
-    dfs.delete(f2, true);
-    // rename bar to bar2
-    final Path bar2 = new Path(foo, "bar2");
-    dfs.rename(bar, bar2);
-    // create snapshot s2
-    SnapshotTestHelper.createSnapshot(dfs, foo, "s2");
-
-    // /foo/.snapshot/s1/bar
-    Path barPath = SnapshotTestHelper.getSnapshotPath(foo, "s1", bar.getName());
-    INode barNode = fsdir.getINode(barPath.toString());
-    Assert.assertTrue(barNode instanceof INodeReference.WithName);
-    INode bar2Node = fsdir.getINode(bar2.toString());
-    Assert.assertTrue(bar2Node instanceof INodeReference.DstReference);
-    byte[][] barComponents = FSDirectory.getPathComponentsWithSnapshot(barNode);
-    byte[][] bar2Components = FSDirectory
-        .getPathComponentsWithSnapshot(bar2Node);
-    DFSTestUtil.checkComponentsEquals(
-        INode.getPathComponents(barPath.toString()), barComponents);
-    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(bar2.toString()),
-        bar2Components);
-
-    byte[][] f2Components = FSDirectory.getPathComponentsWithSnapshot(f2Node);
-    // expected: /foo/.snapshot/s1/bar/f2
-    Path deletedf2 = SnapshotTestHelper.getSnapshotPath(foo, "s1", "bar/f2");
-    DFSTestUtil.checkComponentsEquals(
-        INode.getPathComponents(deletedf2.toString()), f2Components);
-
-    final Path newf1 = new Path(bar2, f1.getName());
-    INode f1Node = fsdir.getINode(newf1.toString());
-    Assert.assertTrue(dfs.delete(newf1, true));
-    Path deletedf1 = SnapshotTestHelper.getSnapshotPath(foo, "s2", "bar2/f1");
-    byte[][] f1Components = FSDirectory.getPathComponentsWithSnapshot(f1Node);
-    DFSTestUtil.checkComponentsEquals(
-        INode.getPathComponents(deletedf1.toString()), f1Components);
-  }
-
-  /**
-   * Similar with testRenamedINode but the rename is across two snapshottable
-   * directory.
-   */
-  @Test
-  public void testRenamedINode2() throws Exception {
-    final Path foo1 = new Path("/foo1");
-    final Path foo2 = new Path("/foo2");
-    final Path bar = new Path(foo1, "bar");
-    final Path f1 = new Path(bar, "f1");
-    final Path f2 = new Path(bar, "f2");
-    dfs.mkdirs(foo2);
-    DFSTestUtil.createFile(dfs, f1, BLOCKSIZE, REPLICATION, 0L);
-    DFSTestUtil.createFile(dfs, f2, BLOCKSIZE, REPLICATION, 0L);
-
-    // create snapshots on foo1 and foo2
-    SnapshotTestHelper.createSnapshot(dfs, foo1, "s1");
-    SnapshotTestHelper.createSnapshot(dfs, foo2, "s2");
-    INode f2Node = fsdir.getINode4Write(f2.toString());
-    // delete /foo1/bar/f2
-    dfs.delete(f2, true);
-    // rename bar to bar2
-    final Path bar2 = new Path(foo2, "bar2");
-    dfs.rename(bar, bar2);
-    // create snapshot s3 and s4 on foo1 and foo2
-    SnapshotTestHelper.createSnapshot(dfs, foo1, "s3");
-    SnapshotTestHelper.createSnapshot(dfs, foo2, "s4");
-
-    // /foo1/.snapshot/s1/bar
-    Path barPath = SnapshotTestHelper
-        .getSnapshotPath(foo1, "s1", bar.getName());
-    INode barNode = fsdir.getINode(barPath.toString());
-    Assert.assertTrue(barNode instanceof INodeReference.WithName);
-    INode bar2Node = fsdir.getINode(bar2.toString());
-    Assert.assertTrue(bar2Node instanceof INodeReference.DstReference);
-    byte[][] barComponents = FSDirectory.getPathComponentsWithSnapshot(barNode);
-    byte[][] bar2Components = FSDirectory
-        .getPathComponentsWithSnapshot(bar2Node);
-    DFSTestUtil.checkComponentsEquals(
-        INode.getPathComponents(barPath.toString()), barComponents);
-    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(bar2.toString()),
-        bar2Components);
-
-    byte[][] f2Components = FSDirectory.getPathComponentsWithSnapshot(f2Node);
-    // expected: /foo1/.snapshot/s1/bar/f2
-    Path deletedf2 = SnapshotTestHelper.getSnapshotPath(foo1, "s1", "bar/f2");
-    DFSTestUtil.checkComponentsEquals(
-        INode.getPathComponents(deletedf2.toString()), f2Components);
-
-    final Path newf1 = new Path(bar2, f1.getName());
-    INode f1Node = fsdir.getINode(newf1.toString());
-    Assert.assertTrue(dfs.delete(newf1, true));
-    // /foo2/.snapshot/s4/bar2/f1
-    Path deletedf1 = SnapshotTestHelper.getSnapshotPath(foo2, "s4", "bar2/f1");
-    byte[][] f1Components = FSDirectory.getPathComponentsWithSnapshot(f1Node);
-    DFSTestUtil.checkComponentsEquals(
-        INode.getPathComponents(deletedf1.toString()), f1Components);
-  }
-
-  /**
-   * Rename a directory to its prior descendant
-   */
-  @Test
-  public void testNestedRename() throws Exception {
-    final Path sdir1 = new Path("/dir1");
-    final Path sdir2 = new Path("/dir2");
-    final Path foo = new Path(sdir1, "foo");
-    final Path bar = new Path(foo, "bar");
-    dfs.mkdirs(bar);
-    dfs.mkdirs(sdir2);
-
-    SnapshotTestHelper.createSnapshot(dfs, sdir1, "s1");
-
-    // /dir1/foo/bar -> /dir2/bar
-    final Path bar2 = new Path(sdir2, "bar");
-    dfs.rename(bar, bar2);
-
-    // /dir1/foo -> /dir2/bar/foo
-    final Path foo2 = new Path(bar2, "foo");
-    dfs.rename(foo, foo2);
-
-    // /dir2/bar
-    INode bar2Node = fsdir.getINode(bar2.toString());
-    Assert.assertTrue(bar2Node instanceof INodeReference.DstReference);
-    byte[][] bar2Components = FSDirectory
-        .getPathComponentsWithSnapshot(bar2Node);
-    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(bar2.toString()),
-        bar2Components);
-
-    // /dir1/.snapshot/s1/foo/bar
-    String oldbar = SnapshotTestHelper.getSnapshotPath(sdir1, "s1", "foo/bar")
-        .toString();
-    INode oldbarNode = fsdir.getINode(oldbar);
-    Assert.assertTrue(oldbarNode instanceof INodeReference.WithName);
-    byte[][] oldbarComponents = FSDirectory
-        .getPathComponentsWithSnapshot(oldbarNode);
-    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(oldbar),
-        oldbarComponents);
-
-    // /dir2/bar/foo
-    INode foo2Node = fsdir.getINode(foo2.toString());
-    Assert.assertTrue(foo2Node instanceof INodeReference.DstReference);
-    byte[][] foo2Components = FSDirectory
-        .getPathComponentsWithSnapshot(foo2Node);
-    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(foo2.toString()),
-        foo2Components);
-
-    // /dir1/.snapshot/s1/foo
-    String oldfoo = SnapshotTestHelper.getSnapshotPath(sdir1, "s1",
-        foo.getName()).toString();
-    INode oldfooNode = fsdir.getINode(oldfoo);
-    Assert.assertTrue(oldfooNode instanceof INodeReference.WithName);
-    byte[][] oldfooComponents = FSDirectory
-        .getPathComponentsWithSnapshot(oldfooNode);
-    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(oldfoo),
-        oldfooComponents);
-  }
-}

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

@@ -169,10 +169,11 @@ public class TestRenameWithSnapshots {
   }
   
   private static boolean existsInDiffReport(List<DiffReportEntry> entries,
-      DiffType type, String relativePath) {
+      DiffType type, String sourcePath, String targetPath) {
     for (DiffReportEntry entry : entries) {
-      if ((entry.getType() == type)
-          && ((new String(entry.getRelativePath())).compareTo(relativePath) == 0)) {
+      if (entry.equals(new DiffReportEntry(type, DFSUtil
+          .string2Bytes(sourcePath), targetPath == null ? null : DFSUtil
+          .string2Bytes(targetPath)))) {
         return true;
       }
     }
@@ -195,8 +196,9 @@ public class TestRenameWithSnapshots {
     SnapshotDiffReport diffReport = hdfs.getSnapshotDiffReport(sub1, snap1, "");
     List<DiffReportEntry> entries = diffReport.getDiffList();
     assertTrue(entries.size() == 2);
-    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, ""));
-    assertTrue(existsInDiffReport(entries, DiffType.CREATE, file2.getName()));
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, "", null));
+    assertTrue(existsInDiffReport(entries, DiffType.CREATE, file2.getName(),
+        null));
   }
 
   /**
@@ -215,10 +217,10 @@ public class TestRenameWithSnapshots {
     SnapshotDiffReport diffReport = hdfs.getSnapshotDiffReport(sub1, snap1, "");
     System.out.println("DiffList is " + diffReport.toString());
     List<DiffReportEntry> entries = diffReport.getDiffList();
-    assertTrue(entries.size() == 3);
-    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, ""));
-    assertTrue(existsInDiffReport(entries, DiffType.CREATE, file2.getName()));
-    assertTrue(existsInDiffReport(entries, DiffType.DELETE, file1.getName()));
+    assertTrue(entries.size() == 2);
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, "", null));
+    assertTrue(existsInDiffReport(entries, DiffType.RENAME, file1.getName(),
+        file2.getName()));
   }
 
   @Test (timeout=60000)
@@ -238,26 +240,26 @@ public class TestRenameWithSnapshots {
     diffReport = hdfs.getSnapshotDiffReport(sub1, snap1, snap2);
     LOG.info("DiffList is " + diffReport.toString());
     List<DiffReportEntry> entries = diffReport.getDiffList();
-    assertTrue(entries.size() == 3);
-    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, ""));
-    assertTrue(existsInDiffReport(entries, DiffType.CREATE, file2.getName()));
-    assertTrue(existsInDiffReport(entries, DiffType.DELETE, file1.getName()));
+    assertTrue(entries.size() == 2);
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, "", null));
+    assertTrue(existsInDiffReport(entries, DiffType.RENAME, file1.getName(),
+        file2.getName()));
     
     diffReport = hdfs.getSnapshotDiffReport(sub1, snap2, "");
     LOG.info("DiffList is " + diffReport.toString());
     entries = diffReport.getDiffList();
-    assertTrue(entries.size() == 3);
-    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, ""));
-    assertTrue(existsInDiffReport(entries, DiffType.CREATE, file3.getName()));
-    assertTrue(existsInDiffReport(entries, DiffType.DELETE, file2.getName()));
+    assertTrue(entries.size() == 2);
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, "", null));
+    assertTrue(existsInDiffReport(entries, DiffType.RENAME, file2.getName(),
+        file3.getName()));
     
     diffReport = hdfs.getSnapshotDiffReport(sub1, snap1, "");
     LOG.info("DiffList is " + diffReport.toString());
     entries = diffReport.getDiffList();
-    assertTrue(entries.size() == 3);
-    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, ""));
-    assertTrue(existsInDiffReport(entries, DiffType.CREATE, file3.getName()));
-    assertTrue(existsInDiffReport(entries, DiffType.DELETE, file1.getName()));
+    assertTrue(entries.size() == 2);
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, "", null));
+    assertTrue(existsInDiffReport(entries, DiffType.RENAME, file1.getName(),
+        file3.getName()));
   }
   
   @Test (timeout=60000)
@@ -280,11 +282,10 @@ public class TestRenameWithSnapshots {
         "");
     LOG.info("DiffList is \n\"" + diffReport.toString() + "\"");
     List<DiffReportEntry> entries = diffReport.getDiffList();
-    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, sub2.getName()));
-    assertTrue(existsInDiffReport(entries, DiffType.CREATE, sub2.getName()
-        + "/" + sub2file2.getName()));
-    assertTrue(existsInDiffReport(entries, DiffType.DELETE, sub2.getName()
-        + "/" + sub2file1.getName()));
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, sub2.getName(),
+        null));
+    assertTrue(existsInDiffReport(entries, DiffType.RENAME, sub2.getName()
+        + "/" + sub2file1.getName(), sub2.getName() + "/" + sub2file2.getName()));
   }
 
   @Test (timeout=60000)
@@ -307,10 +308,10 @@ public class TestRenameWithSnapshots {
         "");
     LOG.info("DiffList is \n\"" + diffReport.toString() + "\"");
     List<DiffReportEntry> entries = diffReport.getDiffList();
-    assertEquals(3, entries.size());
-    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, ""));
-    assertTrue(existsInDiffReport(entries, DiffType.CREATE, sub3.getName()));
-    assertTrue(existsInDiffReport(entries, DiffType.DELETE, sub2.getName()));
+    assertEquals(2, entries.size());
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, "", null));
+    assertTrue(existsInDiffReport(entries, DiffType.RENAME, sub2.getName(),
+        sub3.getName()));
   }
   
   /**
@@ -2406,12 +2407,12 @@ public class TestRenameWithSnapshots {
     LOG.info("DiffList is \n\"" + report.toString() + "\"");
     List<DiffReportEntry> entries = report.getDiffList();
     assertEquals(7, entries.size());
-    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, ""));
-    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, foo.getName()));
-    assertTrue(existsInDiffReport(entries, DiffType.DELETE, bar.getName()));
-    assertTrue(existsInDiffReport(entries, DiffType.CREATE, newDir.getName()));
-    assertTrue(existsInDiffReport(entries, DiffType.DELETE, "foo/file1"));
-    assertTrue(existsInDiffReport(entries, DiffType.DELETE, "foo/file2"));
-    assertTrue(existsInDiffReport(entries, DiffType.DELETE, "foo/file3"));
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, "", null));
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, foo.getName(), null));
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, bar.getName(), null));
+    assertTrue(existsInDiffReport(entries, DiffType.DELETE, "foo/file1", null));
+    assertTrue(existsInDiffReport(entries, DiffType.RENAME, "bar", "newDir"));
+    assertTrue(existsInDiffReport(entries, DiffType.RENAME, "foo/file2", "newDir/file2"));
+    assertTrue(existsInDiffReport(entries, DiffType.RENAME, "foo/file3", "newDir/file1"));
   }
 }

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

@@ -25,6 +25,7 @@ import java.io.IOException;
 import java.util.HashMap;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -143,7 +144,7 @@ public class TestSnapshotDiffReport {
       hdfs.createSnapshot(snapshotDir, genSnapshotName(snapshotDir));
     }
     // modify file10
-    hdfs.setReplication(file10, (short) (REPLICATION - 1));
+    hdfs.setReplication(file10, (short) (REPLICATION + 1));
   }
   
   /** check the correctness of the diff reports */
@@ -166,11 +167,11 @@ public class TestSnapshotDiffReport {
       } else if (entry.getType() == DiffType.DELETE) {
         assertTrue(report.getDiffList().contains(entry));
         assertTrue(inverseReport.getDiffList().contains(
-            new DiffReportEntry(DiffType.CREATE, entry.getRelativePath())));
+            new DiffReportEntry(DiffType.CREATE, entry.getSourcePath())));
       } else if (entry.getType() == DiffType.CREATE) {
         assertTrue(report.getDiffList().contains(entry));
         assertTrue(inverseReport.getDiffList().contains(
-            new DiffReportEntry(DiffType.DELETE, entry.getRelativePath())));
+            new DiffReportEntry(DiffType.DELETE, entry.getSourcePath())));
       }
     }
   }
@@ -329,5 +330,166 @@ public class TestSnapshotDiffReport {
         new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
         new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("subsub1")));
   }
-  
+
+  /**
+   * Rename a directory to its prior descendant, and verify the diff report.
+   */
+  @Test
+  public void testDiffReportWithRename() throws Exception {
+    final Path root = new Path("/");
+    final Path sdir1 = new Path(root, "dir1");
+    final Path sdir2 = new Path(root, "dir2");
+    final Path foo = new Path(sdir1, "foo");
+    final Path bar = new Path(foo, "bar");
+    hdfs.mkdirs(bar);
+    hdfs.mkdirs(sdir2);
+
+    // create snapshot on root
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s1");
+
+    // /dir1/foo/bar -> /dir2/bar
+    final Path bar2 = new Path(sdir2, "bar");
+    hdfs.rename(bar, bar2);
+
+    // /dir1/foo -> /dir2/bar/foo
+    final Path foo2 = new Path(bar2, "foo");
+    hdfs.rename(foo, foo2);
+
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s2");
+    // let's delete /dir2 to make things more complicated
+    hdfs.delete(sdir2, true);
+
+    verifyDiffReport(root, "s1", "s2",
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir1")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("dir1/foo"),
+            DFSUtil.string2Bytes("dir2/bar/foo")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir2")),
+        new DiffReportEntry(DiffType.MODIFY,
+            DFSUtil.string2Bytes("dir1/foo/bar")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir1/foo")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil
+            .string2Bytes("dir1/foo/bar"), DFSUtil.string2Bytes("dir2/bar")));
+  }
+
+  /**
+   * Rename a file/dir outside of the snapshottable dir should be reported as
+   * deleted. Rename a file/dir from outside should be reported as created.
+   */
+  @Test
+  public void testDiffReportWithRenameOutside() throws Exception {
+    final Path root = new Path("/");
+    final Path dir1 = new Path(root, "dir1");
+    final Path dir2 = new Path(root, "dir2");
+    final Path foo = new Path(dir1, "foo");
+    final Path fileInFoo = new Path(foo, "file");
+    final Path bar = new Path(dir2, "bar");
+    final Path fileInBar = new Path(bar, "file");
+    DFSTestUtil.createFile(hdfs, fileInFoo, BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, fileInBar, BLOCKSIZE, REPLICATION, seed);
+
+    // create snapshot on /dir1
+    SnapshotTestHelper.createSnapshot(hdfs, dir1, "s0");
+
+    // move bar into dir1
+    final Path newBar = new Path(dir1, "newBar");
+    hdfs.rename(bar, newBar);
+    // move foo out of dir1 into dir2
+    final Path newFoo = new Path(dir2, "new");
+    hdfs.rename(foo, newFoo);
+
+    SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
+    verifyDiffReport(dir1, "s0", "s1",
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes(newBar
+            .getName())),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes(foo.getName())));
+  }
+
+  /**
+   * Renaming a file/dir then delete the ancestor dir of the rename target
+   * should be reported as deleted.
+   */
+  @Test
+  public void testDiffReportWithRenameAndDelete() throws Exception {
+    final Path root = new Path("/");
+    final Path dir1 = new Path(root, "dir1");
+    final Path dir2 = new Path(root, "dir2");
+    final Path foo = new Path(dir1, "foo");
+    final Path fileInFoo = new Path(foo, "file");
+    final Path bar = new Path(dir2, "bar");
+    final Path fileInBar = new Path(bar, "file");
+    DFSTestUtil.createFile(hdfs, fileInFoo, BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, fileInBar, BLOCKSIZE, REPLICATION, seed);
+
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s0");
+    hdfs.rename(fileInFoo, fileInBar, Rename.OVERWRITE);
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s1");
+    verifyDiffReport(root, "s0", "s1",
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir1/foo")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir2/bar")),
+        new DiffReportEntry(DiffType.DELETE, DFSUtil
+            .string2Bytes("dir2/bar/file")),
+        new DiffReportEntry(DiffType.RENAME,
+            DFSUtil.string2Bytes("dir1/foo/file"),
+            DFSUtil.string2Bytes("dir2/bar/file")));
+
+    // delete bar
+    hdfs.delete(bar, true);
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s2");
+    verifyDiffReport(root, "s0", "s2",
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir1/foo")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir2")),
+        new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("dir2/bar")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("dir1/foo/file")));
+  }
+
+  @Test
+  public void testDiffReportWithRenameToNewDir() throws Exception {
+    final Path root = new Path("/");
+    final Path foo = new Path(root, "foo");
+    final Path fileInFoo = new Path(foo, "file");
+    DFSTestUtil.createFile(hdfs, fileInFoo, BLOCKSIZE, REPLICATION, seed);
+
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s0");
+    final Path bar = new Path(root, "bar");
+    hdfs.mkdirs(bar);
+    final Path fileInBar = new Path(bar, "file");
+    hdfs.rename(fileInFoo, fileInBar);
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s1");
+
+    verifyDiffReport(root, "s0", "s1",
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("foo")),
+        new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("bar")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("foo/file"),
+            DFSUtil.string2Bytes("bar/file")));
+  }
+
+  /**
+   * Rename a file and then append some data to it
+   */
+  @Test
+  public void testDiffReportWithRenameAndAppend() throws Exception {
+    final Path root = new Path("/");
+    final Path foo = new Path(root, "foo");
+    DFSTestUtil.createFile(hdfs, foo, BLOCKSIZE, REPLICATION, seed);
+
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s0");
+    final Path bar = new Path(root, "bar");
+    hdfs.rename(foo, bar);
+    DFSTestUtil.appendFile(hdfs, bar, 10); // append 10 bytes
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s1");
+
+    // we always put modification on the file before rename
+    verifyDiffReport(root, "s0", "s1",
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("foo")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("foo"),
+            DFSUtil.string2Bytes("bar")));
+  }
 }