Browse Source

HDFS-8909. Erasure coding: update BlockInfoContiguousUC and BlockInfoStripedUC to use BlockUnderConstructionFeature. Contributed by Jing Zhao.

Walter Su 9 years ago
parent
commit
164cbe6439
31 changed files with 363 additions and 764 deletions
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
  2. 85 20
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  3. 0 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
  4. 0 281
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
  5. 0 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
  6. 0 84
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
  7. 37 59
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  8. 55 83
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
  9. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  10. 13 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  11. 21 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
  12. 16 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
  13. 9 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  14. 4 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  15. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
  16. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  17. 18 22
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  18. 12 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  19. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
  20. 3 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
  21. 8 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
  22. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
  23. 12 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
  24. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
  25. 17 21
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
  26. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
  27. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
  28. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
  29. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
  30. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
  31. 2 4
      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-HDFS-EC-7285.txt

@@ -403,3 +403,6 @@
 
     HDFS-8838. Erasure Coding: Tolerate datanode failures in DFSStripedOutputStream
     when the data length is small. (szetszwo via waltersu4549)
+
+    HDFS-8909. Erasure coding: update BlockInfoContiguousUC and BlockInfoStripedUC
+    to use BlockUnderConstructionFeature. (Jing Zhao via waltersu4549)

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

@@ -17,8 +17,10 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import java.io.IOException;
 import java.util.LinkedList;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.util.LightWeightGSet;
@@ -52,6 +54,8 @@ public abstract class BlockInfo extends Block
    */
   protected Object[] triplets;
 
+  private BlockUnderConstructionFeature uc;
+
   /**
    * Construct an entry for blocksmap
    * @param size the block's replication factor, or the total number of blocks
@@ -287,26 +291,6 @@ public abstract class BlockInfo extends Block
     return this;
   }
 
-  /**
-   * BlockInfo represents a block that is not being constructed.
-   * In order to start modifying the block, the BlockInfo should be converted to
-   * {@link BlockInfoContiguousUnderConstruction} or
-   * {@link BlockInfoStripedUnderConstruction}.
-   * @return {@link BlockUCState#COMPLETE}
-   */
-  public BlockUCState getBlockUCState() {
-    return BlockUCState.COMPLETE;
-  }
-
-  /**
-   * Is this block complete?
-   *
-   * @return true if the state of the block is {@link BlockUCState#COMPLETE}
-   */
-  public boolean isComplete() {
-    return getBlockUCState().equals(BlockUCState.COMPLETE);
-  }
-
   public boolean isDeleted() {
     return (bc == null);
   }
@@ -332,4 +316,85 @@ public abstract class BlockInfo extends Block
   public void setNext(LightWeightGSet.LinkedElement next) {
     this.nextLinkedElement = next;
   }
+
+  /* UnderConstruction Feature related */
+
+  public BlockUnderConstructionFeature getUnderConstructionFeature() {
+    return uc;
+  }
+
+  public BlockUCState getBlockUCState() {
+    return uc == null ? BlockUCState.COMPLETE : uc.getBlockUCState();
+  }
+
+  /**
+   * Is this block complete?
+   *
+   * @return true if the state of the block is {@link BlockUCState#COMPLETE}
+   */
+  public boolean isComplete() {
+    return getBlockUCState().equals(BlockUCState.COMPLETE);
+  }
+
+  /**
+   * Add/Update the under construction feature.
+   */
+  public void convertToBlockUnderConstruction(BlockUCState s,
+      DatanodeStorageInfo[] targets) {
+    if (isComplete()) {
+      uc = new BlockUnderConstructionFeature(this, s, targets, this.isStriped());
+    } else {
+      // the block is already under construction
+      uc.setBlockUCState(s);
+      uc.setExpectedLocations(this, targets, this.isStriped());
+    }
+  }
+
+  /**
+   * 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";
+    uc = null;
+    return this;
+  }
+
+  /**
+   * Process the recorded replicas. When about to commit or finish the
+   * pipeline recovery sort out bad replicas.
+   * @param genStamp  The final generation stamp for the block.
+   */
+  public void setGenerationStampAndVerifyReplicas(long genStamp) {
+    Preconditions.checkState(uc != null && !isComplete());
+    // Set the generation stamp for the block.
+    setGenerationStamp(genStamp);
+
+    // Remove the replicas with wrong gen stamp
+    uc.removeStaleReplicas(this);
+  }
+
+  /**
+   * 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
+   * @throws IOException if block ids are inconsistent.
+   */
+  void commitBlock(Block block) throws IOException {
+    if (getBlockId() != block.getBlockId()) {
+      throw new IOException("Trying to commit inconsistent block: id = "
+          + block.getBlockId() + ", expected id = " + getBlockId());
+    }
+    Preconditions.checkState(!isComplete());
+    uc.commit();
+    this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp());
+    // Sort out invalid replicas.
+    setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
+  }
 }

+ 0 - 23
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java

@@ -19,7 +19,6 @@ 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.BlockUCState;
 
 /**
  * Subclass of {@link BlockInfo}, used for a block with replication scheme.
@@ -123,28 +122,6 @@ public class BlockInfoContiguous extends BlockInfo {
     }
   }
 
-  /**
-   * Convert a complete block to an under construction block.
-   * @return BlockInfoUnderConstruction -  an under construction block.
-   */
-  public BlockInfoContiguousUnderConstruction convertToBlockUnderConstruction(
-      BlockUCState s, DatanodeStorageInfo[] targets) {
-    if(isComplete()) {
-      BlockInfoContiguousUnderConstruction ucBlock =
-          new BlockInfoContiguousUnderConstruction(this,
-          getBlockCollection().getPreferredBlockReplication(), s, targets);
-      ucBlock.setBlockCollection(getBlockCollection());
-      return ucBlock;
-    }
-    // the block is already under construction
-    BlockInfoContiguousUnderConstruction ucBlock =
-        (BlockInfoContiguousUnderConstruction) this;
-    ucBlock.setBlockUCState(s);
-    ucBlock.setExpectedLocations(targets);
-    ucBlock.setBlockCollection(getBlockCollection());
-    return ucBlock;
-  }
-
   @Override
   public final boolean isStriped() {
     return false;

+ 0 - 281
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java

@@ -1,281 +0,0 @@
-/**
- * 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 java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.hadoop.hdfs.protocol.Block;
-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;
-
-/**
- * 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
-    implements BlockInfoUnderConstruction{
-  /** Block state. See {@link BlockUCState} */
-  private BlockUCState blockUCState;
-
-  /**
-   * Block replicas as assigned when the block was allocated.
-   * This defines the pipeline order.
-   */
-  private List<ReplicaUnderConstruction> replicas;
-
-  /**
-   * Index of the primary data node doing the recovery. Useful for log
-   * messages.
-   */
-  private int primaryNodeIndex = -1;
-
-  /**
-   * The new generation stamp, which this block will have
-   * after the recovery succeeds. Also used as a recovery id to identify
-   * the right recovery if any of the abandoned recoveries re-appear.
-   */
-  private long blockRecoveryId = 0;
-
-  /**
-   * The block source to use in the event of copy-on-write truncate.
-   */
-  private Block truncateBlock;
-
-  /**
-   * Create block and set its state to
-   * {@link BlockUCState#UNDER_CONSTRUCTION}.
-   */
-  public BlockInfoContiguousUnderConstruction(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,
-      BlockUCState state, DatanodeStorageInfo[] targets) {
-    super(blk, replication);
-    assert getBlockUCState() != BlockUCState.COMPLETE :
-      "BlockInfoContiguousUnderConstruction cannot be in COMPLETE state";
-    this.blockUCState = state;
-    setExpectedLocations(targets);
-  }
-
-  @Override
-  public BlockInfoContiguous convertToCompleteBlock() throws IOException {
-    assert getBlockUCState() != BlockUCState.COMPLETE :
-      "Trying to convert a COMPLETE block";
-    return new BlockInfoContiguous(this);
-  }
-
-  @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],
-          ReplicaState.RBW));
-    }
-  }
-
-  @Override
-  public DatanodeStorageInfo[] getExpectedStorageLocations() {
-    int numLocations = replicas == null ? 0 : replicas.size();
-    DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
-    for (int i = 0; i < numLocations; i++) {
-      storages[i] = replicas.get(i).getExpectedStorageLocation();
-    }
-    return storages;
-  }
-
-  @Override
-  public int getNumExpectedLocations() {
-    return replicas == null ? 0 : replicas.size();
-  }
-
-  /**
-   * Return the state of the block under construction.
-   * @see BlockUCState
-   */
-  @Override // BlockInfo
-  public BlockUCState getBlockUCState() {
-    return blockUCState;
-  }
-
-  void setBlockUCState(BlockUCState s) {
-    blockUCState = s;
-  }
-
-  @Override
-  public long getBlockRecoveryId() {
-    return blockRecoveryId;
-  }
-
-  @Override
-  public Block getTruncateBlock() {
-    return truncateBlock;
-  }
-
-  @Override
-  public Block toBlock(){
-    return this;
-  }
-
-  public void setTruncateBlock(Block recoveryBlock) {
-    this.truncateBlock = recoveryBlock;
-  }
-
-  @Override
-  public void setGenerationStampAndVerifyReplicas(long genStamp) {
-    // Set the generation stamp for the block.
-    setGenerationStamp(genStamp);
-    if (replicas == null)
-      return;
-
-    // Remove the replicas with wrong gen stamp.
-    // The replica list is unchanged.
-    for (ReplicaUnderConstruction r : replicas) {
-      if (genStamp != r.getGenerationStamp()) {
-        r.getExpectedStorageLocation().removeBlock(this);
-        NameNode.blockStateChangeLog.debug("BLOCK* Removing stale replica "
-            + "from location: {}", r.getExpectedStorageLocation());
-      }
-    }
-  }
-
-  @Override
-  public void commitBlock(Block block) throws IOException {
-    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.
-    setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
-  }
-
-  @Override
-  public void initializeBlockRecovery(long recoveryId) {
-    setBlockUCState(BlockUCState.UNDER_RECOVERY);
-    blockRecoveryId = recoveryId;
-    if (replicas.size() == 0) {
-      NameNode.blockStateChangeLog.warn("BLOCK*"
-        + " BlockInfoContiguousUnderConstruction.initLeaseRecovery:"
-        + " No blocks found, lease removed.");
-    }
-    boolean allLiveReplicasTriedAsPrimary = true;
-    for (ReplicaUnderConstruction replica : replicas) {
-      // Check if all replicas have been tried or not.
-      if (replica.isAlive()) {
-        allLiveReplicasTriedAsPrimary = (allLiveReplicasTriedAsPrimary &&
-            replica.getChosenAsPrimary());
-      }
-    }
-    if (allLiveReplicasTriedAsPrimary) {
-      // Just set all the replicas to be chosen whether they are alive or not.
-      for (ReplicaUnderConstruction replica : replicas) {
-        replica.setChosenAsPrimary(false);
-      }
-    }
-    long mostRecentLastUpdate = 0;
-    ReplicaUnderConstruction primary = null;
-    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())) {
-        continue;
-      }
-      final ReplicaUnderConstruction ruc = replicas.get(i);
-      final long lastUpdate = ruc.getExpectedStorageLocation()
-          .getDatanodeDescriptor().getLastUpdateMonotonic();
-      if (lastUpdate > mostRecentLastUpdate) {
-        primaryNodeIndex = i;
-        primary = ruc;
-        mostRecentLastUpdate = lastUpdate;
-      }
-    }
-    if (primary != null) {
-      primary.getExpectedStorageLocation().getDatanodeDescriptor()
-          .addBlockToBeRecovered(this);
-      primary.setChosenAsPrimary(true);
-      NameNode.blockStateChangeLog.debug(
-          "BLOCK* {} recovery started, primary={}", this, primary);
-    }
-  }
-
-  @Override
-  public void addReplicaIfNotPresent(DatanodeStorageInfo storage,
-      Block block, ReplicaState rState) {
-    Iterator<ReplicaUnderConstruction> it = replicas.iterator();
-    while (it.hasNext()) {
-      ReplicaUnderConstruction r = it.next();
-      DatanodeStorageInfo expectedLocation = r.getExpectedStorageLocation();
-      if(expectedLocation == storage) {
-        // Record the gen stamp from the report
-        r.setGenerationStamp(block.getGenerationStamp());
-        return;
-      } else if (expectedLocation != null &&
-                 expectedLocation.getDatanodeDescriptor() ==
-                     storage.getDatanodeDescriptor()) {
-
-        // The Datanode reported that the block is on a different storage
-        // than the one chosen by BlockPlacementPolicy. This can occur as
-        // we allow Datanodes to choose the target storage. Update our
-        // state by removing the stale entry and adding a new one.
-        it.remove();
-        break;
-      }
-    }
-    replicas.add(new ReplicaUnderConstruction(block, storage, rState));
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder b = new StringBuilder(100);
-    appendStringTo(b);
-    return b.toString();
-  }
-
-  @Override
-  public void appendStringTo(StringBuilder sb) {
-    super.appendStringTo(sb);
-    appendUCParts(sb);
-  }
-
-  private void appendUCParts(StringBuilder sb) {
-    sb.append("{UCState=").append(blockUCState)
-      .append(", truncateBlock=" + truncateBlock)
-      .append(", primaryNodeIndex=").append(primaryNodeIndex)
-      .append(", replicas=[");
-    if (replicas != null) {
-      Iterator<ReplicaUnderConstruction> iter = replicas.iterator();
-      if (iter.hasNext()) {
-        iter.next().appendStringTo(sb);
-        while (iter.hasNext()) {
-          sb.append(", ");
-          iter.next().appendStringTo(sb);
-        }
-      }
-    }
-    sb.append("]}");
-  }
-}

