You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by wa...@apache.org on 2015/06/12 20:38:45 UTC

[2/2] hadoop git commit: HDFS-8499. Refactor BlockInfo class hierarchy with static helper class. Contributed by Zhe Zhang.

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


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c17439c2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c17439c2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c17439c2

Branch: refs/heads/trunk
Commit: c17439c2ddd921b63b1635e6f1cba634b8da8557
Parents: 46b0b41
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Jun 12 11:35:39 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Jun 12 11:38:39 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../server/blockmanagement/BlockCollection.java |   2 +-
 .../hdfs/server/blockmanagement/BlockInfo.java  |  24 +-
 .../blockmanagement/BlockInfoContiguous.java    |  77 +---
 .../BlockInfoContiguousUnderConstruction.java   | 403 ------------------
 .../BlockInfoUnderConstruction.java             | 405 +++++++++++++++++++
 .../BlockInfoUnderConstructionContiguous.java   | 110 +++++
 .../server/blockmanagement/BlockManager.java    |  40 +-
 .../ContiguousBlockStorageOp.java               | 106 +++++
 .../blockmanagement/DatanodeDescriptor.java     |  13 +-
 .../server/blockmanagement/DatanodeManager.java |   4 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |  15 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |  15 +-
 .../hdfs/server/namenode/FSImageFormat.java     |   7 +-
 .../server/namenode/FSImageFormatPBINode.java   |   6 +-
 .../server/namenode/FSImageSerialization.java   |   4 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  24 +-
 .../namenode/FileUnderConstructionFeature.java  |  10 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  14 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java |   4 +-
 .../server/namenode/snapshot/FileDiffList.java  |   4 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   8 +-
 .../TestBlockInfoUnderConstruction.java         |   6 +-
 .../blockmanagement/TestBlockManager.java       |   6 +-
 .../blockmanagement/TestHeartbeatHandling.java  |   8 +-
 .../blockmanagement/TestReplicationPolicy.java  |   5 +-
 .../namenode/TestBlockUnderConstruction.java    |   4 +-
 .../TestCommitBlockSynchronization.java         |   9 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |   6 +-
 .../namenode/ha/TestRetryCacheWithHA.java       |   6 +-
 .../namenode/snapshot/SnapshotTestHelper.java   |   4 +-
 31 files changed, 769 insertions(+), 583 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c17439c2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e315db6..033451e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -626,6 +626,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-7923. The DataNodes should rate-limit their full block reports by
     asking the NN on heartbeat messages (cmccabe)
 
