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 sh...@apache.org on 2009/09/03 02:30:57 UTC

svn commit: r810740 - in /hadoop/hdfs/branches/HDFS-265: ./ src/java/org/apache/hadoop/hdfs/server/namenode/ src/test/aop/org/apache/hadoop/hdfs/protocol/

Author: shv
Date: Thu Sep  3 00:30:57 2009
New Revision: 810740

URL: http://svn.apache.org/viewvc?rev=810740&view=rev
Log:
HDFS-588. Fix TestFiDataTransferProtocol and TestAppend2 failures. Contributed by Konstantin Shvachko.

Modified:
    hadoop/hdfs/branches/HDFS-265/CHANGES.txt
    hadoop/hdfs/branches/HDFS-265/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java
    hadoop/hdfs/branches/HDFS-265/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
    hadoop/hdfs/branches/HDFS-265/src/test/aop/org/apache/hadoop/hdfs/protocol/ClientProtocolAspects.aj

Modified: hadoop/hdfs/branches/HDFS-265/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-265/CHANGES.txt?rev=810740&r1=810739&r2=810740&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-265/CHANGES.txt (original)
+++ hadoop/hdfs/branches/HDFS-265/CHANGES.txt Thu Sep  3 00:30:57 2009
@@ -36,6 +36,8 @@
     HDFS-547. TestHDFSFileSystemContract#testOutputStreamClosedTwice
     sometimes fails with CloseByInterruptException. (hairong)
 
+    HDFS-588. Fix TestFiDataTransferProtocol and TestAppend2 failures. (shv)
+
 Trunk (unreleased changes)
 
   INCOMPATIBLE CHANGES

Modified: hadoop/hdfs/branches/HDFS-265/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-265/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java?rev=810740&r1=810739&r2=810740&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-265/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java (original)
+++ hadoop/hdfs/branches/HDFS-265/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java Thu Sep  3 00:30:57 2009
@@ -237,7 +237,7 @@
   }
 
   /**
-   * Commit the last block of the file and complete the penultimate block.
+   * Commit the last block of the file.
    * 
    * @param fileINode file inode
    * @param commitBlock - contains client reported block length and generation
@@ -255,9 +255,6 @@
       "commitBlock length is less than the stored one "
       + commitBlock.getNumBytes() + " vs. " + lastBlock.getNumBytes();
     lastBlock.commitBlock(commitBlock);
-
-    // complete the penultimate block
-    completeBlock(fileINode, fileINode.numBlocks()-2);
   }
 
   /**

Modified: hadoop/hdfs/branches/HDFS-265/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-265/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=810740&r1=810739&r2=810740&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-265/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/hdfs/branches/HDFS-265/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Thu Sep  3 00:30:57 2009
@@ -1101,7 +1101,7 @@
 
       INodeFileUnderConstruction pendingFile  = checkLease(src, clientName);
 
-      // commit the last block and complete the penultimate block
+      // commit the last block
       blockManager.commitLastBlock(pendingFile, previous);
 
       //
@@ -1137,6 +1137,9 @@
         throw new NotReplicatedYetException("Not replicated yet:" + src);
       }
 
+      // complete the penultimate block
+      blockManager.completeBlock(pendingFile, pendingFile.numBlocks()-2);
+
       // allocate new block record block locations in INode.
       newBlock = allocateBlock(src, pathINodes, targets);
       
@@ -1252,7 +1255,7 @@
       return CompleteFileStatus.OPERATION_FAILED;
     } 
 
-    // commit the last block and complete the penultimate block
+    // commit the last block
     blockManager.commitLastBlock(pendingFile, last);
 
     if (!checkFileProgress(pendingFile, true)) {
@@ -1594,6 +1597,9 @@
       INodeFileUnderConstruction pendingFile) throws IOException {
     leaseManager.removeLease(pendingFile.clientName, src);
 
+    // complete the penultimate block
+    blockManager.completeBlock(pendingFile, pendingFile.numBlocks()-2);
+
     // The file is no longer pending.
     // Create permanent INode, update blocks
     INodeFile newFile = pendingFile.convertToInodeFile();
@@ -1680,9 +1686,9 @@
       return;
     }
 
-    // commit the last block and complete the penultimate block
+    // commit the last block
     blockManager.commitLastBlock(pendingFile, lastblock);
-    
+
     //remove lease, close file
     finalizeINodeFileUnderConstruction(src, pendingFile);
     getEditLog().logSync();

Modified: hadoop/hdfs/branches/HDFS-265/src/test/aop/org/apache/hadoop/hdfs/protocol/ClientProtocolAspects.aj
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-265/src/test/aop/org/apache/hadoop/hdfs/protocol/ClientProtocolAspects.aj?rev=810740&r1=810739&r2=810740&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-265/src/test/aop/org/apache/hadoop/hdfs/protocol/ClientProtocolAspects.aj (original)
+++ hadoop/hdfs/branches/HDFS-265/src/test/aop/org/apache/hadoop/hdfs/protocol/ClientProtocolAspects.aj Thu Sep  3 00:30:57 2009
@@ -27,7 +27,7 @@
   public static final Log LOG = LogFactory.getLog(ClientProtocolAspects.class);
 
   pointcut addBlock():
-    call(LocatedBlock ClientProtocol.addBlock(String, String));
+    call(LocatedBlock ClientProtocol.addBlock(String, String,..));
 
   after() returning(LocatedBlock lb): addBlock() {
     PipelineTest pipelineTest = DataTransferTestUtil.getPipelineTest();