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 ji...@apache.org on 2015/06/01 20:43:24 UTC

hadoop git commit: HDFS-8489. Subclass BlockInfo to represent contiguous blocks. Contributed by Zhe Zhang.

Repository: hadoop
Updated Branches:
  refs/heads/trunk 0b5cfacde -> cdc13efb1


HDFS-8489. Subclass BlockInfo to represent contiguous blocks. 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/cdc13efb
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cdc13efb
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cdc13efb

Branch: refs/heads/trunk
Commit: cdc13efb1af54d931585d25c5ba696a012412828
Parents: 0b5cfac
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Jun 1 11:37:47 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Mon Jun 1 11:42:58 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/server/blockmanagement/BlockInfo.java  |  74 +++--------
 .../blockmanagement/BlockInfoContiguous.java    | 123 +++++++++++++++++++
 .../BlockInfoContiguousUnderConstruction.java   |   7 +-
 .../server/blockmanagement/BlockManager.java    |   5 +-
 .../hdfs/server/blockmanagement/BlocksMap.java  |  13 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |   3 +-
 .../hdfs/server/namenode/FSImageFormat.java     |   3 +-
 .../server/namenode/FSImageFormatPBINode.java   |   4 +-
 .../server/namenode/FSImageSerialization.java   |   3 +-
 .../snapshot/FSImageFormatPBSnapshot.java       |   3 +-
 .../server/blockmanagement/TestBlockInfo.java   |  14 +--
 .../blockmanagement/TestBlockManager.java       |   4 +-
 .../blockmanagement/TestDatanodeDescriptor.java |   4 +-
 .../blockmanagement/TestPendingReplication.java |   2 +-
 .../blockmanagement/TestReplicationPolicy.java  |   2 +-
 .../hdfs/server/namenode/CreateEditsLog.java    |   5 +-
 .../TestCommitBlockSynchronization.java         |   5 +-
 .../hdfs/server/namenode/TestINodeFile.java     |   3 +-
 .../namenode/TestTruncateQuotaUpdate.java       |   3 +-
 .../snapshot/TestFileWithSnapshotFeature.java   |   3 +-
 21 files changed, 186 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/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 10e5da4..1f7a36a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -588,6 +588,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8443. Document dfs.namenode.service.handler.count in hdfs-site.xml.
     (J.Andreina via aajisaka)
 