+    HDFS-8499. Refactor BlockInfo class hierarchy with static helper class.
+    (Zhe Zhang via wang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c17439c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
index 02a1d05..f11a825 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
@@ -79,7 +79,7 @@ public interface BlockCollection {
    * Convert the last block of the collection to an under-construction block
    * and set the locations.
    */
-  public BlockInfoContiguousUnderConstruction setLastBlock(BlockInfo lastBlock,
+  public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
       DatanodeStorageInfo[] targets) throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c17439c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
index dea31c4..4cc2791 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@ -51,7 +51,7 @@ public abstract class  BlockInfo extends Block
    * per replica is 42 bytes (LinkedList#Entry object per replica) versus 16
    * bytes using the triplets.
    */
-  protected Object[] triplets;
+  Object[] triplets;
 
   /**
    * Construct an entry for blocksmap
@@ -295,7 +295,7 @@ public abstract class  BlockInfo extends Block
   /**
    * BlockInfo represents a block that is not being constructed.
    * In order to start modifying the block, the BlockInfo should be converted
-   * to {@link BlockInfoContiguousUnderConstruction}.
+   * to {@link BlockInfoUnderConstruction}.
    * @return {@link BlockUCState#COMPLETE}
    */
   public BlockUCState getBlockUCState() {
@@ -312,27 +312,29 @@ public abstract class  BlockInfo extends Block
   }
 
   /**
-   * Convert a complete block to an under construction block.
+   * Convert a block to an under construction block.
    * @return BlockInfoUnderConstruction -  an under construction block.
    */
-  public BlockInfoContiguousUnderConstruction convertToBlockUnderConstruction(
+  public BlockInfoUnderConstruction convertToBlockUnderConstruction(
       BlockUCState s, DatanodeStorageInfo[] targets) {
     if(isComplete()) {
-      BlockInfoContiguousUnderConstruction ucBlock =
-          new BlockInfoContiguousUnderConstruction(this,
-          getBlockCollection().getPreferredBlockReplication(), s, targets);
-      ucBlock.setBlockCollection(getBlockCollection());
-      return ucBlock;
+      return convertCompleteBlockToUC(s, targets);
     }
     // the block is already under construction
-    BlockInfoContiguousUnderConstruction ucBlock =
-        (BlockInfoContiguousUnderConstruction)this;
+    BlockInfoUnderConstruction ucBlock =
+        (BlockInfoUnderConstruction)this;
     ucBlock.setBlockUCState(s);
     ucBlock.setExpectedLocations(targets);
     ucBlock.setBlockCollection(getBlockCollection());
     return ucBlock;
   }
 
+  /**
+   * Convert a complete block to an under construction block.
+   */
+  abstract BlockInfoUnderConstruction convertCompleteBlockToUC(
+      BlockUCState s, DatanodeStorageInfo[] targets);
+
   @Override
   public int hashCode() {
     // Super implementation is sufficient

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c17439c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
deleted file mode 100644
index b757171..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
+++ /dev/null
@@ -1,403 +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 {
-  /** 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;
-
-  /**
-   * ReplicaUnderConstruction contains information about replicas while
-   * they are under construction.
-   * The GS, the length and the state of the replica is as reported by 
-   * the data-node.
-   * It is not guaranteed, but expected, that data-nodes actually have
-   * corresponding replicas.
-   */
-  static class ReplicaUnderConstruction extends Block {
-    private final DatanodeStorageInfo expectedLocation;
-    private ReplicaState state;
-    private boolean chosenAsPrimary;
-
-    ReplicaUnderConstruction(Block block,
-                             DatanodeStorageInfo target,
-                             ReplicaState state) {
-      super(block);
-      this.expectedLocation = target;
-      this.state = state;
-      this.chosenAsPrimary = false;
-    }
-
-    /**
-     * Expected block replica location as assigned when the block was allocated.
-     * This defines the pipeline order.
-     * It is not guaranteed, but expected, that the data-node actually has
-     * the replica.
-     */
-    private DatanodeStorageInfo getExpectedStorageLocation() {
-      return expectedLocation;
-    }
-
-    /**
-     * Get replica state as reported by the data-node.
-     */
-    ReplicaState getState() {
-      return state;
-    }
-
-    /**
-     * Whether the replica was chosen for recovery.
-     */
-    boolean getChosenAsPrimary() {
-      return chosenAsPrimary;
-    }
-
-    /**
-     * Set replica state.
-     */
-    void setState(ReplicaState s) {
-      state = s;
-    }
-
-    /**
-     * Set whether this replica was chosen for recovery.
-     */
-    void setChosenAsPrimary(boolean chosenAsPrimary) {
-      this.chosenAsPrimary = chosenAsPrimary;
-    }
-
-    /**
-     * Is data-node the replica belongs to alive.
-     */
-    boolean isAlive() {
-      return expectedLocation.getDatanodeDescriptor().isAlive;
-    }
-
-    @Override // Block
-    public int hashCode() {
-      return super.hashCode();
-    }
-
-    @Override // Block
-    public boolean equals(Object obj) {
-      // Sufficient to rely on super's implementation
-      return (this == obj) || super.equals(obj);
-    }
-
-    @Override
-    public String toString() {
-      final StringBuilder b = new StringBuilder(50);
-      appendStringTo(b);
-      return b.toString();
-    }
-    
-    @Override
-    public void appendStringTo(StringBuilder sb) {
-      sb.append("ReplicaUC[")
-        .append(expectedLocation)
-        .append("|")
-        .append(state)
-        .append("]");
-    }
-  }
-
-  /**
-   * 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 :
-      "BlockInfoUnderConstruction cannot be in COMPLETE state";
-    this.blockUCState = state;
-    setExpectedLocations(targets);
-  }
-
-  /**
-   * Convert an under construction block to a complete block.
-   * 
-   * @return BlockInfo - a complete block.
-   * @throws IOException if the state of the block 
-   * (the generation stamp and the length) has not been committed by 
-   * the client or it does not have at least a minimal number of replicas 
-   * reported from data-nodes. 
-   */
-  BlockInfo convertToCompleteBlock() throws IOException {
-    assert getBlockUCState() != BlockUCState.COMPLETE :
-      "Trying to convert a COMPLETE block";
-    return new BlockInfoContiguous(this);
-  }
-
-  /** Set expected locations */
-  public void setExpectedLocations(DatanodeStorageInfo[] targets) {
-    int numLocations = targets == null ? 0 : targets.length;
-    this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations);
-    for(int i = 0; i < numLocations; i++)
-      replicas.add(
-        new ReplicaUnderConstruction(this, targets[i], ReplicaState.RBW));
-  }
-
-  /**
-   * Create array of expected replica locations
-   * (as has been assigned by chooseTargets()).
-   */
-  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;
-  }
-
-  /** Get the number of expected locations */
-  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;
-  }
-
-  /** Get block recovery ID */
-  public long getBlockRecoveryId() {
-    return blockRecoveryId;
-  }
-
-  /** Get recover block */
-  public Block getTruncateBlock() {
-    return truncateBlock;
-  }
-
-  public void setTruncateBlock(Block recoveryBlock) {
-    this.truncateBlock = recoveryBlock;
-  }
-
-  /**
-   * 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) {
-    // 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());
-      }
-    }
-  }
-
-  /**
-   * 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());
-    blockUCState = BlockUCState.COMMITTED;
-    this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp());
-    // Sort out invalid replicas.
-    setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
-  }
-
-  /**
-   * 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) {
-    setBlockUCState(BlockUCState.UNDER_RECOVERY);
-    blockRecoveryId = recoveryId;
-    if (replicas.size() == 0) {
-      NameNode.blockStateChangeLog.warn("BLOCK*"
-        + " BlockInfoUnderConstruction.initLeaseRecovery:"
-        + " No blocks found, lease removed.");
-    }
-    boolean allLiveReplicasTriedAsPrimary = true;
-    for (int i = 0; i < replicas.size(); i++) {
-      // Check if all replicas have been tried or not.
-      if (replicas.get(i).isAlive()) {
-        allLiveReplicasTriedAsPrimary =
-            (allLiveReplicasTriedAsPrimary && replicas.get(i).getChosenAsPrimary());
-      }
-    }
-    if (allLiveReplicasTriedAsPrimary) {
-      // Just set all the replicas to be chosen whether they are alive or not.
-      for (int i = 0; i < replicas.size(); i++) {
-        replicas.get(i).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.info(
-          "BLOCK* {} recovery started, primary={}", this, primary);
-    }
-  }
-
-  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 // BlockInfo
-  // BlockInfoUnderConstruction participates in maps the same way as BlockInfo
-  public int hashCode() {
-    return super.hashCode();
-  }
-
-  @Override // BlockInfo
-  public boolean equals(Object obj) {
-    // Sufficient to rely on super's implementation
-    return (this == obj) || super.equals(obj);
-  }
-
-  @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("]}");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c17439c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
new file mode 100644
index 0000000..9cd3987
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
@@ -0,0 +1,405 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+
+/**
+ * Represents a block that is currently being constructed.<br>
+ * This is usually the last block of a file opened for write or append.
+ */
+public abstract class BlockInfoUnderConstruction extends BlockInfo {
+  /** Block state. See {@link BlockUCState} */
+  protected BlockUCState blockUCState;
+
+  /**
+   * Block replicas as assigned when the block was allocated.
+   * This defines the pipeline order.
+   */
+  protected 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.
+   */
+  protected Block truncateBlock;
+
+  /**
+   * ReplicaUnderConstruction contains information about replicas while
+   * they are under construction.
+   * The GS, the length and the state of the replica is as reported by
+   * the data-node.
+   * It is not guaranteed, but expected, that data-nodes actually have
+   * corresponding replicas.
+   */
+  static class ReplicaUnderConstruction extends Block {
+    private final DatanodeStorageInfo expectedLocation;
+    private ReplicaState state;
+    private boolean chosenAsPrimary;
+
+    ReplicaUnderConstruction(Block block,
+                             DatanodeStorageInfo target,
+                             ReplicaState state) {
+      super(block);
+      this.expectedLocation = target;
+      this.state = state;
+      this.chosenAsPrimary = false;
+    }
+
+    /**
+     * Expected block replica location as assigned when the block was allocated.
+     * This defines the pipeline order.
+     * It is not guaranteed, but expected, that the data-node actually has
+     * the replica.
+     */
+    private DatanodeStorageInfo getExpectedStorageLocation() {
+      return expectedLocation;
+    }
+
+    /**
+     * Get replica state as reported by the data-node.
+     */
+    ReplicaState getState() {
+      return state;
+    }
+
+    /**
+     * Whether the replica was chosen for recovery.
+     */
+    boolean getChosenAsPrimary() {
+      return chosenAsPrimary;
+    }
+
+    /**
+     * Set replica state.
+     */
+    void setState(ReplicaState s) {
+      state = s;
+    }
+
+    /**
+     * Set whether this replica was chosen for recovery.
+     */
+    void setChosenAsPrimary(boolean chosenAsPrimary) {
+      this.chosenAsPrimary = chosenAsPrimary;
+    }
+
+    /**
+     * Is data-node the replica belongs to alive.
+     */
+    boolean isAlive() {
+      return expectedLocation.getDatanodeDescriptor().isAlive;
+    }
+
+    @Override // Block
+    public int hashCode() {
+      return super.hashCode();
+    }
+
+    @Override // Block
+    public boolean equals(Object obj) {
+      // Sufficient to rely on super's implementation
+      return (this == obj) || super.equals(obj);
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder b = new StringBuilder(50);
+      appendStringTo(b);
+      return b.toString();
+    }
+
+    @Override
+    public void appendStringTo(StringBuilder sb) {
+      sb.append("ReplicaUC[")
+        .append(expectedLocation)
+        .append("|")
+        .append(state)
+        .append("]");
+    }
+  }
+
+  /**
+   * Create block and set its state to
+   * {@link BlockUCState#UNDER_CONSTRUCTION}.
+   */
+  public BlockInfoUnderConstruction(Block blk, short replication) {
+    this(blk, replication, BlockUCState.UNDER_CONSTRUCTION, null);
+  }
+
+  /**
+   * Create a block that is currently being constructed.
+   */
+  public BlockInfoUnderConstruction(Block blk, short replication,
+      BlockUCState state, DatanodeStorageInfo[] targets) {
+    super(blk, replication);
+    Preconditions.checkState(getBlockUCState() != BlockUCState.COMPLETE,
+        "BlockInfoUnderConstruction cannot be in COMPLETE state");
+    this.blockUCState = state;
+    setExpectedLocations(targets);
+  }
+
+  /** Set expected locations. */
+  public abstract void setExpectedLocations(DatanodeStorageInfo[] targets);
+
+  /**
+   * Create array of expected replica locations
+   * (as has been assigned by chooseTargets()).
+   */
+  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;
+  }
+
+  /** Get the number of expected locations. */
+  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;
+  }
+
+  /** Get block recovery ID. */
+  public long getBlockRecoveryId() {
+    return blockRecoveryId;
+  }
+
+  /** Get recover block. */
+  public abstract Block getTruncateBlock();
+
+  public abstract void setTruncateBlock(Block recoveryBlock);
+
+  /**
+   * 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) {
+    // 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());
+      }
+    }
+  }
+
+  /**
+   * 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());
+    }
+    blockUCState = BlockUCState.COMMITTED;
+    this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp());
+    // Sort out invalid replicas.
+    setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
+  }
+
+  /**
+   * 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) {
+    setBlockUCState(BlockUCState.UNDER_RECOVERY);
+    blockRecoveryId = recoveryId;
+    if (replicas.size() == 0) {
+      NameNode.blockStateChangeLog.warn("BLOCK* " +
+          "BlockInfoUnderConstruction.initLeaseRecovery: " +
+          "No blocks found, lease removed.");
+    }
+    boolean allLiveReplicasTriedAsPrimary = true;
+    for (int i = 0; i < replicas.size(); i++) {
+      // Check if all replicas have been tried or not.
+      if (replicas.get(i).isAlive()) {
+        allLiveReplicasTriedAsPrimary =
+            (allLiveReplicasTriedAsPrimary &&
+                replicas.get(i).getChosenAsPrimary());
+      }
+    }
+    if (allLiveReplicasTriedAsPrimary) {
+      // Just set all the replicas to be chosen whether they are alive or not.
+      for (int i = 0; i < replicas.size(); i++) {
+        replicas.get(i).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.info(
+          "BLOCK* {} recovery started, primary={}", this, primary);
+    }
+  }
+
+  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));
+  }
+
+  /**
+   * Convert an under construction block to a complete block.
+   *
+   * @return a complete block.
+   * @throws IOException
+   *           if the state of the block (the generation stamp and the length)
+   *           has not been committed by the client or it does not have at
+   *           least a minimal number of replicas reported from data-nodes.
+   */
+  public abstract BlockInfo convertToCompleteBlock();
+
+  @Override
+  BlockInfoUnderConstruction convertCompleteBlockToUC
+      (HdfsServerConstants.BlockUCState s, DatanodeStorageInfo[] targets) {
+    BlockManager.LOG.error("convertCompleteBlockToUC should only be applied " +
+        "on complete blocks.");
+    return null;
+  }
+
+  @Override // BlockInfo
+  // BlockInfoUnderConstruction participates in maps the same way as BlockInfo
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  @Override // BlockInfo
+  public boolean equals(Object obj) {
+    // Sufficient to rely on super's implementation
+    return (this == obj) || super.equals(obj);
+  }
+
+  @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("]}");
+  }
+}

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c17439c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 02abc1d..dd7b301 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -219,8 +219,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /** A queue of blocks to be replicated by this datanode */
   private final BlockQueue<BlockTargetPair> replicateBlocks = new BlockQueue<BlockTargetPair>();
   /** A queue of blocks to be recovered by this datanode */
