Selaa lähdekoodia

HDFS-4357. Fix a bug that if an inode is replaced, further INode operations should apply to the new inode. Contributed by Jing Zhao

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1428780 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 vuotta sitten
vanhempi
commit
f96d0a3585

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

@@ -89,3 +89,6 @@ Branch-2802 Snapshot (Unreleased)
   HDFS-4103. Support O(1) snapshot creation. (szetszwo)
 
   HDFS-4330. Support snapshots up to the snapshot limit. (szetszwo)
+
+  HDFS-4357. Fix a bug that if an inode is replaced, further INode operations
+  should apply to the new inode. (Jing Zhao via szetszwo)

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

@@ -928,12 +928,12 @@ public class FSDirectory implements Closeable {
       SnapshotAccessControlException {
     assert hasWriteLock();
     final INodesInPath inodesInPath = rootDir.getMutableINodesInPath(src, true);
-    final INode inode = inodesInPath.getLastINode();
+    INode inode = inodesInPath.getLastINode();
     if (inode == null) {
       throw new FileNotFoundException("File does not exist: " + src);
     }
     if (username != null) {
-      inode.setUser(username, inodesInPath.getLatestSnapshot());
+      inode = inode.setUser(username, inodesInPath.getLatestSnapshot());
     }
     if (groupname != null) {
       inode.setGroup(groupname, inodesInPath.getLatestSnapshot());
@@ -1859,6 +1859,7 @@ public class FSDirectory implements Closeable {
     INode removedNode = ((INodeDirectory)inodes[pos-1]).removeChild(
         inodes[pos], inodesInPath.getLatestSnapshot());
     if (removedNode != null) {
+      inodesInPath.setINode(pos - 1, removedNode.getParent());
       INode.DirCounts counts = new INode.DirCounts();
       removedNode.spaceConsumedInTree(counts);
       updateCountNoQuotaCheck(inodesInPath, pos,
@@ -2088,7 +2089,7 @@ public class FSDirectory implements Closeable {
     assert hasWriteLock();
     boolean status = false;
     if (mtime != -1) {
-      inode.setModificationTime(mtime, latest);
+      inode = inode.setModificationTime(mtime, latest);
       status = true;
     }
     if (atime != -1) {

+ 22 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.util.StringUtils;
@@ -224,10 +225,12 @@ public abstract class INode implements Comparable<byte[]> {
   public PermissionStatus getPermissionStatus() {
     return getPermissionStatus(null);
   }
-  private void updatePermissionStatus(PermissionStatusFormat f, long n,
+  private INode updatePermissionStatus(PermissionStatusFormat f, long n,
       Snapshot latest) {
-    recordModification(latest);
-    permission = f.combine(n, permission);
+    Pair<? extends INode, ? extends INode> pair = recordModification(latest);
+    INode nodeToUpdate = pair != null ? pair.left : this;
+    nodeToUpdate.permission = f.combine(n, permission);
+    return nodeToUpdate;
   }
   /**
    * @param snapshot
@@ -244,9 +247,9 @@ public abstract class INode implements Comparable<byte[]> {
     return getUserName(null);
   }
   /** Set user */
-  protected void setUser(String user, Snapshot latest) {
+  protected INode setUser(String user, Snapshot latest) {
     int n = SerialNumberManager.INSTANCE.getUserSerialNumber(user);
-    updatePermissionStatus(PermissionStatusFormat.USER, n, latest);
+    return updatePermissionStatus(PermissionStatusFormat.USER, n, latest);
   }
   /**
    * @param snapshot
@@ -263,9 +266,9 @@ public abstract class INode implements Comparable<byte[]> {
     return getGroupName(null);
   }
   /** Set group */
-  protected void setGroup(String group, Snapshot latest) {
+  protected INode setGroup(String group, Snapshot latest) {
     int n = SerialNumberManager.INSTANCE.getGroupSerialNumber(group);
-    updatePermissionStatus(PermissionStatusFormat.GROUP, n, latest);
+    return updatePermissionStatus(PermissionStatusFormat.GROUP, n, latest);
   }
   /**
    * @param snapshot
@@ -285,9 +288,9 @@ public abstract class INode implements Comparable<byte[]> {
     return (short)PermissionStatusFormat.MODE.retrieve(permission);
   }
   /** Set the {@link FsPermission} of this {@link INode} */
-  void setPermission(FsPermission permission, Snapshot latest) {
+  INode setPermission(FsPermission permission, Snapshot latest) {
     final short mode = permission.toShort();
-    updatePermissionStatus(PermissionStatusFormat.MODE, mode, latest);
+    return updatePermissionStatus(PermissionStatusFormat.MODE, mode, latest);
   }
 
   /**
@@ -470,9 +473,11 @@ public abstract class INode implements Comparable<byte[]> {
   /**
    * Always set the last modification time of inode.
    */
-  public void setModificationTime(long modtime, Snapshot latest) {
-    recordModification(latest);
-    this.modificationTime = modtime;
+  public INode setModificationTime(long modtime, Snapshot latest) {
+    Pair<? extends INode, ? extends INode> pair = recordModification(latest);
+    INode nodeToUpdate = pair != null ? pair.left : this;
+    nodeToUpdate.modificationTime = modtime;
+    return nodeToUpdate;
   }
 
   /**
@@ -493,9 +498,11 @@ public abstract class INode implements Comparable<byte[]> {
   /**
    * Set last access time of inode.
    */
-  void setAccessTime(long atime, Snapshot latest) {
-    recordModification(latest);
-    accessTime = atime;
+  INode setAccessTime(long atime, Snapshot latest) {
+    Pair<? extends INode, ? extends INode> pair = recordModification(latest);
+    INode nodeToUpdate = pair != null ? pair.left : this;    
+    nodeToUpdate.accessTime = atime;
+    return nodeToUpdate;
   }
 
   /**

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

@@ -127,8 +127,8 @@ public class INodeFile extends INode implements BlockCollection {
    * the {@link FsAction#EXECUTE} action, if any, is ignored.
    */
   @Override
-  void setPermission(FsPermission permission, Snapshot latest) {
-    super.setPermission(permission.applyUMask(UMASK), latest);
+  INode setPermission(FsPermission permission, Snapshot latest) {
+    return super.setPermission(permission.applyUMask(UMASK), latest);
   }
 
   /** @return the replication factor of the file. */

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

@@ -375,6 +375,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
       if (snapshotCopy == null) {
         snapshotCopy = new INodeDirectory(dir, false);
       }
+      snapshotINode = snapshotCopy;
       return new Pair<INodeDirectory, INodeDirectory>(dir, snapshotCopy);
     }
 

+ 7 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java

@@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue;
 import java.io.PrintWriter;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Random;
 
@@ -107,7 +108,8 @@ public class SnapshotTestHelper {
     Path rootParent = snapshotRoot.getParent();
     if (rootParent != null && rootParent.getName().equals(".snapshot")) {
       Path snapshotDir = rootParent.getParent();
-      if (file.toString().contains(snapshotDir.toString())) {
+      if (file.toString().contains(snapshotDir.toString())
+          && !file.equals(snapshotDir)) {
         String fileName = file.toString().substring(
             snapshotDir.toString().length() + 1);
         Path snapshotFile = new Path(snapshotRoot, fileName);
@@ -186,14 +188,13 @@ public class SnapshotTestHelper {
      *          cannot be one of the nodes in this list.
      * @return a random node from the tree.
      */
-    Node getRandomDirNode(Random random,
-        ArrayList<Node> excludedList) {
+    Node getRandomDirNode(Random random, List<Node> excludedList) {
       while (true) {
         int level = random.nextInt(height);
         ArrayList<Node> levelList = levelMap.get(level);
         int index = random.nextInt(levelList.size());
         Node randomNode = levelList.get(index);
-        if (!excludedList.contains(randomNode)) {
+        if (excludedList == null || !excludedList.contains(randomNode)) {
           return randomNode;
         }
       }
@@ -261,8 +262,8 @@ public class SnapshotTestHelper {
        * Create files and add them in the fileList. Initially the last element
        * in the fileList is set to null (where we start file creation).
        */
-      void initFileList(String namePrefix, long fileLen, short replication, long seed, int numFiles)
-          throws Exception {
+      void initFileList(String namePrefix, long fileLen, short replication,
+          long seed, int numFiles) throws Exception {
         fileList = new ArrayList<Path>(numFiles);
         for (int i = 0; i < numFiles; i++) {
           Path file = new Path(nodePath, namePrefix + "-f" + i);

+ 39 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java

@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Random;
 
@@ -145,10 +146,11 @@ public class TestSnapshot {
   
   /**
    * Main test, where we will go in the following loop:
-   * 
+   * <pre>
    *    Create snapshot and check the creation <--+  
    * -> Change the current/live files/dir         | 
    * -> Check previous snapshots -----------------+
+   * </pre>
    */
   @Test
   public void testSnapshot() throws Exception {
@@ -169,13 +171,48 @@ public class TestSnapshot {
       modNodes[modNodes.length - 1] = dirTree.getRandomDirNode(random,
           excludedList);
       Modification[] mods = prepareModifications(modNodes);
-      // make changes to the current directory
+      // make changes to the directories/files
       modifyCurrentDirAndCheckSnapshots(mods);
+      
+      // also update the metadata of directories
+      TestDirectoryTree.Node chmodDir = dirTree.getRandomDirNode(random, null);
+      Modification chmod = new FileChangePermission(chmodDir.nodePath, hdfs,
+          genRandomPermission());
+      String[] userGroup = genRandomOwner();
+      TestDirectoryTree.Node chownDir = dirTree.getRandomDirNode(random,
+          Arrays.asList(chmodDir));
+      Modification chown = new FileChown(chownDir.nodePath, hdfs, userGroup[0],
+          userGroup[1]);
+      modifyCurrentDirAndCheckSnapshots(new Modification[]{chmod, chown});
     }
     System.out.println("XXX done:");
     SnapshotTestHelper.dumpTreeRecursively(fsn.getFSDirectory().getINode("/"));
   }
   
+  /**
+   * A simple test that updates a sub-directory of a snapshottable directory
+   * with snapshots
+   */
+  @Test
+  public void testUpdateDirectory() throws Exception {
+    Path dir = new Path("/dir");
+    Path sub = new Path(dir, "sub");
+    Path subFile = new Path(sub, "file");
+    DFSTestUtil.createFile(hdfs, subFile, BLOCKSIZE, REPLICATION, seed);
+
+    FileStatus oldStatus = hdfs.getFileStatus(sub);
+
+    hdfs.allowSnapshot(dir.toString());
+    hdfs.createSnapshot("s1", dir.toString());
+    hdfs.setTimes(sub, 100L, 100L);
+
+    Path snapshotPath = SnapshotTestHelper.getSnapshotPath(dir, "s1", "sub");
+    FileStatus snapshotStatus = hdfs.getFileStatus(snapshotPath);
+    assertEquals(oldStatus.getModificationTime(),
+        snapshotStatus.getModificationTime());
+    assertEquals(oldStatus.getAccessTime(), snapshotStatus.getAccessTime());
+  }
+  
   /**
    * Creating snapshots for a directory that is not snapshottable must fail.
    *