Browse Source

HDFS-8499. Refactor BlockInfo class hierarchy with static helper class. Contributed by Zhe Zhang.

Andrew Wang 10 năm trước cách đây
mục cha
commit
c17439c2dd
30 tập tin đã thay đổi với 418 bổ sung232 xóa
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
  3. 13 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  4. 15 62
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
  5. 54 52
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
  6. 110 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java
  7. 20 20
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  8. 106 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ContiguousBlockStorageOp.java
  9. 7 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  10. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  11. 8 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
  12. 8 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  13. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  14. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
  15. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  16. 13 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  17. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
  18. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  19. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
  20. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
  21. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
  22. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
  23. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
  24. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
  25. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
  26. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
  27. 6 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
  28. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
  29. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
  30. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java

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

@@ -626,6 +626,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-7923. The DataNodes should rate-limit their full block reports by
     asking the NN on heartbeat messages (cmccabe)
 
+    HDFS-8499. Refactor BlockInfo class hierarchy with static helper class.
+    (Zhe Zhang via wang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

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

@@ -79,7 +79,7 @@ public interface BlockCollection {
    * Convert the last block of the collection to an under-construction block
    * and set the locations.
    */
-  public BlockInfoContiguousUnderConstruction setLastBlock(BlockInfo lastBlock,
+  public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
       DatanodeStorageInfo[] targets) throws IOException;
 
   /**

+ 13 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java

@@ -51,7 +51,7 @@ public abstract class  BlockInfo extends Block
    * per replica is 42 bytes (LinkedList#Entry object per replica) versus 16
    * bytes using the triplets.
    */
-  protected Object[] triplets;
+  Object[] triplets;
 
   /**
    * Construct an entry for blocksmap
@@ -295,7 +295,7 @@ public abstract class  BlockInfo extends Block
   /**
    * BlockInfo represents a block that is not being constructed.
    * In order to start modifying the block, the BlockInfo should be converted
-   * to {@link BlockInfoContiguousUnderConstruction}.
+   * to {@link BlockInfoUnderConstruction}.
    * @return {@link BlockUCState#COMPLETE}
    */
   public BlockUCState getBlockUCState() {
@@ -312,27 +312,29 @@ public abstract class  BlockInfo extends Block
   }
 
   /**
-   * Convert a complete block to an under construction block.
+   * Convert a block to an under construction block.
    * @return BlockInfoUnderConstruction -  an under construction block.
    */
-  public BlockInfoContiguousUnderConstruction convertToBlockUnderConstruction(
+  public BlockInfoUnderConstruction convertToBlockUnderConstruction(
       BlockUCState s, DatanodeStorageInfo[] targets) {
     if(isComplete()) {
-      BlockInfoContiguousUnderConstruction ucBlock =
-          new BlockInfoContiguousUnderConstruction(this,
-          getBlockCollection().getPreferredBlockReplication(), s, targets);
-      ucBlock.setBlockCollection(getBlockCollection());
-      return ucBlock;
+      return convertCompleteBlockToUC(s, targets);
     }
     // the block is already under construction
-    BlockInfoContiguousUnderConstruction ucBlock =
-        (BlockInfoContiguousUnderConstruction)this;
+    BlockInfoUnderConstruction ucBlock =
+        (BlockInfoUnderConstruction)this;
     ucBlock.setBlockUCState(s);
     ucBlock.setExpectedLocations(targets);
     ucBlock.setBlockCollection(getBlockCollection());
     return ucBlock;
   }
 
+  /**
+   * Convert a complete block to an under construction block.
+   */
+  abstract BlockInfoUnderConstruction convertCompleteBlockToUC(
+      BlockUCState s, DatanodeStorageInfo[] targets);
+
   @Override
   public int hashCode() {
     // Super implementation is sufficient

+ 15 - 62
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java

@@ -19,13 +19,13 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
 /**
  * Subclass of {@link BlockInfo}, used for a block with replication scheme.
  */
 @InterfaceAudience.Private
 public class BlockInfoContiguous extends BlockInfo {
-  public static final BlockInfoContiguous[] EMPTY_ARRAY = {};
 
   public BlockInfoContiguous(short size) {
     super(size);
@@ -40,84 +40,37 @@ public class BlockInfoContiguous extends BlockInfo {
    * This is used to convert BlockReplicationInfoUnderConstruction
    * @param from BlockReplicationInfo to copy from.
    */
-  protected BlockInfoContiguous(BlockInfoContiguous from) {
+  protected BlockInfoContiguous(BlockInfo from) {
     super(from);
   }
 
-  /**
-   * Ensure that there is enough  space to include num more triplets.
-   * @return first free triplet index.
-   */
-  private int ensureCapacity(int num) {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    int last = numNodes();
-    if (triplets.length >= (last+num)*3) {
-      return last;
-    }
-    /* Not enough space left. Create a new array. Should normally
-     * happen only when replication is manually increased by the user. */
-    Object[] old = triplets;
-    triplets = new Object[(last+num)*3];
-    System.arraycopy(old, 0, triplets, 0, last * 3);
-    return last;
-  }
-
   @Override
   boolean addStorage(DatanodeStorageInfo storage) {
-    // find the last null node
-    int lastNode = ensureCapacity(1);
-    setStorageInfo(lastNode, storage);
-    setNext(lastNode, null);
-    setPrevious(lastNode, null);
-    return true;
+    return ContiguousBlockStorageOp.addStorage(this, storage);
   }
 
   @Override
   boolean removeStorage(DatanodeStorageInfo storage) {
-    int dnIndex = findStorageInfo(storage);
-    if (dnIndex < 0) { // the node is not found
-      return false;
-    }
-    assert getPrevious(dnIndex) == null && getNext(dnIndex) == null :
-        "Block is still in the list and must be removed first.";
-    // find the last not null node
-    int lastNode = numNodes()-1;
-    // replace current node triplet by the lastNode one
-    setStorageInfo(dnIndex, getStorageInfo(lastNode));
-    setNext(dnIndex, getNext(lastNode));
-    setPrevious(dnIndex, getPrevious(lastNode));
-    // set the last triplet to null
-    setStorageInfo(lastNode, null);
-    setNext(lastNode, null);
-    setPrevious(lastNode, null);
-    return true;
+    return ContiguousBlockStorageOp.removeStorage(this, storage);
   }
 
   @Override
   public int numNodes() {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    assert triplets.length % 3 == 0 : "Malformed BlockInfo";
-
-    for (int idx = getCapacity()-1; idx >= 0; idx--) {
-      if (getDatanode(idx) != null) {
-        return idx + 1;
-      }
-    }
-    return 0;
+    return ContiguousBlockStorageOp.numNodes(this);
   }
 
   @Override
   void replaceBlock(BlockInfo newBlock) {
-    assert newBlock instanceof BlockInfoContiguous;
-    for (int i = this.numNodes() - 1; i >= 0; i--) {
-      final DatanodeStorageInfo storage = this.getStorageInfo(i);
-      final boolean removed = storage.removeBlock(this);
-      assert removed : "currentBlock not found.";
+    ContiguousBlockStorageOp.replaceBlock(this, newBlock);
+  }
 
-      final DatanodeStorageInfo.AddBlockResult result = storage.addBlock(
-          newBlock);
-      assert result == DatanodeStorageInfo.AddBlockResult.ADDED :
-          "newBlock already exists.";
-    }
+  @Override
+  BlockInfoUnderConstruction convertCompleteBlockToUC(
+      HdfsServerConstants.BlockUCState s, DatanodeStorageInfo[] targets) {
+    BlockInfoUnderConstructionContiguous ucBlock =
+        new BlockInfoUnderConstructionContiguous(this,
+            getBlockCollection().getPreferredBlockReplication(), s, targets);
+    ucBlock.setBlockCollection(getBlockCollection());
+    return ucBlock;
   }
 }

+ 54 - 52
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java

@@ -22,7 +22,9 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -31,15 +33,15 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
  * Represents a block that is currently being constructed.<br>
  * This is usually the last block of a file opened for write or append.
  */
-public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
+public abstract class BlockInfoUnderConstruction extends BlockInfo {
   /** Block state. See {@link BlockUCState} */
-  private BlockUCState blockUCState;
+  protected BlockUCState blockUCState;
 
   /**
    * Block replicas as assigned when the block was allocated.
    * This defines the pipeline order.
    */
-  private List<ReplicaUnderConstruction> replicas;
+  protected List<ReplicaUnderConstruction> replicas;
 
   /**
    * Index of the primary data node doing the recovery. Useful for log
@@ -57,12 +59,12 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
   /**
    * The block source to use in the event of copy-on-write truncate.
    */
-  private Block truncateBlock;
+  protected Block truncateBlock;
 
   /**
    * ReplicaUnderConstruction contains information about replicas while
    * they are under construction.
-   * The GS, the length and the state of the replica is as reported by 
+   * The GS, the length and the state of the replica is as reported by
    * the data-node.
    * It is not guaranteed, but expected, that data-nodes actually have
    * corresponding replicas.
@@ -143,7 +145,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
       appendStringTo(b);
       return b.toString();
     }
-    
+
     @Override
     public void appendStringTo(StringBuilder sb) {
       sb.append("ReplicaUC[")
@@ -158,45 +160,24 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
    * Create block and set its state to
    * {@link BlockUCState#UNDER_CONSTRUCTION}.
    */
-  public BlockInfoContiguousUnderConstruction(Block blk, short replication) {
+  public BlockInfoUnderConstruction(Block blk, short replication) {
     this(blk, replication, BlockUCState.UNDER_CONSTRUCTION, null);
   }
 
   /**
    * Create a block that is currently being constructed.
    */
-  public BlockInfoContiguousUnderConstruction(Block blk, short replication,
+  public BlockInfoUnderConstruction(Block blk, short replication,
       BlockUCState state, DatanodeStorageInfo[] targets) {
     super(blk, replication);
-    assert getBlockUCState() != BlockUCState.COMPLETE :
-      "BlockInfoUnderConstruction cannot be in COMPLETE state";
+    Preconditions.checkState(getBlockUCState() != BlockUCState.COMPLETE,
+        "BlockInfoUnderConstruction cannot be in COMPLETE state");
     this.blockUCState = state;
     setExpectedLocations(targets);
   }
 
-  /**
-   * Convert an under construction block to a complete block.
-   * 
-   * @return BlockInfo - a complete block.
-   * @throws IOException if the state of the block 
-   * (the generation stamp and the length) has not been committed by 
-   * the client or it does not have at least a minimal number of replicas 
-   * reported from data-nodes. 
-   */
-  BlockInfo convertToCompleteBlock() throws IOException {
-    assert getBlockUCState() != BlockUCState.COMPLETE :
-      "Trying to convert a COMPLETE block";
-    return new BlockInfoContiguous(this);
-  }
-
-  /** Set expected locations */
-  public void setExpectedLocations(DatanodeStorageInfo[] targets) {
-    int numLocations = targets == null ? 0 : targets.length;
-    this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations);
-    for(int i = 0; i < numLocations; i++)
-      replicas.add(
-        new ReplicaUnderConstruction(this, targets[i], ReplicaState.RBW));
-  }
+  /** Set expected locations. */
+  public abstract void setExpectedLocations(DatanodeStorageInfo[] targets);
 
   /**
    * Create array of expected replica locations
@@ -205,12 +186,13 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
   public DatanodeStorageInfo[] getExpectedStorageLocations() {
     int numLocations = replicas == null ? 0 : replicas.size();
     DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
-    for(int i = 0; i < numLocations; i++)
+    for(int i = 0; i < numLocations; i++) {
       storages[i] = replicas.get(i).getExpectedStorageLocation();
+    }
     return storages;
   }
 
-  /** Get the number of expected locations */
+  /** Get the number of expected locations. */
   public int getNumExpectedLocations() {
     return replicas == null ? 0 : replicas.size();
   }
@@ -228,19 +210,15 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
     blockUCState = s;
   }
 
-  /** Get block recovery ID */
+  /** Get block recovery ID. */
   public long getBlockRecoveryId() {
     return blockRecoveryId;
   }
 
-  /** Get recover block */
-  public Block getTruncateBlock() {
-    return truncateBlock;
-  }
+  /** Get recover block. */
+  public abstract Block getTruncateBlock();
 
-  public void setTruncateBlock(Block recoveryBlock) {
-    this.truncateBlock = recoveryBlock;
-  }
+  public abstract void setTruncateBlock(Block recoveryBlock);
 
   /**
    * Process the recorded replicas. When about to commit or finish the
@@ -250,8 +228,9 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
   public void setGenerationStampAndVerifyReplicas(long genStamp) {
     // Set the generation stamp for the block.
     setGenerationStamp(genStamp);
-    if (replicas == null)
+    if (replicas == null) {
       return;
+    }
 
     // Remove the replicas with wrong gen stamp.
     // The replica list is unchanged.
@@ -267,13 +246,14 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
   /**
    * Commit block's length and generation stamp as reported by the client.
    * Set block state to {@link BlockUCState#COMMITTED}.
-   * @param block - contains client reported block length and generation 
+   * @param block - contains client reported block length and generation
    * @throws IOException if block ids are inconsistent.
    */
   void commitBlock(Block block) throws IOException {
-    if(getBlockId() != block.getBlockId())
+    if(getBlockId() != block.getBlockId()) {
       throw new IOException("Trying to commit inconsistent block: id = "
           + block.getBlockId() + ", expected id = " + getBlockId());
+    }
     blockUCState = BlockUCState.COMMITTED;
     this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp());
     // Sort out invalid replicas.
@@ -289,16 +269,17 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
     setBlockUCState(BlockUCState.UNDER_RECOVERY);
     blockRecoveryId = recoveryId;
     if (replicas.size() == 0) {
-      NameNode.blockStateChangeLog.warn("BLOCK*"
-        + " BlockInfoUnderConstruction.initLeaseRecovery:"
-        + " No blocks found, lease removed.");
+      NameNode.blockStateChangeLog.warn("BLOCK* " +
+          "BlockInfoUnderConstruction.initLeaseRecovery: " +
+          "No blocks found, lease removed.");
     }
     boolean allLiveReplicasTriedAsPrimary = true;
     for (int i = 0; i < replicas.size(); i++) {
       // Check if all replicas have been tried or not.
       if (replicas.get(i).isAlive()) {
         allLiveReplicasTriedAsPrimary =
-            (allLiveReplicasTriedAsPrimary && replicas.get(i).getChosenAsPrimary());
+            (allLiveReplicasTriedAsPrimary &&
+                replicas.get(i).getChosenAsPrimary());
       }
     }
     if (allLiveReplicasTriedAsPrimary) {
@@ -312,7 +293,8 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
     primaryNodeIndex = -1;
     for(int i = 0; i < replicas.size(); i++) {
       // Skip alive replicas which have been chosen for recovery.
-      if (!(replicas.get(i).isAlive() && !replicas.get(i).getChosenAsPrimary())) {
+      if (!(replicas.get(i).isAlive() &&
+          !replicas.get(i).getChosenAsPrimary())) {
         continue;
       }
       final ReplicaUnderConstruction ruc = replicas.get(i);
@@ -325,7 +307,8 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
       }
     }
     if (primary != null) {
-      primary.getExpectedStorageLocation().getDatanodeDescriptor().addBlockToBeRecovered(this);
+      primary.getExpectedStorageLocation().
+          getDatanodeDescriptor().addBlockToBeRecovered(this);
       primary.setChosenAsPrimary(true);
       NameNode.blockStateChangeLog.info(
           "BLOCK* {} recovery started, primary={}", this, primary);
@@ -358,6 +341,25 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
     replicas.add(new ReplicaUnderConstruction(block, storage, rState));
   }
 
+  /**
+   * Convert an under construction block to a complete block.
+   *
+   * @return a complete block.
+   * @throws IOException
+   *           if the state of the block (the generation stamp and the length)
+   *           has not been committed by the client or it does not have at
+   *           least a minimal number of replicas reported from data-nodes.
+   */
+  public abstract BlockInfo convertToCompleteBlock();
+
+  @Override
+  BlockInfoUnderConstruction convertCompleteBlockToUC
+      (HdfsServerConstants.BlockUCState s, DatanodeStorageInfo[] targets) {
+    BlockManager.LOG.error("convertCompleteBlockToUC should only be applied " +
+        "on complete blocks.");
+    return null;
+  }
+
   @Override // BlockInfo
   // BlockInfoUnderConstruction participates in maps the same way as BlockInfo
   public int hashCode() {

+ 110 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java

@@ -0,0 +1,110 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+
+import java.util.ArrayList;
+
+/**
+ * Subclass of {@link BlockInfoUnderConstruction}, representing a block under
+ * the contiguous (instead of striped) layout.
+ */
+public class BlockInfoUnderConstructionContiguous extends
+    BlockInfoUnderConstruction {
+  /**
+   * Create block and set its state to
+   * {@link HdfsServerConstants.BlockUCState#UNDER_CONSTRUCTION}.
+   */
+  public BlockInfoUnderConstructionContiguous(Block blk, short replication) {
+    this(blk, replication, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
+        null);
+  }
+
+  /**
+   * Create a block that is currently being constructed.
+   */
+  public BlockInfoUnderConstructionContiguous(Block blk, short replication,
+      HdfsServerConstants.BlockUCState state, DatanodeStorageInfo[] targets) {
+    super(blk, replication);
+    Preconditions.checkState(getBlockUCState() !=
+        HdfsServerConstants.BlockUCState.COMPLETE,
+        "BlockInfoUnderConstructionContiguous cannot be in COMPLETE state");
+    this.blockUCState = state;
+    setExpectedLocations(targets);
+  }
+
+  /**
+   * Convert an under construction block to a complete block.
+   *
+   * @return BlockInfo - a complete block.
+   * @throws IOException if the state of the block
+   * (the generation stamp and the length) has not been committed by
+   * the client or it does not have at least a minimal number of replicas
+   * reported from data-nodes.
+   */
+  @Override
+  public BlockInfoContiguous convertToCompleteBlock() {
+    Preconditions.checkState(getBlockUCState() !=
+        HdfsServerConstants.BlockUCState.COMPLETE,
+        "Trying to convert a COMPLETE block");
+    return new BlockInfoContiguous(this);
+  }
+
+  @Override
+  boolean addStorage(DatanodeStorageInfo storage) {
+    return ContiguousBlockStorageOp.addStorage(this, storage);
+  }
+
+  @Override
+  boolean removeStorage(DatanodeStorageInfo storage) {
+    return ContiguousBlockStorageOp.removeStorage(this, storage);
+  }
+
+  @Override
+  public int numNodes() {
+    return ContiguousBlockStorageOp.numNodes(this);
+  }
+
+  @Override
+  void replaceBlock(BlockInfo newBlock) {
+    ContiguousBlockStorageOp.replaceBlock(this, newBlock);
+  }
+
+  @Override
+  public void setExpectedLocations(DatanodeStorageInfo[] targets) {
+    int numLocations = targets == null ? 0 : targets.length;
+    this.replicas = new ArrayList<>(numLocations);
+    for(int i = 0; i < numLocations; i++) {
+      replicas.add(
+          new ReplicaUnderConstruction(this, targets[i], HdfsServerConstants.ReplicaState.RBW));
+    }
+  }
+
+  @Override
+  public Block getTruncateBlock() {
+    return truncateBlock;
+  }
+
+  @Override
+  public void setTruncateBlock(Block recoveryBlock) {
+    this.truncateBlock = recoveryBlock;
+  }
+}

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

@@ -602,7 +602,7 @@ public class BlockManager {
    * of replicas reported from data-nodes.
    */
   private static boolean commitBlock(
-      final BlockInfoContiguousUnderConstruction block, final Block commitBlock)
+      final BlockInfoUnderConstruction block, final Block commitBlock)
       throws IOException {
     if (block.getBlockUCState() == BlockUCState.COMMITTED)
       return false;
@@ -634,7 +634,7 @@ public class BlockManager {
       return false; // already completed (e.g. by syncBlock)
     
     final boolean b = commitBlock(
-        (BlockInfoContiguousUnderConstruction) lastBlock, commitBlock);
+        (BlockInfoUnderConstruction) lastBlock, commitBlock);
     if(countNodes(lastBlock).liveReplicas() >= minReplication)
       completeBlock(bc, bc.numBlocks()-1, false);
     return b;
@@ -654,8 +654,8 @@ public class BlockManager {
     BlockInfo curBlock = bc.getBlocks()[blkIndex];
     if(curBlock.isComplete())
       return curBlock;
-    BlockInfoContiguousUnderConstruction ucBlock =
-        (BlockInfoContiguousUnderConstruction) curBlock;
+    BlockInfoUnderConstruction ucBlock =
+        (BlockInfoUnderConstruction) curBlock;
     int numNodes = ucBlock.numNodes();
     if (!force && numNodes < minReplication)
       throw new IOException("Cannot complete block: " +
@@ -697,7 +697,7 @@ public class BlockManager {
    * when tailing edit logs as a Standby.
    */
   public BlockInfo forceCompleteBlock(final BlockCollection bc,
-      final BlockInfoContiguousUnderConstruction block) throws IOException {
+      final BlockInfoUnderConstruction block) throws IOException {
     block.commitBlock(block);
     return completeBlock(bc, block, true);
   }
@@ -728,7 +728,7 @@ public class BlockManager {
 
     DatanodeStorageInfo[] targets = getStorages(oldBlock);
 
-    BlockInfoContiguousUnderConstruction ucBlock =
+    BlockInfoUnderConstruction ucBlock =
         bc.setLastBlock(oldBlock, targets);
     blocksMap.replaceBlock(ucBlock);
 
@@ -830,14 +830,14 @@ public class BlockManager {
   /** @return a LocatedBlock for the given block */
   private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos
       ) throws IOException {
-    if (blk instanceof BlockInfoContiguousUnderConstruction) {
+    if (blk instanceof BlockInfoUnderConstruction) {
       if (blk.isComplete()) {
         throw new IOException(
             "blk instanceof BlockInfoUnderConstruction && blk.isComplete()"
             + ", blk=" + blk);
       }
-      final BlockInfoContiguousUnderConstruction uc =
-          (BlockInfoContiguousUnderConstruction) blk;
+      final BlockInfoUnderConstruction uc =
+          (BlockInfoUnderConstruction) blk;
       final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
       final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
       return newLocatedBlock(eb, storages, pos, false);
@@ -1744,11 +1744,11 @@ public class BlockManager {
    * reported by the datanode in the block report. 
    */
   static class StatefulBlockInfo {
-    final BlockInfoContiguousUnderConstruction storedBlock;
+    final BlockInfoUnderConstruction storedBlock;
     final Block reportedBlock;
     final ReplicaState reportedState;
     
-    StatefulBlockInfo(BlockInfoContiguousUnderConstruction storedBlock,
+    StatefulBlockInfo(BlockInfoUnderConstruction storedBlock,
         Block reportedBlock, ReplicaState reportedState) {
       this.storedBlock = storedBlock;
       this.reportedBlock = reportedBlock;
@@ -1789,7 +1789,7 @@ public class BlockManager {
 
     BlockToMarkCorrupt(BlockInfo stored, long gs, String reason,
         Reason reasonCode) {
-      this(new BlockInfoContiguous((BlockInfoContiguous)stored), stored,
+      this(new BlockInfoContiguous(stored), stored,
           reason, reasonCode);
       //the corrupted block in datanode has a different generation stamp
       corrupted.setGenerationStamp(gs);
@@ -2148,13 +2148,13 @@ public class BlockManager {
       
       // If block is under construction, add this replica to its list
       if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
-        ((BlockInfoContiguousUnderConstruction)storedBlock)
+        ((BlockInfoUnderConstruction)storedBlock)
             .addReplicaIfNotPresent(storageInfo, iblk, reportedState);
         // OpenFileBlocks only inside snapshots also will be added to safemode
         // threshold. So we need to update such blocks to safemode
         // refer HDFS-5283
-        BlockInfoContiguousUnderConstruction blockUC =
-            (BlockInfoContiguousUnderConstruction) storedBlock;
+        BlockInfoUnderConstruction blockUC =
+            (BlockInfoUnderConstruction) storedBlock;
         if (namesystem.isInSnapshot(blockUC)) {
           int numOfReplicas = blockUC.getNumExpectedLocations();
           namesystem.incrementSafeBlockCount(numOfReplicas);
@@ -2309,7 +2309,7 @@ public class BlockManager {
 
     if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
       toUC.add(new StatefulBlockInfo(
-          (BlockInfoContiguousUnderConstruction) storedBlock,
+          (BlockInfoUnderConstruction) storedBlock,
           new Block(block), reportedState));
       return storedBlock;
     }
@@ -2500,7 +2500,7 @@ public class BlockManager {
 
   void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
       DatanodeStorageInfo storageInfo) throws IOException {
-    BlockInfoContiguousUnderConstruction block = ucBlock.storedBlock;
+    BlockInfoUnderConstruction block = ucBlock.storedBlock;
     block.addReplicaIfNotPresent(
         storageInfo, ucBlock.reportedBlock, ucBlock.reportedState);
 
@@ -2561,7 +2561,7 @@ public class BlockManager {
     assert block != null && namesystem.hasWriteLock();
     BlockInfo storedBlock;
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
-    if (block instanceof BlockInfoContiguousUnderConstruction) {
+    if (block instanceof BlockInfoUnderConstruction) {
       //refresh our copy in case the block got completed in another thread
       storedBlock = blocksMap.getStoredBlock(block);
     } else {
@@ -3499,8 +3499,8 @@ public class BlockManager {
       String src, BlockInfo[] blocks) {
     for (BlockInfo b: blocks) {
       if (!b.isComplete()) {
-        final BlockInfoContiguousUnderConstruction uc =
-            (BlockInfoContiguousUnderConstruction)b;
+        final BlockInfoUnderConstruction uc =
+            (BlockInfoUnderConstruction)b;
         final int numNodes = b.numNodes();
         LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = "
           + uc.getBlockUCState() + ", replication# = " + numNodes

+ 106 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ContiguousBlockStorageOp.java

@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Utility class with logic on managing storage locations shared between
+ * complete and under-construction blocks under the contiguous format --
+ * {@link BlockInfoContiguous} and
+ * {@link BlockInfoUnderConstructionContiguous}.
+ */
+class ContiguousBlockStorageOp {
+  /**
+   * Ensure that there is enough  space to include num more triplets.
+   * @return first free triplet index.
+   */
+  private static int ensureCapacity(BlockInfo b, int num) {
+    Preconditions.checkArgument(b.triplets != null,
+        "BlockInfo is not initialized");
+    int last = b.numNodes();
+    if (b.triplets.length >= (last+num)*3) {
+      return last;
+    }
+    /* Not enough space left. Create a new array. Should normally
+     * happen only when replication is manually increased by the user. */
+    Object[] old = b.triplets;
+    b.triplets = new Object[(last+num)*3];
+    System.arraycopy(old, 0, b.triplets, 0, last * 3);
+    return last;
+  }
+
+  static boolean addStorage(BlockInfo b, DatanodeStorageInfo storage) {
+    // find the last null node
+    int lastNode = ensureCapacity(b, 1);
+    b.setStorageInfo(lastNode, storage);
+    b.setNext(lastNode, null);
+    b.setPrevious(lastNode, null);
+    return true;
+  }
+
+  static boolean removeStorage(BlockInfo b,
+      DatanodeStorageInfo storage) {
+    int dnIndex = b.findStorageInfo(storage);
+    if (dnIndex < 0) { // the node is not found
+      return false;
+    }
+    Preconditions.checkArgument(b.getPrevious(dnIndex) == null &&
+            b.getNext(dnIndex) == null,
+        "Block is still in the list and must be removed first.");
+    // find the last not null node
+    int lastNode = b.numNodes()-1;
+    // replace current node triplet by the lastNode one
+    b.setStorageInfo(dnIndex, b.getStorageInfo(lastNode));
+    b.setNext(dnIndex, b.getNext(lastNode));
+    b.setPrevious(dnIndex, b.getPrevious(lastNode));
+    // set the last triplet to null
+    b.setStorageInfo(lastNode, null);
+    b.setNext(lastNode, null);
+    b.setPrevious(lastNode, null);
+    return true;
+  }
+
+  static int numNodes(BlockInfo b) {
+    Preconditions.checkArgument(b.triplets != null,
+        "BlockInfo is not initialized");
+    Preconditions.checkArgument(b.triplets.length % 3 == 0,
+        "Malformed BlockInfo");
+
+    for (int idx = b.getCapacity()-1; idx >= 0; idx--) {
+      if (b.getDatanode(idx) != null) {
+        return idx + 1;
+      }
+    }
+    return 0;
+  }
+
+  static void replaceBlock(BlockInfo b, BlockInfo newBlock) {
+    for (int i = b.numNodes() - 1; i >= 0; i--) {
+      final DatanodeStorageInfo storage = b.getStorageInfo(i);
+      final boolean removed = storage.removeBlock(b);
+      Preconditions.checkState(removed, "currentBlock not found.");
+
+      final DatanodeStorageInfo.AddBlockResult result = storage.addBlock(
+          newBlock);
+      Preconditions.checkState(
+          result == DatanodeStorageInfo.AddBlockResult.ADDED,
+          "newBlock already exists.");
+    }
+  }
+}

+ 7 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

@@ -219,8 +219,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /** A queue of blocks to be replicated by this datanode */
   private final BlockQueue<BlockTargetPair> replicateBlocks = new BlockQueue<BlockTargetPair>();
   /** A queue of blocks to be recovered by this datanode */
-  private final BlockQueue<BlockInfoContiguousUnderConstruction> recoverBlocks =
-                                new BlockQueue<BlockInfoContiguousUnderConstruction>();
+  private final BlockQueue<BlockInfoUnderConstruction> recoverBlocks =
+                                new BlockQueue<BlockInfoUnderConstruction>();
   /** A set of blocks to be invalidated by this datanode */
   private final LightWeightHashSet<Block> invalidateBlocks = new LightWeightHashSet<Block>();
 
@@ -599,7 +599,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * Store block recovery work.
    */
-  void addBlockToBeRecovered(BlockInfoContiguousUnderConstruction block) {
+  void addBlockToBeRecovered(BlockInfoUnderConstruction block) {
     if(recoverBlocks.contains(block)) {
       // this prevents adding the same block twice to the recovery queue
       BlockManager.LOG.info(block + " is already in the recovery queue");
@@ -641,11 +641,12 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return replicateBlocks.poll(maxTransfers);
   }
 
-  public BlockInfoContiguousUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
-    List<BlockInfoContiguousUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
+  public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(
+      int maxTransfers) {
+    List<BlockInfoUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
     if(blocks == null)
       return null;
-    return blocks.toArray(new BlockInfoContiguousUnderConstruction[blocks.size()]);
+    return blocks.toArray(new BlockInfoUnderConstruction[blocks.size()]);
   }
 
   /**

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

@@ -1380,12 +1380,12 @@ public class DatanodeManager {
         }
 
         //check lease recovery
-        BlockInfoContiguousUnderConstruction[] blocks = nodeinfo
+        BlockInfoUnderConstruction[] blocks = nodeinfo
             .getLeaseRecoveryCommand(Integer.MAX_VALUE);
         if (blocks != null) {
           BlockRecoveryCommand brCommand = new BlockRecoveryCommand(
               blocks.length);
-          for (BlockInfoContiguousUnderConstruction b : blocks) {
+          for (BlockInfoUnderConstruction b : blocks) {
             final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations();
             // Skip stale nodes during recovery - not heart beated for some time (30s by default).
             final List<DatanodeStorageInfo> recoveryLocations =

+ 8 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java

@@ -43,7 +43,8 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -73,7 +74,7 @@ class FSDirWriteFileOp {
       Block block) throws IOException {
     // modify file-> block and blocksMap
     // fileNode should be under construction
-    BlockInfoContiguousUnderConstruction uc = fileNode.removeLastBlock(block);
+    BlockInfoUnderConstruction uc = fileNode.removeLastBlock(block);
     if (uc == null) {
       return false;
     }
@@ -236,7 +237,7 @@ class FSDirWriteFileOp {
       } else {
         // add new chosen targets to already allocated block and return
         BlockInfo lastBlockInFile = pendingFile.getLastBlock();
-        ((BlockInfoContiguousUnderConstruction) lastBlockInFile)
+        ((BlockInfoUnderConstruction) lastBlockInFile)
             .setExpectedLocations(targets);
         offset = pendingFile.computeFileSize();
         return makeLocatedBlock(fsn, lastBlockInFile, targets, offset);
@@ -520,8 +521,8 @@ class FSDirWriteFileOp {
           fileINode.getPreferredBlockReplication(), true);
 
       // associate new last block for the file
-      BlockInfoContiguousUnderConstruction blockInfo =
-        new BlockInfoContiguousUnderConstruction(
+      BlockInfoUnderConstruction blockInfo =
+        new BlockInfoUnderConstructionContiguous(
             block,
             fileINode.getFileReplication(),
             HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
@@ -662,8 +663,8 @@ class FSDirWriteFileOp {
             "allocation of a new block in " + src + ". Returning previously" +
             " allocated block " + lastBlockInFile);
         long offset = file.computeFileSize();
-        BlockInfoContiguousUnderConstruction lastBlockUC =
-            (BlockInfoContiguousUnderConstruction) lastBlockInFile;
+        BlockInfoUnderConstruction lastBlockUC =
+            (BlockInfoUnderConstruction) lastBlockInFile;
         onRetryBlock[0] = makeLocatedBlock(fsn, lastBlockInFile,
             lastBlockUC.getExpectedStorageLocations(), offset);
         return new FileState(file, src, iip);

+ 8 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -45,7 +45,8 @@ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -960,16 +961,16 @@ public class FSEditLogLoader {
       }
       
       oldLastBlock.setNumBytes(pBlock.getNumBytes());
-      if (oldLastBlock instanceof BlockInfoContiguousUnderConstruction) {
+      if (oldLastBlock instanceof BlockInfoUnderConstruction) {
         fsNamesys.getBlockManager().forceCompleteBlock(file,
-            (BlockInfoContiguousUnderConstruction) oldLastBlock);
+            (BlockInfoUnderConstruction) oldLastBlock);
         fsNamesys.getBlockManager().processQueuedMessagesForBlock(pBlock);
       }
     } else { // the penultimate block is null
       Preconditions.checkState(oldBlocks == null || oldBlocks.length == 0);
     }
     // add the new block
-    BlockInfo newBI = new BlockInfoContiguousUnderConstruction(
+    BlockInfo newBI = new BlockInfoUnderConstructionContiguous(
           newBlock, file.getPreferredBlockReplication());
     fsNamesys.getBlockManager().addBlockCollection(newBI, file);
     file.addBlock(newBI);
@@ -1010,11 +1011,11 @@ public class FSEditLogLoader {
         oldBlock.getGenerationStamp() != newBlock.getGenerationStamp();
       oldBlock.setGenerationStamp(newBlock.getGenerationStamp());
       
-      if (oldBlock instanceof BlockInfoContiguousUnderConstruction &&
+      if (oldBlock instanceof BlockInfoUnderConstruction &&
           (!isLastBlock || op.shouldCompleteLastBlock())) {
         changeMade = true;
         fsNamesys.getBlockManager().forceCompleteBlock(file,
-            (BlockInfoContiguousUnderConstruction) oldBlock);
+            (BlockInfoUnderConstruction) oldBlock);
       }
       if (changeMade) {
         // The state or gen-stamp of the block has changed. So, we may be
@@ -1049,7 +1050,7 @@ public class FSEditLogLoader {
           // TODO: shouldn't this only be true for the last block?
           // what about an old-version fsync() where fsync isn't called
           // until several blocks in?
-          newBI = new BlockInfoContiguousUnderConstruction(
+          newBI = new BlockInfoUnderConstructionContiguous(
               newBlock, file.getPreferredBlockReplication());
         } else {
           // OP_CLOSE should add finalized blocks. This code path

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -54,7 +54,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -777,8 +777,9 @@ public class FSImageFormat {
             // convert the last block to BlockUC
             if (blocks.length > 0) {
               BlockInfo lastBlk = blocks[blocks.length - 1];
-              blocks[blocks.length - 1] = new BlockInfoContiguousUnderConstruction(
-                  lastBlk, replication);
+              blocks[blocks.length - 1] =
+                  new BlockInfoUnderConstructionContiguous(
+                      lastBlk, replication);
             }
           }
         }

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java

@@ -44,7 +44,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext;
@@ -364,8 +364,8 @@ public final class FSImageFormatPBINode {
         if (blocks.length > 0) {
           BlockInfo lastBlk = file.getLastBlock();
           // replace the last block of file
-          file.setBlock(file.numBlocks() - 1, new BlockInfoContiguousUnderConstruction(
-              lastBlk, replication));
+          file.setBlock(file.numBlocks() - 1,
+              new BlockInfoUnderConstructionContiguous(lastBlk, replication));
         }
       }
       return file;

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -35,7 +35,7 @@ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
@@ -137,7 +137,7 @@ public class FSImageSerialization {
     // last block is UNDER_CONSTRUCTION
     if(numBlocks > 0) {
       blk.readFields(in);
-      blocks[i] = new BlockInfoContiguousUnderConstruction(
+      blocks[i] = new BlockInfoUnderConstructionContiguous(
         blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
     }
     PermissionStatus perm = PermissionStatus.read(in);

+ 13 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -204,7 +204,8 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretMan
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -2003,7 +2004,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final BlockInfo last = file.getLastBlock();
     if (last != null && last.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
       final Block truncateBlock
-          = ((BlockInfoContiguousUnderConstruction)last).getTruncateBlock();
+          = ((BlockInfoUnderConstruction)last).getTruncateBlock();
       if (truncateBlock != null) {
         final long truncateLength = file.computeFileSize(false, false)
             + truncateBlock.getNumBytes();
@@ -2082,11 +2083,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
               nextGenerationStamp(blockIdManager.isLegacyBlock(oldBlock)));
     }
 
-    BlockInfoContiguousUnderConstruction truncatedBlockUC;
+    BlockInfoUnderConstruction truncatedBlockUC;
     if(shouldCopyOnTruncate) {
       // Add new truncateBlock into blocksMap and
       // use oldBlock as a source for copy-on-truncate recovery
-      truncatedBlockUC = new BlockInfoContiguousUnderConstruction(newBlock,
+      truncatedBlockUC = new BlockInfoUnderConstructionContiguous(newBlock,
           file.getPreferredBlockReplication());
       truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta);
       truncatedBlockUC.setTruncateBlock(oldBlock);
@@ -2102,7 +2103,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       blockManager.convertLastBlockToUnderConstruction(file, lastBlockDelta);
       oldBlock = file.getLastBlock();
       assert !oldBlock.isComplete() : "oldBlock should be under construction";
-      truncatedBlockUC = (BlockInfoContiguousUnderConstruction) oldBlock;
+      truncatedBlockUC = (BlockInfoUnderConstruction) oldBlock;
       truncatedBlockUC.setTruncateBlock(new Block(oldBlock));
       truncatedBlockUC.getTruncateBlock().setNumBytes(
           oldBlock.getNumBytes() - lastBlockDelta);
@@ -3519,7 +3520,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throw new AlreadyBeingCreatedException(message);
     case UNDER_CONSTRUCTION:
     case UNDER_RECOVERY:
-      final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)lastBlock;
+      final BlockInfoUnderConstruction uc =
+          (BlockInfoUnderConstruction)lastBlock;
       // determine if last block was intended to be truncated
       Block recoveryBlock = uc.getTruncateBlock();
       boolean truncateRecovery = recoveryBlock != null;
@@ -3635,7 +3637,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
   
   @Override
-  public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC) {
+  public boolean isInSnapshot(BlockInfoUnderConstruction blockUC) {
     assert hasReadLock();
     final BlockCollection bc = blockUC.getBlockCollection();
     if (bc == null || !(bc instanceof INodeFile)
@@ -3682,7 +3684,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     waitForLoadingFSImage();
     writeLock();
     boolean copyTruncate = false;
-    BlockInfoContiguousUnderConstruction truncatedBlock = null;
+    BlockInfoUnderConstruction truncatedBlock = null;
     try {
       checkOperation(OperationCategory.WRITE);
       // If a DN tries to commit to the standby, the recovery will
@@ -3739,7 +3741,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         return;
       }
 
-      truncatedBlock = (BlockInfoContiguousUnderConstruction) iFile
+      truncatedBlock = (BlockInfoUnderConstruction) iFile
           .getLastBlock();
       long recoveryId = truncatedBlock.getBlockRecoveryId();
       copyTruncate = truncatedBlock.getBlockId() != storedBlock.getBlockId();
@@ -5774,8 +5776,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     assert hasWriteLock();
     // check the vadility of the block and lease holder name
     final INodeFile pendingFile = checkUCBlock(oldBlock, clientName);
-    final BlockInfoContiguousUnderConstruction blockinfo
-        = (BlockInfoContiguousUnderConstruction)pendingFile.getLastBlock();
+    final BlockInfoUnderConstruction blockinfo
+        = (BlockInfoUnderConstruction)pendingFile.getLastBlock();
 
     // check new GS & length: this is not expected
     if (newBlock.getGenerationStamp() <= blockinfo.getGenerationStamp() ||

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

@@ -21,7 +21,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 
 /**
@@ -61,7 +61,7 @@ public class FileUnderConstructionFeature implements INode.Feature {
     BlockInfo lastBlock = f.getLastBlock();
     assert (lastBlock != null) : "The last block for path "
         + f.getFullPathName() + " is null when updating its length";
-    assert (lastBlock instanceof BlockInfoContiguousUnderConstruction)
+    assert (lastBlock instanceof BlockInfoUnderConstruction)
         : "The last block for path " + f.getFullPathName()
             + " is not a BlockInfoUnderConstruction when updating its length";
     lastBlock.setNumBytes(lastBlockLength);
@@ -76,9 +76,9 @@ public class FileUnderConstructionFeature implements INode.Feature {
       final BlocksMapUpdateInfo collectedBlocks) {
     final BlockInfo[] blocks = f.getBlocks();
     if (blocks != null && blocks.length > 0
-        && blocks[blocks.length - 1] instanceof BlockInfoContiguousUnderConstruction) {
-      BlockInfoContiguousUnderConstruction lastUC =
-          (BlockInfoContiguousUnderConstruction) blocks[blocks.length - 1];
+        && blocks[blocks.length - 1] instanceof BlockInfoUnderConstruction) {
+      BlockInfoUnderConstruction lastUC =
+          (BlockInfoUnderConstruction) blocks[blocks.length - 1];
       if (lastUC.getNumBytes() == 0) {
         // this is a 0-sized block. do not need check its UC state here
         collectedBlocks.addDeleteBlock(lastUC);

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

@@ -37,7 +37,7 @@ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
@@ -231,7 +231,7 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   @Override // BlockCollection, the file should be under construction
-  public BlockInfoContiguousUnderConstruction setLastBlock(
+  public BlockInfoUnderConstruction setLastBlock(
       BlockInfo lastBlock, DatanodeStorageInfo[] locations)
       throws IOException {
     Preconditions.checkState(isUnderConstruction(),
@@ -240,7 +240,7 @@ public class INodeFile extends INodeWithAdditionalFields
     if (numBlocks() == 0) {
       throw new IOException("Failed to set last block: File is empty.");
     }
-    BlockInfoContiguousUnderConstruction ucBlock =
+    BlockInfoUnderConstruction ucBlock =
       lastBlock.convertToBlockUnderConstruction(
           BlockUCState.UNDER_CONSTRUCTION, locations);
     setBlock(numBlocks() - 1, ucBlock);
@@ -251,7 +251,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * Remove a block from the block list. This block should be
    * the last one on the list.
    */
-  BlockInfoContiguousUnderConstruction removeLastBlock(Block oldblock) {
+  BlockInfoUnderConstruction removeLastBlock(Block oldblock) {
     Preconditions.checkState(isUnderConstruction(),
         "file is no longer under construction");
     if (blocks == null || blocks.length == 0) {
@@ -262,8 +262,8 @@ public class INodeFile extends INodeWithAdditionalFields
       return null;
     }
 
-    BlockInfoContiguousUnderConstruction uc =
-        (BlockInfoContiguousUnderConstruction)blocks[size_1];
+    BlockInfoUnderConstruction uc =
+        (BlockInfoUnderConstruction)blocks[size_1];
     //copy to a new list
     BlockInfo[] newlist = new BlockInfo[size_1];
     System.arraycopy(blocks, 0, newlist, 0, size_1);
@@ -689,7 +689,7 @@ public class INodeFile extends INodeWithAdditionalFields
     final int last = blocks.length - 1;
     //check if the last block is BlockInfoUnderConstruction
     long size = blocks[last].getNumBytes();
-    if (blocks[last] instanceof BlockInfoContiguousUnderConstruction) {
+    if (blocks[last] instanceof BlockInfoUnderConstruction) {
        if (!includesLastUcBlock) {
          size = 0;
        } else if (usePreferredBlockSize4LastUcBlock) {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java

@@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.apache.hadoop.ipc.StandbyException;
@@ -45,5 +45,5 @@ public interface Namesystem extends RwLock, SafeMode {
 
   public void checkOperation(OperationCategory read) throws StandbyException;
 
-  public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC);
+  public boolean isInSnapshot(BlockInfoUnderConstruction blockUC);
 }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java

@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@@ -133,7 +133,7 @@ public class FileDiffList extends
     Block dontRemoveBlock = null;
     if (lastBlock != null && lastBlock.getBlockUCState().equals(
         HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) {
-      dontRemoveBlock = ((BlockInfoContiguousUnderConstruction) lastBlock)
+      dontRemoveBlock = ((BlockInfoUnderConstruction) lastBlock)
           .getTruncateBlock();
     }
     // Collect the remaining blocks of the file, ignoring truncate block

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -109,7 +109,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -1612,9 +1612,9 @@ public class DFSTestUtil {
     BlockInfo storedBlock = bm0.getStoredBlock(blk.getLocalBlock());
     assertTrue("Block " + blk + " should be under construction, " +
         "got: " + storedBlock,
-        storedBlock instanceof BlockInfoContiguousUnderConstruction);
-    BlockInfoContiguousUnderConstruction ucBlock =
-      (BlockInfoContiguousUnderConstruction)storedBlock;
+        storedBlock instanceof BlockInfoUnderConstruction);
+    BlockInfoUnderConstruction ucBlock =
+      (BlockInfoUnderConstruction)storedBlock;
     // We expect that the replica with the most recent heart beat will be
     // the one to be in charge of the synchronization / recovery protocol.
     final DatanodeStorageInfo[] storages = ucBlock.getExpectedStorageLocations();

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java

@@ -23,7 +23,6 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-import org.apache.hadoop.util.Time;
 import org.junit.Test;
 
 /**
@@ -40,7 +39,8 @@ public class TestBlockInfoUnderConstruction {
     DatanodeDescriptor dd3 = s3.getDatanodeDescriptor();
 
     dd1.isAlive = dd2.isAlive = dd3.isAlive = true;
-    BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
+    BlockInfoUnderConstruction blockInfo =
+        new BlockInfoUnderConstructionContiguous(
         new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP),
         (short) 3,
         BlockUCState.UNDER_CONSTRUCTION,
@@ -51,7 +51,7 @@ public class TestBlockInfoUnderConstruction {
     DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd3, -2 * 1000);
     blockInfo.initializeBlockRecovery(1);
-    BlockInfoContiguousUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
+    BlockInfoUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
     assertEquals(blockInfoRecovery[0], blockInfo);
 
     // Recovery attempt #2.

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java

@@ -726,7 +726,7 @@ public class TestBlockManager {
     // verify the storage info is correct
     assertTrue(bm.getStoredBlock(new Block(receivedBlockId)).findStorageInfo
         (ds) >= 0);
-    assertTrue(((BlockInfoContiguousUnderConstruction) bm.
+    assertTrue(((BlockInfoUnderConstruction) bm.
         getStoredBlock(new Block(receivingBlockId))).getNumExpectedLocations() > 0);
     assertTrue(bm.getStoredBlock(new Block(receivingReceivedBlockId))
         .findStorageInfo(ds) >= 0);
@@ -747,8 +747,8 @@ public class TestBlockManager {
 
   private BlockInfo addUcBlockToBM(long blkId) {
     Block block = new Block(blkId);
-    BlockInfoContiguousUnderConstruction blockInfo =
-        new BlockInfoContiguousUnderConstruction(block, (short) 3);
+    BlockInfoUnderConstruction blockInfo =
+        new BlockInfoUnderConstructionContiguous(block, (short) 3);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
     Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication();
     bm.blocksMap.addBlockCollection(blockInfo, bc);

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java

@@ -39,7 +39,6 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
-import org.apache.hadoop.util.Time;
 import org.junit.Test;
 
 /**
@@ -173,7 +172,8 @@ public class TestHeartbeatHandling {
               dd1.getStorageInfos()[0],
               dd2.getStorageInfos()[0],
               dd3.getStorageInfos()[0]};
-          BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
+          BlockInfoUnderConstruction blockInfo =
+              new BlockInfoUnderConstructionContiguous(
               new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
               BlockUCState.UNDER_RECOVERY, storages);
           dd1.addBlockToBeRecovered(blockInfo);
@@ -195,7 +195,7 @@ public class TestHeartbeatHandling {
           // More than the default stale interval of 30 seconds.
           DFSTestUtil.resetLastUpdatesWithOffset(dd2, -40 * 1000);
           DFSTestUtil.resetLastUpdatesWithOffset(dd3, 0);
-          blockInfo = new BlockInfoContiguousUnderConstruction(
+          blockInfo = new BlockInfoUnderConstructionContiguous(
               new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
               BlockUCState.UNDER_RECOVERY, storages);
           dd1.addBlockToBeRecovered(blockInfo);
@@ -216,7 +216,7 @@ public class TestHeartbeatHandling {
           // More than the default stale interval of 30 seconds.
           DFSTestUtil.resetLastUpdatesWithOffset(dd2, - 40 * 1000);
           DFSTestUtil.resetLastUpdatesWithOffset(dd3, - 80 * 1000);
-          blockInfo = new BlockInfoContiguousUnderConstruction(
+          blockInfo = new BlockInfoUnderConstructionContiguous(
               new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
               BlockUCState.UNDER_RECOVERY, storages);
           dd1.addBlockToBeRecovered(blockInfo);

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java

@@ -1176,7 +1176,8 @@ public class TestReplicationPolicy {
     // block under construction, the BlockManager will realize the expected
     // replication has been achieved and remove it from the under-replicated
     // queue.
-    BlockInfoContiguousUnderConstruction info = new BlockInfoContiguousUnderConstruction(block1, (short) 1);
+    BlockInfoUnderConstruction info =
+        new BlockInfoUnderConstructionContiguous(block1, (short) 1);
     BlockCollection bc = mock(BlockCollection.class);
     when(bc.getPreferredBlockReplication()).thenReturn((short)1);
     bm.addBlockCollection(info, bc);
@@ -1232,7 +1233,7 @@ public class TestReplicationPolicy {
 
     DatanodeStorageInfo[] storageAry = {new DatanodeStorageInfo(
         dataNodes[0], new DatanodeStorage("s1"))};
-    final BlockInfoContiguousUnderConstruction ucBlock =
+    final BlockInfoUnderConstruction ucBlock =
         info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
             storageAry);
     DatanodeStorageInfo storage = mock(DatanodeStorageInfo.class);

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

@@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.junit.AfterClass;
@@ -170,7 +170,7 @@ public class TestBlockUnderConstruction {
       final List<LocatedBlock> blocks = lb.getLocatedBlocks();
       assertEquals(i, blocks.size());
       final Block b = blocks.get(blocks.size() - 1).getBlock().getLocalBlock();
-      assertTrue(b instanceof BlockInfoContiguousUnderConstruction);
+      assertTrue(b instanceof BlockInfoUnderConstruction);
 
       if (++i < NUM_BLOCKS) {
         // write one more block

+ 6 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java

@@ -24,7 +24,8 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.junit.Test;
@@ -68,8 +69,10 @@ public class TestCommitBlockSynchronization {
     namesystem.dir.getINodeMap().put(file);
 
     FSNamesystem namesystemSpy = spy(namesystem);
-    BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
-        block, (short) 1, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
+    BlockInfoUnderConstruction blockInfo =
+        new BlockInfoUnderConstructionContiguous(
+        block, (short) 1, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
+            targets);
     blockInfo.setBlockCollection(file);
     blockInfo.setGenerationStamp(genStamp);
     blockInfo.initializeBlockRecovery(genStamp);

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java

@@ -54,7 +54,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -1019,7 +1019,7 @@ public class TestFileTruncate {
           is(fsn.getBlockIdManager().getGenerationStampV2()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
-      long blockRecoveryId = ((BlockInfoContiguousUnderConstruction) file.getLastBlock())
+      long blockRecoveryId = ((BlockInfoUnderConstruction) file.getLastBlock())
           .getBlockRecoveryId();
       assertThat(blockRecoveryId, is(initialGenStamp + 1));
       fsn.getEditLog().logTruncate(
@@ -1052,7 +1052,7 @@ public class TestFileTruncate {
           is(fsn.getBlockIdManager().getGenerationStampV2()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
-      long blockRecoveryId = ((BlockInfoContiguousUnderConstruction) file.getLastBlock())
+      long blockRecoveryId = ((BlockInfoUnderConstruction) file.getLastBlock())
           .getBlockRecoveryId();
       assertThat(blockRecoveryId, is(initialGenStamp + 1));
       fsn.getEditLog().logTruncate(

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java

@@ -72,7 +72,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
@@ -752,8 +752,8 @@ public class TestRetryCacheWithHA {
     boolean checkNamenodeBeforeReturn() throws Exception {
       INodeFile fileNode = cluster.getNamesystem(0).getFSDirectory()
           .getINode4Write(file).asFile();
-      BlockInfoContiguousUnderConstruction blkUC =
-          (BlockInfoContiguousUnderConstruction) (fileNode.getBlocks())[1];
+      BlockInfoUnderConstruction blkUC =
+          (BlockInfoUnderConstruction) (fileNode.getBlocks())[1];
       int datanodeNum = blkUC.getExpectedStorageLocations().length;
       for (int i = 0; i < CHECKTIMES && datanodeNum != 2; i++) {
         Thread.sleep(1000);

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

@@ -44,7 +44,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage;
 import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
@@ -177,7 +177,7 @@ public class SnapshotTestHelper {
    * Specific information for different types of INode: 
    * {@link INodeDirectory}:childrenSize 
    * {@link INodeFile}: fileSize, block list. Check {@link BlockInfo#toString()}
-   * and {@link BlockInfoContiguousUnderConstruction#toString()} for detailed information.
+   * and {@link BlockInfoUnderConstruction#toString()} for detailed information.
    * {@link FileWithSnapshot}: next link
    * </pre>
    * @see INode#dumpTreeRecursively()