+ 0 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java

@@ -245,27 +245,6 @@ public class BlockInfoStriped extends BlockInfo {
     return num;
   }
 
-  /**
-   * Convert a complete block to an under construction block.
-   * @return BlockInfoUnderConstruction -  an under construction block.
-   */
-  public BlockInfoStripedUnderConstruction convertToBlockUnderConstruction(
-      BlockUCState s, DatanodeStorageInfo[] targets) {
-    final BlockInfoStripedUnderConstruction ucBlock;
-    if(isComplete()) {
-      ucBlock = new BlockInfoStripedUnderConstruction(this, ecPolicy,
-          s, targets);
-      ucBlock.setBlockCollection(getBlockCollection());
-    } else {
-      // the block is already under construction
-      ucBlock = (BlockInfoStripedUnderConstruction) this;
-      ucBlock.setBlockUCState(s);
-      ucBlock.setExpectedLocations(targets);
-      ucBlock.setBlockCollection(getBlockCollection());
-    }
-    return ucBlock;
-  }
-
   @Override
   final boolean hasNoStorage() {
     final int len = getCapacity();

+ 0 - 84
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java

@@ -1,84 +0,0 @@
-/**
- * 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 java.io.IOException;
-
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
-
-public interface BlockInfoUnderConstruction {
-  /**
-   * Create array of expected replica locations
-   * (as has been assigned by chooseTargets()).
-   */
-  public DatanodeStorageInfo[] getExpectedStorageLocations();
-
-  /** Get recover block */
-  public Block getTruncateBlock();
-
-  /** Convert to a Block object */
-  public Block toBlock();
-
-  /** Get block recovery ID */
-  public long getBlockRecoveryId();
-
-  /** Get the number of expected locations */
-  public int getNumExpectedLocations();
-
-  /** Set expected locations */
-  public void setExpectedLocations(DatanodeStorageInfo[] targets);
-
-  /**
-   * Process the recorded replicas. When about to commit or finish the
-   * pipeline recovery sort out bad replicas.
-   * @param genStamp  The final generation stamp for the block.
-   */
-  public void setGenerationStampAndVerifyReplicas(long genStamp);
-
-  /**
-   * Initialize lease recovery for this block.
-   * Find the first alive data-node starting from the previous primary and
-   * make it primary.
-   */
-  public void initializeBlockRecovery(long recoveryId);
-  
-  /** Add the reported replica if it is not already in the replica list. */
-  public void addReplicaIfNotPresent(DatanodeStorageInfo storage,
-      Block reportedBlock, ReplicaState rState);
-
-  /**
-   * 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 
-   * @throws IOException if block ids are inconsistent.
-   */
-  public void commitBlock(Block block) throws IOException;
-
-  /**
-   * 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 BlockInfo convertToCompleteBlock() throws IOException;
-}

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

@@ -644,19 +644,13 @@ public class BlockManager implements BlockStatsMXBean {
    */
   private static boolean commitBlock(final BlockInfo block,
       final Block commitBlock) throws IOException {
-    if (block instanceof BlockInfoUnderConstruction
-        && block.getBlockUCState() != BlockUCState.COMMITTED) {
-      final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)block;
-
-      assert block.getNumBytes() <= commitBlock.getNumBytes() :
+    if (block.getBlockUCState() == BlockUCState.COMMITTED)
+      return false;
+    assert block.getNumBytes() <= commitBlock.getNumBytes() :
         "commitBlock length is less than the stored one "
-        + commitBlock.getNumBytes() + " vs. " + block.getNumBytes();
-
-      uc.commitBlock(commitBlock);
-      return true;
-    }
-
-    return false;
+            + commitBlock.getNumBytes() + " vs. " + block.getNumBytes();
+    block.commitBlock(commitBlock);
+    return true;
   }
   
   /**
@@ -713,9 +707,7 @@ public class BlockManager implements BlockStatsMXBean {
           "Cannot complete block: block has not been COMMITTED by the client");
     }
 
-    final BlockInfo completeBlock
-        = !(curBlock instanceof BlockInfoUnderConstruction)? curBlock
-            : ((BlockInfoUnderConstruction)curBlock).convertToCompleteBlock();
+    final BlockInfo completeBlock = curBlock.convertToCompleteBlock();
 
     // replace penultimate block in file
     bc.setBlock(blkIndex, completeBlock);
@@ -754,9 +746,7 @@ public class BlockManager implements BlockStatsMXBean {
    */
   public BlockInfo forceCompleteBlock(final BlockCollection bc,
       final BlockInfo block) throws IOException {
-    if (block instanceof BlockInfoUnderConstruction) {
-      ((BlockInfoUnderConstruction)block).commitBlock(block);
-    }
+    block.commitBlock(block);
     return completeBlock(bc, block, true);
   }
 