-  private final BlockQueue<BlockInfoContiguousUnderConstruction> recoverBlocks =
-                                new BlockQueue<BlockInfoContiguousUnderConstruction>();
+  private final BlockQueue<BlockInfoUnderConstruction> recoverBlocks =
+                                new BlockQueue<BlockInfoUnderConstruction>();
   /** A set of blocks to be invalidated by this datanode */
   private final LightWeightHashSet<Block> invalidateBlocks = new LightWeightHashSet<Block>();
 
@@ -599,7 +599,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * Store block recovery work.
    */
-  void addBlockToBeRecovered(BlockInfoContiguousUnderConstruction block) {
+  void addBlockToBeRecovered(BlockInfoUnderConstruction block) {
     if(recoverBlocks.contains(block)) {
       // this prevents adding the same block twice to the recovery queue
       BlockManager.LOG.info(block + " is already in the recovery queue");
@@ -641,11 +641,12 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return replicateBlocks.poll(maxTransfers);
   }
 
-  public BlockInfoContiguousUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
-    List<BlockInfoContiguousUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
+  public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(
+      int maxTransfers) {
+    List<BlockInfoUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
     if(blocks == null)
       return null;
-    return blocks.toArray(new BlockInfoContiguousUnderConstruction[blocks.size()]);
+    return blocks.toArray(new BlockInfoUnderConstruction[blocks.size()]);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c17439c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 58349cc..8143fb4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -1380,12 +1380,12 @@ public class DatanodeManager {
         }
 
         //check lease recovery
-        BlockInfoContiguousUnderConstruction[] blocks = nodeinfo
+        BlockInfoUnderConstruction[] blocks = nodeinfo
             .getLeaseRecoveryCommand(Integer.MAX_VALUE);
         if (blocks != null) {
           BlockRecoveryCommand brCommand = new BlockRecoveryCommand(
               blocks.length);
-          for (BlockInfoContiguousUnderConstruction b : blocks) {
+          for (BlockInfoUnderConstruction b : blocks) {
             final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations();
             // Skip stale nodes during recovery - not heart beated for some time (30s by default).
             final List<DatanodeStorageInfo> recoveryLocations =

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c17439c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
index 30517d0..2305e31 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
@@ -54,7 +54,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -777,8 +777,9 @@ public class FSImageFormat {
             // convert the last block to BlockUC
             if (blocks.length > 0) {
               BlockInfo lastBlk = blocks[blocks.length - 1];
-              blocks[blocks.length - 1] = new BlockInfoContiguousUnderConstruction(
-                  lastBlk, replication);
+              blocks[blocks.length - 1] =
+                  new BlockInfoUnderConstructionContiguous(
+                      lastBlk, replication);
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c17439c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
index e8378e5..d90751c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext;
@@ -364,8 +364,8 @@ public final class FSImageFormatPBINode {
         if (blocks.length > 0) {
           BlockInfo lastBlk = file.getLastBlock();
           // replace the last block of file
-          file.setBlock(file.numBlocks() - 1, new BlockInfoContiguousUnderConstruction(
-              lastBlk, replication));
+          file.setBlock(file.numBlocks() - 1,
+              new BlockInfoUnderConstructionContiguous(lastBlk, replication));
         }
       }
       return file;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c17439c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
index f71cf0b..2dc6252 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
@@ -137,7 +137,7 @@ public class FSImageSerialization {
     // last block is UNDER_CONSTRUCTION
     if(numBlocks > 0) {
       blk.readFields(in);
-      blocks[i] = new BlockInfoContiguousUnderConstruction(
+      blocks[i] = new BlockInfoUnderConstructionContiguous(
         blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
     }
     PermissionStatus perm = PermissionStatus.read(in);

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c17439c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index 3442e7b..40c4765 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.apache.hadoop.ipc.StandbyException;
@@ -45,5 +45,5 @@ public interface Namesystem extends RwLock, SafeMode {
 
   public void checkOperation(OperationCategory read) throws StandbyException;
 
-  public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC);
+  public boolean isInSnapshot(BlockInfoUnderConstruction blockUC);
 }
\ No newline at end of file