+    HDFS-8489. Subclass BlockInfo to represent contiguous blocks.
+    (Zhe Zhang via jing9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/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 0370c57..dea31c4 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
@@ -30,7 +30,7 @@ import org.apache.hadoop.util.LightWeightGSet;
  * the block are stored.
  */
 @InterfaceAudience.Private
-public class BlockInfo extends Block
+public abstract class  BlockInfo extends Block
     implements LightWeightGSet.LinkedElement {
   public static final BlockInfo[] EMPTY_ARRAY = {};
 
@@ -51,7 +51,7 @@ public class BlockInfo extends Block
    * per replica is 42 bytes (LinkedList#Entry object per replica) versus 16
    * bytes using the triplets.
    */
-  private Object[] triplets;
+  protected Object[] triplets;
 
   /**
    * Construct an entry for blocksmap
@@ -102,7 +102,7 @@ public class BlockInfo extends Block
     return (DatanodeStorageInfo)triplets[index*3];
   }
 
-  private BlockInfo getPrevious(int index) {
+  BlockInfo getPrevious(int index) {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
     BlockInfo info = (BlockInfo)triplets[index*3+1];
@@ -122,7 +122,7 @@ public class BlockInfo extends Block
     return info;
   }
 
-  private void setStorageInfo(int index, DatanodeStorageInfo storage) {
+  void setStorageInfo(int index, DatanodeStorageInfo storage) {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
     triplets[index*3] = storage;
@@ -136,7 +136,7 @@ public class BlockInfo extends Block
    * @param to - block to be set to previous on the list of blocks
    * @return current previous block on the list of blocks
    */
-  private BlockInfo setPrevious(int index, BlockInfo to) {
+  BlockInfo setPrevious(int index, BlockInfo to) {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
     BlockInfo info = (BlockInfo)triplets[index*3+1];
@@ -152,7 +152,7 @@ public class BlockInfo extends Block
    * @param to - block to be set to next on the list of blocks
    *    * @return current next block on the list of blocks
    */
-  private BlockInfo setNext(int index, BlockInfo to) {
+  BlockInfo setNext(int index, BlockInfo to) {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
     BlockInfo info = (BlockInfo)triplets[index*3+2];
@@ -167,68 +167,26 @@ public class BlockInfo extends Block
   }
 
   /**
-   * Ensure that there is enough  space to include num more triplets.
-   * @return first free triplet index.
+   * Count the number of data-nodes the block belongs to.
    */
-  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;
-  }
+  public abstract int numNodes();
 
   /**
-   * Count the number of data-nodes the block belongs to.
+   * Add a {@link DatanodeStorageInfo} location for a block.
    */
-  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;
-  }
+  abstract boolean addStorage(DatanodeStorageInfo storage);
 
   /**
-   * Add a {@link DatanodeStorageInfo} location for a block
+   * Remove {@link DatanodeStorageInfo} location for a block
    */
-  boolean addStorage(DatanodeStorageInfo storage) {
-    // find the last null node
-    int lastNode = ensureCapacity(1);
-    setStorageInfo(lastNode, storage);
-    setNext(lastNode, null);
-    setPrevious(lastNode, null);
-    return true;
-  }
+  abstract boolean removeStorage(DatanodeStorageInfo storage);
+
 
   /**
-   * Remove {@link DatanodeStorageInfo} location for a block
+   * Replace the current BlockInfo with the new one in corresponding
+   * DatanodeStorageInfo's linked list
    */
-  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;
-  }
+  abstract void replaceBlock(BlockInfo newBlock);
 
   /**
    * Find specified DatanodeStorageInfo.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/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
new file mode 100644
index 0000000..eff89a8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
@@ -0,0 +1,123 @@
+/**
+ * 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 org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.Block;
+
+/**
+ * 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);
+  }
+
+  public BlockInfoContiguous(Block blk, short size) {
+    super(blk, size);
+  }
+
+  /**
+   * Copy construction.
+   * This is used to convert BlockReplicationInfoUnderConstruction
+   * @param from BlockReplicationInfo to copy from.
+   */
+  protected BlockInfoContiguous(BlockInfoContiguous 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;
+  }
+
+  @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;
+  }
+
+  @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;
+  }
+
+  @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.";
+
+      final DatanodeStorageInfo.AddBlockResult result = storage.addBlock(
+          newBlock);
+      assert result == DatanodeStorageInfo.AddBlockResult.ADDED :
+          "newBlock already exists.";
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/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
index 58020a6..b757171 100644
--- 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
@@ -31,7 +31,7 @@ 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 BlockInfo {
+public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
   /** Block state. See {@link BlockUCState} */
   private BlockUCState blockUCState;
 
@@ -165,7 +165,8 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfo {
   /**
    * Create a block that is currently being constructed.
    */
-  public BlockInfoContiguousUnderConstruction(Block blk, short replication, BlockUCState state, DatanodeStorageInfo[] targets) {
+  public BlockInfoContiguousUnderConstruction(Block blk, short replication,
+      BlockUCState state, DatanodeStorageInfo[] targets) {
     super(blk, replication);
     assert getBlockUCState() != BlockUCState.COMPLETE :
       "BlockInfoUnderConstruction cannot be in COMPLETE state";
@@ -185,7 +186,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfo {
   BlockInfo convertToCompleteBlock() throws IOException {
     assert getBlockUCState() != BlockUCState.COMPLETE :
       "Trying to convert a COMPLETE block";
-    return new BlockInfo(this);
+    return new BlockInfoContiguous(this);
   }
 
   /** Set expected locations */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/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 45a485b..d18d7fe 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
@@ -1765,7 +1765,8 @@ public class BlockManager {
 
     BlockToMarkCorrupt(BlockInfo stored, long gs, String reason,
         Reason reasonCode) {
-      this(new BlockInfo(stored), stored, reason, reasonCode);
+      this(new BlockInfoContiguous((BlockInfoContiguous)stored), stored,
+          reason, reasonCode);
       //the corrupted block in datanode has a different generation stamp
       corrupted.setGenerationStamp(gs);
     }
@@ -2144,7 +2145,7 @@ public class BlockManager {
 
     // place a delimiter in the list which separates blocks 
     // that have been reported from those that have not
-    BlockInfo delimiter = new BlockInfo(new Block(), (short) 1);
+    BlockInfo delimiter = new BlockInfoContiguous(new Block(), (short) 1);
     AddBlockResult result = storageInfo.addBlock(delimiter);
     assert result == AddBlockResult.ADDED 
         : "Delimiting block cannot be present in the node";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
index d06d503..9eb1059 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
@@ -20,12 +20,10 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import java.util.Iterator;
 
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.util.GSet;
 import org.apache.hadoop.util.LightWeightGSet;
 
-import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 
@@ -222,16 +220,7 @@ class BlocksMap {
     BlockInfo currentBlock = blocks.get(newBlock);
     assert currentBlock != null : "the block if not in blocksMap";
     // replace block in data-node lists
-    for (int i = currentBlock.numNodes() - 1; i >= 0; i--) {
-      final DatanodeDescriptor dn = currentBlock.getDatanode(i);
-      final DatanodeStorageInfo storage = currentBlock.findStorageInfo(dn);
-      final boolean removed = storage.removeBlock(currentBlock);
-      Preconditions.checkState(removed, "currentBlock not found.");
-
-      final AddBlockResult result = storage.addBlock(newBlock);
-      Preconditions.checkState(result == AddBlockResult.ADDED,
-          "newBlock already exists.");
-    }
+    currentBlock.replaceBlock(newBlock);
     // replace block in the map itself
     blocks.put(newBlock);
     return newBlock;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/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 9acd81f..3de6760 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
@@ -44,6 +44,7 @@ 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.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.common.HdfsServerConstants.RollingUpgradeStartupOption;
@@ -1055,7 +1056,7 @@ public class FSEditLogLoader {
           // is only executed when loading edits written by prior
           // versions of Hadoop. Current versions always log
           // OP_ADD operations as each block is allocated.
-          newBI = new BlockInfo(newBlock,
+          newBI = new BlockInfoContiguous(newBlock,
               file.getPreferredBlockReplication());
         }
         fsNamesys.getBlockManager().addBlockCollection(newBI, file);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/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 a46f2b6..30517d0 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
@@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutFlags;
 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;
@@ -755,7 +756,7 @@ public class FSImageFormat {
       // read blocks
       BlockInfo[] blocks = new BlockInfo[numBlocks];
       for (int j = 0; j < numBlocks; j++) {
-        blocks[j] = new BlockInfo(replication);
+        blocks[j] = new BlockInfoContiguous(replication);
         blocks[j].readFields(in);
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/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 29713a2..1c14220 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
@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
@@ -325,7 +326,8 @@ public final class FSImageFormatPBINode {
 
       BlockInfo[] blocks = new BlockInfo[bp.size()];
       for (int i = 0, e = bp.size(); i < e; ++i) {
-        blocks[i] = new BlockInfo(PBHelper.convert(bp.get(i)), replication);
+        blocks[i] =
+            new BlockInfoContiguous(PBHelper.convert(bp.get(i)), replication);
       }
       final PermissionStatus permissions = loadPermission(f.getPermission(),
           parent.getLoaderContext().getStringTable());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/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 d790b8f..f71cf0b 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
@@ -34,6 +34,7 @@ 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.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.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
@@ -131,7 +132,7 @@ public class FSImageSerialization {
     int i = 0;
     for (; i < numBlocks-1; i++) {
       blk.readFields(in);
-      blocks[i] = new BlockInfo(blk, blockReplication);
+      blocks[i] = new BlockInfoContiguous(blk, blockReplication);
     }
     // last block is UNDER_CONSTRUCTION
     if(numBlocks > 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
index 363c208..64ad1f6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.namenode.AclEntryStatusFormat;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
@@ -245,7 +246,7 @@ public class FSImageFormatPBSnapshot {
           BlockInfo storedBlock =  fsn.getBlockManager().getStoredBlock(blk);
           if(storedBlock == null) {
             storedBlock = fsn.getBlockManager().addBlockCollection(
-                new BlockInfo(blk, copy.getFileReplication()), file);
+                new BlockInfoContiguous(blk, copy.getFileReplication()), file);
           }
           blocks[j] = storedBlock;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
index 96d4d8b..5126aa7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
@@ -49,7 +49,7 @@ public class TestBlockInfo {
 
   @Test
   public void testIsDeleted() {
-    BlockInfo blockInfo = new BlockInfo((short) 3);
+    BlockInfo blockInfo = new BlockInfoContiguous((short) 3);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
     blockInfo.setBlockCollection(bc);
     Assert.assertFalse(blockInfo.isDeleted());
@@ -59,7 +59,7 @@ public class TestBlockInfo {
 
   @Test
   public void testAddStorage() throws Exception {
-    BlockInfo blockInfo = new BlockInfo((short) 3);
+    BlockInfo blockInfo = new BlockInfoContiguous((short) 3);
 
     final DatanodeStorageInfo storage = DFSTestUtil.createDatanodeStorageInfo("storageID", "127.0.0.1");
 
@@ -71,9 +71,9 @@ public class TestBlockInfo {
 
   @Test
   public void testCopyConstructor() {
-    BlockInfo old = new BlockInfo((short) 3);
+    BlockInfo old = new BlockInfoContiguous((short) 3);
     try {
-      BlockInfo copy = new BlockInfo(old);
+      BlockInfo copy = new BlockInfoContiguous((BlockInfoContiguous)old);
       assertEquals(old.getBlockCollection(), copy.getBlockCollection());
       assertEquals(old.getCapacity(), copy.getCapacity());
     } catch (Exception e) {
@@ -92,7 +92,7 @@ public class TestBlockInfo {
 
     // Create a few dummy blocks and add them to the first storage.
     for (int i = 0; i < NUM_BLOCKS; ++i) {
-      blockInfos[i] = new BlockInfo((short) 3);
+      blockInfos[i] = new BlockInfoContiguous((short) 3);
       storage1.addBlock(blockInfos[i]);
     }
 
@@ -118,7 +118,7 @@ public class TestBlockInfo {
     LOG.info("Building block list...");
     for (int i = 0; i < MAX_BLOCKS; i++) {
       blockList.add(new Block(i, 0, GenerationStamp.LAST_RESERVED_STAMP));
-      blockInfoList.add(new BlockInfo(blockList.get(i), (short) 3));
+      blockInfoList.add(new BlockInfoContiguous(blockList.get(i), (short) 3));
       dd.addBlock(blockInfoList.get(i));
 
       // index of the datanode should be 0
@@ -182,4 +182,4 @@ public class TestBlockInfo {
           blockInfoList.get(j), dd.getBlockListHeadForTesting());
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index bf46ed7..fd03745 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -389,7 +389,7 @@ public class TestBlockManager {
 
   private BlockInfo blockOnNodes(long blkId, List<DatanodeDescriptor> nodes) {
     Block block = new Block(blkId);
-    BlockInfo blockInfo = new BlockInfo(block, (short) 3);
+    BlockInfo blockInfo = new BlockInfoContiguous(block, (short) 3);
 
     for (DatanodeDescriptor dn : nodes) {
       for (DatanodeStorageInfo storage : dn.getStorageInfos()) {
@@ -738,7 +738,7 @@ public class TestBlockManager {
   private BlockInfo addBlockToBM(long blkId) {
     Block block = new Block(blkId);
     BlockInfo blockInfo =
-        new BlockInfo(block, (short) 3);
+        new BlockInfoContiguous(block, (short) 3);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
     Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication();
     bm.blocksMap.addBlockCollection(blockInfo, bc);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeDescriptor.java
index fe639e4..ca27bb7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeDescriptor.java
@@ -58,8 +58,8 @@ public class TestDatanodeDescriptor {
   public void testBlocksCounter() throws Exception {
     DatanodeDescriptor dd = BlockManagerTestUtil.getLocalDatanodeDescriptor(true);
     assertEquals(0, dd.numBlocks());
-    BlockInfo blk = new BlockInfo(new Block(1L), (short) 1);
-    BlockInfo blk1 = new BlockInfo(new Block(2L), (short) 2);
+    BlockInfo blk = new BlockInfoContiguous(new Block(1L), (short) 1);
+    BlockInfo blk1 = new BlockInfoContiguous(new Block(2L), (short) 2);
     DatanodeStorageInfo[] storages = dd.getStorageInfos();
     assertTrue(storages.length > 0);
     // add first block

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
index 844c5d0..090c6e8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
@@ -178,7 +178,7 @@ public class TestPendingReplication {
       //
 
       block = new Block(1, 1, 0);
-      blockInfo = new BlockInfo(block, (short) 3);
+      blockInfo = new BlockInfoContiguous(block, (short) 3);
 
       pendingReplications.increment(block,
           DatanodeStorageInfo.toDatanodeDescriptors(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index f317723..fccaf3c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@ -1218,7 +1218,7 @@ public class TestReplicationPolicy {
     chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
 
-    final BlockInfo info = new BlockInfo(block1, (short) 1);
+    final BlockInfo info = new BlockInfoContiguous(block1, (short) 1);
     final BlockCollection mbc = mock(BlockCollection.class);
     when(mbc.getLastBlock()).thenReturn(info);
     when(mbc.getPreferredBlockSize()).thenReturn(block1.getNumBytes() + 1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
index 3f96c0c..8a606f5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 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.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.Storage;
 
@@ -69,8 +70,8 @@ public class CreateEditsLog {
     BlockInfo[] blocks = new BlockInfo[blocksPerFile];
     for (int iB = 0; iB < blocksPerFile; ++iB) {
       blocks[iB] = 
-       new BlockInfo(new Block(0, blockSize, BLOCK_GENERATION_STAMP),
-                               replication);
+       new BlockInfoContiguous(new Block(0, blockSize, BLOCK_GENERATION_STAMP),
+           replication);
     }
     
     long currentBlockId = startingBlockId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
index 8c9da01..bea7241 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 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;
@@ -106,7 +107,7 @@ public class TestCommitBlockSynchronization {
         lastBlock, genStamp, length, false, false, newTargets, null);
 
     // Simulate 'completing' the block.
-    BlockInfo completedBlockInfo = new BlockInfo(block, (short) 1);
+    BlockInfo completedBlockInfo = new BlockInfoContiguous(block, (short) 1);
     completedBlockInfo.setBlockCollection(file);
     completedBlockInfo.setGenerationStamp(genStamp);
     doReturn(completedBlockInfo).when(namesystemSpy)
@@ -178,7 +179,7 @@ public class TestCommitBlockSynchronization {
     namesystemSpy.commitBlockSynchronization(
         lastBlock, genStamp, length, true, false, newTargets, null);
 
-    BlockInfo completedBlockInfo = new BlockInfo(block, (short) 1);
+    BlockInfo completedBlockInfo = new BlockInfoContiguous(block, (short) 1);
     completedBlockInfo.setBlockCollection(file);
     completedBlockInfo.setGenerationStamp(genStamp);
     doReturn(completedBlockInfo).when(namesystemSpy)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
index 21b67a7..ec4de1b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
@@ -61,6 +61,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.io.IOUtils;
@@ -290,7 +291,7 @@ public class TestINodeFile {
       iNodes[i] = new INodeFile(i, null, perm, 0L, 0L, null, replication,
           preferredBlockSize);
       iNodes[i].setLocalName(DFSUtil.string2Bytes(fileNamePrefix + i));
-      BlockInfo newblock = new BlockInfo(replication);
+      BlockInfo newblock = new BlockInfoContiguous(replication);
       iNodes[i].addBlock(newblock);
     }
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
index 0d726e9..106edad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 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.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature;
@@ -148,7 +149,7 @@ public class TestTruncateQuotaUpdate {
 
   private BlockInfo newBlock(long size, short replication) {
     Block b = new Block(++nextMockBlockId, size, ++nextMockGenstamp);
-    return new BlockInfo(b, replication);
+    return new BlockInfoContiguous(b, replication);
   }
 
   private static void addSnapshotFeature(INodeFile file, BlockInfo[] blocks) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdc13efb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java
index ac81488..0d6298f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java
@@ -21,6 +21,7 @@ import com.google.common.collect.Lists;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
@@ -50,7 +51,7 @@ public class TestFileWithSnapshotFeature {
     BlockStoragePolicySuite bsps = mock(BlockStoragePolicySuite.class);
     BlockStoragePolicy bsp = mock(BlockStoragePolicy.class);
     BlockInfo[] blocks = new BlockInfo[] {
-        new BlockInfo(new Block(1, BLOCK_SIZE, 1), REPL_1)
+        new BlockInfoContiguous(new Block(1, BLOCK_SIZE, 1), REPL_1)
     };
 
     // No snapshot