@@ -777,30 +767,28 @@ public class BlockManager implements BlockStatsMXBean {
    */
   public LocatedBlock convertLastBlockToUnderConstruction(
       BlockCollection bc, long bytesToRemove) throws IOException {
-    BlockInfo oldBlock = bc.getLastBlock();
-    if(oldBlock == null ||
-       bc.getPreferredBlockSize() == oldBlock.getNumBytes() - bytesToRemove)
+    BlockInfo lastBlock = bc.getLastBlock();
+    if(lastBlock == null ||
+       bc.getPreferredBlockSize() == lastBlock.getNumBytes() - bytesToRemove)
       return null;
-    assert oldBlock == getStoredBlock(oldBlock) :
+    assert lastBlock == getStoredBlock(lastBlock) :
       "last block of the file is not in blocksMap";
 
-    DatanodeStorageInfo[] targets = getStorages(oldBlock);
+    DatanodeStorageInfo[] targets = getStorages(lastBlock);
 
-    // convert the last block to UC
-    bc.convertLastBlockToUC(oldBlock, targets);
-    // get the new created uc block
-    BlockInfo ucBlock = bc.getLastBlock();
-    blocksMap.replaceBlock(ucBlock);
+    // convert the last block to under construction. note no block replacement
+    // is happening
+    bc.convertLastBlockToUC(lastBlock, targets);
 
     // Remove block from replication queue.
-    NumberReplicas replicas = countNodes(ucBlock);
-    neededReplications.remove(ucBlock, replicas.liveReplicas(),
-        replicas.decommissionedAndDecommissioning(), getReplication(ucBlock));
-    pendingReplications.remove(ucBlock);
+    NumberReplicas replicas = countNodes(lastBlock);
+    neededReplications.remove(lastBlock, replicas.liveReplicas(),
+        replicas.decommissionedAndDecommissioning(), getReplication(lastBlock));
+    pendingReplications.remove(lastBlock);
 
     // remove this block from the list of pending blocks to be deleted. 
     for (DatanodeStorageInfo storage : targets) {
-      final Block b = getBlockOnStorage(oldBlock, storage);
+      final Block b = getBlockOnStorage(lastBlock, storage);
       if (b != null) {
         invalidateBlocks.remove(storage.getDatanodeDescriptor(), b);
       }
@@ -810,13 +798,15 @@ public class BlockManager implements BlockStatsMXBean {
     // count in safe-mode.
     namesystem.adjustSafeModeBlockTotals(
         // decrement safe if we had enough
-        hasMinStorage(oldBlock, targets.length) ? -1 : 0,
+        hasMinStorage(lastBlock, targets.length) ? -1 : 0,
         // always decrement total blocks
         -1);
 
-    final long fileLength = bc.computeContentSummary(getStoragePolicySuite()).getLength();
-    final long pos = fileLength - ucBlock.getNumBytes();
-    return createLocatedBlock(ucBlock, pos, BlockTokenIdentifier.AccessMode.WRITE);
+    final long fileLength = bc.computeContentSummary(
+        getStoragePolicySuite()).getLength();
+    final long pos = fileLength - lastBlock.getNumBytes();
+    return createLocatedBlock(lastBlock, pos,
+        BlockTokenIdentifier.AccessMode.WRITE);
   }
 
   /**
@@ -895,18 +885,14 @@ public class BlockManager implements BlockStatsMXBean {
   private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos)
       throws IOException {
     if (!blk.isComplete()) {
+      final BlockUnderConstructionFeature uc = blk.getUnderConstructionFeature();
       if (blk.isStriped()) {
-        final BlockInfoStripedUnderConstruction uc =
-            (BlockInfoStripedUnderConstruction) blk;
         final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
         final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
             blk);
         return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
             false);
       } else {
-        assert blk instanceof BlockInfoContiguousUnderConstruction;
-        final BlockInfoContiguousUnderConstruction uc =
-            (BlockInfoContiguousUnderConstruction) blk;
         final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
         final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
             blk);
@@ -1923,9 +1909,7 @@ public class BlockManager implements BlockStatsMXBean {
     
     StatefulBlockInfo(BlockInfo storedBlock,
         Block reportedBlock, ReplicaState reportedState) {
-      Preconditions.checkArgument(
-          storedBlock instanceof BlockInfoContiguousUnderConstruction ||
-          storedBlock instanceof BlockInfoStripedUnderConstruction);
+      Preconditions.checkArgument(!storedBlock.isComplete());
       this.storedBlock = storedBlock;
       this.reportedBlock = reportedBlock;
       this.reportedState = reportedState;
@@ -2335,13 +2319,14 @@ public class BlockManager implements BlockStatsMXBean {
       
       // If block is under construction, add this replica to its list
       if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
-        final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)storedBlock;
-        uc.addReplicaIfNotPresent(storageInfo, iblk, reportedState);
+        storedBlock.getUnderConstructionFeature()
+            .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
         if (namesystem.isInSnapshot(storedBlock.getBlockCollection())) {
-          int numOfReplicas = uc.getNumExpectedLocations();
+          int numOfReplicas = storedBlock.getUnderConstructionFeature()
+              .getNumExpectedLocations();
           namesystem.incrementSafeBlockCount(numOfReplicas, storedBlock);
         }
         //and fall through to next clause
@@ -2469,11 +2454,6 @@ public class BlockManager implements BlockStatsMXBean {
 
     // Ignore replicas already scheduled to be removed from the DN
     if(invalidateBlocks.contains(dn, block)) {
-      /*
-       * TODO: following assertion is incorrect, see HDFS-2668 assert
-       * storedBlock.findDatanode(dn) < 0 : "Block " + block +
-       * " in recentInvalidatesSet should not appear in DN " + dn;
-       */
       return storedBlock;
     }
 
@@ -2704,9 +2684,8 @@ public class BlockManager implements BlockStatsMXBean {
   void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
       DatanodeStorageInfo storageInfo) throws IOException {
     BlockInfo block = ucBlock.storedBlock;
-    final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)block;
-    uc.addReplicaIfNotPresent(storageInfo, ucBlock.reportedBlock,
-        ucBlock.reportedState);
+    block.getUnderConstructionFeature().addReplicaIfNotPresent(storageInfo,
+        ucBlock.reportedBlock, ucBlock.reportedState);
 
     if (ucBlock.reportedState == ReplicaState.FINALIZED &&
         (block.findStorageInfo(storageInfo) < 0)) {
@@ -2766,8 +2745,7 @@ public class BlockManager implements BlockStatsMXBean {
     assert block != null && namesystem.hasWriteLock();
     BlockInfo storedBlock;
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
-    if (block instanceof BlockInfoContiguousUnderConstruction ||
-        block instanceof BlockInfoStripedUnderConstruction) {
+    if (!block.isComplete()) {
       //refresh our copy in case the block got completed in another thread
       storedBlock = getStoredBlock(block);
     } else {
@@ -4221,7 +4199,7 @@ public class BlockManager implements BlockStatsMXBean {
     final LocatedBlock lb;
     if (info.isStriped()) {
       lb = newLocatedStripedBlock(eb, locs,
-          ((BlockInfoStripedUnderConstruction)info).getBlockIndices(),
+          info.getUnderConstructionFeature().getBlockIndices(),
           offset, false);
     } else {
       lb = newLocatedBlock(eb, locs, offset, false);

+ 55 - 83
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java

@@ -21,19 +21,14 @@ import org.apache.hadoop.hdfs.protocol.Block;
 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;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-
-import java.io.IOException;
 
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.COMPLETE;
-import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION;
 
 /**
- * Represents a striped block that is currently being constructed.
+ * Represents the under construction feature of a Block.
  * This is usually the last block of a file opened for write or append.
  */
-public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
-    implements BlockInfoUnderConstruction{
+public class BlockUnderConstructionFeature {
   private BlockUCState blockUCState;
 
   /**
@@ -55,41 +50,30 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
   private long blockRecoveryId = 0;
 
   /**
-   * Constructor with null storage targets.
+   * The block source to use in the event of copy-on-write truncate.
    */
-  public BlockInfoStripedUnderConstruction(Block blk, ErasureCodingPolicy ecPolicy) {
-    this(blk, ecPolicy, UNDER_CONSTRUCTION, null);
-  }
+  private Block truncateBlock;
 
-  /**
-   * Create a striped block that is currently being constructed.
-   */
-  public BlockInfoStripedUnderConstruction(Block blk, ErasureCodingPolicy ecPolicy,
-      BlockUCState state, DatanodeStorageInfo[] targets) {
-    super(blk, ecPolicy);
+  public BlockUnderConstructionFeature(Block blk,
+      BlockUCState state, DatanodeStorageInfo[] targets, boolean isStriped) {
     assert getBlockUCState() != COMPLETE :
-      "BlockInfoStripedUnderConstruction cannot be in COMPLETE state";
+      "BlockUnderConstructionFeature cannot be in COMPLETE state";
     this.blockUCState = state;
-    setExpectedLocations(targets);
-  }
-
-  @Override
-  public BlockInfoStriped convertToCompleteBlock() throws IOException {
-    assert getBlockUCState() != COMPLETE :
-      "Trying to convert a COMPLETE block";
-    return new BlockInfoStriped(this);
+    setExpectedLocations(blk, targets, isStriped);
   }
 
   /** Set expected locations */
-  @Override
-  public void setExpectedLocations(DatanodeStorageInfo[] targets) {
+  public void setExpectedLocations(Block block, DatanodeStorageInfo[] targets,
+      boolean isStriped) {
     int numLocations = targets == null ? 0 : targets.length;
     this.replicas = new ReplicaUnderConstruction[numLocations];
     for(int i = 0; i < numLocations; i++) {
-      // when creating a new block we simply sequentially assign block index to
-      // each storage
-      Block blk = new Block(this.getBlockId() + i, 0, this.getGenerationStamp());
-      replicas[i] = new ReplicaUnderConstruction(blk, targets[i],
+      // when creating a new striped block we simply sequentially assign block
+      // index to each storage
+      Block replicaBlock = isStriped ?
+          new Block(block.getBlockId() + i, 0, block.getGenerationStamp()) :
+          block;
+      replicas[i] = new ReplicaUnderConstruction(replicaBlock, targets[i],
           ReplicaState.RBW);
     }
   }
@@ -98,7 +82,6 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
    * Create array of expected replica locations
    * (as has been assigned by chooseTargets()).
    */
-  @Override
   public DatanodeStorageInfo[] getExpectedStorageLocations() {
     int numLocations = getNumExpectedLocations();
     DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
@@ -108,7 +91,10 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
     return storages;
   }
 
-  /** @return the index array indicating the block index in each storage */
+  /**
+   * @return the index array indicating the block index in each storage. Used
+   * only by striped blocks.
+   */
   public int[] getBlockIndices() {
     int numLocations = getNumExpectedLocations();
     int[] indices = new int[numLocations];
@@ -118,7 +104,6 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
     return indices;
   }
 
-  @Override
   public int getNumExpectedLocations() {
     return replicas == null ? 0 : replicas.length;
   }
@@ -127,7 +112,6 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
    * Return the state of the block under construction.
    * @see BlockUCState
    */
-  @Override // BlockInfo
   public BlockUCState getBlockUCState() {
     return blockUCState;
   }
@@ -136,58 +120,51 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
     blockUCState = s;
   }
 
-  @Override
   public long getBlockRecoveryId() {
     return blockRecoveryId;
   }
 
-  @Override
+  /** Get recover block */
   public Block getTruncateBlock() {
-    return null;
+    return truncateBlock;
   }
 
-  @Override
-  public Block toBlock(){
-    return this;
+  public void setTruncateBlock(Block recoveryBlock) {
+    this.truncateBlock = recoveryBlock;
   }
 
-  @Override
-  public void setGenerationStampAndVerifyReplicas(long genStamp) {
-    // Set the generation stamp for the block.
-    setGenerationStamp(genStamp);
-    if (replicas == null)
-      return;
-
-    // Remove the replicas with wrong gen stamp.
-    // The replica list is unchanged.
-    for (ReplicaUnderConstruction r : replicas) {
-      if (genStamp != r.getGenerationStamp()) {
-        r.getExpectedStorageLocation().removeBlock(this);
-        NameNode.blockStateChangeLog.info("BLOCK* Removing stale replica "
-            + "from location: {}", r.getExpectedStorageLocation());
-      }
-    }
+  /**
+   * Set {@link #blockUCState} to {@link BlockUCState#COMMITTED}.
+   */
+  void commit() {
+    blockUCState = BlockUCState.COMMITTED;
   }
 
-  @Override
-  public void commitBlock(Block block) throws IOException {
-    if (getBlockId() != block.getBlockId()) {
-      throw new IOException("Trying to commit inconsistent block: id = "
-          + block.getBlockId() + ", expected id = " + getBlockId());
+  void removeStaleReplicas(BlockInfo block) {
+    final long genStamp = block.getGenerationStamp();
+    if (replicas != null) {
+      // Remove replicas with wrong gen stamp. The replica list is unchanged.
+      for (ReplicaUnderConstruction r : replicas) {
+        if (genStamp != r.getGenerationStamp()) {
+          r.getExpectedStorageLocation().removeBlock(block);
+          NameNode.blockStateChangeLog.debug("BLOCK* Removing stale replica "
+              + "from location: {}", r.getExpectedStorageLocation());
+        }
+      }
     }
-    blockUCState = BlockUCState.COMMITTED;
-    this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp());
-    // Sort out invalid replicas.
-    setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
   }
 
-  @Override
-  public void initializeBlockRecovery(long recoveryId) {
+  /**
+   * Initialize lease recovery for this block.
+   * Find the first alive data-node starting from the previous primary and
+   * make it primary.
+   */
+  public void initializeBlockRecovery(BlockInfo blockInfo, long recoveryId) {
     setBlockUCState(BlockUCState.UNDER_RECOVERY);
     blockRecoveryId = recoveryId;
     if (replicas == null || replicas.length == 0) {
       NameNode.blockStateChangeLog.warn("BLOCK*" +
-          " BlockInfoStripedUnderConstruction.initLeaseRecovery:" +
+          " BlockUnderConstructionFeature.initLeaseRecovery:" +
           " No blocks found, lease removed.");
       // sets primary node index and return.
       primaryNodeIndex = -1;
@@ -226,15 +203,15 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
     }
     if (primary != null) {
       primary.getExpectedStorageLocation().getDatanodeDescriptor()
-          .addBlockToBeRecovered(this);
+          .addBlockToBeRecovered(blockInfo);
       primary.setChosenAsPrimary(true);
       NameNode.blockStateChangeLog.info(
           "BLOCK* {} recovery started, primary={}", this, primary);
     }
   }
 
-  @Override
-  public void addReplicaIfNotPresent(DatanodeStorageInfo storage,
+  /** Add the reported replica if it is not already in the replica list. */
+  void addReplicaIfNotPresent(DatanodeStorageInfo storage,
       Block reportedBlock, ReplicaState rState) {
     if (replicas == null) {
       replicas = new ReplicaUnderConstruction[1];
@@ -269,20 +246,15 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
   @Override
   public String toString() {
     final StringBuilder b = new StringBuilder(100);
-    appendStringTo(b);
+    appendUCParts(b);
     return b.toString();
   }
 
-  @Override
-  public void appendStringTo(StringBuilder sb) {
-    super.appendStringTo(sb);
-    appendUCParts(sb);
-  }
-
   private void appendUCParts(StringBuilder sb) {
-    sb.append("{UCState=").append(blockUCState).
-        append(", primaryNodeIndex=").append(primaryNodeIndex).
-        append(", replicas=[");
+    sb.append("{UCState=").append(blockUCState)
+      .append(", truncateBlock=").append(truncateBlock)
+      .append(", primaryNodeIndex=").append(primaryNodeIndex)
+      .append(", replicas=[");
     if (replicas != null) {
       int i = 0;
       for (ReplicaUnderConstruction r : replicas) {

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

@@ -227,7 +227,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private final BlockQueue<BlockECRecoveryInfo> erasurecodeBlocks =
       new BlockQueue<>();
   /** A queue of blocks to be recovered by this datanode */
-  private final BlockQueue<BlockInfoUnderConstruction> recoverBlocks =
+  private final BlockQueue<BlockInfo> recoverBlocks =
       new BlockQueue<>();
   /** A set of blocks to be invalidated by this datanode */
   private final LightWeightHashSet<Block> invalidateBlocks =
@@ -624,7 +624,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * Store block recovery work.
    */
-  void addBlockToBeRecovered(BlockInfoUnderConstruction block) {
+  void addBlockToBeRecovered(BlockInfo 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");
@@ -678,11 +678,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return erasurecodeBlocks.poll(maxTransfers);
   }
 
-  public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
-    List<BlockInfoUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
+  public BlockInfo[] getLeaseRecoveryCommand(int maxTransfers) {
+    List<BlockInfo> blocks = recoverBlocks.poll(maxTransfers);
     if(blocks == null)
       return null;
-    return blocks.toArray(new BlockInfoUnderConstruction[blocks.size()]);
+    return blocks.toArray(new BlockInfo[blocks.size()]);
   }
 
   /**

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

@@ -1381,13 +1381,15 @@ public class DatanodeManager {
         }
 
         //check lease recovery
-        BlockInfoUnderConstruction[] blocks = nodeinfo
-            .getLeaseRecoveryCommand(Integer.MAX_VALUE);
+        BlockInfo[] blocks = nodeinfo.getLeaseRecoveryCommand(Integer.MAX_VALUE);
         if (blocks != null) {
           BlockRecoveryCommand brCommand = new BlockRecoveryCommand(
               blocks.length);
-          for (BlockInfoUnderConstruction b : blocks) {
-            final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations();
+          for (BlockInfo b : blocks) {
+            final BlockUnderConstructionFeature uc =
+                b.getUnderConstructionFeature();
+            assert uc != null;
+            final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
             // Skip stale nodes during recovery - not heart beated for some time (30s by default).
             final List<DatanodeStorageInfo> recoveryLocations =
                 new ArrayList<>(storages.length);
@@ -1398,12 +1400,12 @@ public class DatanodeManager {
             }
             // If we are performing a truncate recovery than set recovery fields
             // to old block.
-            boolean truncateRecovery = b.getTruncateBlock() != null;
+            boolean truncateRecovery = uc.getTruncateBlock() != null;
             boolean copyOnTruncateRecovery = truncateRecovery &&
-                b.getTruncateBlock().getBlockId() != b.toBlock().getBlockId();
+                uc.getTruncateBlock().getBlockId() != b.getBlockId();
             ExtendedBlock primaryBlock = (copyOnTruncateRecovery) ?
-                new ExtendedBlock(blockPoolId, b.getTruncateBlock()) :
-                new ExtendedBlock(blockPoolId, b.toBlock());
+                new ExtendedBlock(blockPoolId, uc.getTruncateBlock()) :
+                new ExtendedBlock(blockPoolId, b);
             // If we only get 1 replica after eliminating stale nodes, then choose all
             // replicas for recovery and let the primary data node handle failures.
             DatanodeInfo[] recoveryInfos;
@@ -1420,13 +1422,13 @@ public class DatanodeManager {
               recoveryInfos = DatanodeStorageInfo.toDatanodeInfos(storages);
             }
             if(truncateRecovery) {
-              Block recoveryBlock = (copyOnTruncateRecovery) ? b.toBlock() :
-                  b.getTruncateBlock();
+              Block recoveryBlock = (copyOnTruncateRecovery) ? b :
+                  uc.getTruncateBlock();
               brCommand.add(new RecoveringBlock(primaryBlock, recoveryInfos,
                                                 recoveryBlock));
             } else {
               brCommand.add(new RecoveringBlock(primaryBlock, recoveryInfos,
-                                                b.getBlockRecoveryId()));
+                                                uc.getBlockRecoveryId()));
             }
           }
           return new DatanodeCommand[] { brCommand };

+ 21 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java

@@ -28,8 +28,9 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.RecoverLeaseOp;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@@ -102,7 +103,7 @@ final class FSDirTruncateOp {
       final BlockInfo last = file.getLastBlock();
       if (last != null && last.getBlockUCState()
           == BlockUCState.UNDER_RECOVERY) {
-        final Block truncatedBlock = ((BlockInfoContiguousUnderConstruction) last)
+        final Block truncatedBlock = last.getUnderConstructionFeature()
             .getTruncateBlock();
         if (truncatedBlock != null) {
           final long truncateLength = file.computeFileSize(false, false)
@@ -231,43 +232,42 @@ final class FSDirTruncateOp {
               oldBlock)));
     }
 
-    BlockInfoContiguousUnderConstruction truncatedBlockUC;
+    final BlockInfo truncatedBlockUC;
     BlockManager blockManager = fsn.getFSDirectory().getBlockManager();
     if (shouldCopyOnTruncate) {
       // Add new truncateBlock into blocksMap and
       // use oldBlock as a source for copy-on-truncate recovery
-      truncatedBlockUC = new BlockInfoContiguousUnderConstruction(newBlock,
+      truncatedBlockUC = new BlockInfoContiguous(newBlock,
           file.getPreferredBlockReplication());
+      truncatedBlockUC.convertToBlockUnderConstruction(
+          BlockUCState.UNDER_CONSTRUCTION, blockManager.getStorages(oldBlock));
       truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta);
-      truncatedBlockUC.setTruncateBlock(oldBlock);
-      file.convertLastBlockToUC(truncatedBlockUC,
-          blockManager.getStorages(oldBlock));
+      truncatedBlockUC.getUnderConstructionFeature().setTruncateBlock(oldBlock);
+      file.setLastBlock(truncatedBlockUC);
       blockManager.addBlockCollection(truncatedBlockUC, file);
 
       NameNode.stateChangeLog.debug(
           "BLOCK* prepareFileForTruncate: Scheduling copy-on-truncate to new"
               + " size {}  new block {} old block {}",
-          truncatedBlockUC.getNumBytes(), newBlock,
-          truncatedBlockUC.getTruncateBlock());
+          truncatedBlockUC.getNumBytes(), newBlock, oldBlock);
     } else {
       // Use new generation stamp for in-place truncate recovery
       blockManager.convertLastBlockToUnderConstruction(file, lastBlockDelta);
       oldBlock = file.getLastBlock();
       assert !oldBlock.isComplete() : "oldBlock should be under construction";
-      truncatedBlockUC = (BlockInfoContiguousUnderConstruction) oldBlock;
-      truncatedBlockUC.setTruncateBlock(new Block(oldBlock));
-      truncatedBlockUC.getTruncateBlock().setNumBytes(
-          oldBlock.getNumBytes() - lastBlockDelta);
-      truncatedBlockUC.getTruncateBlock().setGenerationStamp(
-          newBlock.getGenerationStamp());
-
-      NameNode.stateChangeLog.debug(
-          "BLOCK* prepareFileForTruncate: {} Scheduling in-place block "
-              + "truncate to new size {}", truncatedBlockUC.getTruncateBlock()
-              .getNumBytes(), truncatedBlockUC);
+      BlockUnderConstructionFeature uc = oldBlock.getUnderConstructionFeature();
+      uc.setTruncateBlock(new Block(oldBlock));
+      uc.getTruncateBlock().setNumBytes(oldBlock.getNumBytes() - lastBlockDelta);
+      uc.getTruncateBlock().setGenerationStamp(newBlock.getGenerationStamp());
+      truncatedBlockUC = oldBlock;
+
+      NameNode.stateChangeLog.debug("BLOCK* prepareFileForTruncate: " +
+              "{} Scheduling in-place block truncate to new size {}",
+          uc, uc.getTruncateBlock().getNumBytes());
     }
     if (shouldRecoverNow) {
-      truncatedBlockUC.initializeBlockRecovery(newBlock.getGenerationStamp());
+      truncatedBlockUC.getUnderConstructionFeature().initializeBlockRecovery(
+          truncatedBlockUC, newBlock.getGenerationStamp());
     }
 
     return newBlock;

+ 16 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java

@@ -45,10 +45,10 @@ 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.BlockInfoStripedUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
@@ -77,7 +77,7 @@ class FSDirWriteFileOp {
       Block block) throws IOException {
     // modify file-> block and blocksMap
     // fileNode should be under construction
-    BlockInfoUnderConstruction uc = fileNode.removeLastBlock(block);
+    BlockInfo uc = fileNode.removeLastBlock(block);
     if (uc == null) {
       return false;
     }
@@ -214,8 +214,8 @@ class FSDirWriteFileOp {
 
   static LocatedBlock makeLocatedBlock(FSNamesystem fsn, BlockInfo blk,
       DatanodeStorageInfo[] locs, long offset) throws IOException {
-    LocatedBlock lBlk = BlockManager.newLocatedBlock(fsn.getExtendedBlock(blk),
-        blk, locs, offset);
+    LocatedBlock lBlk = BlockManager.newLocatedBlock(
+        fsn.getExtendedBlock(new Block(blk)), blk, locs, offset);
     fsn.getBlockManager().setBlockToken(lBlk,
         BlockTokenIdentifier.AccessMode.WRITE);
     return lBlk;
@@ -247,8 +247,8 @@ class FSDirWriteFileOp {
       } else {
         // add new chosen targets to already allocated block and return
         BlockInfo lastBlockInFile = pendingFile.getLastBlock();
-        ((BlockInfoContiguousUnderConstruction) lastBlockInFile)
-            .setExpectedLocations(targets);
+        lastBlockInFile.getUnderConstructionFeature().setExpectedLocations(
+            lastBlockInFile, targets, pendingFile.isStriped());
         offset = pendingFile.computeFileSize();
         return makeLocatedBlock(fsn, lastBlockInFile, targets, offset);
       }
@@ -542,7 +542,8 @@ class FSDirWriteFileOp {
         // check quota limits and updated space consumed
         fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
             numLocations, true);
-        blockInfo = new BlockInfoStripedUnderConstruction(block, ecPolicy,
+        blockInfo = new BlockInfoStriped(block, ecPolicy);
+        blockInfo.convertToBlockUnderConstruction(
             HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
       } else {
         // check quota limits and updated space consumed
@@ -550,9 +551,9 @@ class FSDirWriteFileOp {
             fileINode.getPreferredBlockReplication(), true);
 
         short numLocations = fileINode.getFileReplication();
-        blockInfo = new BlockInfoContiguousUnderConstruction(block,
-            numLocations, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
-            targets);
+        blockInfo = new BlockInfoContiguous(block, numLocations);
+        blockInfo.convertToBlockUnderConstruction(
+            HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
       }
       fsd.getBlockManager().addBlockCollection(blockInfo, fileINode);
       fileINode.addBlock(blockInfo);
@@ -692,10 +693,10 @@ class FSDirWriteFileOp {
             "allocation of a new block in " + src + ". Returning previously" +
             " allocated block " + lastBlockInFile);
         long offset = file.computeFileSize();
-        BlockInfoUnderConstruction lastBlockUC =
-            (BlockInfoUnderConstruction) lastBlockInFile;
+        BlockUnderConstructionFeature uc =
+            lastBlockInFile.getUnderConstructionFeature();
         onRetryBlock[0] = makeLocatedBlock(fsn, lastBlockInFile,
-            lastBlockUC.getExpectedStorageLocations(), offset);
+            uc.getExpectedStorageLocations(), offset);
         return new FileState(file, src, iip);
       } else {
         // Case 3

+ 9 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -42,15 +42,14 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 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.RollingUpgradeStartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -991,12 +990,14 @@ public class FSEditLogLoader {
     final BlockInfo newBlockInfo;
     boolean isStriped = ecZone != null;
     if (isStriped) {
-      newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock,
+      newBlockInfo = new BlockInfoStriped(newBlock,
           ecZone.getErasureCodingPolicy());
     } else {
-      newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock,
+      newBlockInfo = new BlockInfoContiguous(newBlock,
           file.getPreferredBlockReplication());
     }
+    newBlockInfo.convertToBlockUnderConstruction(
+        BlockUCState.UNDER_CONSTRUCTION, null);
     fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBlockInfo, file);
     file.addBlock(newBlockInfo);
     fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
@@ -1077,12 +1078,14 @@ public class FSEditLogLoader {
           // what about an old-version fsync() where fsync isn't called
           // until several blocks in?
           if (isStriped) {
-            newBI = new BlockInfoStripedUnderConstruction(newBlock,
+            newBI = new BlockInfoStriped(newBlock,
                 ecZone.getErasureCodingPolicy());
           } else {
-            newBI = new BlockInfoContiguousUnderConstruction(newBlock,
+            newBI = new BlockInfoContiguous(newBlock,
                 file.getPreferredBlockReplication());
           }
+          newBI.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
+              null);
         } else {
           // OP_CLOSE should add finalized blocks. This code path
           // is only executed when loading edits written by prior

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

@@ -55,7 +55,6 @@ 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.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -756,7 +755,7 @@ public class FSImageFormat {
       // file
       
       // read blocks
-      Block[] blocks = new BlockInfoContiguous[numBlocks];
+      BlockInfo[] blocks = new BlockInfoContiguous[numBlocks];
       for (int j = 0; j < numBlocks; j++) {
         blocks[j] = new BlockInfoContiguous(replication);
         blocks[j].readFields(in);
@@ -778,9 +777,9 @@ public class FSImageFormat {
             clientMachine = FSImageSerialization.readString(in);
             // convert the last block to BlockUC
             if (blocks.length > 0) {
-              Block lastBlk = blocks[blocks.length - 1];
-              blocks[blocks.length - 1] =
-                  new BlockInfoContiguousUnderConstruction(lastBlk, replication);
+              BlockInfo lastBlk = blocks[blocks.length - 1];
+              lastBlk.convertToBlockUnderConstruction(
+                  HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null);
             }
           }
         }

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

@@ -45,10 +45,9 @@ 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.BlockInfoStriped;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
@@ -378,11 +377,13 @@ public final class FSImageFormatPBINode {
           final BlockInfo ucBlk;
           if (isStriped) {
             BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
-            ucBlk = new BlockInfoStripedUnderConstruction(striped, ecPolicy);
+            ucBlk = new BlockInfoStriped(striped, ecPolicy);
           } else {
-            ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk,
+            ucBlk = new BlockInfoContiguous(lastBlk,
                 replication);
           }
+          ucBlk.convertToBlockUnderConstruction(
+              HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null);
           file.setBlock(file.numBlocks() - 1, ucBlk);
         }
       }

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

@@ -34,7 +34,6 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 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;
@@ -138,8 +137,9 @@ public class FSImageSerialization {
     // last block is UNDER_CONSTRUCTION
     if(numBlocks > 0) {
       blk.readFields(in);
-      blocksContiguous[i] = new BlockInfoContiguousUnderConstruction(
-          blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
+      blocksContiguous[i] = new BlockInfoContiguous(blk, blockReplication);
+      blocksContiguous[i].convertToBlockUnderConstruction(
+          BlockUCState.UNDER_CONSTRUCTION, null);
     }
 
     PermissionStatus perm = PermissionStatus.read(in);

+ 18 - 22
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -142,7 +142,6 @@ import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
 import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
@@ -204,10 +203,9 @@ 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.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
@@ -3124,28 +3122,25 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throw new AlreadyBeingCreatedException(message);
     case UNDER_CONSTRUCTION:
     case UNDER_RECOVERY:
-      // TODO support truncate of striped blocks
-      final BlockInfoUnderConstruction uc =
-          (BlockInfoUnderConstruction)lastBlock;
+      final BlockUnderConstructionFeature uc = lastBlock.getUnderConstructionFeature();
       // determine if last block was intended to be truncated
       Block recoveryBlock = uc.getTruncateBlock();
       boolean truncateRecovery = recoveryBlock != null;
       boolean copyOnTruncate = truncateRecovery &&
-          recoveryBlock.getBlockId() != uc.toBlock().getBlockId();
+          recoveryBlock.getBlockId() != lastBlock.getBlockId();
       assert !copyOnTruncate ||
-          recoveryBlock.getBlockId() < uc.toBlock().getBlockId() &&
-          recoveryBlock.getGenerationStamp() < uc.toBlock().
-              getGenerationStamp() &&
-          recoveryBlock.getNumBytes() > uc.toBlock().getNumBytes() :
+          recoveryBlock.getBlockId() < lastBlock.getBlockId() &&
+          recoveryBlock.getGenerationStamp() < lastBlock.getGenerationStamp() &&
+          recoveryBlock.getNumBytes() > lastBlock.getNumBytes() :
             "wrong recoveryBlock";
 
       // setup the last block locations from the blockManager if not known
       if (uc.getNumExpectedLocations() == 0) {
-        uc.setExpectedLocations(blockManager.getStorages(lastBlock));
+        uc.setExpectedLocations(lastBlock, blockManager.getStorages(lastBlock),
+            lastBlock.isStriped());
       }
 
-      if (uc.getNumExpectedLocations() == 0 &&
-          uc.toBlock().getNumBytes() == 0) {
+      if (uc.getNumExpectedLocations() == 0 && lastBlock.getNumBytes() == 0) {
         // There is no datanode reported to this block.
         // may be client have crashed before writing data to pipeline.
         // This blocks doesn't need any recovery.
@@ -3159,14 +3154,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
       // start recovery of the last block for this file
       long blockRecoveryId =
-          nextGenerationStamp(blockIdManager.isLegacyBlock(uc.toBlock()));
+          nextGenerationStamp(blockIdManager.isLegacyBlock(lastBlock));
       lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
       if(copyOnTruncate) {
-        uc.toBlock().setGenerationStamp(blockRecoveryId);
+        lastBlock.setGenerationStamp(blockRecoveryId);
       } else if(truncateRecovery) {
         recoveryBlock.setGenerationStamp(blockRecoveryId);
       }
-      uc.initializeBlockRecovery(blockRecoveryId);
+      uc.initializeBlockRecovery(lastBlock, blockRecoveryId);
       leaseManager.renewLease(lease);
       // Cannot close file right now, since the last block requires recovery.
       // This may potentially cause infinite loop in lease recovery
@@ -3371,8 +3366,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
 
       truncatedBlock = iFile.getLastBlock();
-      final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)truncatedBlock;
-      final long recoveryId = uc.getBlockRecoveryId();
+      final long recoveryId = truncatedBlock.getUnderConstructionFeature()
+          .getBlockRecoveryId();
       copyTruncate = truncatedBlock.getBlockId() != storedBlock.getBlockId();
       if(recoveryId != newgenerationstamp) {
         throw new IOException("The recovery id " + newgenerationstamp
@@ -5424,7 +5419,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     // check the vadility of the block and lease holder name
     final INodeFile pendingFile = checkUCBlock(oldBlock, clientName);
     final BlockInfo lastBlock = pendingFile.getLastBlock();
-    final BlockInfoUnderConstruction blockinfo = (BlockInfoUnderConstruction)lastBlock;
+    assert !lastBlock.isComplete();
 
     // check new GS & length: this is not expected
     if (newBlock.getGenerationStamp() <= lastBlock.getGenerationStamp()) {
@@ -5444,12 +5439,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     // Update old block with the new generation stamp and new length
     lastBlock.setNumBytes(newBlock.getNumBytes());
-    blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp());
+    lastBlock.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp());
 
     // find the DatanodeDescriptor objects
     final DatanodeStorageInfo[] storages = blockManager.getDatanodeManager()
         .getDatanodeStorageInfos(newNodes, newStorageIDs);
-    blockinfo.setExpectedLocations(storages);
+    lastBlock.getUnderConstructionFeature().setExpectedLocations(lastBlock,
+        storages, lastBlock.isStriped());
 
     String src = pendingFile.getFullPathName();
     FSDirWriteFileOp.persistBlocks(dir, src, pendingFile, logRetryCache);

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

@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
-import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID;
 
@@ -39,12 +38,10 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 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;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature;
@@ -86,7 +83,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * [4-bit storagePolicyID][1-bit isStriped]
    * [11-bit replication][48-bit preferredBlockSize]
    */
-  static enum HeaderFormat {
+  enum HeaderFormat {
     PREFERRED_BLOCK_SIZE(null, 48, 1),
     REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 11, 0),
     IS_STRIPED(REPLICATION.BITS, 1, 0),
@@ -264,25 +261,20 @@ public class INodeFile extends INodeWithAdditionalFields
     if (numBlocks() == 0) {
       throw new IOException("Failed to set last block: File is empty.");
     }
+    lastBlock.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
+        locations);
+  }
 
-    final BlockInfo ucBlock;
-    if (isStriped()) {
-      Preconditions.checkState(lastBlock.isStriped());
-      ucBlock = ((BlockInfoStriped) lastBlock)
-          .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
-    } else {
-      Preconditions.checkState(!lastBlock.isStriped());
-      ucBlock = ((BlockInfoContiguous) lastBlock)
-          .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
-    }
-    setBlock(numBlocks() - 1, ucBlock);
+  void setLastBlock(BlockInfo blk) {
+    blk.setBlockCollection(this);
+    setBlock(numBlocks() - 1, blk);
   }
 
   /**
    * Remove a block from the block list. This block should be
    * the last one on the list.
    */
-  BlockInfoUnderConstruction removeLastBlock(Block oldblock) {
+  BlockInfo removeLastBlock(Block oldblock) {
     Preconditions.checkState(isUnderConstruction(),
         "file is no longer under construction");
     if (blocks == null || blocks.length == 0) {
@@ -293,13 +285,12 @@ public class INodeFile extends INodeWithAdditionalFields
       return null;
     }
 
-    BlockInfoUnderConstruction uc =
-        (BlockInfoUnderConstruction)blocks[size_1];
+    BlockInfo ucBlock = blocks[size_1];
     //copy to a new list
     BlockInfo[] newlist = new BlockInfo[size_1];
     System.arraycopy(blocks, 0, newlist, 0, size_1);
     setBlocks(newlist);
-    return uc;
+    return ucBlock;
   }
 
   /* End of Under-Construction Feature */
@@ -758,7 +749,7 @@ public class INodeFile extends INodeWithAdditionalFields
     //check if the last block is BlockInfoUnderConstruction
     BlockInfo lastBlk = blocks[last];
     long size = lastBlk.getNumBytes();
-    if (lastBlk instanceof BlockInfoUnderConstruction) {
+    if (!lastBlk.isComplete()) {
        if (!includesLastUcBlock) {
          size = 0;
        } else if (usePreferredBlockSize4LastUcBlock) {

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

@@ -23,7 +23,6 @@ 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.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 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;
@@ -136,7 +135,7 @@ public class FileDiffList extends
     Block dontRemoveBlock = null;
     if (lastBlock != null && lastBlock.getBlockUCState().equals(
         HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) {
-      dontRemoveBlock = ((BlockInfoContiguousUnderConstruction) lastBlock)
+      dontRemoveBlock = lastBlock.getUnderConstructionFeature()
           .getTruncateBlock();
     }
     // Collect the remaining blocks of the file, ignoring truncate block

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

@@ -117,7 +117,6 @@ 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.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -1650,13 +1649,11 @@ public class DFSTestUtil {
     BlockManager bm0 = nn.getNamesystem().getBlockManager();
     BlockInfo storedBlock = bm0.getStoredBlock(blk.getLocalBlock());
     assertTrue("Block " + blk + " should be under construction, " +
-        "got: " + storedBlock,
-        storedBlock instanceof BlockInfoContiguousUnderConstruction);
-    BlockInfoContiguousUnderConstruction ucBlock =
-      (BlockInfoContiguousUnderConstruction)storedBlock;
+        "got: " + storedBlock, !storedBlock.isComplete());
     // 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();
+    final DatanodeStorageInfo[] storages = storedBlock
+        .getUnderConstructionFeature().getExpectedStorageLocations();
     DatanodeStorageInfo expectedPrimary = storages[0];
     long mostRecentLastUpdate = expectedPrimary.getDatanodeDescriptor()
         .getLastUpdateMonotonic();

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

@@ -39,25 +39,24 @@ public class TestBlockInfoUnderConstruction {
     DatanodeDescriptor dd3 = s3.getDatanodeDescriptor();
 
     dd1.isAlive = dd2.isAlive = dd3.isAlive = true;
-    BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
-        new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP),
-        (short) 3,
-        BlockUCState.UNDER_CONSTRUCTION,
+    BlockInfoContiguous blockInfo = new BlockInfoContiguous(
+        new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3);
+    blockInfo.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
         new DatanodeStorageInfo[] {s1, s2, s3});
 
     // Recovery attempt #1.
     DFSTestUtil.resetLastUpdatesWithOffset(dd1, -3 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd3, -2 * 1000);
-    blockInfo.initializeBlockRecovery(1);
-    BlockInfoUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
+    blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 1);
+    BlockInfo[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
     assertEquals(blockInfoRecovery[0], blockInfo);
 
     // Recovery attempt #2.
     DFSTestUtil.resetLastUpdatesWithOffset(dd1, -2 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd3, -3 * 1000);
-    blockInfo.initializeBlockRecovery(2);
+    blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 2);
     blockInfoRecovery = dd1.getLeaseRecoveryCommand(1);
     assertEquals(blockInfoRecovery[0], blockInfo);
 
@@ -65,7 +64,7 @@ public class TestBlockInfoUnderConstruction {
     DFSTestUtil.resetLastUpdatesWithOffset(dd1, -2 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd3, -3 * 1000);
-    blockInfo.initializeBlockRecovery(3);
+    blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 3);
     blockInfoRecovery = dd3.getLeaseRecoveryCommand(1);
     assertEquals(blockInfoRecovery[0], blockInfo);
 
@@ -74,7 +73,7 @@ public class TestBlockInfoUnderConstruction {
     DFSTestUtil.resetLastUpdatesWithOffset(dd1, -2 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd3, 0);
-    blockInfo.initializeBlockRecovery(3);
+    blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 3);
     blockInfoRecovery = dd3.getLeaseRecoveryCommand(1);
     assertEquals(blockInfoRecovery[0], blockInfo);
   }

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

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -727,8 +728,8 @@ public class TestBlockManager {
     // verify the storage info is correct
     assertTrue(bm.getStoredBlock(new Block(receivedBlockId)).findStorageInfo
         (ds) >= 0);
-    assertTrue(((BlockInfoContiguousUnderConstruction) bm.
-        getStoredBlock(new Block(receivingBlockId))).getNumExpectedLocations() > 0);
+    assertTrue(bm.getStoredBlock(new Block(receivingBlockId))
+        .getUnderConstructionFeature().getNumExpectedLocations() > 0);
     assertTrue(bm.getStoredBlock(new Block(receivingReceivedBlockId))
         .findStorageInfo(ds) >= 0);
     assertNull(bm.getStoredBlock(new Block(ReceivedDeletedBlockId)));
@@ -748,8 +749,8 @@ public class TestBlockManager {
 
   private BlockInfo addUcBlockToBM(long blkId) {
     Block block = new Block(blkId);
-    BlockInfoContiguousUnderConstruction blockInfo =
-        new BlockInfoContiguousUnderConstruction(block, (short) 3);
+    BlockInfo blockInfo = new BlockInfoContiguous(block, (short) 3);
+    blockInfo.convertToBlockUnderConstruction(UNDER_CONSTRUCTION, null);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
     Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication();
     bm.blocksMap.addBlockCollection(blockInfo, bc);

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

@@ -172,9 +172,10 @@ public class TestHeartbeatHandling {
               dd1.getStorageInfos()[0],
               dd2.getStorageInfos()[0],
               dd3.getStorageInfos()[0]};
-          BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
-              new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
-              BlockUCState.UNDER_RECOVERY, storages);
+          BlockInfo blockInfo = new BlockInfoContiguous(
+              new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3);
+          blockInfo.convertToBlockUnderConstruction(BlockUCState.UNDER_RECOVERY,
+              storages);
           dd1.addBlockToBeRecovered(blockInfo);
           DatanodeCommand[] cmds =
               NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands();
@@ -194,9 +195,10 @@ public class TestHeartbeatHandling {
           // More than the default stale interval of 30 seconds.
           DFSTestUtil.resetLastUpdatesWithOffset(dd2, -40 * 1000);
           DFSTestUtil.resetLastUpdatesWithOffset(dd3, 0);
-          blockInfo = new BlockInfoContiguousUnderConstruction(
-              new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
-              BlockUCState.UNDER_RECOVERY, storages);
+          blockInfo = new BlockInfoContiguous(
+              new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3);
+          blockInfo.convertToBlockUnderConstruction(BlockUCState.UNDER_RECOVERY,
+              storages);
           dd1.addBlockToBeRecovered(blockInfo);
           cmds = NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands();
           assertEquals(1, cmds.length);
@@ -215,9 +217,10 @@ 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(
-              new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
-              BlockUCState.UNDER_RECOVERY, storages);
+          blockInfo = new BlockInfoContiguous(
+              new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3);
+          blockInfo.convertToBlockUnderConstruction(BlockUCState.UNDER_RECOVERY,
+              storages);
           dd1.addBlockToBeRecovered(blockInfo);
           cmds = NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands();
           assertEquals(1, cmds.length);

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

@@ -1182,7 +1182,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);
+    BlockInfoContiguous info = new BlockInfoContiguous(block1, (short) 1);
+    info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION, null);
     BlockCollection bc = mock(BlockCollection.class);
     when(bc.getPreferredBlockReplication()).thenReturn((short)1);
     bm.addBlockCollection(info, bc);
@@ -1247,9 +1248,8 @@ public class TestReplicationPolicy {
 
     DatanodeStorageInfo[] storageAry = {new DatanodeStorageInfo(
         dataNodes[0], new DatanodeStorage("s1"))};
-    final BlockInfoContiguousUnderConstruction ucBlock =
-        info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
-            storageAry);
+    info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
+        storageAry);
     DatanodeStorageInfo storage = mock(DatanodeStorageInfo.class);
     DatanodeDescriptor dn = mock(DatanodeDescriptor.class);
     when(dn.isDecommissioned()).thenReturn(true);
@@ -1258,10 +1258,10 @@ public class TestReplicationPolicy {
     when(storage.removeBlock(any(BlockInfo.class))).thenReturn(true);
     when(storage.addBlock(any(BlockInfo.class))).thenReturn
         (DatanodeStorageInfo.AddBlockResult.ADDED);
-    ucBlock.addStorage(storage, ucBlock);
+    info.addStorage(storage, info);
 
     BlockInfo lastBlk = mbc.getLastBlock();
-    when(mbc.getLastBlock()).thenReturn(lastBlk, ucBlock);
+    when(mbc.getLastBlock()).thenReturn(lastBlk, info);
 
     bm.convertLastBlockToUnderConstruction(mbc, 0L);
 

+ 17 - 21
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java

@@ -34,7 +34,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
@@ -170,13 +169,13 @@ public class TestAddStripedBlocks {
     Assert.assertEquals(0,
         block.getBlockId() & HdfsServerConstants.BLOCK_GROUP_INDEX_MASK);
 
-    final BlockInfoStripedUnderConstruction blockUC =
-        (BlockInfoStripedUnderConstruction) block;
     Assert.assertEquals(HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
-        blockUC.getBlockUCState());
+        block.getBlockUCState());
     if (checkReplica) {
-      Assert.assertEquals(GROUP_SIZE, blockUC.getNumExpectedLocations());
-      DatanodeStorageInfo[] storages = blockUC.getExpectedStorageLocations();
+      Assert.assertEquals(GROUP_SIZE,
+          block.getUnderConstructionFeature().getNumExpectedLocations());
+      DatanodeStorageInfo[] storages = block.getUnderConstructionFeature()
+          .getExpectedStorageLocations();
       for (DataNode dn : cluster.getDataNodes()) {
         Assert.assertTrue(includeDataNode(dn.getDatanodeId(), storages));
       }
@@ -205,11 +204,10 @@ public class TestAddStripedBlocks {
 
       FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
       INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
-      BlockInfoStripedUnderConstruction lastBlk =
-          (BlockInfoStripedUnderConstruction) fileNode.getLastBlock();
-      DatanodeInfo[] expectedDNs = DatanodeStorageInfo
-          .toDatanodeInfos(lastBlk.getExpectedStorageLocations());
-      int[] indices = lastBlk.getBlockIndices();
+      BlockInfoStriped lastBlk = (BlockInfoStriped) fileNode.getLastBlock();
+      DatanodeInfo[] expectedDNs = DatanodeStorageInfo.toDatanodeInfos(
+          lastBlk.getUnderConstructionFeature().getExpectedStorageLocations());
+      int[] indices = lastBlk.getUnderConstructionFeature().getBlockIndices();
 
       LocatedBlocks blks = dfs.getClient().getLocatedBlocks(file.toString(), 0L);
       Assert.assertEquals(1, blks.locatedBlockCount());
@@ -246,11 +244,10 @@ public class TestAddStripedBlocks {
       cluster.getNamesystem().getAdditionalBlock(file.toString(),
           fileNode.getId(), dfs.getClient().getClientName(), null, null, null);
       BlockInfo lastBlock = fileNode.getLastBlock();
-      BlockInfoStripedUnderConstruction ucBlock =
-          (BlockInfoStripedUnderConstruction) lastBlock;
 
-      DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations();
-      int[] indices = ucBlock.getBlockIndices();
+      DatanodeStorageInfo[] locs = lastBlock.getUnderConstructionFeature()
+          .getExpectedStorageLocations();
+      int[] indices = lastBlock.getUnderConstructionFeature().getBlockIndices();
       Assert.assertEquals(GROUP_SIZE, locs.length);
       Assert.assertEquals(GROUP_SIZE, indices.length);
 
@@ -272,8 +269,8 @@ public class TestAddStripedBlocks {
       }
 
       // make sure lastBlock is correct and the storages have been updated
-      locs = ucBlock.getExpectedStorageLocations();
-      indices = ucBlock.getBlockIndices();
+      locs = lastBlock.getUnderConstructionFeature().getExpectedStorageLocations();
+      indices = lastBlock.getUnderConstructionFeature().getBlockIndices();
       Assert.assertEquals(GROUP_SIZE, locs.length);
       Assert.assertEquals(GROUP_SIZE, indices.length);
       for (DatanodeStorageInfo newstorage : locs) {
@@ -307,10 +304,9 @@ public class TestAddStripedBlocks {
           bpId, reports, null);
     }
 
-    BlockInfoStripedUnderConstruction ucBlock =
-        (BlockInfoStripedUnderConstruction) lastBlock;
-    DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations();
-    int[] indices = ucBlock.getBlockIndices();
+    DatanodeStorageInfo[] locs = lastBlock.getUnderConstructionFeature()
+        .getExpectedStorageLocations();
+    int[] indices = lastBlock.getUnderConstructionFeature().getBlockIndices();
     Assert.assertEquals(GROUP_SIZE, locs.length);
     Assert.assertEquals(GROUP_SIZE, indices.length);
     for (i = 0; i < GROUP_SIZE; i++) {

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

@@ -18,6 +18,7 @@
 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;
@@ -36,7 +37,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.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.junit.AfterClass;
@@ -156,6 +157,7 @@ public class TestBlockUnderConstruction {
   @Test
   public void testGetBlockLocations() throws IOException {
     final NamenodeProtocols namenode = cluster.getNameNodeRpc();
+    final BlockManager blockManager = cluster.getNamesystem().getBlockManager();
     final Path p = new Path(BASE_DIR, "file2.dat");
     final String src = p.toString();
     final FSDataOutputStream out = TestFileCreation.createFile(hdfs, p, 3);
@@ -170,7 +172,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);
+      assertFalse(blockManager.getStoredBlock(b).isComplete());
 
       if (++i < NUM_BLOCKS) {
         // write one more block

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java

@@ -24,7 +24,6 @@ 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.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.junit.Test;
@@ -68,11 +67,13 @@ public class TestCommitBlockSynchronization {
     namesystem.dir.getINodeMap().put(file);
 
     FSNamesystem namesystemSpy = spy(namesystem);
-    BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
-        block, (short) 1, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
+    BlockInfo blockInfo = new BlockInfoContiguous(block, (short) 1);
+    blockInfo.convertToBlockUnderConstruction(
+        HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
     blockInfo.setBlockCollection(file);
     blockInfo.setGenerationStamp(genStamp);
-    blockInfo.initializeBlockRecovery(genStamp);
+    blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo,
+        genStamp);
     doReturn(blockInfo).when(file).removeLastBlock(any(Block.class));
     doReturn(true).when(file).isUnderConstruction();
     doReturn(new BlockInfoContiguous[1]).when(file).getBlocks();

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

@@ -55,7 +55,6 @@ 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.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -1017,7 +1016,7 @@ public class TestFileTruncate {
           is(fsn.getBlockIdManager().getGenerationStampV2()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
-      long blockRecoveryId = ((BlockInfoContiguousUnderConstruction) file.getLastBlock())
+      long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature()
           .getBlockRecoveryId();
       assertThat(blockRecoveryId, is(initialGenStamp + 1));
       fsn.getEditLog().logTruncate(
@@ -1051,7 +1050,7 @@ public class TestFileTruncate {
           is(fsn.getBlockIdManager().getGenerationStampV2()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
-      long blockRecoveryId = ((BlockInfoContiguousUnderConstruction) file.getLastBlock())
+      long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature()
           .getBlockRecoveryId();
       assertThat(blockRecoveryId, is(initialGenStamp + 1));
       fsn.getEditLog().logTruncate(

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java

@@ -39,7 +39,6 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
@@ -157,8 +156,9 @@ public class TestStripedINodeFile {
       throws IOException, InterruptedException {
     INodeFile inf = createStripedINodeFile();
     Block blk = new Block(1);
-    BlockInfoStripedUnderConstruction bInfoUCStriped
-        = new BlockInfoStripedUnderConstruction(blk, testECPolicy);
+    BlockInfoStriped bInfoUCStriped = new BlockInfoStriped(blk, testECPolicy);
+    bInfoUCStriped.convertToBlockUnderConstruction(
+        HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null);
     bInfoUCStriped.setNumBytes(100);
     inf.addBlock(bInfoUCStriped);
     assertEquals(100, inf.computeFileSize());
@@ -191,8 +191,9 @@ public class TestStripedINodeFile {
       throws IOException, InterruptedException {
     INodeFile inf = createStripedINodeFile();
     Block blk = new Block(1);
-    BlockInfoStripedUnderConstruction bInfoUCStriped
-        = new BlockInfoStripedUnderConstruction(blk, testECPolicy);
+    BlockInfoStriped bInfoUCStriped = new BlockInfoStriped(blk, testECPolicy);
+    bInfoUCStriped.convertToBlockUnderConstruction(
+        HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null);
     bInfoUCStriped.setNumBytes(100);
     inf.addBlock(bInfoUCStriped);
 

+ 6 - 5
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.BlockInfo;
 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,12 +752,13 @@ public class TestRetryCacheWithHA {
     boolean checkNamenodeBeforeReturn() throws Exception {
       INodeFile fileNode = cluster.getNamesystem(0).getFSDirectory()
           .getINode4Write(file).asFile();
-      BlockInfoContiguousUnderConstruction blkUC =
-          (BlockInfoContiguousUnderConstruction) (fileNode.getBlocks())[1];
-      int datanodeNum = blkUC.getExpectedStorageLocations().length;
+      BlockInfo blkUC = (fileNode.getBlocks())[1];
+      int datanodeNum = blkUC.getUnderConstructionFeature()
+          .getExpectedStorageLocations().length;
       for (int i = 0; i < CHECKTIMES && datanodeNum != 2; i++) {
         Thread.sleep(1000);
-        datanodeNum = blkUC.getExpectedStorageLocations().length;
+        datanodeNum = blkUC.getUnderConstructionFeature()
+            .getExpectedStorageLocations().length;
       }
       return datanodeNum == 2;
     }

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

@@ -43,10 +43,9 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 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.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage;
 import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -178,8 +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.
-   * {@link FileWithSnapshot}: next link
+   * and {@link BlockUnderConstructionFeature#toString()} for detailed information.
    * </pre>
    * @see INode#dumpTreeRecursively()
    */