瀏覽代碼

HDFS-4243. When replacing an INodeDirectory, the parent pointers of the children of the child have to be updated to the new child. Contributed by Jing Zhao

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1416709 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 年之前
父節點
當前提交
c0a8957c2b

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -655,6 +655,10 @@ Release 2.0.3-alpha - Unreleased
 
     HDFS-4231. BackupNode: Introduce BackupState. (shv)
 
+    HDFS-4243. When replacing an INodeDirectory, the parent pointers of the
+    children of the child have to be updated to the new child.  (Jing Zhao
+    via szetszwo)
+
 Release 2.0.2-alpha - 2012-09-07 
 
   INCOMPATIBLE CHANGES

+ 29 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -1075,31 +1075,39 @@ public class FSDirectory implements Closeable {
       throws IOException, UnresolvedLinkException {    
     writeLock();
     try {
-      //
-      // Remove the node from the namespace 
-      //
-      if (!oldnode.removeNode()) {
-        NameNode.stateChangeLog.warn("DIR* FSDirectory.replaceNode: " +
-                                     "failed to remove " + path);
-        throw new IOException("FSDirectory.replaceNode: " +
-                              "failed to remove " + path);
-      } 
-      
-      /* Currently oldnode and newnode are assumed to contain the same
-       * blocks. Otherwise, blocks need to be removed from the blocksMap.
-       */
-      rootDir.addINode(path, newnode); 
-
-      int index = 0;
-      for (BlockInfo b : newnode.getBlocks()) {
-        BlockInfo info = getBlockManager().addBlockCollection(b, newnode);
-        newnode.setBlock(index, info); // inode refers to the block in BlocksMap
-        index++;
-      }
+      unprotectedReplaceNode(path, oldnode, newnode);
     } finally {
       writeUnlock();
     }
   }
+  
+  void unprotectedReplaceNode(String path, INodeFile oldnode, INodeFile newnode)
+      throws IOException, UnresolvedLinkException {
+    assert hasWriteLock();
+    INodeDirectory parent = oldnode.parent;
+    // Remove the node from the namespace 
+    if (!oldnode.removeNode()) {
+      NameNode.stateChangeLog.warn("DIR* FSDirectory.replaceNode: " +
+                                   "failed to remove " + path);
+      throw new IOException("FSDirectory.replaceNode: " +
+                            "failed to remove " + path);
+    } 
+    
+    // Parent should be non-null, otherwise oldnode.removeNode() will return
+    // false
+    newnode.setLocalName(oldnode.getLocalNameBytes());
+    parent.addChild(newnode, true);
+    
+    /* Currently oldnode and newnode are assumed to contain the same
+     * blocks. Otherwise, blocks need to be removed from the blocksMap.
+     */
+    int index = 0;
+    for (BlockInfo b : newnode.getBlocks()) {
+      BlockInfo info = getBlockManager().addBlockCollection(b, newnode);
+      newnode.setBlock(index, info); // inode refers to the block in BlocksMap
+      index++;
+    }
+  }
 
   /**
    * Get a partial listing of the indicated directory

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

@@ -321,7 +321,7 @@ public class FSEditLogLoader {
         INodeFileUnderConstruction ucFile = (INodeFileUnderConstruction) oldFile;
         fsNamesys.leaseManager.removeLeaseWithPrefixPath(addCloseOp.path);
         INodeFile newFile = ucFile.convertToInodeFile();
-        fsDir.replaceNode(addCloseOp.path, ucFile, newFile);
+        fsDir.unprotectedReplaceNode(addCloseOp.path, ucFile, newFile);
       }
       break;
     }

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

@@ -73,6 +73,11 @@ class INodeDirectory extends INode {
   INodeDirectory(INodeDirectory other) {
     super(other);
     this.children = other.children;
+    if (this.children != null) {
+      for (INode child : children) {
+        child.parent = this;
+      }
+    }
   }
   
   /** @return true unconditionally. */
@@ -106,6 +111,7 @@ class INodeDirectory extends INode {
 
     final int low = searchChildren(newChild);
     if (low>=0) { // an old child exists so replace by the newChild
+      children.get(low).parent = null;
       children.set(low, newChild);
     } else {
       throw new IllegalArgumentException("No child exists to be replaced");

+ 47 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -25,10 +25,15 @@ import static org.junit.Assert.fail;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.junit.Test;
 
@@ -157,6 +162,48 @@ public class TestINodeFile {
     
   }
   
+  /**
+   * FSDirectory#unprotectedSetQuota creates a new INodeDirectoryWithQuota to
+   * replace the original INodeDirectory. Before HDFS-4243, the parent field of
+   * all the children INodes of the target INodeDirectory is not changed to
+   * point to the new INodeDirectoryWithQuota. This testcase tests this
+   * scenario.
+   */
+  @Test
+  public void testGetFullPathNameAfterSetQuota() throws Exception {
+    long fileLen = 1024;
+    replication = 3;
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
+        replication).build();
+    cluster.waitActive();
+    FSNamesystem fsn = cluster.getNamesystem();
+    FSDirectory fsdir = fsn.getFSDirectory();
+    DistributedFileSystem dfs = cluster.getFileSystem();
+    
+    // Create a file for test
+    final Path dir = new Path("/dir");
+    final Path file = new Path(dir, "file");
+    DFSTestUtil.createFile(dfs, file, fileLen, replication, 0L);
+    
+    // Check the full path name of the INode associating with the file
+    INode fnode = fsdir.getINode(file.toString());
+    assertEquals(file.toString(), fnode.getFullPathName());
+    
+    // Call FSDirectory#unprotectedSetQuota which calls
+    // INodeDirectory#replaceChild
+    dfs.setQuota(dir, Long.MAX_VALUE - 1, replication * fileLen * 10);
+    final Path newDir = new Path("/newdir");
+    final Path newFile = new Path(newDir, "file");
+    // Also rename dir
+    dfs.rename(dir, newDir, Options.Rename.OVERWRITE);
+    // /dir/file now should be renamed to /newdir/file
+    fnode = fsdir.getINode(newFile.toString());
+    // getFullPathName can return correct result only if the parent field of
+    // child node is set correctly
+    assertEquals(newFile.toString(), fnode.getFullPathName());
+  }
+  
   @Test
   public void testAppendBlocks() {
     INodeFile origFile = createINodeFiles(1, "origfile")[0];