浏览代码

HDFS-652. Replace BlockInfo.isUnderConstruction() with isComplete(). Contributed by Konstantin Shvachko.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/branches/HDFS-265@818676 13f79535-47bb-0310-9956-ffa450edef68
Konstantin Shvachko 15 年之前
父节点
当前提交
c4b7ee0dcf

+ 2 - 0
CHANGES.txt

@@ -55,6 +55,8 @@ Append branch (unreleased changes)
     HDFS-589. Change block write protocol to support pipeline recovery. 
     (hairong)  
 
+    HDFS-652. Replace BlockInfo.isUnderConstruction() with isComplete() (shv)
+
   BUG FIXES
 
     HDFS-547. TestHDFSFileSystemContract#testOutputStreamClosedTwice

+ 6 - 4
src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfo.java

@@ -284,10 +284,12 @@ class BlockInfo extends Block {
   }
 
   /**
-   * Is this block being constructed?
+   * Is this block complete?
+   * 
+   * @return true if the state of the block is {@link BlockUCState#COMPLETE}
    */
-  boolean isUnderConstruction() {
-    return !getBlockUCState().equals(BlockUCState.COMPLETE);
+  boolean isComplete() {
+    return getBlockUCState().equals(BlockUCState.COMPLETE);
   }
 
   /**
@@ -297,7 +299,7 @@ class BlockInfo extends Block {
    */
   BlockInfoUnderConstruction convertToBlockUnderConstruction(
       BlockUCState s, DatanodeDescriptor[] targets) {
-    if(!isUnderConstruction()) {
+    if(isComplete()) {
       return new BlockInfoUnderConstruction(
           this, getINode().getReplication(), s, targets);
     }

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

@@ -254,7 +254,7 @@ public class BlockManager {
     BlockInfo lastBlock = fileINode.getLastBlock();
     if(lastBlock == null)
       return; // no blocks in file yet
-    if(!lastBlock.isUnderConstruction())
+    if(lastBlock.isComplete())
       return; // already completed (e.g. by syncBlock)
     assert lastBlock.getNumBytes() <= commitBlock.getNumBytes() :
       "commitBlock length is less than the stored one "
@@ -274,7 +274,7 @@ public class BlockManager {
     if(blkIndex < 0)
       return null;
     BlockInfo curBlock = fileINode.getBlocks()[blkIndex];
-    if(!curBlock.isUnderConstruction())
+    if(curBlock.isComplete())
       return curBlock;
     BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)curBlock;
     if(ucBlock.numNodes() < minReplication)
@@ -1108,7 +1108,7 @@ public class BlockManager {
 
     // check whether safe replication is reached for the block
     // only complete blocks are counted towards that
-    if(!storedBlock.isUnderConstruction())
+    if(storedBlock.isComplete())
       namesystem.incrementSafeBlockCount(numCurrentReplica);
 
     // if file is under construction, then check whether the block

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

@@ -490,7 +490,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
       break;
     case RBW:
     case RWR:
-      if(storedBlock.isUnderConstruction())
+      if(!storedBlock.isComplete())
         ((BlockInfoUnderConstruction)storedBlock).addReplicaIfNotPresent(
                                                       this, block, rState);
       else

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

@@ -567,14 +567,14 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     BlockInfo curBlock;
     while(totalSize<size && iter.hasNext()) {
       curBlock = iter.next();
-      if(curBlock.isUnderConstruction())  continue;
+      if(!curBlock.isComplete())  continue;
       totalSize += addBlock(curBlock, results);
     }
     if(totalSize<size) {
       iter = node.getBlockIterator(); // start from the beginning
       for(int i=0; i<startBlock&&totalSize<size; i++) {
         curBlock = iter.next();
-        if(curBlock.isUnderConstruction())  continue;
+        if(!curBlock.isComplete())  continue;
         totalSize += addBlock(curBlock, results);
       }
     }
@@ -3399,7 +3399,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
         if(blocks == null)
           continue;
         for(BlockInfo b : blocks) {
-          if(b.isUnderConstruction())
+          if(!b.isComplete())
             numUCBlocks++;
         }
       }
@@ -3739,7 +3739,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       throw new IOException(msg);
     }
     INodeFile fileINode = storedBlock.getINode();
-    if(!fileINode.isUnderConstruction() || !storedBlock.isUnderConstruction()) {
+    if(!fileINode.isUnderConstruction() || storedBlock.isComplete()) {
       String msg = block + 
             " is already commited, file or block is not under construction().";
       LOG.info(msg);

+ 7 - 8
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java

@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
@@ -94,8 +93,8 @@ public class TestBlockUnderConstruction {
     // all blocks but the last two should be regular blocks
     for(; idx < blocks.length - 2; idx++) {
       curBlock = blocks[idx];
-      assertFalse("Block is not under construction: " + curBlock,
-          curBlock.isUnderConstruction());
+      assertTrue("Block is not complete: " + curBlock,
+          curBlock.isComplete());
       assertTrue("Block is not in BlocksMap: " + curBlock,
           ns.blockManager.getStoredBlock(curBlock) == curBlock);
     }
@@ -107,8 +106,8 @@ public class TestBlockUnderConstruction {
       assertTrue("Block " + curBlock +
           " isUnderConstruction = " + inode.isUnderConstruction() +
           " expected to be " + isFileOpen,
-          (isFileOpen && !curBlock.isUnderConstruction()) ||
-          (!isFileOpen && curBlock.isUnderConstruction() == 
+          (isFileOpen && curBlock.isComplete()) ||
+          (!isFileOpen && !curBlock.isComplete() == 
             (curBlock.getBlockUCState() ==
               BlockUCState.COMMITTED)));
       assertTrue("Block is not in BlocksMap: " + curBlock,
@@ -117,10 +116,10 @@ public class TestBlockUnderConstruction {
 
     // the last block is under construction if the file is not closed
     curBlock = blocks[idx]; // last block
-    assertTrue("Block " + curBlock +
-        " isUnderConstruction = " + inode.isUnderConstruction() +
+    assertEquals("Block " + curBlock +
+        " isComplete = " + curBlock.isComplete() +
         " expected to be " + isFileOpen,
-        curBlock.isUnderConstruction() == isFileOpen);
+        isFileOpen, !curBlock.isComplete());
     assertTrue("Block is not in BlocksMap: " + curBlock,
         ns.blockManager.getStoredBlock(curBlock) == curBlock);
   }