You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by sz...@apache.org on 2013/04/25 20:53:15 UTC

svn commit: r1475903 - in /hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/

Author: szetszwo
Date: Thu Apr 25 18:53:14 2013
New Revision: 1475903

URL: http://svn.apache.org/r1475903
Log:
HDFS-4742. Fix appending to a renamed file with snapshot.  Contributed by Jing Zhao

Modified:
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt?rev=1475903&r1=1475902&r2=1475903&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt Thu Apr 25 18:53:14 2013
@@ -302,3 +302,6 @@ Branch-2802 Snapshot (Unreleased)
 
   HDFS-4749. Use INodeId to identify the corresponding directory node in
   FSImage saving/loading.  (Jing Zhao via szetszwo)
+
+  HDFS-4742. Fix appending to a renamed file with snapshot.  (Jing Zhao via
+  szetszwo)

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java?rev=1475903&r1=1475902&r2=1475903&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java Thu Apr 25 18:53:14 2013
@@ -199,16 +199,18 @@ public class INodeDirectory extends INod
   }
 
   /** Replace the given child with a new child. */
-  public void replaceChild(final INode oldChild, final INode newChild) {
+  public void replaceChild(INode oldChild, final INode newChild) {
     Preconditions.checkNotNull(children);
     final int i = searchChildren(newChild.getLocalNameBytes());
     Preconditions.checkState(i >= 0);
-    Preconditions.checkState(oldChild == children.get(i));
+    Preconditions.checkState(oldChild == children.get(i)
+        || oldChild == children.get(i).asReference().getReferredINode()
+            .asReference().getReferredINode());
+    oldChild = children.get(i);
     
     if (oldChild.isReference() && !newChild.isReference()) {
       // replace the referred inode, e.g., 
       // INodeFileWithSnapshot -> INodeFileUnderConstructionWithSnapshot
-      // TODO: add a unit test for rename + append
       final INode withCount = oldChild.asReference().getReferredINode();
       withCount.asReference().setReferredINode(newChild);
     } else {
@@ -219,8 +221,7 @@ public class INodeDirectory extends INod
         withCount.removeReference(oldChild.asReference());
       }
       // do the replacement
-      final INode removed = children.set(i, newChild);
-      Preconditions.checkState(removed == oldChild);
+      children.set(i, newChild);
     }
   }
 

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java?rev=1475903&r1=1475902&r2=1475903&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java Thu Apr 25 18:53:14 2013
@@ -109,7 +109,8 @@ public abstract class INodeWithAdditiona
   
   /** @param other Other node to be copied */
   INodeWithAdditionalFields(INodeWithAdditionalFields other) {
-    this(other.getParent(), other.getId(), other.getLocalNameBytes(),
+    this(other.getParentReference() != null ? other.getParentReference()
+        : other.getParent(), other.getId(), other.getLocalNameBytes(),
         other.permission, other.modificationTime, other.accessTime);
   }
 

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java?rev=1475903&r1=1475902&r2=1475903&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java Thu Apr 25 18:53:14 2013
@@ -256,7 +256,9 @@ public class LeaseManager {
     private String findPath(INodeFileUnderConstruction pendingFile) {
       try {
         for (String src : paths) {
-          if (fsnamesystem.dir.getINode(src) == pendingFile) {
+          INode node = fsnamesystem.dir.getINode(src);
+          if (node == pendingFile
+              || (node.isFile() && node.asFile() == pendingFile)) {
             return src;
           }
         }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java?rev=1475903&r1=1475902&r2=1475903&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java Thu Apr 25 18:53:14 2013
@@ -30,10 +30,12 @@ import static org.mockito.Mockito.spy;
 import java.io.File;
 import java.io.IOException;
 import java.util.List;
+import java.util.Random;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.Path;
@@ -1185,6 +1187,50 @@ public class TestRenameWithSnapshots {
   }
   
   /**
+   * Rename a file and then append the same file. 
+   */
+  @Test
+  public void testRenameAndAppend() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    hdfs.mkdirs(sdir1);
+    hdfs.mkdirs(sdir2);
+    
+    final Path foo = new Path(sdir1, "foo");
+    DFSTestUtil.createFile(hdfs, foo, BLOCKSIZE, REPL, SEED);
+    
+    SnapshotTestHelper.createSnapshot(hdfs, sdir1, snap1);
+
+    final Path foo2 = new Path(sdir2, "foo");
+    hdfs.rename(foo, foo2);
+    
+    INode fooRef = fsdir.getINode4Write(foo2.toString());
+    assertTrue(fooRef instanceof INodeReference.DstReference);
+    
+    FSDataOutputStream out = hdfs.append(foo2);
+    try {
+      byte[] content = new byte[1024];
+      (new Random()).nextBytes(content);
+      out.write(content);
+      fooRef = fsdir.getINode4Write(foo2.toString());
+      assertTrue(fooRef instanceof INodeReference.DstReference);
+      INode fooNode = fooRef.asFile();
+      assertTrue(fooNode instanceof INodeFileUnderConstructionWithSnapshot);
+    } finally {
+      if (out != null) {
+        out.close();
+      }
+    }
+    
+    fooRef = fsdir.getINode4Write(foo2.toString());
+    assertTrue(fooRef instanceof INodeReference.DstReference);
+    INode fooNode = fooRef.asFile();
+    assertTrue(fooNode instanceof INodeFileWithSnapshot);
+    
+    restartClusterAndCheckImage();
+  }
+  
+  /**
    * Test the undo section of rename. Before the rename, we create the renamed 
    * file/dir before taking the snapshot.
    */