Преглед изворни кода

HDFS-8917. Cleanup BlockInfoUnderConstruction from comments and tests. Contributed by Zhe Zhang.

(cherry picked from commit 4e14f7982a6e57bf08deb3b266806c2b779a157d)
Jing Zhao пре 9 година
родитељ
комит
62dfe1b7e4

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

@@ -471,6 +471,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8803. Move DfsClientConf to hdfs-client. (Mingliang Liu via wheat9)
 
+    HDFS-8917. Cleanup BlockInfoUnderConstruction from comments and tests.
+    (Zhe Zhang via jing9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java

@@ -81,7 +81,6 @@ public abstract class  BlockInfo extends Block
 
   /**
    * Copy construction.
-   * This is used to convert BlockInfoUnderConstruction
    * @param from BlockInfo to copy from.
    */
   protected BlockInfo(BlockInfo from) {

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java

@@ -37,8 +37,7 @@ public class BlockInfoContiguous extends BlockInfo {
 
   /**
    * Copy construction.
-   * This is used to convert BlockReplicationInfoUnderConstruction
-   * @param from BlockReplicationInfo to copy from.
+   * @param from BlockInfoContiguous to copy from.
    */
   protected BlockInfoContiguous(BlockInfoContiguous from) {
     super(from);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -2225,7 +2225,7 @@ public class BlockManager implements BlockStatsMXBean {
    * is fully replicated.</li>
    * <li>If the reported replica is for a block currently marked "under
    * construction" in the NN, then it should be added to the 
-   * BlockInfoUnderConstruction's list of replicas.</li>
+   * BlockUnderConstructionFeature's list of replicas.</li>
    * </ol>
    * 
    * @param storageInfo DatanodeStorageInfo that sent the report.

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java

@@ -151,7 +151,7 @@ public class BlockUnderConstructionFeature {
   public BlockUnderConstructionFeature(Block block, BlockUCState state,
       DatanodeStorageInfo[] targets) {
     assert getBlockUCState() != BlockUCState.COMPLETE :
-      "BlockInfoUnderConstruction cannot be in COMPLETE state";
+      "BlockUnderConstructionFeature cannot be in COMPLETE state";
     this.blockUCState = state;
     setExpectedLocations(block.getGenerationStamp(), targets);
   }
@@ -241,7 +241,7 @@ public class BlockUnderConstructionFeature {
     blockRecoveryId = recoveryId;
     if (replicas.size() == 0) {
       NameNode.blockStateChangeLog.warn("BLOCK*"
-        + " BlockInfoUnderConstruction.initLeaseRecovery:"
+        + " BlockUnderConstructionFeature.initLeaseRecovery:"
         + " No blocks found, lease removed.");
     }
     boolean allLiveReplicasTriedAsPrimary = true;

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

@@ -62,7 +62,7 @@ public class FileUnderConstructionFeature implements INode.Feature {
         + f.getFullPathName() + " is null when updating its length";
     assert !lastBlock.isComplete()
         : "The last block for path " + f.getFullPathName()
-            + " is not a BlockInfoUnderConstruction when updating its length";
+            + " is not under-construction when updating its length";
     lastBlock.setNumBytes(lastBlockLength);
   }
 

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

@@ -693,7 +693,7 @@ public class INodeFile extends INodeWithAdditionalFields
       return 0;
     }
     final int last = blocks.length - 1;
-    //check if the last block is BlockInfoUnderConstruction
+    //check if the last block is under-construction
     long size = blocks[last].getNumBytes();
     if (!blocks[last].isComplete()) {
        if (!includesLastUcBlock) {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java

@@ -331,7 +331,7 @@ public class TestFileAppend{
   
       //1st append does not add any data so that the last block remains full
       //and the last block in INodeFileUnderConstruction is a BlockInfo
-      //but not BlockInfoUnderConstruction. 
+      //but does not have a BlockUnderConstructionFeature.
       fs2.append(p);
       
       //2nd append should get AlreadyBeingCreatedException
@@ -369,7 +369,7 @@ public class TestFileAppend{
   
       //1st append does not add any data so that the last block remains full
       //and the last block in INodeFileUnderConstruction is a BlockInfo
-      //but not BlockInfoUnderConstruction.
+      //but does not have a BlockUnderConstructionFeature.
       ((DistributedFileSystem) fs2).append(p,
           EnumSet.of(CreateFlag.APPEND, CreateFlag.NEW_BLOCK), 4096, null);
 

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockUnderConstructionFeature.java

@@ -26,9 +26,9 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.junit.Test;
 
 /**
- * This class provides tests for BlockInfoUnderConstruction class
+ * This class provides tests for {@link BlockUnderConstructionFeature} class
  */
-public class TestBlockInfoUnderConstruction {
+public class TestBlockUnderConstructionFeature {
   @Test
   public void testInitializeBlockRecovery() throws Exception {
     DatanodeStorageInfo s1 = DFSTestUtil.createDatanodeStorageInfo("10.10.1.1", "s1");

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java

@@ -227,8 +227,8 @@ public class SnapshotTestHelper {
           line2 = line2.replaceAll("Quota\\[.*\\]", "Quota[]");
         }
         
-        // skip the specific fields of BlockInfoUnderConstruction when the node
-        // is an INodeFileSnapshot or an INodeFileUnderConstructionSnapshot
+        // skip the specific fields of BlockUnderConstructionFeature when the
+        // node is an INodeFileSnapshot or INodeFileUnderConstructionSnapshot
         if (line1.contains("(INodeFileSnapshot)")
             || line1.contains("(INodeFileUnderConstructionSnapshot)")) {
           line1 = line1.replaceAll(