Bladeren bron

HDFS-4612. Not to use INode.getParent() when generating snapshot diff report. Contributed by Jing Zhao

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1460590 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 jaren geleden
bovenliggende
commit
cc2f96f655

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

@@ -210,3 +210,6 @@ Branch-2802 Snapshot (Unreleased)
 
   HDFS-4627. Fix FSImageFormat#Loader NPE and synchronization issues.
   (Jing Zhao via suresh)
+
+  HDFS-4612. Not to use INode.getParent() when generating snapshot diff report.
+  (Jing Zhao via szetszwo)

+ 0 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -1621,21 +1621,6 @@ public class FSDirectory implements Closeable {
     return getFullPathName(inodes, inodes.length - 1);
   }
   
-  /**
-   * For a given inode, get its relative path from its ancestor.
-   * @param inode The given inode.
-   * @param ancestor An ancestor inode of the given inode.
-   * @return The relative path name represented in an array of byte array.
-   */
-  static byte[][] getRelativePathNameBytes(INode inode, INode ancestor) {
-    INode[] inodes = getRelativePathINodes(inode, ancestor);
-    byte[][] path = new byte[inodes.length][];
-    for (int i = 0; i < inodes.length; i++) {
-      path[i] = inodes[i].getLocalNameBytes();
-    }
-    return path;
-  }
-  
   /**
    * Create a directory 
    * If ancestor directories do not exist, automatically create them.

+ 0 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -413,13 +413,6 @@ public abstract class INode implements Diff.Element<byte[]> {
     // Get the full path name of this inode.
     return FSDirectory.getFullPathName(this);
   }
-
-  /** 
-   * @return The full path name represented in a list of byte array
-   */
-  public final byte[][] getRelativePathNameBytes(INode ancestor) {
-    return FSDirectory.getRelativePathNameBytes(this, ancestor);
-  }
   
   @Override
   public String toString() {

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

@@ -26,6 +26,8 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -100,17 +102,18 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
     /** The end point of the difference */
     private final Snapshot to;
     /**
-     * The list recording modified INodeFile and INodeDirectory. Sorted based on
-     * their names.
+     * A map recording modified INodeFile and INodeDirectory and their relative
+     * path corresponding to the snapshot root. Sorted based on their names.
      */ 
-    private final List<INode> diffList = new ArrayList<INode>();;
+    private final SortedMap<INode, byte[][]> diffMap = 
+        new TreeMap<INode, byte[][]>(INODE_COMPARATOR);
     /**
      * A map capturing the detailed difference about file creation/deletion.
      * Each key indicates a directory whose children have been changed between
      * the two snapshots, while its associated value is a {@link ChildrenDiff}
      * storing the changes (creation/deletion) happened to the children (files).
      */
-    private final Map<INodeDirectoryWithSnapshot, ChildrenDiff> diffMap = 
+    private final Map<INodeDirectoryWithSnapshot, ChildrenDiff> dirDiffMap = 
         new HashMap<INodeDirectoryWithSnapshot, ChildrenDiff>();
     
     SnapshotDiffInfo(INodeDirectorySnapshottable snapshotRoot, Snapshot start,
@@ -121,20 +124,15 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
     }
     
     /** Add a dir-diff pair */
-    private void addDirDiff(INodeDirectoryWithSnapshot dir, ChildrenDiff diff) {
-      diffMap.put(dir, diff);
-      int i = Collections.binarySearch(diffList, dir, INODE_COMPARATOR);
-      if (i < 0) {
-        diffList.add(-i - 1, dir);
-      }
+    private void addDirDiff(INodeDirectoryWithSnapshot dir,
+        byte[][] relativePath, ChildrenDiff diff) {
+      dirDiffMap.put(dir, diff);
+      diffMap.put(dir, relativePath);
     }
     
     /** Add a modified file */ 
-    private void addFileDiff(INodeFile file) {
-      int i = Collections.binarySearch(diffList, file, INODE_COMPARATOR);
-      if (i < 0) {
-        diffList.add(-i - 1, file);
-      }
+    private void addFileDiff(INodeFile file, byte[][] relativePath) {
+      diffMap.put(file, relativePath);
     }
     
     /** @return True if {@link #from} is earlier than {@link #to} */
@@ -148,13 +146,14 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
      */
     public SnapshotDiffReport generateReport() {
       List<DiffReportEntry> diffReportList = new ArrayList<DiffReportEntry>();
-      for (INode node : diffList) {
-        diffReportList.add(new DiffReportEntry(DiffType.MODIFY, node
-            .getRelativePathNameBytes(snapshotRoot)));
+      for (INode node : diffMap.keySet()) {
+        diffReportList.add(new DiffReportEntry(DiffType.MODIFY, diffMap
+            .get(node)));
         if (node.isDirectory()) {
-          ChildrenDiff dirDiff = diffMap.get(node);
-          List<DiffReportEntry> subList = dirDiff.generateReport(snapshotRoot,
-              (INodeDirectoryWithSnapshot) node, isFromEarlier());
+          ChildrenDiff dirDiff = dirDiffMap.get(node);
+          List<DiffReportEntry> subList = dirDiff.generateReport(
+              diffMap.get(node), (INodeDirectoryWithSnapshot) node,
+              isFromEarlier());
           diffReportList.addAll(subList);
         }
       }
@@ -371,7 +370,7 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
     Snapshot toSnapshot = getSnapshotByName(to); 
     SnapshotDiffInfo diffs = new SnapshotDiffInfo(this, fromSnapshot,
         toSnapshot);
-    computeDiffRecursively(this, diffs);
+    computeDiffRecursively(this, new ArrayList<byte[]>(), diffs);
     return diffs;
   }
   
@@ -400,12 +399,15 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
   /**
    * Recursively compute the difference between snapshots under a given
    * directory/file.
-   * @param node The directory/file under which the diff is computed.
+   * @param node The directory/file under which the diff is computed. 
+   * @param parentPath Relative path (corresponding to the snapshot root) of 
+   *                   the node's parent.
    * @param diffReport data structure used to store the diff.
    */
-  private void computeDiffRecursively(INode node, 
+  private void computeDiffRecursively(INode node, List<byte[]> parentPath,
       SnapshotDiffInfo diffReport) {
     ChildrenDiff diff = new ChildrenDiff();
+    byte[][] relativePath = parentPath.toArray(new byte[parentPath.size()][]);
     if (node.isDirectory()) {
       INodeDirectory dir = node.asDirectory();
       if (dir instanceof INodeDirectoryWithSnapshot) {
@@ -413,7 +415,7 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
         boolean change = sdir.computeDiffBetweenSnapshots(
             diffReport.from, diffReport.to, diff);
         if (change) {
-          diffReport.addDirDiff(sdir, diff);
+          diffReport.addDirDiff(sdir, relativePath, diff);
         }
       }
       ReadOnlyList<INode> children = dir.getChildrenList(diffReport
@@ -422,7 +424,9 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
         final byte[] name = child.getLocalNameBytes();
         if (diff.searchIndex(ListType.CREATED, name) < 0
             && diff.searchIndex(ListType.DELETED, name) < 0) {
-          computeDiffRecursively(child, diffReport);
+          parentPath.add(name);
+          computeDiffRecursively(child, parentPath, diffReport);
+          parentPath.remove(parentPath.size() - 1);
         }
       }
     } else if (node.isFile() && node.asFile() instanceof FileWithSnapshot) {
@@ -434,7 +438,7 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
       boolean change = file.getDiffs().changedBetweenSnapshots(earlierSnapshot,
           laterSnapshot);
       if (change) {
-        diffReport.addFileDiff(file.asINodeFile());
+        diffReport.addFileDiff(file.asINodeFile(), relativePath);
       }
     }
   }

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

@@ -159,21 +159,19 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     
     /**
      * Interpret the diff and generate a list of {@link DiffReportEntry}.
-     * @root The snapshot root of the diff report.
+     * @param parentPath The relative path of the parent.
      * @param parent The directory that the diff belongs to.
      * @param fromEarlier True indicates {@code diff=later-earlier}, 
-     *                            False indicates {@code diff=earlier-later}
+     *                    False indicates {@code diff=earlier-later}
      * @return A list of {@link DiffReportEntry} as the diff report.
      */
-    public List<DiffReportEntry> generateReport(
-        INodeDirectorySnapshottable root, INodeDirectoryWithSnapshot parent,
-        boolean fromEarlier) {
+    public List<DiffReportEntry> generateReport(byte[][] parentPath,
+        INodeDirectoryWithSnapshot parent, boolean fromEarlier) {
       List<DiffReportEntry> cList = new ArrayList<DiffReportEntry>();
       List<DiffReportEntry> dList = new ArrayList<DiffReportEntry>();
       int c = 0, d = 0;
       List<INode> created = getList(ListType.CREATED);
       List<INode> deleted = getList(ListType.DELETED);
-      byte[][] parentPath = parent.getRelativePathNameBytes(root);
       byte[][] fullPath = new byte[parentPath.length + 1][];
       System.arraycopy(parentPath, 0, fullPath, 0, parentPath.length);
       for (; c < created.size() && d < deleted.size(); ) {