Przeglądaj źródła

HDFS-588. Fix TestFiDataTransferProtocol and TestAppend2 failures. Contributed by Konstantin Shvachko.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/branches/HDFS-265@810740 13f79535-47bb-0310-9956-ffa450edef68
Konstantin Shvachko 16 lat temu
rodzic
commit
b7906a0d78

+ 2 - 0
CHANGES.txt

@@ -36,6 +36,8 @@ Append branch (unreleased changes)
     HDFS-547. TestHDFSFileSystemContract#testOutputStreamClosedTwice
     sometimes fails with CloseByInterruptException. (hairong)
 
+    HDFS-588. Fix TestFiDataTransferProtocol and TestAppend2 failures. (shv)
+
 Trunk (unreleased changes)
 
   INCOMPATIBLE CHANGES

+ 1 - 4
src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java

@@ -237,7 +237,7 @@ public class BlockManager {
   }
 
   /**
-   * 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 @@ public class BlockManager {
       "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);
   }
 
   /**

+ 10 - 4
src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -1101,7 +1101,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
 
       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 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
         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 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
       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 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
       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 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
       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();

+ 1 - 1
src/test/aop/org/apache/hadoop/hdfs/protocol/ClientProtocolAspects.aj

@@ -27,7 +27,7 @@ public aspect ClientProtocolAspects {
   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();