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/02/08 20:53:02 UTC

[1/3] hadoop git commit: HDFS-7743. Code cleanup of BlockInfo and rename BlockInfo to BlockInfoContiguous. Contributed by Jing Zhao.

Repository: hadoop
Updated Branches:
  refs/heads/trunk ef0176833 -> 1382ae525


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 8d0589c..81f6ae5 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
@@ -33,8 +33,8 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 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.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
@@ -115,17 +115,17 @@ public class INodeFile extends INodeWithAdditionalFields
 
   private long header = 0L;
 
-  private BlockInfo[] blocks;
+  private BlockInfoContiguous[] blocks;
 
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
-            long atime, BlockInfo[] blklist, short replication,
+            long atime, BlockInfoContiguous[] blklist, short replication,
             long preferredBlockSize) {
     this(id, name, permissions, mtime, atime, blklist, replication,
          preferredBlockSize, (byte) 0);
   }
 
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
-      long atime, BlockInfo[] blklist, short replication,
+      long atime, BlockInfoContiguous[] blklist, short replication,
       long preferredBlockSize, byte storagePolicyID) {
     super(id, name, permissions, mtime, atime);
     header = HeaderFormat.toLong(preferredBlockSize, replication, storagePolicyID);
@@ -220,20 +220,21 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   @Override // BlockCollection
-  public void setBlock(int index, BlockInfo blk) {
+  public void setBlock(int index, BlockInfoContiguous blk) {
     this.blocks[index] = blk;
   }
 
   @Override // BlockCollection, the file should be under construction
-  public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
-      DatanodeStorageInfo[] locations) throws IOException {
+  public BlockInfoContiguousUnderConstruction setLastBlock(
+      BlockInfoContiguous lastBlock, DatanodeStorageInfo[] locations)
+      throws IOException {
     Preconditions.checkState(isUnderConstruction(),
         "file is no longer under construction");
 
     if (numBlocks() == 0) {
       throw new IOException("Failed to set last block: File is empty.");
     }
-    BlockInfoUnderConstruction ucBlock =
+    BlockInfoContiguousUnderConstruction ucBlock =
       lastBlock.convertToBlockUnderConstruction(
           BlockUCState.UNDER_CONSTRUCTION, locations);
     setBlock(numBlocks() - 1, ucBlock);
@@ -256,7 +257,7 @@ public class INodeFile extends INodeWithAdditionalFields
     }
 
     //copy to a new list
-    BlockInfo[] newlist = new BlockInfo[size_1];
+    BlockInfoContiguous[] newlist = new BlockInfoContiguous[size_1];
     System.arraycopy(blocks, 0, newlist, 0, size_1);
     setBlocks(newlist);
     return true;
@@ -416,16 +417,17 @@ public class INodeFile extends INodeWithAdditionalFields
 
   /** @return the blocks of the file. */
   @Override
-  public BlockInfo[] getBlocks() {
+  public BlockInfoContiguous[] getBlocks() {
     return this.blocks;
   }
 
   /** @return blocks of the file corresponding to the snapshot. */
-  public BlockInfo[] getBlocks(int snapshot) {
+  public BlockInfoContiguous[] getBlocks(int snapshot) {
     if(snapshot == CURRENT_STATE_ID || getDiffs() == null)
       return getBlocks();
     FileDiff diff = getDiffs().getDiffById(snapshot);
-    BlockInfo[] snapshotBlocks = diff == null ? getBlocks() : diff.getBlocks();
+    BlockInfoContiguous[] snapshotBlocks =
+        diff == null ? getBlocks() : diff.getBlocks();
     if(snapshotBlocks != null)
       return snapshotBlocks;
     // Blocks are not in the current snapshot
@@ -436,7 +438,7 @@ public class INodeFile extends INodeWithAdditionalFields
 
   void updateBlockCollection() {
     if (blocks != null) {
-      for(BlockInfo b : blocks) {
+      for(BlockInfoContiguous b : blocks) {
         b.setBlockCollection(this);
       }
     }
@@ -452,7 +454,8 @@ public class INodeFile extends INodeWithAdditionalFields
       totalAddedBlocks += f.blocks.length;
     }
     
-    BlockInfo[] newlist = new BlockInfo[size + totalAddedBlocks];
+    BlockInfoContiguous[] newlist =
+        new BlockInfoContiguous[size + totalAddedBlocks];
     System.arraycopy(this.blocks, 0, newlist, 0, size);
     
     for(INodeFile in: inodes) {
@@ -467,12 +470,12 @@ public class INodeFile extends INodeWithAdditionalFields
   /**
    * add a block to the block list
    */
-  void addBlock(BlockInfo newblock) {
+  void addBlock(BlockInfoContiguous newblock) {
     if (this.blocks == null) {
-      this.setBlocks(new BlockInfo[]{newblock});
+      this.setBlocks(new BlockInfoContiguous[]{newblock});
     } else {
       int size = this.blocks.length;
-      BlockInfo[] newlist = new BlockInfo[size + 1];
+      BlockInfoContiguous[] newlist = new BlockInfoContiguous[size + 1];
       System.arraycopy(this.blocks, 0, newlist, 0, size);
       newlist[size] = newblock;
       this.setBlocks(newlist);
@@ -480,7 +483,7 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   /** Set the blocks. */
-  public void setBlocks(BlockInfo[] blocks) {
+  public void setBlocks(BlockInfoContiguous[] blocks) {
     this.blocks = blocks;
   }
 
@@ -516,7 +519,7 @@ public class INodeFile extends INodeWithAdditionalFields
   public void destroyAndCollectBlocks(BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) {
     if (blocks != null && collectedBlocks != null) {
-      for (BlockInfo blk : blocks) {
+      for (BlockInfoContiguous blk : blocks) {
         collectedBlocks.addDeleteBlock(blk);
         blk.setBlockCollection(null);
       }
@@ -637,7 +640,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 BlockInfoUnderConstruction) {
+    if (blocks[last] instanceof BlockInfoContiguousUnderConstruction) {
        if (!includesLastUcBlock) {
          size = 0;
        } else if (usePreferredBlockSize4LastUcBlock) {
@@ -667,7 +670,7 @@ public class INodeFile extends INodeWithAdditionalFields
     Set<Block> allBlocks = new HashSet<Block>(Arrays.asList(getBlocks()));
     List<FileDiff> diffs = sf.getDiffs().asList();
     for(FileDiff diff : diffs) {
-      BlockInfo[] diffBlocks = diff.getBlocks();
+      BlockInfoContiguous[] diffBlocks = diff.getBlocks();
       if (diffBlocks != null) {
         allBlocks.addAll(Arrays.asList(diffBlocks));
       }
@@ -676,8 +679,9 @@ public class INodeFile extends INodeWithAdditionalFields
       size += block.getNumBytes();
     }
     // check if the last block is under construction
-    BlockInfo lastBlock = getLastBlock();
-    if(lastBlock != null && lastBlock instanceof BlockInfoUnderConstruction) {
+    BlockInfoContiguous lastBlock = getLastBlock();
+    if(lastBlock != null &&
+        lastBlock instanceof BlockInfoContiguousUnderConstruction) {
       size += getPreferredBlockSize() - lastBlock.getNumBytes();
     }
     return size * getBlockReplication();
@@ -695,7 +699,7 @@ public class INodeFile extends INodeWithAdditionalFields
   /**
    * Return the penultimate allocated block for this file.
    */
-  BlockInfo getPenultimateBlock() {
+  BlockInfoContiguous getPenultimateBlock() {
     if (blocks == null || blocks.length <= 1) {
       return null;
     }
@@ -703,7 +707,7 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   @Override
-  public BlockInfo getLastBlock() {
+  public BlockInfoContiguous getLastBlock() {
     return blocks == null || blocks.length == 0? null: blocks[blocks.length-1];
   }
 
@@ -730,7 +734,7 @@ public class INodeFile extends INodeWithAdditionalFields
    */
   public long collectBlocksBeyondMax(final long max,
       final BlocksMapUpdateInfo collectedBlocks) {
-    final BlockInfo[] oldBlocks = getBlocks();
+    final BlockInfoContiguous[] oldBlocks = getBlocks();
     if (oldBlocks == null)
       return 0;
     // find the minimum n such that the size of the first n blocks > max
@@ -756,20 +760,20 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   void truncateBlocksTo(int n) {
-    final BlockInfo[] newBlocks;
+    final BlockInfoContiguous[] newBlocks;
     if (n == 0) {
-      newBlocks = BlockInfo.EMPTY_ARRAY;
+      newBlocks = BlockInfoContiguous.EMPTY_ARRAY;
     } else {
-      newBlocks = new BlockInfo[n];
+      newBlocks = new BlockInfoContiguous[n];
       System.arraycopy(getBlocks(), 0, newBlocks, 0, n);
     }
     // set new blocks
     setBlocks(newBlocks);
   }
 
-  public void collectBlocksBeyondSnapshot(BlockInfo[] snapshotBlocks,
+  public void collectBlocksBeyondSnapshot(BlockInfoContiguous[] snapshotBlocks,
                                           BlocksMapUpdateInfo collectedBlocks) {
-    BlockInfo[] oldBlocks = getBlocks();
+    BlockInfoContiguous[] oldBlocks = getBlocks();
     if(snapshotBlocks == null || oldBlocks == null)
       return;
     // Skip blocks in common between the file and the snapshot
@@ -793,7 +797,7 @@ public class INodeFile extends INodeWithAdditionalFields
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if(sf == null)
       return;
-    BlockInfo[] snapshotBlocks = 
+    BlockInfoContiguous[] snapshotBlocks =
         getDiffs().findEarlierSnapshotBlocks(snapshotId);
     if(snapshotBlocks == null)
       return;
@@ -807,14 +811,14 @@ public class INodeFile extends INodeWithAdditionalFields
   /**
    * @return true if the block is contained in a snapshot or false otherwise.
    */
-  boolean isBlockInLatestSnapshot(BlockInfo block) {
+  boolean isBlockInLatestSnapshot(BlockInfoContiguous block) {
     FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
-    if (sf == null || sf.getDiffs() == null)
+    if (sf == null || sf.getDiffs() == null) {
       return false;
-    BlockInfo[] snapshotBlocks =
-        getDiffs().findEarlierSnapshotBlocks(getDiffs().getLastSnapshotId());
-    if(snapshotBlocks == null)
-      return false;
-    return Arrays.asList(snapshotBlocks).contains(block);
+    }
+    BlockInfoContiguous[] snapshotBlocks = getDiffs()
+        .findEarlierSnapshotBlocks(getDiffs().getLastSnapshotId());
+    return snapshotBlocks != null &&
+        Arrays.asList(snapshotBlocks).contains(block);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
index f076215..0dafaae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.util.Daemon;
 
@@ -120,10 +120,10 @@ public class LeaseManager {
         } catch (UnresolvedLinkException e) {
           throw new AssertionError("Lease files should reside on this FS");
         }
-        BlockInfo[] blocks = cons.getBlocks();
+        BlockInfoContiguous[] blocks = cons.getBlocks();
         if(blocks == null)
           continue;
-        for(BlockInfo b : blocks) {
+        for(BlockInfoContiguous b : blocks) {
           if(!b.isComplete())
             numUCBlocks++;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 5eddeea..dc9494d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -61,7 +61,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactor
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus;
@@ -230,7 +230,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       //get blockInfo
       Block block = new Block(Block.getBlockId(blockId));
       //find which file this block belongs to
-      BlockInfo blockInfo = bm.getStoredBlock(block);
+      BlockInfoContiguous blockInfo = bm.getStoredBlock(block);
       if(blockInfo == null) {
         out.println("Block "+ blockId +" " + NONEXISTENT_STATUS);
         LOG.warn("Block "+ blockId + " " + NONEXISTENT_STATUS);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 40c4765..3442e7b 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.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 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(BlockInfoUnderConstruction blockUC);
+  public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 7ac85cd..ebe4603 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
@@ -39,7 +39,7 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 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;
@@ -234,13 +234,13 @@ public class FSImageFormatPBSnapshot {
         FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,
             pbf.getFileSize());
         List<BlockProto> bpl = pbf.getBlocksList();
-        BlockInfo[] blocks = new BlockInfo[bpl.size()];
+        BlockInfoContiguous[] blocks = new BlockInfoContiguous[bpl.size()];
         for(int j = 0, e = bpl.size(); j < e; ++j) {
           Block blk = PBHelper.convert(bpl.get(j));
-          BlockInfo storedBlock =  fsn.getBlockManager().getStoredBlock(blk);
+          BlockInfoContiguous 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/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java
index 7b52dc9..10bba67 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
 
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@@ -40,7 +40,7 @@ public class FileDiff extends
   /** The file size at snapshot creation time. */
   private final long fileSize;
   /** A copy of the INodeFile block list. Used in truncate. */
-  private BlockInfo[] blocks;
+  private BlockInfoContiguous[] blocks;
 
   FileDiff(int snapshotId, INodeFile file) {
     super(snapshotId, null, null);
@@ -66,7 +66,7 @@ public class FileDiff extends
    * up to the current {@link #fileSize}.
    * Should be done only once.
    */
-  public void setBlocks(BlockInfo[] blocks) {
+  public void setBlocks(BlockInfoContiguous[] blocks) {
     if(this.blocks != null)
       return;
     int numBlocks = 0;
@@ -75,7 +75,7 @@ public class FileDiff extends
     this.blocks = Arrays.copyOf(blocks, numBlocks);
   }
 
-  public BlockInfo[] getBlocks() {
+  public BlockInfoContiguous[] getBlocks() {
     return blocks;
   }
 
@@ -121,7 +121,7 @@ public class FileDiff extends
       BlocksMapUpdateInfo collectedBlocks) {
     if(blocks == null || collectedBlocks == null)
       return;
-    for(BlockInfo blk : blocks)
+    for(BlockInfoContiguous blk : blocks)
       collectedBlocks.addDeleteBlock(blk);
     blocks = null;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
index c7af52c..5ea23dc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
@@ -54,14 +54,14 @@ public class FileDiffList extends
       diff.setBlocks(iNodeFile.getBlocks());
   }
 
-  public BlockInfo[] findEarlierSnapshotBlocks(int snapshotId) {
+  public BlockInfoContiguous[] findEarlierSnapshotBlocks(int snapshotId) {
     assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id";
     if(snapshotId == Snapshot.CURRENT_STATE_ID) {
       return null;
     }
     List<FileDiff> diffs = this.asList();
     int i = Collections.binarySearch(diffs, snapshotId);
-    BlockInfo[] blocks = null;
+    BlockInfoContiguous[] blocks = null;
     for(i = i >= 0 ? i : -i; i < diffs.size(); i--) {
       blocks = diffs.get(i).getBlocks();
       if(blocks != null) {
@@ -71,14 +71,14 @@ public class FileDiffList extends
     return blocks;
   }
 
-  public BlockInfo[] findLaterSnapshotBlocks(int snapshotId) {
+  public BlockInfoContiguous[] findLaterSnapshotBlocks(int snapshotId) {
     assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id";
     if(snapshotId == Snapshot.CURRENT_STATE_ID) {
       return null;
     }
     List<FileDiff> diffs = this.asList();
     int i = Collections.binarySearch(diffs, snapshotId);
-    BlockInfo[] blocks = null;
+    BlockInfoContiguous[] blocks = null;
     for(i = i >= 0 ? i+1 : -i-1; i < diffs.size(); i++) {
       blocks = diffs.get(i).getBlocks();
       if(blocks != null) {
@@ -97,7 +97,7 @@ public class FileDiffList extends
                                        FileDiff removed,
                                        BlocksMapUpdateInfo collectedBlocks,
                                        List<INode> removedINodes) {
-    BlockInfo[] removedBlocks = removed.getBlocks();
+    BlockInfoContiguous[] removedBlocks = removed.getBlocks();
     if(removedBlocks == null) {
       FileWithSnapshotFeature sf = file.getFileWithSnapshotFeature();
       assert sf != null : "FileWithSnapshotFeature is null";
@@ -110,10 +110,10 @@ public class FileDiffList extends
     // Copy blocks to the previous snapshot if not set already
     if(earlierDiff != null)
       earlierDiff.setBlocks(removedBlocks);
-    BlockInfo[] earlierBlocks =
-        (earlierDiff == null ? new BlockInfo[]{} : earlierDiff.getBlocks());
+    BlockInfoContiguous[] earlierBlocks =
+        (earlierDiff == null ? new BlockInfoContiguous[]{} : earlierDiff.getBlocks());
     // Find later snapshot (or file itself) with blocks
-    BlockInfo[] laterBlocks = findLaterSnapshotBlocks(removed.getSnapshotId());
+    BlockInfoContiguous[] laterBlocks = findLaterSnapshotBlocks(removed.getSnapshotId());
     laterBlocks = (laterBlocks==null) ? file.getBlocks() : laterBlocks;
     // Skip blocks, which belong to either the earlier or the later lists
     int i = 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
index 61e31c2..419557b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@@ -183,7 +183,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
 
     // Collect blocks that should be deleted
     FileDiff last = diffs.getLast();
-    BlockInfo[] snapshotBlocks = last == null ? null : last.getBlocks();
+    BlockInfoContiguous[] snapshotBlocks = last == null ? null : last.getBlocks();
     if(snapshotBlocks == null)
       file.collectBlocksBeyondMax(max, info);
     else

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 1af3ef2..e8fc662 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -47,8 +47,8 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+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.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -1521,12 +1521,12 @@ public class DFSTestUtil {
   public static DatanodeDescriptor getExpectedPrimaryNode(NameNode nn,
       ExtendedBlock blk) {
     BlockManager bm0 = nn.getNamesystem().getBlockManager();
-    BlockInfo storedBlock = bm0.getStoredBlock(blk.getLocalBlock());
+    BlockInfoContiguous storedBlock = bm0.getStoredBlock(blk.getLocalBlock());
     assertTrue("Block " + blk + " should be under construction, " +
         "got: " + storedBlock,
-        storedBlock instanceof BlockInfoUnderConstruction);
-    BlockInfoUnderConstruction ucBlock =
-      (BlockInfoUnderConstruction)storedBlock;
+        storedBlock instanceof BlockInfoContiguousUnderConstruction);
+    BlockInfoContiguousUnderConstruction ucBlock =
+      (BlockInfoContiguousUnderConstruction)storedBlock;
     // We expect that the replica with the most recent heart beat will be
     // the one to be in charge of the synchronization / recovery protocol.
     final DatanodeStorageInfo[] storages = ucBlock.getExpectedStorageLocations();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 41c8f8a..7425c6a 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 testAddStorage() throws Exception {
-    BlockInfo blockInfo = new BlockInfo((short) 3);
+    BlockInfoContiguous blockInfo = new BlockInfoContiguous((short) 3);
 
     final DatanodeStorageInfo storage = DFSTestUtil.createDatanodeStorageInfo("storageID", "127.0.0.1");
 
@@ -67,11 +67,11 @@ public class TestBlockInfo {
     final DatanodeStorageInfo storage1 = DFSTestUtil.createDatanodeStorageInfo("storageID1", "127.0.0.1");
     final DatanodeStorageInfo storage2 = new DatanodeStorageInfo(storage1.getDatanodeDescriptor(), new DatanodeStorage("storageID2"));
     final int NUM_BLOCKS = 10;
-    BlockInfo[] blockInfos = new BlockInfo[NUM_BLOCKS];
+    BlockInfoContiguous[] blockInfos = new BlockInfoContiguous[NUM_BLOCKS];
 
     // 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]);
     }
 
@@ -90,14 +90,14 @@ public class TestBlockInfo {
 
     DatanodeStorageInfo dd = DFSTestUtil.createDatanodeStorageInfo("s1", "1.1.1.1");
     ArrayList<Block> blockList = new ArrayList<Block>(MAX_BLOCKS);
-    ArrayList<BlockInfo> blockInfoList = new ArrayList<BlockInfo>();
+    ArrayList<BlockInfoContiguous> blockInfoList = new ArrayList<BlockInfoContiguous>();
     int headIndex;
     int curIndex;
 
     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
@@ -108,7 +108,7 @@ public class TestBlockInfo {
     // list length should be equal to the number of blocks we inserted
     LOG.info("Checking list length...");
     assertEquals("Length should be MAX_BLOCK", MAX_BLOCKS, dd.numBlocks());
-    Iterator<BlockInfo> it = dd.getBlockIterator();
+    Iterator<BlockInfoContiguous> it = dd.getBlockIterator();
     int len = 0;
     while (it.hasNext()) {
       it.next();
@@ -130,7 +130,7 @@ public class TestBlockInfo {
     // move head of the list to the head - this should not change the list
     LOG.info("Moving head to the head...");
 
-    BlockInfo temp = dd.getBlockListHeadForTesting();
+    BlockInfoContiguous temp = dd.getBlockListHeadForTesting();
     curIndex = 0;
     headIndex = 0;
     dd.moveBlockToHead(temp, curIndex, headIndex);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
index 4c36448..453f411 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
@@ -39,7 +39,7 @@ public class TestBlockInfoUnderConstruction {
     DatanodeDescriptor dd3 = s3.getDatanodeDescriptor();
 
     dd1.isAlive = dd2.isAlive = dd3.isAlive = true;
-    BlockInfoUnderConstruction blockInfo = new BlockInfoUnderConstruction(
+    BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
         new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP),
         (short) 3,
         BlockUCState.UNDER_CONSTRUCTION,
@@ -51,7 +51,7 @@ public class TestBlockInfoUnderConstruction {
     dd2.setLastUpdate(currentTime - 1 * 1000);
     dd3.setLastUpdate(currentTime - 2 * 1000);
     blockInfo.initializeBlockRecovery(1);
-    BlockInfoUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
+    BlockInfoContiguousUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
     assertEquals(blockInfoRecovery[0], blockInfo);
 
     // Recovery attempt #2.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 024d00e..c7dfcf9 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
@@ -141,7 +141,7 @@ public class TestBlockManager {
   private void doBasicTest(int testIndex) {
     List<DatanodeStorageInfo> origStorages = getStorages(0, 1);
     List<DatanodeDescriptor> origNodes = getNodes(origStorages);
-    BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
+    BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes);
 
     DatanodeStorageInfo[] pipeline = scheduleSingleReplication(blockInfo);
     assertEquals(2, pipeline.length);
@@ -173,7 +173,7 @@ public class TestBlockManager {
     // Block originally on A1, A2, B1
     List<DatanodeStorageInfo> origStorages = getStorages(0, 1, 3);
     List<DatanodeDescriptor> origNodes = getNodes(origStorages);
-    BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
+    BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes);
     
     // Decommission two of the nodes (A1, A2)
     List<DatanodeDescriptor> decomNodes = startDecommission(0, 1);
@@ -217,7 +217,7 @@ public class TestBlockManager {
     // Block originally on A1, A2, B1
     List<DatanodeStorageInfo> origStorages = getStorages(0, 1, 3);
     List<DatanodeDescriptor> origNodes = getNodes(origStorages);
-    BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
+    BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes);
     
     // Decommission all of the nodes
     List<DatanodeDescriptor> decomNodes = startDecommission(0, 1, 3);
@@ -270,7 +270,7 @@ public class TestBlockManager {
     // Block originally on A1, A2, B1
     List<DatanodeStorageInfo> origStorages = getStorages(0, 1, 3);
     List<DatanodeDescriptor> origNodes = getNodes(origStorages);
-    BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
+    BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes);
     
     // Decommission all of the nodes in rack A
     List<DatanodeDescriptor> decomNodes = startDecommission(0, 1, 2);
@@ -329,7 +329,7 @@ public class TestBlockManager {
   private void doTestSufficientlyReplBlocksUsesNewRack(int testIndex) {
     // Originally on only nodes in rack A.
     List<DatanodeDescriptor> origNodes = rackA;
-    BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
+    BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes);
     DatanodeStorageInfo pipeline[] = scheduleSingleReplication(blockInfo);
     
     assertEquals(2, pipeline.length); // single new copy
@@ -372,7 +372,7 @@ public class TestBlockManager {
    * Tell the block manager that replication is completed for the given
    * pipeline.
    */
-  private void fulfillPipeline(BlockInfo blockInfo,
+  private void fulfillPipeline(BlockInfoContiguous blockInfo,
       DatanodeStorageInfo[] pipeline) throws IOException {
     for (int i = 1; i < pipeline.length; i++) {
       DatanodeStorageInfo storage = pipeline[i];
@@ -381,9 +381,9 @@ public class TestBlockManager {
     }
   }
 
-  private BlockInfo blockOnNodes(long blkId, List<DatanodeDescriptor> nodes) {
+  private BlockInfoContiguous blockOnNodes(long blkId, List<DatanodeDescriptor> nodes) {
     Block block = new Block(blkId);
-    BlockInfo blockInfo = new BlockInfo(block, (short) 3);
+    BlockInfoContiguous blockInfo = new BlockInfoContiguous(block, (short) 3);
 
     for (DatanodeDescriptor dn : nodes) {
       for (DatanodeStorageInfo storage : dn.getStorageInfos()) {
@@ -425,10 +425,10 @@ public class TestBlockManager {
     return nodes;
   }
   
-  private BlockInfo addBlockOnNodes(long blockId, List<DatanodeDescriptor> nodes) {
+  private BlockInfoContiguous addBlockOnNodes(long blockId, List<DatanodeDescriptor> nodes) {
     BlockCollection bc = Mockito.mock(BlockCollection.class);
     Mockito.doReturn((short)3).when(bc).getBlockReplication();
-    BlockInfo blockInfo = blockOnNodes(blockId, nodes);
+    BlockInfoContiguous blockInfo = blockOnNodes(blockId, nodes);
 
     bm.blocksMap.addBlockCollection(blockInfo, bc);
     return blockInfo;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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..7cdb423 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);
+    BlockInfoContiguous blk = new BlockInfoContiguous(new Block(1L), (short) 1);
+    BlockInfoContiguous 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/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
index 988a0ed..efd1feb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
@@ -171,7 +171,7 @@ public class TestHeartbeatHandling {
               dd1.getStorageInfos()[0],
               dd2.getStorageInfos()[0],
               dd3.getStorageInfos()[0]};
-          BlockInfoUnderConstruction blockInfo = new BlockInfoUnderConstruction(
+          BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
               new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
               BlockUCState.UNDER_RECOVERY, storages);
           dd1.addBlockToBeRecovered(blockInfo);
@@ -193,7 +193,7 @@ public class TestHeartbeatHandling {
           // More than the default stale interval of 30 seconds.
           dd2.setLastUpdate(System.currentTimeMillis() - 40 * 1000);
           dd3.setLastUpdate(System.currentTimeMillis());
-          blockInfo = new BlockInfoUnderConstruction(
+          blockInfo = new BlockInfoContiguousUnderConstruction(
               new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
               BlockUCState.UNDER_RECOVERY, storages);
           dd1.addBlockToBeRecovered(blockInfo);
@@ -214,7 +214,7 @@ public class TestHeartbeatHandling {
           // More than the default stale interval of 30 seconds.
           dd2.setLastUpdate(System.currentTimeMillis() - 40 * 1000);
           dd3.setLastUpdate(System.currentTimeMillis() - 80 * 1000);
-          blockInfo = new BlockInfoUnderConstruction(
+          blockInfo = new BlockInfoContiguousUnderConstruction(
               new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
               BlockUCState.UNDER_RECOVERY, storages);
           dd1.addBlockToBeRecovered(blockInfo);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 7a4960e..1c008ac 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
@@ -1167,7 +1167,7 @@ public class TestReplicationPolicy {
     // block under construction, the BlockManager will realize the expected
     // replication has been achieved and remove it from the under-replicated
     // queue.
-    BlockInfoUnderConstruction info = new BlockInfoUnderConstruction(block1, (short) 1);
+    BlockInfoContiguousUnderConstruction info = new BlockInfoContiguousUnderConstruction(block1, (short) 1);
     BlockCollection bc = mock(BlockCollection.class);
     when(bc.getBlockReplication()).thenReturn((short)1);
     bm.addBlockCollection(info, bc);
@@ -1209,7 +1209,7 @@ public class TestReplicationPolicy {
     chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
 
-    final BlockInfo info = new BlockInfo(block1, (short) 1);
+    final BlockInfoContiguous info = new BlockInfoContiguous(block1, (short) 1);
     final BlockCollection mbc = mock(BlockCollection.class);
     when(mbc.getLastBlock()).thenReturn(info);
     when(mbc.getPreferredBlockSize()).thenReturn(block1.getNumBytes() + 1);
@@ -1223,7 +1223,7 @@ public class TestReplicationPolicy {
 
     DatanodeStorageInfo[] storageAry = {new DatanodeStorageInfo(
         dataNodes[0], new DatanodeStorage("s1"))};
-    final BlockInfoUnderConstruction ucBlock =
+    final BlockInfoContiguousUnderConstruction ucBlock =
         info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
             storageAry);
     DatanodeStorageInfo storage = mock(DatanodeStorageInfo.class);
@@ -1231,12 +1231,12 @@ public class TestReplicationPolicy {
     when(dn.isDecommissioned()).thenReturn(true);
     when(storage.getState()).thenReturn(DatanodeStorage.State.NORMAL);
     when(storage.getDatanodeDescriptor()).thenReturn(dn);
-    when(storage.removeBlock(any(BlockInfo.class))).thenReturn(true);
-    when(storage.addBlock(any(BlockInfo.class))).thenReturn
+    when(storage.removeBlock(any(BlockInfoContiguous.class))).thenReturn(true);
+    when(storage.addBlock(any(BlockInfoContiguous.class))).thenReturn
         (DatanodeStorageInfo.AddBlockResult.ADDED);
     ucBlock.addStorage(storage);
 
-    when(mbc.setLastBlock((BlockInfo) any(), (DatanodeStorageInfo[]) any()))
+    when(mbc.setLastBlock((BlockInfoContiguous) any(), (DatanodeStorageInfo[]) any()))
     .thenReturn(ucBlock);
 
     bm.convertLastBlockToUnderConstruction(mbc, 0L);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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..0349251 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
@@ -23,7 +23,7 @@ import java.io.IOException;
 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;
 
@@ -66,10 +66,10 @@ public class CreateEditsLog {
     INodeDirectory dirInode = new INodeDirectory(inodeId.nextValue(), null, p,
       0L);
     editLog.logMkDir(BASE_PATH, dirInode);
-    BlockInfo[] blocks = new BlockInfo[blocksPerFile];
+    BlockInfoContiguous[] blocks = new BlockInfoContiguous[blocksPerFile];
     for (int iB = 0; iB < blocksPerFile; ++iB) {
       blocks[iB] = 
-       new BlockInfo(new Block(0, blockSize, BLOCK_GENERATION_STAMP),
+       new BlockInfoContiguous(new Block(0, blockSize, BLOCK_GENERATION_STAMP),
                                replication);
     }
     
@@ -97,7 +97,7 @@ public class CreateEditsLog {
         editLog.logMkDir(currentDir, dirInode);
       }
       INodeFile fileUc = new INodeFile(inodeId.nextValue(), null,
-          p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize);
+          p, 0L, 0L, BlockInfoContiguous.EMPTY_ARRAY, replication, blockSize);
       fileUc.toUnderConstruction("", "");
       editLog.logOpenFile(filePath, fileUc, false, false);
       editLog.logCloseFile(filePath, inode);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java
index 301ee25..a417c3d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.junit.After;
 import org.junit.Before;
@@ -87,21 +87,21 @@ public class TestAddBlock {
     
     // check file1
     INodeFile file1Node = fsdir.getINode4Write(file1.toString()).asFile();
-    BlockInfo[] file1Blocks = file1Node.getBlocks();
+    BlockInfoContiguous[] file1Blocks = file1Node.getBlocks();
     assertEquals(1, file1Blocks.length);
     assertEquals(BLOCKSIZE - 1, file1Blocks[0].getNumBytes());
     assertEquals(BlockUCState.COMPLETE, file1Blocks[0].getBlockUCState());
     
     // check file2
     INodeFile file2Node = fsdir.getINode4Write(file2.toString()).asFile();
-    BlockInfo[] file2Blocks = file2Node.getBlocks();
+    BlockInfoContiguous[] file2Blocks = file2Node.getBlocks();
     assertEquals(1, file2Blocks.length);
     assertEquals(BLOCKSIZE, file2Blocks[0].getNumBytes());
     assertEquals(BlockUCState.COMPLETE, file2Blocks[0].getBlockUCState());
     
     // check file3
     INodeFile file3Node = fsdir.getINode4Write(file3.toString()).asFile();
-    BlockInfo[] file3Blocks = file3Node.getBlocks();
+    BlockInfoContiguous[] file3Blocks = file3Node.getBlocks();
     assertEquals(2, file3Blocks.length);
     assertEquals(BLOCKSIZE, file3Blocks[0].getNumBytes());
     assertEquals(BlockUCState.COMPLETE, file3Blocks[0].getBlockUCState());
@@ -110,7 +110,7 @@ public class TestAddBlock {
     
     // check file4
     INodeFile file4Node = fsdir.getINode4Write(file4.toString()).asFile();
-    BlockInfo[] file4Blocks = file4Node.getBlocks();
+    BlockInfoContiguous[] file4Blocks = file4Node.getBlocks();
     assertEquals(2, file4Blocks.length);
     assertEquals(BLOCKSIZE, file4Blocks[0].getNumBytes());
     assertEquals(BlockUCState.COMPLETE, file4Blocks[0].getBlockUCState());
@@ -141,7 +141,7 @@ public class TestAddBlock {
       FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
       
       INodeFile fileNode = fsdir.getINode4Write(file1.toString()).asFile();
-      BlockInfo[] fileBlocks = fileNode.getBlocks();
+      BlockInfoContiguous[] fileBlocks = fileNode.getBlocks();
       assertEquals(2, fileBlocks.length);
       assertEquals(BLOCKSIZE, fileBlocks[0].getNumBytes());
       assertEquals(BlockUCState.COMPLETE, fileBlocks[0].getBlockUCState());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
index 872ff9c..1fbe160 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
@@ -35,8 +35,8 @@ import org.apache.hadoop.hdfs.TestFileCreation;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+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.protocol.NamenodeProtocols;
 import org.junit.AfterClass;
@@ -91,12 +91,12 @@ public class TestBlockUnderConstruction {
         " isUnderConstruction = " + inode.isUnderConstruction() +
         " expected to be " + isFileOpen,
         inode.isUnderConstruction() == isFileOpen);
-    BlockInfo[] blocks = inode.getBlocks();
+    BlockInfoContiguous[] blocks = inode.getBlocks();
     assertTrue("File does not have blocks: " + inode.toString(),
         blocks != null && blocks.length > 0);
     
     int idx = 0;
-    BlockInfo curBlock;
+    BlockInfoContiguous curBlock;
     // all blocks but the last two should be regular blocks
     for(; idx < blocks.length - 2; idx++) {
       curBlock = blocks[idx];
@@ -170,7 +170,7 @@ public class TestBlockUnderConstruction {
       final List<LocatedBlock> blocks = lb.getLocatedBlocks();
       assertEquals(i, blocks.size());
       final Block b = blocks.get(blocks.size() - 1).getBlock().getLocalBlock();
-      assertTrue(b instanceof BlockInfoUnderConstruction);
+      assertTrue(b instanceof BlockInfoContiguousUnderConstruction);
 
       if (++i < NUM_BLOCKS) {
         // write one more block

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 d88227a..b7e8c25 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
@@ -22,8 +22,8 @@ import org.apache.hadoop.conf.Configuration;
 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.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.junit.Test;
@@ -64,7 +64,7 @@ public class TestCommitBlockSynchronization {
     namesystem.dir.getINodeMap().put(file);
 
     FSNamesystem namesystemSpy = spy(namesystem);
-    BlockInfoUnderConstruction blockInfo = new BlockInfoUnderConstruction(
+    BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
         block, (short) 1, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
     blockInfo.setBlockCollection(file);
     blockInfo.setGenerationStamp(genStamp);
@@ -75,7 +75,7 @@ public class TestCommitBlockSynchronization {
     doReturn(blockInfo).when(namesystemSpy).getStoredBlock(any(Block.class));
     doReturn(blockInfo).when(file).getLastBlock();
     doReturn("").when(namesystemSpy).closeFileCommitBlocks(
-        any(INodeFile.class), any(BlockInfo.class));
+        any(INodeFile.class), any(BlockInfoContiguous.class));
     doReturn(mock(FSEditLog.class)).when(namesystemSpy).getEditLog();
 
     return namesystemSpy;
@@ -103,7 +103,7 @@ public class TestCommitBlockSynchronization {
         lastBlock, genStamp, length, false, false, newTargets, null);
 
     // Simulate 'completing' the block.
-    BlockInfo completedBlockInfo = new BlockInfo(block, (short) 1);
+    BlockInfoContiguous completedBlockInfo = new BlockInfoContiguous(block, (short) 1);
     completedBlockInfo.setBlockCollection(file);
     completedBlockInfo.setGenerationStamp(genStamp);
     doReturn(completedBlockInfo).when(namesystemSpy)
@@ -175,7 +175,7 @@ public class TestCommitBlockSynchronization {
     namesystemSpy.commitBlockSynchronization(
         lastBlock, genStamp, length, true, false, newTargets, null);
 
-    BlockInfo completedBlockInfo = new BlockInfo(block, (short) 1);
+    BlockInfoContiguous 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/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
index 2a8f289..6d8d205 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
@@ -70,7 +70,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
@@ -205,7 +205,7 @@ public class TestEditLog {
 
       for (int i = 0; i < numTransactions; i++) {
         INodeFile inode = new INodeFile(namesystem.dir.allocateNewInodeId(), null,
-            p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize);
+            p, 0L, 0L, BlockInfoContiguous.EMPTY_ARRAY, replication, blockSize);
         inode.toUnderConstruction("", "");
 
         editLog.logOpenFile("/filename" + (startIndex + i), inode, false, false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
index f21834e..c68ae04 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
@@ -97,7 +97,7 @@ public class TestFSImage {
       INodeFile file2Node = fsn.dir.getINode4Write(file2.toString()).asFile();
       assertEquals("hello".length(), file2Node.computeFileSize());
       assertTrue(file2Node.isUnderConstruction());
-      BlockInfo[] blks = file2Node.getBlocks();
+      BlockInfoContiguous[] blks = file2Node.getBlocks();
       assertEquals(1, blks.length);
       assertEquals(BlockUCState.UNDER_CONSTRUCTION, blks[0].getBlockUCState());
       // check lease manager

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index 2bbb4b7..253727d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -54,7 +54,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -773,7 +773,7 @@ public class TestFileTruncate {
           is(fsn.getBlockIdManager().getGenerationStampV2()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
-      long blockRecoveryId = ((BlockInfoUnderConstruction) file.getLastBlock())
+      long blockRecoveryId = ((BlockInfoContiguousUnderConstruction) file.getLastBlock())
           .getBlockRecoveryId();
       assertThat(blockRecoveryId, is(initialGenStamp + 1));
       fsn.getEditLog().logTruncate(
@@ -806,7 +806,7 @@ public class TestFileTruncate {
           is(fsn.getBlockIdManager().getGenerationStampV2()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
-      long blockRecoveryId = ((BlockInfoUnderConstruction) file.getLastBlock())
+      long blockRecoveryId = ((BlockInfoContiguousUnderConstruction) file.getLastBlock())
           .getBlockRecoveryId();
       assertThat(blockRecoveryId, is(initialGenStamp + 1));
       fsn.getEditLog().logTruncate(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index d67bd72..5574a30 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@ -65,7 +65,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -714,7 +714,7 @@ public class TestFsck {
       // intentionally corrupt NN data structure
       INodeFile node = (INodeFile) cluster.getNamesystem().dir.getINode
           (fileName, true);
-      final BlockInfo[] blocks = node.getBlocks(); 
+      final BlockInfoContiguous[] blocks = node.getBlocks();
       assertEquals(blocks.length, 1);
       blocks[0].setNumBytes(-1L);  // set the block length to be negative
       

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 f79277a..61d2b3e 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
@@ -59,7 +59,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 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;
@@ -285,7 +285,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);
+      BlockInfoContiguous newblock = new BlockInfoContiguous(replication);
       iNodes[i].addBlock(newblock);
     }
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
index 218a0d5..18d6dfc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
@@ -28,7 +28,6 @@ import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -42,12 +41,8 @@ import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -61,7 +56,6 @@ import org.apache.hadoop.test.MultithreadedTestUtil.RepeatingTestThread;
 import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
 import org.junit.Test;
 import org.mockito.Mockito;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
index 916893c..c0d320c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
@@ -71,7 +71,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
@@ -743,8 +743,8 @@ public class TestRetryCacheWithHA {
     boolean checkNamenodeBeforeReturn() throws Exception {
       INodeFile fileNode = cluster.getNamesystem(0).getFSDirectory()
           .getINode4Write(file).asFile();
-      BlockInfoUnderConstruction blkUC = 
-          (BlockInfoUnderConstruction) (fileNode.getBlocks())[1];
+      BlockInfoContiguousUnderConstruction blkUC =
+          (BlockInfoContiguousUnderConstruction) (fileNode.getBlocks())[1];
       int datanodeNum = blkUC.getExpectedStorageLocations().length;
       for (int i = 0; i < CHECKTIMES && datanodeNum != 2; i++) {
         Thread.sleep(1000);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
index 4c703ba..11b19f3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
@@ -34,7 +34,6 @@ import java.util.Random;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -44,8 +43,8 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+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.datanode.BlockPoolSliceStorage;
 import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
@@ -63,7 +62,6 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine.Server;
 import org.apache.hadoop.metrics2.impl.MetricsSystemImpl;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.log4j.Level;
 import org.junit.Assert;
 
 /**
@@ -178,8 +176,8 @@ public class SnapshotTestHelper {
    * 
    * Specific information for different types of INode: 
    * {@link INodeDirectory}:childrenSize 
-   * {@link INodeFile}: fileSize, block list. Check {@link BlockInfo#toString()} 
-   * and {@link BlockInfoUnderConstruction#toString()} for detailed information.
+   * {@link INodeFile}: fileSize, block list. Check {@link BlockInfoContiguous#toString()}
+   * and {@link BlockInfoContiguousUnderConstruction#toString()} for detailed information.
    * {@link FileWithSnapshot}: next link
    * </pre>
    * @see INode#dumpTreeRecursively()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java
index c7b6b7f..c6c8dad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -108,14 +108,14 @@ public class TestSnapshotBlocksMap {
      final FSDirectory dir, final BlockManager blkManager) throws Exception {
     final INodeFile file = INodeFile.valueOf(dir.getINode(path), path);
     assertEquals(numBlocks, file.getBlocks().length);
-    for(BlockInfo b : file.getBlocks()) {
+    for(BlockInfoContiguous b : file.getBlocks()) {
       assertBlockCollection(blkManager, file, b);
     }
     return file;
   }
 
   static void assertBlockCollection(final BlockManager blkManager,
-      final INodeFile file, final BlockInfo b) { 
+      final INodeFile file, final BlockInfoContiguous b) {
     Assert.assertSame(b, blkManager.getStoredBlock(b));
     Assert.assertSame(file, blkManager.getBlockCollection(b));
     Assert.assertSame(file, b.getBlockCollection());
@@ -146,10 +146,10 @@ public class TestSnapshotBlocksMap {
     {
       final INodeFile f2 = assertBlockCollection(file2.toString(), 3, fsdir,
           blockmanager);
-      BlockInfo[] blocks = f2.getBlocks();
+      BlockInfoContiguous[] blocks = f2.getBlocks();
       hdfs.delete(sub2, true);
       // The INode should have been removed from the blocksMap
-      for(BlockInfo b : blocks) {
+      for(BlockInfoContiguous b : blocks) {
         assertNull(blockmanager.getBlockCollection(b));
       }
     }
@@ -177,7 +177,7 @@ public class TestSnapshotBlocksMap {
     // Check the block information for file0
     final INodeFile f0 = assertBlockCollection(file0.toString(), 4, fsdir,
         blockmanager);
-    BlockInfo[] blocks0 = f0.getBlocks();
+    BlockInfoContiguous[] blocks0 = f0.getBlocks();
     
     // Also check the block information for snapshot of file0
     Path snapshotFile0 = SnapshotTestHelper.getSnapshotPath(sub1, "s0",
@@ -187,7 +187,7 @@ public class TestSnapshotBlocksMap {
     // Delete file0
     hdfs.delete(file0, true);
     // Make sure the blocks of file0 is still in blocksMap
-    for(BlockInfo b : blocks0) {
+    for(BlockInfoContiguous b : blocks0) {
       assertNotNull(blockmanager.getBlockCollection(b));
     }
     assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager);
@@ -201,7 +201,7 @@ public class TestSnapshotBlocksMap {
     hdfs.deleteSnapshot(sub1, "s1");
 
     // Make sure the first block of file0 is still in blocksMap
-    for(BlockInfo b : blocks0) {
+    for(BlockInfoContiguous b : blocks0) {
       assertNotNull(blockmanager.getBlockCollection(b));
     }
     assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager);
@@ -293,7 +293,7 @@ public class TestSnapshotBlocksMap {
     hdfs.append(bar);
 
     INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
-    BlockInfo[] blks = barNode.getBlocks();
+    BlockInfoContiguous[] blks = barNode.getBlocks();
     assertEquals(1, blks.length);
     assertEquals(BLOCKSIZE, blks[0].getNumBytes());
     ExtendedBlock previous = new ExtendedBlock(fsn.getBlockPoolId(), blks[0]);
@@ -331,7 +331,7 @@ public class TestSnapshotBlocksMap {
     hdfs.append(bar);
 
     INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
-    BlockInfo[] blks = barNode.getBlocks();
+    BlockInfoContiguous[] blks = barNode.getBlocks();
     assertEquals(1, blks.length);
     ExtendedBlock previous = new ExtendedBlock(fsn.getBlockPoolId(), blks[0]);
     cluster.getNameNodeRpc()
@@ -370,7 +370,7 @@ public class TestSnapshotBlocksMap {
     hdfs.append(bar);
 
     INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
-    BlockInfo[] blks = barNode.getBlocks();
+    BlockInfoContiguous[] blks = barNode.getBlocks();
     assertEquals(1, blks.length);
     ExtendedBlock previous = new ExtendedBlock(fsn.getBlockPoolId(), blks[0]);
     cluster.getNameNodeRpc()
@@ -421,7 +421,7 @@ public class TestSnapshotBlocksMap {
     out.write(testData);
     out.close();
     INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
-    BlockInfo[] blks = barNode.getBlocks();
+    BlockInfoContiguous[] blks = barNode.getBlocks();
     assertEquals(1, blks.length);
     assertEquals(testData.length, blks[0].getNumBytes());
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
index 76e8392..bc4ec90 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
@@ -42,7 +42,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -260,12 +260,12 @@ public class TestSnapshotDeletion {
     DFSTestUtil.createFile(hdfs, tempFile, BLOCKSIZE, REPLICATION, seed);
     final INodeFile temp = TestSnapshotBlocksMap.assertBlockCollection(
         tempFile.toString(), 1, fsdir, blockmanager);
-    BlockInfo[] blocks = temp.getBlocks();
+    BlockInfoContiguous[] blocks = temp.getBlocks();
     hdfs.delete(tempDir, true);
     // check dir's quota usage
     checkQuotaUsageComputation(dir, 8, BLOCKSIZE * REPLICATION * 3);
     // check blocks of tempFile
-    for (BlockInfo b : blocks) {
+    for (BlockInfoContiguous b : blocks) {
       assertNull(blockmanager.getBlockCollection(b));
     }
     
@@ -342,7 +342,7 @@ public class TestSnapshotDeletion {
     // while deletion, we add diff for subsub and metaChangeFile1, and remove
     // newFile
     checkQuotaUsageComputation(dir, 9L, BLOCKSIZE * REPLICATION * 4);
-    for (BlockInfo b : blocks) {
+    for (BlockInfoContiguous b : blocks) {
       assertNull(blockmanager.getBlockCollection(b));
     }
     
@@ -479,7 +479,7 @@ public class TestSnapshotDeletion {
     
     final INodeFile toDeleteFileNode = TestSnapshotBlocksMap
         .assertBlockCollection(toDeleteFile.toString(), 1, fsdir, blockmanager);
-    BlockInfo[] blocks = toDeleteFileNode.getBlocks();
+    BlockInfoContiguous[] blocks = toDeleteFileNode.getBlocks();
     
     // create snapshot s0 on dir
     SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
@@ -505,7 +505,7 @@ public class TestSnapshotDeletion {
     // metaChangeDir's diff, dir's diff. diskspace: remove toDeleteFile, and 
     // metaChangeFile's replication factor decreases
     checkQuotaUsageComputation(dir, 6, 2 * BLOCKSIZE * REPLICATION - BLOCKSIZE);
-    for (BlockInfo b : blocks) {
+    for (BlockInfoContiguous b : blocks) {
       assertNull(blockmanager.getBlockCollection(b));
     }
     
@@ -799,7 +799,7 @@ public class TestSnapshotDeletion {
     FileStatus statusBeforeDeletion13 = hdfs.getFileStatus(file13_s1);
     INodeFile file14Node = TestSnapshotBlocksMap.assertBlockCollection(
         file14_s2.toString(), 1, fsdir, blockmanager);
-    BlockInfo[] blocks_14 = file14Node.getBlocks();
+    BlockInfoContiguous[] blocks_14 = file14Node.getBlocks();
     TestSnapshotBlocksMap.assertBlockCollection(file15_s2.toString(), 1, fsdir,
         blockmanager);
     
@@ -836,7 +836,7 @@ public class TestSnapshotDeletion {
         modDirStr + "file15");
     assertFalse(hdfs.exists(file14_s1));
     assertFalse(hdfs.exists(file15_s1));
-    for (BlockInfo b : blocks_14) {
+    for (BlockInfoContiguous b : blocks_14) {
       assertNull(blockmanager.getBlockCollection(b));
     }
     


[2/3] hadoop git commit: HDFS-7743. Code cleanup of BlockInfo and rename BlockInfo to BlockInfoContiguous. Contributed by Jing Zhao.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 92031c1..3fe47af 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
@@ -535,8 +535,8 @@ public class BlockManager {
     int usableReplicas = numReplicas.liveReplicas() +
                          numReplicas.decommissionedReplicas();
     
-    if (block instanceof BlockInfo) {
-      BlockCollection bc = ((BlockInfo) block).getBlockCollection();
+    if (block instanceof BlockInfoContiguous) {
+      BlockCollection bc = ((BlockInfoContiguous) block).getBlockCollection();
       String fileName = (bc == null) ? "[orphaned]" : bc.getName();
       out.print(fileName + ": ");
     }
@@ -590,8 +590,9 @@ public class BlockManager {
    * @throws IOException if the block does not have at least a minimal number
    * of replicas reported from data-nodes.
    */
-  private static boolean commitBlock(final BlockInfoUnderConstruction block,
-      final Block commitBlock) throws IOException {
+  private static boolean commitBlock(
+      final BlockInfoContiguousUnderConstruction block, final Block commitBlock)
+      throws IOException {
     if (block.getBlockUCState() == BlockUCState.COMMITTED)
       return false;
     assert block.getNumBytes() <= commitBlock.getNumBytes() :
@@ -615,13 +616,14 @@ public class BlockManager {
       Block commitBlock) throws IOException {
     if(commitBlock == null)
       return false; // not committing, this is a block allocation retry
-    BlockInfo lastBlock = bc.getLastBlock();
+    BlockInfoContiguous lastBlock = bc.getLastBlock();
     if(lastBlock == null)
       return false; // no blocks in file yet
     if(lastBlock.isComplete())
       return false; // already completed (e.g. by syncBlock)
     
-    final boolean b = commitBlock((BlockInfoUnderConstruction)lastBlock, commitBlock);
+    final boolean b = commitBlock(
+        (BlockInfoContiguousUnderConstruction) lastBlock, commitBlock);
     if(countNodes(lastBlock).liveReplicas() >= minReplication)
       completeBlock(bc, bc.numBlocks()-1, false);
     return b;
@@ -634,14 +636,15 @@ public class BlockManager {
    * @throws IOException if the block does not have at least a minimal number
    * of replicas reported from data-nodes.
    */
-  private BlockInfo completeBlock(final BlockCollection bc,
+  private BlockInfoContiguous completeBlock(final BlockCollection bc,
       final int blkIndex, boolean force) throws IOException {
     if(blkIndex < 0)
       return null;
-    BlockInfo curBlock = bc.getBlocks()[blkIndex];
+    BlockInfoContiguous curBlock = bc.getBlocks()[blkIndex];
     if(curBlock.isComplete())
       return curBlock;
-    BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)curBlock;
+    BlockInfoContiguousUnderConstruction ucBlock =
+        (BlockInfoContiguousUnderConstruction) curBlock;
     int numNodes = ucBlock.numNodes();
     if (!force && numNodes < minReplication)
       throw new IOException("Cannot complete block: " +
@@ -649,7 +652,7 @@ public class BlockManager {
     if(!force && ucBlock.getBlockUCState() != BlockUCState.COMMITTED)
       throw new IOException(
           "Cannot complete block: block has not been COMMITTED by the client");
-    BlockInfo completeBlock = ucBlock.convertToCompleteBlock();
+    BlockInfoContiguous completeBlock = ucBlock.convertToCompleteBlock();
     // replace penultimate block in file
     bc.setBlock(blkIndex, completeBlock);
     
@@ -667,9 +670,9 @@ public class BlockManager {
     return blocksMap.replaceBlock(completeBlock);
   }
 
-  private BlockInfo completeBlock(final BlockCollection bc,
-      final BlockInfo block, boolean force) throws IOException {
-    BlockInfo[] fileBlocks = bc.getBlocks();
+  private BlockInfoContiguous completeBlock(final BlockCollection bc,
+      final BlockInfoContiguous block, boolean force) throws IOException {
+    BlockInfoContiguous[] fileBlocks = bc.getBlocks();
     for(int idx = 0; idx < fileBlocks.length; idx++)
       if(fileBlocks[idx] == block) {
         return completeBlock(bc, idx, force);
@@ -682,8 +685,8 @@ public class BlockManager {
    * regardless of whether enough replicas are present. This is necessary
    * when tailing edit logs as a Standby.
    */
-  public BlockInfo forceCompleteBlock(final BlockCollection bc,
-      final BlockInfoUnderConstruction block) throws IOException {
+  public BlockInfoContiguous forceCompleteBlock(final BlockCollection bc,
+      final BlockInfoContiguousUnderConstruction block) throws IOException {
     block.commitBlock(block);
     return completeBlock(bc, block, true);
   }
@@ -705,7 +708,7 @@ public class BlockManager {
    */
   public LocatedBlock convertLastBlockToUnderConstruction(
       BlockCollection bc, long bytesToRemove) throws IOException {
-    BlockInfo oldBlock = bc.getLastBlock();
+    BlockInfoContiguous oldBlock = bc.getLastBlock();
     if(oldBlock == null ||
        bc.getPreferredBlockSize() == oldBlock.getNumBytes() - bytesToRemove)
       return null;
@@ -714,7 +717,8 @@ public class BlockManager {
 
     DatanodeStorageInfo[] targets = getStorages(oldBlock);
 
-    BlockInfoUnderConstruction ucBlock = bc.setLastBlock(oldBlock, targets);
+    BlockInfoContiguousUnderConstruction ucBlock =
+        bc.setLastBlock(oldBlock, targets);
     blocksMap.replaceBlock(ucBlock);
 
     // Remove block from replication queue.
@@ -756,7 +760,8 @@ public class BlockManager {
     return locations;
   }
   
-  private List<LocatedBlock> createLocatedBlockList(final BlockInfo[] blocks,
+  private List<LocatedBlock> createLocatedBlockList(
+      final BlockInfoContiguous[] blocks,
       final long offset, final long length, final int nrBlocksToReturn,
       final AccessMode mode) throws IOException {
     int curBlk = 0;
@@ -786,7 +791,7 @@ public class BlockManager {
     return results;
   }
 
-  private LocatedBlock createLocatedBlock(final BlockInfo[] blocks,
+  private LocatedBlock createLocatedBlock(final BlockInfoContiguous[] blocks,
       final long endPos, final AccessMode mode) throws IOException {
     int curBlk = 0;
     long curPos = 0;
@@ -802,7 +807,7 @@ public class BlockManager {
     return createLocatedBlock(blocks[curBlk], curPos, mode);
   }
   
-  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos,
+  private LocatedBlock createLocatedBlock(final BlockInfoContiguous blk, final long pos,
     final BlockTokenSecretManager.AccessMode mode) throws IOException {
     final LocatedBlock lb = createLocatedBlock(blk, pos);
     if (mode != null) {
@@ -812,15 +817,16 @@ public class BlockManager {
   }
 
   /** @return a LocatedBlock for the given block */
-  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos
+  private LocatedBlock createLocatedBlock(final BlockInfoContiguous blk, final long pos
       ) throws IOException {
-    if (blk instanceof BlockInfoUnderConstruction) {
+    if (blk instanceof BlockInfoContiguousUnderConstruction) {
       if (blk.isComplete()) {
         throw new IOException(
             "blk instanceof BlockInfoUnderConstruction && blk.isComplete()"
             + ", blk=" + blk);
       }
-      final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)blk;
+      final BlockInfoContiguousUnderConstruction uc =
+          (BlockInfoContiguousUnderConstruction) blk;
       final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
       final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
       return new LocatedBlock(eb, storages, pos, false);
@@ -859,7 +865,7 @@ public class BlockManager {
   }
 
   /** Create a LocatedBlocks. */
-  public LocatedBlocks createLocatedBlocks(final BlockInfo[] blocks,
+  public LocatedBlocks createLocatedBlocks(final BlockInfoContiguous[] blocks,
       final long fileSizeExcludeBlocksUnderConstruction,
       final boolean isFileUnderConstruction, final long offset,
       final long length, final boolean needBlockToken,
@@ -882,7 +888,7 @@ public class BlockManager {
       final LocatedBlock lastlb;
       final boolean isComplete;
       if (!inSnapshot) {
-        final BlockInfo last = blocks[blocks.length - 1];
+        final BlockInfoContiguous last = blocks[blocks.length - 1];
         final long lastPos = last.isComplete()?
             fileSizeExcludeBlocksUnderConstruction - last.getNumBytes()
             : fileSizeExcludeBlocksUnderConstruction;
@@ -971,7 +977,7 @@ public class BlockManager {
   /**
    * Check if a block is replicated to at least the minimum replication.
    */
-  public boolean isSufficientlyReplicated(BlockInfo b) {
+  public boolean isSufficientlyReplicated(BlockInfoContiguous b) {
     // Compare against the lesser of the minReplication and number of live DNs.
     final int replication =
         Math.min(minReplication, getDatanodeManager().getNumLiveDataNodes());
@@ -1012,7 +1018,7 @@ public class BlockManager {
     if(numBlocks == 0) {
       return new BlocksWithLocations(new BlockWithLocations[0]);
     }
-    Iterator<BlockInfo> iter = node.getBlockIterator();
+    Iterator<BlockInfoContiguous> iter = node.getBlockIterator();
     int startBlock = DFSUtil.getRandom().nextInt(numBlocks); // starting from a random block
     // skip blocks
     for(int i=0; i<startBlock; i++) {
@@ -1020,7 +1026,7 @@ public class BlockManager {
     }
     List<BlockWithLocations> results = new ArrayList<BlockWithLocations>();
     long totalSize = 0;
-    BlockInfo curBlock;
+    BlockInfoContiguous curBlock;
     while(totalSize<size && iter.hasNext()) {
       curBlock = iter.next();
       if(!curBlock.isComplete())  continue;
@@ -1119,7 +1125,7 @@ public class BlockManager {
   public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk,
       final DatanodeInfo dn, String storageID, String reason) throws IOException {
     assert namesystem.hasWriteLock();
-    final BlockInfo storedBlock = getStoredBlock(blk.getLocalBlock());
+    final BlockInfoContiguous storedBlock = getStoredBlock(blk.getLocalBlock());
     if (storedBlock == null) {
       // Check if the replica is in the blockMap, if not
       // ignore the request for now. This could happen when BlockScanner
@@ -1696,11 +1702,11 @@ public class BlockManager {
    * reported by the datanode in the block report. 
    */
   static class StatefulBlockInfo {
-    final BlockInfoUnderConstruction storedBlock;
+    final BlockInfoContiguousUnderConstruction storedBlock;
     final Block reportedBlock;
     final ReplicaState reportedState;
     
-    StatefulBlockInfo(BlockInfoUnderConstruction storedBlock, 
+    StatefulBlockInfo(BlockInfoContiguousUnderConstruction storedBlock,
         Block reportedBlock, ReplicaState reportedState) {
       this.storedBlock = storedBlock;
       this.reportedBlock = reportedBlock;
@@ -1714,15 +1720,16 @@ public class BlockManager {
    */
   private static class BlockToMarkCorrupt {
     /** The corrupted block in a datanode. */
-    final BlockInfo corrupted;
+    final BlockInfoContiguous corrupted;
     /** The corresponding block stored in the BlockManager. */
-    final BlockInfo stored;
+    final BlockInfoContiguous stored;
     /** The reason to mark corrupt. */
     final String reason;
     /** The reason code to be stored */
     final Reason reasonCode;
 
-    BlockToMarkCorrupt(BlockInfo corrupted, BlockInfo stored, String reason,
+    BlockToMarkCorrupt(BlockInfoContiguous corrupted,
+        BlockInfoContiguous stored, String reason,
         Reason reasonCode) {
       Preconditions.checkNotNull(corrupted, "corrupted is null");
       Preconditions.checkNotNull(stored, "stored is null");
@@ -1733,13 +1740,14 @@ public class BlockManager {
       this.reasonCode = reasonCode;
     }
 
-    BlockToMarkCorrupt(BlockInfo stored, String reason, Reason reasonCode) {
+    BlockToMarkCorrupt(BlockInfoContiguous stored, String reason,
+        Reason reasonCode) {
       this(stored, stored, reason, reasonCode);
     }
 
-    BlockToMarkCorrupt(BlockInfo stored, long gs, String reason,
+    BlockToMarkCorrupt(BlockInfoContiguous stored, long gs, String reason,
         Reason reasonCode) {
-      this(new BlockInfo(stored), stored, reason, reasonCode);
+      this(new BlockInfoContiguous(stored), stored, reason, reasonCode);
       //the corrupted block in datanode has a different generation stamp
       corrupted.setGenerationStamp(gs);
     }
@@ -1864,7 +1872,7 @@ public class BlockManager {
           break;
         }
 
-        BlockInfo bi = blocksMap.getStoredBlock(b);
+        BlockInfoContiguous bi = blocksMap.getStoredBlock(b);
         if (bi == null) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
@@ -1904,7 +1912,7 @@ public class BlockManager {
     // Modify the (block-->datanode) map, according to the difference
     // between the old and new block report.
     //
-    Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
+    Collection<BlockInfoContiguous> toAdd = new LinkedList<BlockInfoContiguous>();
     Collection<Block> toRemove = new TreeSet<Block>();
     Collection<Block> toInvalidate = new LinkedList<Block>();
     Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
@@ -1921,7 +1929,7 @@ public class BlockManager {
       removeStoredBlock(b, node);
     }
     int numBlocksLogged = 0;
-    for (BlockInfo b : toAdd) {
+    for (BlockInfoContiguous b : toAdd) {
       addStoredBlock(b, storageInfo, null, numBlocksLogged < maxNumBlocksToLog);
       numBlocksLogged++;
     }
@@ -1969,7 +1977,7 @@ public class BlockManager {
         continue;
       }
       
-      BlockInfo storedBlock = blocksMap.getStoredBlock(iblk);
+      BlockInfoContiguous storedBlock = blocksMap.getStoredBlock(iblk);
       // If block does not belong to any file, we are done.
       if (storedBlock == null) continue;
       
@@ -1992,12 +2000,13 @@ public class BlockManager {
       
       // If block is under construction, add this replica to its list
       if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
-        ((BlockInfoUnderConstruction)storedBlock).addReplicaIfNotPresent(
-            storageInfo, iblk, reportedState);
+        ((BlockInfoContiguousUnderConstruction)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
-        BlockInfoUnderConstruction blockUC = (BlockInfoUnderConstruction) storedBlock;
+        BlockInfoContiguousUnderConstruction blockUC =
+            (BlockInfoContiguousUnderConstruction) storedBlock;
         if (namesystem.isInSnapshot(blockUC)) {
           int numOfReplicas = blockUC.getNumExpectedLocations();
           namesystem.incrementSafeBlockCount(numOfReplicas);
@@ -2013,7 +2022,7 @@ public class BlockManager {
 
   private void reportDiff(DatanodeStorageInfo storageInfo, 
       BlockListAsLongs newReport, 
-      Collection<BlockInfo> toAdd,              // add to DatanodeDescriptor
+      Collection<BlockInfoContiguous> toAdd,              // add to DatanodeDescriptor
       Collection<Block> toRemove,           // remove from DatanodeDescriptor
       Collection<Block> toInvalidate,       // should be removed from DN
       Collection<BlockToMarkCorrupt> toCorrupt, // add to corrupt replicas list
@@ -2021,7 +2030,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);
+    BlockInfoContiguous delimiter = new BlockInfoContiguous(new Block(), (short) 1);
     AddBlockResult result = storageInfo.addBlock(delimiter);
     assert result == AddBlockResult.ADDED 
         : "Delimiting block cannot be present in the node";
@@ -2036,7 +2045,7 @@ public class BlockManager {
     while(itBR.hasNext()) {
       Block iblk = itBR.next();
       ReplicaState iState = itBR.getCurrentReplicaState();
-      BlockInfo storedBlock = processReportedBlock(storageInfo,
+      BlockInfoContiguous storedBlock = processReportedBlock(storageInfo,
           iblk, iState, toAdd, toInvalidate, toCorrupt, toUC);
 
       // move block to the head of the list
@@ -2048,7 +2057,8 @@ public class BlockManager {
 
     // collect blocks that have not been reported
     // all of them are next to the delimiter
-    Iterator<BlockInfo> it = storageInfo.new BlockIterator(delimiter.getNext(0));
+    Iterator<BlockInfoContiguous> it =
+        storageInfo.new BlockIterator(delimiter.getNext(0));
     while(it.hasNext())
       toRemove.add(it.next());
     storageInfo.removeBlock(delimiter);
@@ -2085,10 +2095,10 @@ public class BlockManager {
    * @return the up-to-date stored block, if it should be kept.
    *         Otherwise, null.
    */
-  private BlockInfo processReportedBlock(
+  private BlockInfoContiguous processReportedBlock(
       final DatanodeStorageInfo storageInfo,
       final Block block, final ReplicaState reportedState, 
-      final Collection<BlockInfo> toAdd, 
+      final Collection<BlockInfoContiguous> toAdd,
       final Collection<Block> toInvalidate, 
       final Collection<BlockToMarkCorrupt> toCorrupt,
       final Collection<StatefulBlockInfo> toUC) {
@@ -2109,7 +2119,7 @@ public class BlockManager {
     }
     
     // find block by blockId
-    BlockInfo storedBlock = blocksMap.getStoredBlock(block);
+    BlockInfoContiguous storedBlock = blocksMap.getStoredBlock(block);
     if(storedBlock == null) {
       // If blocksMap does not contain reported block id,
       // the replica should be removed from the data-node.
@@ -2152,7 +2162,8 @@ public class BlockManager {
     }
 
     if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
-      toUC.add(new StatefulBlockInfo((BlockInfoUnderConstruction) storedBlock,
+      toUC.add(new StatefulBlockInfo(
+          (BlockInfoContiguousUnderConstruction) storedBlock,
           new Block(block), reportedState));
       return storedBlock;
     }
@@ -2240,7 +2251,7 @@ public class BlockManager {
    */
   private BlockToMarkCorrupt checkReplicaCorrupt(
       Block reported, ReplicaState reportedState, 
-      BlockInfo storedBlock, BlockUCState ucState, 
+      BlockInfoContiguous storedBlock, BlockUCState ucState,
       DatanodeDescriptor dn) {
     switch(reportedState) {
     case FINALIZED:
@@ -2313,7 +2324,7 @@ public class BlockManager {
     }
   }
 
-  private boolean isBlockUnderConstruction(BlockInfo storedBlock, 
+  private boolean isBlockUnderConstruction(BlockInfoContiguous storedBlock,
       BlockUCState ucState, ReplicaState reportedState) {
     switch(reportedState) {
     case FINALIZED:
@@ -2336,7 +2347,7 @@ public class BlockManager {
 
   void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
       DatanodeStorageInfo storageInfo) throws IOException {
-    BlockInfoUnderConstruction block = ucBlock.storedBlock;
+    BlockInfoContiguousUnderConstruction block = ucBlock.storedBlock;
     block.addReplicaIfNotPresent(
         storageInfo, ucBlock.reportedBlock, ucBlock.reportedState);
 
@@ -2347,9 +2358,8 @@ public class BlockManager {
   } 
 
   /**
-   * Faster version of
-   * {@link #addStoredBlock(BlockInfo, DatanodeStorageInfo, DatanodeDescriptor, boolean)}
-   * , intended for use with initial block report at startup. If not in startup
+   * Faster version of {@link #addStoredBlock},
+   * intended for use with initial block report at startup. If not in startup
    * safe mode, will call standard addStoredBlock(). Assumes this method is
    * called "immediately" so there is no need to refresh the storedBlock from
    * blocksMap. Doesn't handle underReplication/overReplication, or worry about
@@ -2358,7 +2368,7 @@ public class BlockManager {
    * 
    * @throws IOException
    */
-  private void addStoredBlockImmediate(BlockInfo storedBlock,
+  private void addStoredBlockImmediate(BlockInfoContiguous storedBlock,
       DatanodeStorageInfo storageInfo)
   throws IOException {
     assert (storedBlock != null && namesystem.hasWriteLock());
@@ -2390,15 +2400,15 @@ public class BlockManager {
    * needed replications if this takes care of the problem.
    * @return the block that is stored in blockMap.
    */
-  private Block addStoredBlock(final BlockInfo block,
+  private Block addStoredBlock(final BlockInfoContiguous block,
                                DatanodeStorageInfo storageInfo,
                                DatanodeDescriptor delNodeHint,
                                boolean logEveryBlock)
   throws IOException {
     assert block != null && namesystem.hasWriteLock();
-    BlockInfo storedBlock;
+    BlockInfoContiguous storedBlock;
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
-    if (block instanceof BlockInfoUnderConstruction) {
+    if (block instanceof BlockInfoContiguousUnderConstruction) {
       //refresh our copy in case the block got completed in another thread
       storedBlock = blocksMap.getStoredBlock(block);
     } else {
@@ -2497,7 +2507,8 @@ public class BlockManager {
     return storedBlock;
   }
 
-  private void logAddStoredBlock(BlockInfo storedBlock, DatanodeDescriptor node) {
+  private void logAddStoredBlock(BlockInfoContiguous storedBlock,
+      DatanodeDescriptor node) {
     if (!blockLog.isInfoEnabled()) {
       return;
     }
@@ -2524,7 +2535,7 @@ public class BlockManager {
    *
    * @param blk Block whose corrupt replicas need to be invalidated
    */
-  private void invalidateCorruptReplicas(BlockInfo blk) {
+  private void invalidateCorruptReplicas(BlockInfoContiguous blk) {
     Collection<DatanodeDescriptor> nodes = corruptReplicas.getNodes(blk);
     boolean removedFromBlocksMap = true;
     if (nodes == null)
@@ -2603,7 +2614,7 @@ public class BlockManager {
     long nrInvalid = 0, nrOverReplicated = 0;
     long nrUnderReplicated = 0, nrPostponed = 0, nrUnderConstruction = 0;
     long startTimeMisReplicatedScan = Time.now();
-    Iterator<BlockInfo> blocksItr = blocksMap.getBlocks().iterator();
+    Iterator<BlockInfoContiguous> blocksItr = blocksMap.getBlocks().iterator();
     long totalBlocks = blocksMap.size();
     replicationQueuesInitProgress = 0;
     long totalProcessed = 0;
@@ -2615,7 +2626,7 @@ public class BlockManager {
       namesystem.writeLockInterruptibly();
       try {
         while (processed < numBlocksPerIteration && blocksItr.hasNext()) {
-          BlockInfo block = blocksItr.next();
+          BlockInfoContiguous block = blocksItr.next();
           MisReplicationResult res = processMisReplicatedBlock(block);
           if (LOG.isTraceEnabled()) {
             LOG.trace("block " + block + ": " + res);
@@ -2689,7 +2700,7 @@ public class BlockManager {
    * appropriate queues if necessary, and returns a result code indicating
    * what happened with it.
    */
-  private MisReplicationResult processMisReplicatedBlock(BlockInfo block) {
+  private MisReplicationResult processMisReplicatedBlock(BlockInfoContiguous block) {
     BlockCollection bc = block.getBlockCollection();
     if (bc == null) {
       // block does not belong to any file
@@ -3018,7 +3029,7 @@ public class BlockManager {
       ReplicaState reportedState, DatanodeDescriptor delHintNode)
       throws IOException {
     // blockReceived reports a finalized block
-    Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
+    Collection<BlockInfoContiguous> toAdd = new LinkedList<BlockInfoContiguous>();
     Collection<Block> toInvalidate = new LinkedList<Block>();
     Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
     Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
@@ -3035,7 +3046,7 @@ public class BlockManager {
       addStoredBlockUnderConstruction(b, storageInfo);
     }
     long numBlocksLogged = 0;
-    for (BlockInfo b : toAdd) {
+    for (BlockInfoContiguous b : toAdd) {
       addStoredBlock(b, storageInfo, delHintNode, numBlocksLogged < maxNumBlocksToLog);
       numBlocksLogged++;
     }
@@ -3159,7 +3170,7 @@ public class BlockManager {
    * @param b - the block being tested
    * @return count of live nodes for this block
    */
-  int countLiveNodes(BlockInfo b) {
+  int countLiveNodes(BlockInfoContiguous b) {
     if (!namesystem.isInStartupSafeMode()) {
       return countNodes(b).liveReplicas();
     }
@@ -3314,7 +3325,7 @@ public class BlockManager {
     return blocksMap.size();
   }
 
-  public DatanodeStorageInfo[] getStorages(BlockInfo block) {
+  public DatanodeStorageInfo[] getStorages(BlockInfoContiguous block) {
     final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[block.numNodes()];
     int i = 0;
     for(DatanodeStorageInfo s : blocksMap.getStorages(block)) {
@@ -3344,7 +3355,7 @@ public class BlockManager {
     }
   }
 
-  public BlockInfo getStoredBlock(Block block) {
+  public BlockInfoContiguous getStoredBlock(Block block) {
     return blocksMap.getStoredBlock(block);
   }
 
@@ -3492,7 +3503,8 @@ public class BlockManager {
     return this.neededReplications.getCorruptReplOneBlockSize();
   }
 
-  public BlockInfo addBlockCollection(BlockInfo block, BlockCollection bc) {
+  public BlockInfoContiguous addBlockCollection(BlockInfoContiguous block,
+      BlockCollection bc) {
     return blocksMap.addBlockCollection(block, bc);
   }
 
@@ -3701,7 +3713,7 @@ public class BlockManager {
 
   /**
    * A simple result enum for the result of
-   * {@link BlockManager#processMisReplicatedBlock(BlockInfo)}.
+   * {@link BlockManager#processMisReplicatedBlock(BlockInfoContiguous)}.
    */
   enum MisReplicationResult {
     /** The block should be invalidated since it belongs to a deleted file. */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 d532e74..806a4cb 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
@@ -36,10 +36,10 @@ import com.google.common.collect.Iterables;
  */
 class BlocksMap {
   private static class StorageIterator implements Iterator<DatanodeStorageInfo> {
-    private final BlockInfo blockInfo;
+    private final BlockInfoContiguous blockInfo;
     private int nextIdx = 0;
       
-    StorageIterator(BlockInfo blkInfo) {
+    StorageIterator(BlockInfoContiguous blkInfo) {
       this.blockInfo = blkInfo;
     }
 
@@ -63,14 +63,14 @@ class BlocksMap {
   /** Constant {@link LightWeightGSet} capacity. */
   private final int capacity;
   
-  private GSet<Block, BlockInfo> blocks;
+  private GSet<Block, BlockInfoContiguous> blocks;
 
   BlocksMap(int capacity) {
     // Use 2% of total memory to size the GSet capacity
     this.capacity = capacity;
-    this.blocks = new LightWeightGSet<Block, BlockInfo>(capacity) {
+    this.blocks = new LightWeightGSet<Block, BlockInfoContiguous>(capacity) {
       @Override
-      public Iterator<BlockInfo> iterator() {
+      public Iterator<BlockInfoContiguous> iterator() {
         SetIterator iterator = new SetIterator();
         /*
          * Not tracking any modifications to set. As this set will be used
@@ -97,15 +97,15 @@ class BlocksMap {
   }
 
   BlockCollection getBlockCollection(Block b) {
-    BlockInfo info = blocks.get(b);
+    BlockInfoContiguous info = blocks.get(b);
     return (info != null) ? info.getBlockCollection() : null;
   }
 
   /**
    * Add block b belonging to the specified block collection to the map.
    */
-  BlockInfo addBlockCollection(BlockInfo b, BlockCollection bc) {
-    BlockInfo info = blocks.get(b);
+  BlockInfoContiguous addBlockCollection(BlockInfoContiguous b, BlockCollection bc) {
+    BlockInfoContiguous info = blocks.get(b);
     if (info != b) {
       info = b;
       blocks.put(info);
@@ -120,7 +120,7 @@ class BlocksMap {
    * and remove all data-node locations associated with the block.
    */
   void removeBlock(Block block) {
-    BlockInfo blockInfo = blocks.remove(block);
+    BlockInfoContiguous blockInfo = blocks.remove(block);
     if (blockInfo == null)
       return;
 
@@ -132,7 +132,7 @@ class BlocksMap {
   }
   
   /** Returns the block object it it exists in the map. */
-  BlockInfo getStoredBlock(Block b) {
+  BlockInfoContiguous getStoredBlock(Block b) {
     return blocks.get(b);
   }
 
@@ -164,7 +164,7 @@ class BlocksMap {
    * For a block that has already been retrieved from the BlocksMap
    * returns {@link Iterable} of the storages the block belongs to.
    */
-  Iterable<DatanodeStorageInfo> getStorages(final BlockInfo storedBlock) {
+  Iterable<DatanodeStorageInfo> getStorages(final BlockInfoContiguous storedBlock) {
     return new Iterable<DatanodeStorageInfo>() {
       @Override
       public Iterator<DatanodeStorageInfo> iterator() {
@@ -175,7 +175,7 @@ class BlocksMap {
 
   /** counts number of containing nodes. Better than using iterator. */
   int numNodes(Block b) {
-    BlockInfo info = blocks.get(b);
+    BlockInfoContiguous info = blocks.get(b);
     return info == null ? 0 : info.numNodes();
   }
 
@@ -185,7 +185,7 @@ class BlocksMap {
    * only if it does not belong to any file and data-nodes.
    */
   boolean removeNode(Block b, DatanodeDescriptor node) {
-    BlockInfo info = blocks.get(b);
+    BlockInfoContiguous info = blocks.get(b);
     if (info == null)
       return false;
 
@@ -203,7 +203,7 @@ class BlocksMap {
     return blocks.size();
   }
 
-  Iterable<BlockInfo> getBlocks() {
+  Iterable<BlockInfoContiguous> getBlocks() {
     return blocks;
   }
   
@@ -218,8 +218,8 @@ class BlocksMap {
    * @param newBlock - block for replacement
    * @return new block
    */
-  BlockInfo replaceBlock(BlockInfo newBlock) {
-    BlockInfo currentBlock = blocks.get(newBlock);
+  BlockInfoContiguous replaceBlock(BlockInfoContiguous newBlock) {
+    BlockInfoContiguous 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--) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
index a0f3503..bf5ece9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
@@ -369,7 +369,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
    * @param file The file.
    */
   private void rescanFile(CacheDirective directive, INodeFile file) {
-    BlockInfo[] blockInfos = file.getBlocks();
+    BlockInfoContiguous[] blockInfos = file.getBlocks();
 
     // Increment the "needed" statistics
     directive.addFilesNeeded(1);
@@ -394,7 +394,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
     }
 
     long cachedTotal = 0;
-    for (BlockInfo blockInfo : blockInfos) {
+    for (BlockInfoContiguous blockInfo : blockInfos) {
       if (!blockInfo.getBlockUCState().equals(BlockUCState.COMPLETE)) {
         // We don't try to cache blocks that are under construction.
         LOG.trace("Directive {}: can't cache block {} because it is in state "
@@ -453,7 +453,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
   }
 
   private String findReasonForNotCaching(CachedBlock cblock, 
-          BlockInfo blockInfo) {
+          BlockInfoContiguous blockInfo) {
     if (blockInfo == null) {
       // Somehow, a cache report with the block arrived, but the block
       // reports from the DataNode haven't (yet?) described such a block.
@@ -513,7 +513,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
           iter.remove();
         }
       }
-      BlockInfo blockInfo = blockManager.
+      BlockInfoContiguous blockInfo = blockManager.
             getStoredBlock(new Block(cblock.getBlockId()));
       String reason = findReasonForNotCaching(cblock, blockInfo);
       int neededCached = 0;
@@ -628,7 +628,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
       List<DatanodeDescriptor> pendingCached) {
     // To figure out which replicas can be cached, we consult the
     // blocksMap.  We don't want to try to cache a corrupt replica, though.
-    BlockInfo blockInfo = blockManager.
+    BlockInfoContiguous blockInfo = blockManager.
           getStoredBlock(new Block(cachedBlock.getBlockId()));
     if (blockInfo == null) {
       LOG.debug("Block {}: can't add new cached replicas," +
@@ -667,7 +667,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
       Iterator<CachedBlock> it = datanode.getPendingCached().iterator();
       while (it.hasNext()) {
         CachedBlock cBlock = it.next();
-        BlockInfo info =
+        BlockInfoContiguous info =
             blockManager.getStoredBlock(new Block(cBlock.getBlockId()));
         if (info != null) {
           pendingBytes -= info.getNumBytes();
@@ -677,7 +677,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
       // Add pending uncached blocks from effective capacity
       while (it.hasNext()) {
         CachedBlock cBlock = it.next();
-        BlockInfo info =
+        BlockInfoContiguous info =
             blockManager.getStoredBlock(new Block(cBlock.getBlockId()));
         if (info != null) {
           pendingBytes += info.getNumBytes();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 644404b..833ab56 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
@@ -197,8 +197,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<BlockInfoUnderConstruction> recoverBlocks =
-                                new BlockQueue<BlockInfoUnderConstruction>();
+  private final BlockQueue<BlockInfoContiguousUnderConstruction> recoverBlocks =
+                                new BlockQueue<BlockInfoContiguousUnderConstruction>();
   /** A set of blocks to be invalidated by this datanode */
   private final LightWeightHashSet<Block> invalidateBlocks = new LightWeightHashSet<Block>();
 
@@ -284,7 +284,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * Remove block from the list of blocks belonging to the data-node. Remove
    * data-node from the block.
    */
-  boolean removeBlock(BlockInfo b) {
+  boolean removeBlock(BlockInfoContiguous b) {
     final DatanodeStorageInfo s = b.findStorageInfo(this);
     // if block exists on this datanode
     if (s != null) {
@@ -297,7 +297,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * Remove block from the list of blocks belonging to the data-node. Remove
    * data-node from the block.
    */
-  boolean removeBlock(String storageID, BlockInfo b) {
+  boolean removeBlock(String storageID, BlockInfoContiguous b) {
     DatanodeStorageInfo s = getStorageInfo(storageID);
     if (s != null) {
       return s.removeBlock(b);
@@ -468,12 +468,12 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
-  private static class BlockIterator implements Iterator<BlockInfo> {
+  private static class BlockIterator implements Iterator<BlockInfoContiguous> {
     private int index = 0;
-    private final List<Iterator<BlockInfo>> iterators;
+    private final List<Iterator<BlockInfoContiguous>> iterators;
     
     private BlockIterator(final DatanodeStorageInfo... storages) {
-      List<Iterator<BlockInfo>> iterators = new ArrayList<Iterator<BlockInfo>>();
+      List<Iterator<BlockInfoContiguous>> iterators = new ArrayList<Iterator<BlockInfoContiguous>>();
       for (DatanodeStorageInfo e : storages) {
         iterators.add(e.getBlockIterator());
       }
@@ -487,7 +487,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
 
     @Override
-    public BlockInfo next() {
+    public BlockInfoContiguous next() {
       update();
       return iterators.get(index).next();
     }
@@ -504,10 +504,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
-  Iterator<BlockInfo> getBlockIterator() {
+  Iterator<BlockInfoContiguous> getBlockIterator() {
     return new BlockIterator(getStorageInfos());
   }
-  Iterator<BlockInfo> getBlockIterator(final String storageID) {
+  Iterator<BlockInfoContiguous> getBlockIterator(final String storageID) {
     return new BlockIterator(getStorageInfo(storageID));
   }
 
@@ -530,7 +530,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * Store block recovery work.
    */
-  void addBlockToBeRecovered(BlockInfoUnderConstruction block) {
+  void addBlockToBeRecovered(BlockInfoContiguousUnderConstruction 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");
@@ -572,11 +572,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return replicateBlocks.poll(maxTransfers);
   }
 
-  public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
-    List<BlockInfoUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
+  public BlockInfoContiguousUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
+    List<BlockInfoContiguousUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
     if(blocks == null)
       return null;
-    return blocks.toArray(new BlockInfoUnderConstruction[blocks.size()]);
+    return blocks.toArray(new BlockInfoContiguousUnderConstruction[blocks.size()]);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 a3ebb68..a33d990 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
@@ -1417,12 +1417,12 @@ public class DatanodeManager {
         }
 
         //check lease recovery
-        BlockInfoUnderConstruction[] blocks = nodeinfo
+        BlockInfoContiguousUnderConstruction[] blocks = nodeinfo
             .getLeaseRecoveryCommand(Integer.MAX_VALUE);
         if (blocks != null) {
           BlockRecoveryCommand brCommand = new BlockRecoveryCommand(
               blocks.length);
-          for (BlockInfoUnderConstruction b : blocks) {
+          for (BlockInfoContiguousUnderConstruction 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/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
index 3ab10b4..d5ad5fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
@@ -80,10 +80,10 @@ public class DatanodeStorageInfo {
   /**
    * Iterates over the list of blocks belonging to the data-node.
    */
-  class BlockIterator implements Iterator<BlockInfo> {
-    private BlockInfo current;
+  class BlockIterator implements Iterator<BlockInfoContiguous> {
+    private BlockInfoContiguous current;
 
-    BlockIterator(BlockInfo head) {
+    BlockIterator(BlockInfoContiguous head) {
       this.current = head;
     }
 
@@ -91,8 +91,8 @@ public class DatanodeStorageInfo {
       return current != null;
     }
 
-    public BlockInfo next() {
-      BlockInfo res = current;
+    public BlockInfoContiguous next() {
+      BlockInfoContiguous res = current;
       current = current.getNext(current.findStorageInfo(DatanodeStorageInfo.this));
       return res;
     }
@@ -112,7 +112,7 @@ public class DatanodeStorageInfo {
   private volatile long remaining;
   private long blockPoolUsed;
 
-  private volatile BlockInfo blockList = null;
+  private volatile BlockInfoContiguous blockList = null;
   private int numBlocks = 0;
 
   /** The number of block reports received */
@@ -215,7 +215,7 @@ public class DatanodeStorageInfo {
     return blockPoolUsed;
   }
 
-  public AddBlockResult addBlock(BlockInfo b) {
+  public AddBlockResult addBlock(BlockInfoContiguous b) {
     // First check whether the block belongs to a different storage
     // on the same DN.
     AddBlockResult result = AddBlockResult.ADDED;
@@ -240,7 +240,7 @@ public class DatanodeStorageInfo {
     return result;
   }
 
-  public boolean removeBlock(BlockInfo b) {
+  public boolean removeBlock(BlockInfoContiguous b) {
     blockList = b.listRemove(blockList, this);
     if (b.removeStorage(this)) {
       numBlocks--;
@@ -254,7 +254,7 @@ public class DatanodeStorageInfo {
     return numBlocks;
   }
   
-  Iterator<BlockInfo> getBlockIterator() {
+  Iterator<BlockInfoContiguous> getBlockIterator() {
     return new BlockIterator(blockList);
 
   }
@@ -263,7 +263,7 @@ public class DatanodeStorageInfo {
    * Move block to the head of the list of blocks belonging to the data-node.
    * @return the index of the head of the blockList
    */
-  int moveBlockToHead(BlockInfo b, int curIndex, int headIndex) {
+  int moveBlockToHead(BlockInfoContiguous b, int curIndex, int headIndex) {
     blockList = b.moveBlockToHead(blockList, this, curIndex, headIndex);
     return curIndex;
   }
@@ -273,7 +273,7 @@ public class DatanodeStorageInfo {
    * @return the head of the blockList
    */
   @VisibleForTesting
-  BlockInfo getBlockListHeadForTesting(){
+  BlockInfoContiguous getBlockListHeadForTesting(){
     return blockList;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 6389e65..7450249 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -52,8 +52,8 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+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.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
@@ -364,7 +364,7 @@ public class FSDirectory implements Closeable {
       long mtime, long atime, short replication, long preferredBlockSize,
       byte storagePolicyId) {
     return new INodeFile(id, null, permissions, mtime, atime,
-        BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize,
+        BlockInfoContiguous.EMPTY_ARRAY, replication, preferredBlockSize,
         storagePolicyId);
   }
 
@@ -444,8 +444,8 @@ public class FSDirectory implements Closeable {
   /**
    * Add a block to the file. Returns a reference to the added block.
    */
-  BlockInfo addBlock(String path, INodesInPath inodesInPath, Block block,
-      DatanodeStorageInfo[] targets) throws IOException {
+  BlockInfoContiguous addBlock(String path, INodesInPath inodesInPath,
+      Block block, DatanodeStorageInfo[] targets) throws IOException {
     writeLock();
     try {
       final INodeFile fileINode = inodesInPath.getLastINode().asFile();
@@ -455,8 +455,8 @@ public class FSDirectory implements Closeable {
       updateCount(inodesInPath, 0, fileINode.getPreferredBlockDiskspace(), true);
 
       // associate new last block for the file
-      BlockInfoUnderConstruction blockInfo =
-        new BlockInfoUnderConstruction(
+      BlockInfoContiguousUnderConstruction blockInfo =
+        new BlockInfoContiguousUnderConstruction(
             block,
             fileINode.getFileReplication(),
             BlockUCState.UNDER_CONSTRUCTION,
@@ -974,7 +974,7 @@ public class FSDirectory implements Closeable {
         unprotectedTruncate(iip, newLength, collectedBlocks, mtime);
 
     if(! onBlockBoundary) {
-      BlockInfo oldBlock = file.getLastBlock();
+      BlockInfoContiguous oldBlock = file.getLastBlock();
       Block tBlk =
       getFSNamesystem().prepareFileForTruncate(iip,
           clientName, clientMachine, file.computeFileSize() - newLength,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index 3c7eae4..9ce3fa9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -52,7 +52,6 @@ import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
@@ -772,10 +771,10 @@ public class FSEditLog implements LogsPurgeable {
   
   public void logAddBlock(String path, INodeFile file) {
     Preconditions.checkArgument(file.isUnderConstruction());
-    BlockInfo[] blocks = file.getBlocks();
+    BlockInfoContiguous[] blocks = file.getBlocks();
     Preconditions.checkState(blocks != null && blocks.length > 0);
-    BlockInfo pBlock = blocks.length > 1 ? blocks[blocks.length - 2] : null;
-    BlockInfo lastBlock = blocks[blocks.length - 1];
+    BlockInfoContiguous pBlock = blocks.length > 1 ? blocks[blocks.length - 2] : null;
+    BlockInfoContiguous lastBlock = blocks[blocks.length - 1];
     AddBlockOp op = AddBlockOp.getInstance(cache.get()).setPath(path)
         .setPenultimateBlock(pBlock).setLastBlock(lastBlock);
     logEdit(op);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 7cb6486..5fcad74 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
@@ -42,8 +42,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -933,7 +933,7 @@ public class FSEditLogLoader {
    */
   private void addNewBlock(FSDirectory fsDir, AddBlockOp op, INodeFile file)
       throws IOException {
-    BlockInfo[] oldBlocks = file.getBlocks();
+    BlockInfoContiguous[] oldBlocks = file.getBlocks();
     Block pBlock = op.getPenultimateBlock();
     Block newBlock= op.getLastBlock();
     
@@ -950,16 +950,16 @@ public class FSEditLogLoader {
       }
       
       oldLastBlock.setNumBytes(pBlock.getNumBytes());
-      if (oldLastBlock instanceof BlockInfoUnderConstruction) {
+      if (oldLastBlock instanceof BlockInfoContiguousUnderConstruction) {
         fsNamesys.getBlockManager().forceCompleteBlock(file,
-            (BlockInfoUnderConstruction) oldLastBlock);
+            (BlockInfoContiguousUnderConstruction) 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 BlockInfoUnderConstruction(
+    BlockInfoContiguous newBI = new BlockInfoContiguousUnderConstruction(
           newBlock, file.getBlockReplication());
     fsNamesys.getBlockManager().addBlockCollection(newBI, file);
     file.addBlock(newBI);
@@ -973,7 +973,7 @@ public class FSEditLogLoader {
   private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
       INodesInPath iip, INodeFile file) throws IOException {
     // Update its block list
-    BlockInfo[] oldBlocks = file.getBlocks();
+    BlockInfoContiguous[] oldBlocks = file.getBlocks();
     Block[] newBlocks = op.getBlocks();
     String path = op.getPath();
     
@@ -982,7 +982,7 @@ public class FSEditLogLoader {
     
     // First, update blocks in common
     for (int i = 0; i < oldBlocks.length && i < newBlocks.length; i++) {
-      BlockInfo oldBlock = oldBlocks[i];
+      BlockInfoContiguous oldBlock = oldBlocks[i];
       Block newBlock = newBlocks[i];
       
       boolean isLastBlock = i == newBlocks.length - 1;
@@ -1000,11 +1000,11 @@ public class FSEditLogLoader {
         oldBlock.getGenerationStamp() != newBlock.getGenerationStamp();
       oldBlock.setGenerationStamp(newBlock.getGenerationStamp());
       
-      if (oldBlock instanceof BlockInfoUnderConstruction &&
+      if (oldBlock instanceof BlockInfoContiguousUnderConstruction &&
           (!isLastBlock || op.shouldCompleteLastBlock())) {
         changeMade = true;
         fsNamesys.getBlockManager().forceCompleteBlock(file,
-            (BlockInfoUnderConstruction) oldBlock);
+            (BlockInfoContiguousUnderConstruction) oldBlock);
       }
       if (changeMade) {
         // The state or gen-stamp of the block has changed. So, we may be
@@ -1033,19 +1033,19 @@ public class FSEditLogLoader {
       // We're adding blocks
       for (int i = oldBlocks.length; i < newBlocks.length; i++) {
         Block newBlock = newBlocks[i];
-        BlockInfo newBI;
+        BlockInfoContiguous newBI;
         if (!op.shouldCompleteLastBlock()) {
           // 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 BlockInfoUnderConstruction(
+          newBI = new BlockInfoContiguousUnderConstruction(
               newBlock, file.getBlockReplication());
         } else {
           // OP_CLOSE should add finalized blocks. This code path
           // 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, file.getBlockReplication());
+          newBI = new BlockInfoContiguous(newBlock, file.getBlockReplication());
         }
         fsNamesys.getBlockManager().addBlockCollection(newBI, file);
         file.addBlock(newBI);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 0a92054..ed05d40 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
@@ -51,8 +51,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.BlockInfoUnderConstruction;
+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.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@@ -683,7 +683,7 @@ public class FSImageFormat {
 
     public void updateBlocksMap(INodeFile file) {
       // Add file->block mapping
-      final BlockInfo[] blocks = file.getBlocks();
+      final BlockInfoContiguous[] blocks = file.getBlocks();
       if (blocks != null) {
         final BlockManager bm = namesystem.getBlockManager();
         for (int i = 0; i < blocks.length; i++) {
@@ -749,9 +749,9 @@ public class FSImageFormat {
       // file
       
       // read blocks
-      BlockInfo[] blocks = new BlockInfo[numBlocks];
+      BlockInfoContiguous[] blocks = new BlockInfoContiguous[numBlocks];
       for (int j = 0; j < numBlocks; j++) {
-        blocks[j] = new BlockInfo(replication);
+        blocks[j] = new BlockInfoContiguous(replication);
         blocks[j].readFields(in);
       }
 
@@ -771,8 +771,8 @@ public class FSImageFormat {
             clientMachine = FSImageSerialization.readString(in);
             // convert the last block to BlockUC
             if (blocks.length > 0) {
-              BlockInfo lastBlk = blocks[blocks.length - 1]; 
-              blocks[blocks.length - 1] = new BlockInfoUnderConstruction(
+              BlockInfoContiguous lastBlk = blocks[blocks.length - 1];
+              blocks[blocks.length - 1] = new BlockInfoContiguousUnderConstruction(
                   lastBlk, replication);
             }
           }
@@ -947,9 +947,9 @@ public class FSImageFormat {
         FileUnderConstructionFeature uc = cons.getFileUnderConstructionFeature();
         oldnode.toUnderConstruction(uc.getClientName(), uc.getClientMachine());
         if (oldnode.numBlocks() > 0) {
-          BlockInfo ucBlock = cons.getLastBlock();
+          BlockInfoContiguous ucBlock = cons.getLastBlock();
           // we do not replace the inode, just replace the last block of oldnode
-          BlockInfo info = namesystem.getBlockManager().addBlockCollection(
+          BlockInfoContiguous info = namesystem.getBlockManager().addBlockCollection(
               ucBlock, oldnode);
           oldnode.setBlock(oldnode.numBlocks() - 1, info);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 26ca16a..f20add1 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
@@ -40,8 +40,8 @@ import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.hdfs.protocol.Block;
 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.BlockInfoUnderConstruction;
+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;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext;
@@ -170,7 +170,7 @@ public final class FSImageFormatPBINode {
 
     public static void updateBlocksMap(INodeFile file, BlockManager bm) {
       // Add file->block mapping
-      final BlockInfo[] blocks = file.getBlocks();
+      final BlockInfoContiguous[] blocks = file.getBlocks();
       if (blocks != null) {
         for (int i = 0; i < blocks.length; i++) {
           file.setBlock(i, bm.addBlockCollection(blocks[i], file));
@@ -282,9 +282,9 @@ public final class FSImageFormatPBINode {
       short replication = (short) f.getReplication();
       LoaderContext state = parent.getLoaderContext();
 
-      BlockInfo[] blocks = new BlockInfo[bp.size()];
+      BlockInfoContiguous[] blocks = new BlockInfoContiguous[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());
@@ -310,9 +310,9 @@ public final class FSImageFormatPBINode {
         INodeSection.FileUnderConstructionFeature uc = f.getFileUC();
         file.toUnderConstruction(uc.getClientName(), uc.getClientMachine());
         if (blocks.length > 0) {
-          BlockInfo lastBlk = file.getLastBlock();
+          BlockInfoContiguous lastBlk = file.getLastBlock();
           // replace the last block of file
-          file.setBlock(file.numBlocks() - 1, new BlockInfoUnderConstruction(
+          file.setBlock(file.numBlocks() - 1, new BlockInfoContiguousUnderConstruction(
               lastBlk, replication));
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 1c22ee9..33f644d 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
@@ -33,8 +33,8 @@ import org.apache.hadoop.hdfs.protocol.Block;
 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.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
@@ -126,17 +126,17 @@ public class FSImageSerialization {
     long preferredBlockSize = in.readLong();
   
     int numBlocks = in.readInt();
-    BlockInfo[] blocks = new BlockInfo[numBlocks];
+    BlockInfoContiguous[] blocks = new BlockInfoContiguous[numBlocks];
     Block blk = new Block();
     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) {
       blk.readFields(in);
-      blocks[i] = new BlockInfoUnderConstruction(
+      blocks[i] = new BlockInfoContiguousUnderConstruction(
         blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
     }
     PermissionStatus perm = PermissionStatus.read(in);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 98ef1e5..7781244 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
@@ -201,8 +201,8 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretMan
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager.SecretManagerState;
 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.BlockInfoUnderConstruction;
+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.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -2017,7 +2017,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     leaseManager.addLease(
         file.getFileUnderConstructionFeature().getClientName(), src);
     boolean shouldRecoverNow = (newBlock == null);
-    BlockInfo oldBlock = file.getLastBlock();
+    BlockInfoContiguous oldBlock = file.getLastBlock();
     boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file, oldBlock);
     if(newBlock == null) {
       newBlock = (shouldCopyOnTruncate) ? createNewBlock() :
@@ -2025,11 +2025,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
               nextGenerationStamp(blockIdManager.isLegacyBlock(oldBlock)));
     }
 
-    BlockInfoUnderConstruction truncatedBlockUC;
+    BlockInfoContiguousUnderConstruction truncatedBlockUC;
     if(shouldCopyOnTruncate) {
       // Add new truncateBlock into blocksMap and
       // use oldBlock as a source for copy-on-truncate recovery
-      truncatedBlockUC = new BlockInfoUnderConstruction(newBlock,
+      truncatedBlockUC = new BlockInfoContiguousUnderConstruction(newBlock,
           file.getBlockReplication());
       truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta);
       truncatedBlockUC.setTruncateBlock(oldBlock);
@@ -2045,7 +2045,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       blockManager.convertLastBlockToUnderConstruction(file, lastBlockDelta);
       oldBlock = file.getLastBlock();
       assert !oldBlock.isComplete() : "oldBlock should be under construction";
-      truncatedBlockUC = (BlockInfoUnderConstruction) oldBlock;
+      truncatedBlockUC = (BlockInfoContiguousUnderConstruction) oldBlock;
       truncatedBlockUC.setTruncateBlock(new Block(oldBlock));
       truncatedBlockUC.getTruncateBlock().setNumBytes(
           oldBlock.getNumBytes() - lastBlockDelta);
@@ -2071,7 +2071,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * Defines if a replica needs to be copied on truncate or
    * can be truncated in place.
    */
-  boolean shouldCopyOnTruncate(INodeFile file, BlockInfo blk) {
+  boolean shouldCopyOnTruncate(INodeFile file, BlockInfoContiguous blk) {
     if(!isUpgradeFinalized()) {
       return true;
     }
@@ -2625,7 +2625,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       recoverLeaseInternal(RecoverLeaseOp.APPEND_FILE,
           iip, src, holder, clientMachine, false);
       
-      final BlockInfo lastBlock = myFile.getLastBlock();
+      final BlockInfoContiguous lastBlock = myFile.getLastBlock();
       // Check that the block has at least minimum replication.
       if(lastBlock != null && lastBlock.isComplete() &&
           !getBlockManager().isSufficientlyReplicated(lastBlock)) {
@@ -2674,7 +2674,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         dir.updateSpaceConsumed(iip, 0, diff * file.getBlockReplication());
       }
     } else {
-      BlockInfo lastBlock = file.getLastBlock();
+      BlockInfoContiguous lastBlock = file.getLastBlock();
       if (lastBlock != null) {
         ExtendedBlock blk = new ExtendedBlock(this.getBlockPoolId(), lastBlock);
         ret = new LocatedBlock(blk, new DatanodeInfo[0]);
@@ -2807,7 +2807,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                 op.getExceptionMessage(src, holder, clientMachine,
                     "lease recovery is in progress. Try again later."));
         } else {
-          final BlockInfo lastBlock = file.getLastBlock();
+          final BlockInfoContiguous lastBlock = file.getLastBlock();
           if (lastBlock != null
               && lastBlock.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
             throw new RecoveryInProgressException(
@@ -2989,8 +2989,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           return onRetryBlock[0];
         } else {
           // add new chosen targets to already allocated block and return
-          BlockInfo lastBlockInFile = pendingFile.getLastBlock();
-          ((BlockInfoUnderConstruction) lastBlockInFile)
+          BlockInfoContiguous lastBlockInFile = pendingFile.getLastBlock();
+          ((BlockInfoContiguousUnderConstruction) lastBlockInFile)
               .setExpectedLocations(targets);
           offset = pendingFile.computeFileSize();
           return makeLocatedBlock(lastBlockInFile, targets, offset);
@@ -3081,7 +3081,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
     }
     final INodeFile pendingFile = checkLease(src, clientName, inode, fileId);
-    BlockInfo lastBlockInFile = pendingFile.getLastBlock();
+    BlockInfoContiguous lastBlockInFile = pendingFile.getLastBlock();
     if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
       // The block that the client claims is the current last block
       // doesn't match up with what we think is the last block. There are
@@ -3109,7 +3109,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       //    changed the namesystem state yet.
       //    We run this analysis again in Part II where case 4 is impossible.
 
-      BlockInfo penultimateBlock = pendingFile.getPenultimateBlock();
+      BlockInfoContiguous penultimateBlock = pendingFile.getPenultimateBlock();
       if (previous == null &&
           lastBlockInFile != null &&
           lastBlockInFile.getNumBytes() == pendingFile.getPreferredBlockSize() &&
@@ -3136,7 +3136,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             src + ". Returning previously allocated block " + lastBlockInFile);
         long offset = pendingFile.computeFileSize();
         onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
-            ((BlockInfoUnderConstruction)lastBlockInFile).getExpectedStorageLocations(),
+            ((BlockInfoContiguousUnderConstruction)lastBlockInFile).getExpectedStorageLocations(),
             offset);
         return new FileState(pendingFile, src, iip);
       } else {
@@ -3427,11 +3427,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @param targets target datanodes where replicas of the new block is placed
    * @throws QuotaExceededException If addition of block exceeds space quota
    */
-  BlockInfo saveAllocatedBlock(String src, INodesInPath inodesInPath,
+  BlockInfoContiguous saveAllocatedBlock(String src, INodesInPath inodesInPath,
       Block newBlock, DatanodeStorageInfo[] targets)
           throws IOException {
     assert hasWriteLock();
-    BlockInfo b = dir.addBlock(src, inodesInPath, newBlock, targets);
+    BlockInfoContiguous b = dir.addBlock(src, inodesInPath, newBlock, targets);
     NameNode.stateChangeLog.info("BLOCK* allocate " + b + " for " + src);
     DatanodeStorageInfo.incrementBlocksScheduled(targets);
     return b;
@@ -3458,14 +3458,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       if (checkall) {
         // check all blocks of the file.
-        for (BlockInfo block: v.getBlocks()) {
+        for (BlockInfoContiguous block: v.getBlocks()) {
           if (!isCompleteBlock(src, block, blockManager.minReplication)) {
             return false;
           }
         }
       } else {
         // check the penultimate block of this file
-        BlockInfo b = v.getPenultimateBlock();
+        BlockInfoContiguous b = v.getPenultimateBlock();
         if (b != null
             && !isCompleteBlock(src, b, blockManager.minReplication)) {
           return false;
@@ -3477,9 +3477,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
-  private static boolean isCompleteBlock(String src, BlockInfo b, int minRepl) {
+  private static boolean isCompleteBlock(String src, BlockInfoContiguous b, int minRepl) {
     if (!b.isComplete()) {
-      final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)b;
+      final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)b;
       final int numNodes = b.numNodes();
       LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = "
           + uc.getBlockUCState() + ", replication# = " + numNodes
@@ -3669,7 +3669,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     for (Block b : blocks.getToDeleteList()) {
       if (trackBlockCounts) {
-        BlockInfo bi = getStoredBlock(b);
+        BlockInfoContiguous bi = getStoredBlock(b);
         if (bi.isComplete()) {
           numRemovedComplete++;
           if (bi.numNodes() >= blockManager.minReplication) {
@@ -3888,10 +3888,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     final INodeFile pendingFile = iip.getLastINode().asFile();
     int nrBlocks = pendingFile.numBlocks();
-    BlockInfo[] blocks = pendingFile.getBlocks();
+    BlockInfoContiguous[] blocks = pendingFile.getBlocks();
 
     int nrCompleteBlocks;
-    BlockInfo curBlock = null;
+    BlockInfoContiguous curBlock = null;
     for(nrCompleteBlocks = 0; nrCompleteBlocks < nrBlocks; nrCompleteBlocks++) {
       curBlock = blocks[nrCompleteBlocks];
       if(!curBlock.isComplete())
@@ -3926,9 +3926,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     // The last block is not COMPLETE, and
     // that the penultimate block if exists is either COMPLETE or COMMITTED
-    final BlockInfo lastBlock = pendingFile.getLastBlock();
+    final BlockInfoContiguous lastBlock = pendingFile.getLastBlock();
     BlockUCState lastBlockState = lastBlock.getBlockUCState();
-    BlockInfo penultimateBlock = pendingFile.getPenultimateBlock();
+    BlockInfoContiguous penultimateBlock = pendingFile.getPenultimateBlock();
 
     // If penultimate block doesn't exist then its minReplication is met
     boolean penultimateBlockMinReplication = penultimateBlock == null ? true :
@@ -3961,7 +3961,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throw new AlreadyBeingCreatedException(message);
     case UNDER_CONSTRUCTION:
     case UNDER_RECOVERY:
-      final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)lastBlock;
+      final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)lastBlock;
       // determine if last block was intended to be truncated
       Block recoveryBlock = uc.getTruncateBlock();
       boolean truncateRecovery = recoveryBlock != null;
@@ -4072,12 +4072,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @VisibleForTesting
-  BlockInfo getStoredBlock(Block block) {
+  BlockInfoContiguous getStoredBlock(Block block) {
     return blockManager.getStoredBlock(block);
   }
   
   @Override
-  public boolean isInSnapshot(BlockInfoUnderConstruction blockUC) {
+  public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC) {
     assert hasReadLock();
     final BlockCollection bc = blockUC.getBlockCollection();
     if (bc == null || !(bc instanceof INodeFile)
@@ -4130,7 +4130,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   
       checkNameNodeSafeMode(
           "Cannot commitBlockSynchronization while in safe mode");
-      final BlockInfo storedBlock = getStoredBlock(
+      final BlockInfoContiguous storedBlock = getStoredBlock(
           ExtendedBlock.getLocalBlock(oldBlock));
       if (storedBlock == null) {
         if (deleteblock) {
@@ -4178,8 +4178,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         return;
       }
 
-      BlockInfoUnderConstruction truncatedBlock =
-          (BlockInfoUnderConstruction) iFile.getLastBlock();
+      BlockInfoContiguousUnderConstruction truncatedBlock =
+          (BlockInfoContiguousUnderConstruction) iFile.getLastBlock();
       long recoveryId = truncatedBlock.getBlockRecoveryId();
       boolean copyTruncate =
           truncatedBlock.getBlockId() != storedBlock.getBlockId();
@@ -4288,7 +4288,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @throws IOException on error
    */
   @VisibleForTesting
-  String closeFileCommitBlocks(INodeFile pendingFile, BlockInfo storedBlock)
+  String closeFileCommitBlocks(INodeFile pendingFile, BlockInfoContiguous storedBlock)
       throws IOException {
     final INodesInPath iip = INodesInPath.fromINode(pendingFile);
     final String src = iip.getPath();
@@ -4596,7 +4596,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
         while (it.hasNext()) {
           Block b = it.next();
-          BlockInfo blockInfo = blockManager.getStoredBlock(b);
+          BlockInfoContiguous blockInfo = blockManager.getStoredBlock(b);
           if (blockInfo.getBlockCollection().getStoragePolicyID() == lpPolicy.getId()) {
             filesToDelete.add(blockInfo.getBlockCollection());
           }
@@ -5543,7 +5543,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null) // mostly true
       return;
-    BlockInfo storedBlock = getStoredBlock(b);
+    BlockInfoContiguous storedBlock = getStoredBlock(b);
     if (storedBlock.isComplete()) {
       safeMode.decrementSafeBlockCount((short)blockManager.countNodes(b).liveReplicas());
     }
@@ -6060,7 +6060,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         + "access token for block " + block);
     
     // check stored block state
-    BlockInfo storedBlock = getStoredBlock(ExtendedBlock.getLocalBlock(block));
+    BlockInfoContiguous storedBlock = getStoredBlock(ExtendedBlock.getLocalBlock(block));
     if (storedBlock == null || 
         storedBlock.getBlockUCState() != BlockUCState.UNDER_CONSTRUCTION) {
         throw new IOException(block + 
@@ -6189,8 +6189,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 BlockInfoUnderConstruction blockinfo
-        = (BlockInfoUnderConstruction)pendingFile.getLastBlock();
+    final BlockInfoContiguousUnderConstruction blockinfo
+        = (BlockInfoContiguousUnderConstruction)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/1382ae52/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 896bedb..1ebdde6 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
@@ -20,9 +20,8 @@ package org.apache.hadoop.hdfs.server.namenode;
 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.BlockInfoUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 
 /**
@@ -59,10 +58,10 @@ public class FileUnderConstructionFeature implements INode.Feature {
    */
   void updateLengthOfLastBlock(INodeFile f, long lastBlockLength)
       throws IOException {
-    BlockInfo lastBlock = f.getLastBlock();
+    BlockInfoContiguous lastBlock = f.getLastBlock();
     assert (lastBlock != null) : "The last block for path "
         + f.getFullPathName() + " is null when updating its length";
-    assert (lastBlock instanceof BlockInfoUnderConstruction)
+    assert (lastBlock instanceof BlockInfoContiguousUnderConstruction)
         : "The last block for path " + f.getFullPathName()
             + " is not a BlockInfoUnderConstruction when updating its length";
     lastBlock.setNumBytes(lastBlockLength);
@@ -75,11 +74,11 @@ public class FileUnderConstructionFeature implements INode.Feature {
    */
   void cleanZeroSizeBlock(final INodeFile f,
       final BlocksMapUpdateInfo collectedBlocks) {
-    final BlockInfo[] blocks = f.getBlocks();
+    final BlockInfoContiguous[] blocks = f.getBlocks();
     if (blocks != null && blocks.length > 0
-        && blocks[blocks.length - 1] instanceof BlockInfoUnderConstruction) {
-      BlockInfoUnderConstruction lastUC =
-          (BlockInfoUnderConstruction) blocks[blocks.length - 1];
+        && blocks[blocks.length - 1] instanceof BlockInfoContiguousUnderConstruction) {
+      BlockInfoContiguousUnderConstruction lastUC =
+          (BlockInfoContiguousUnderConstruction) 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);


[3/3] hadoop git commit: HDFS-7743. Code cleanup of BlockInfo and rename BlockInfo to BlockInfoContiguous. Contributed by Jing Zhao.

Posted by ji...@apache.org.
HDFS-7743. Code cleanup of BlockInfo and rename BlockInfo to BlockInfoContiguous. Contributed by Jing Zhao.


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

Branch: refs/heads/trunk
Commit: 1382ae525c67bf95d8f3a436b547dbc72cfbb177
Parents: ef01768
Author: Jing Zhao <ji...@apache.org>
Authored: Sun Feb 8 11:51:44 2015 -0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Sun Feb 8 11:51:44 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../server/blockmanagement/BlockCollection.java |   8 +-
 .../hdfs/server/blockmanagement/BlockInfo.java  | 403 ------------------
 .../blockmanagement/BlockInfoContiguous.java    | 412 +++++++++++++++++++
 .../BlockInfoContiguousUnderConstruction.java   | 400 ++++++++++++++++++
 .../BlockInfoUnderConstruction.java             | 402 ------------------
 .../server/blockmanagement/BlockManager.java    | 160 +++----
 .../hdfs/server/blockmanagement/BlocksMap.java  |  32 +-
 .../CacheReplicationMonitor.java                |  14 +-
 .../blockmanagement/DatanodeDescriptor.java     |  28 +-
 .../server/blockmanagement/DatanodeManager.java |   4 +-
 .../blockmanagement/DatanodeStorageInfo.java    |  22 +-
 .../hdfs/server/namenode/FSDirectory.java       |  16 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |   9 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |  26 +-
 .../hdfs/server/namenode/FSImageFormat.java     |  18 +-
 .../server/namenode/FSImageFormatPBINode.java   |  14 +-
 .../server/namenode/FSImageSerialization.java   |  10 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  76 ++--
 .../namenode/FileUnderConstructionFeature.java  |  17 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  84 ++--
 .../hdfs/server/namenode/LeaseManager.java      |   6 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |   4 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java |   4 +-
 .../snapshot/FSImageFormatPBSnapshot.java       |   8 +-
 .../hdfs/server/namenode/snapshot/FileDiff.java |  10 +-
 .../server/namenode/snapshot/FileDiffList.java  |  18 +-
 .../snapshot/FileWithSnapshotFeature.java       |   4 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  12 +-
 .../server/blockmanagement/TestBlockInfo.java   |  14 +-
 .../TestBlockInfoUnderConstruction.java         |   4 +-
 .../blockmanagement/TestBlockManager.java       |  20 +-
 .../blockmanagement/TestDatanodeDescriptor.java |   4 +-
 .../blockmanagement/TestHeartbeatHandling.java  |   6 +-
 .../blockmanagement/TestReplicationPolicy.java  |  12 +-
 .../hdfs/server/namenode/CreateEditsLog.java    |   8 +-
 .../hdfs/server/namenode/TestAddBlock.java      |  12 +-
 .../namenode/TestBlockUnderConstruction.java    |  10 +-
 .../TestCommitBlockSynchronization.java         |  12 +-
 .../hdfs/server/namenode/TestEditLog.java       |   4 +-
 .../hdfs/server/namenode/TestFSImage.java       |   4 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |   6 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |   4 +-
 .../hdfs/server/namenode/TestINodeFile.java     |   4 +-
 .../namenode/ha/TestPipelinesFailover.java      |   6 -
 .../namenode/ha/TestRetryCacheWithHA.java       |   6 +-
 .../namenode/snapshot/SnapshotTestHelper.java   |  10 +-
 .../snapshot/TestSnapshotBlocksMap.java         |  24 +-
 .../namenode/snapshot/TestSnapshotDeletion.java |  16 +-
 49 files changed, 1213 insertions(+), 1197 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 8555862..eda3744 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -591,6 +591,9 @@ Release 2.7.0 - UNRELEASED
     tests such as truncate with HA setup, negative tests, truncate with other
     operations and multiple truncates.  (szetszwo)
 
+    HDFS-7743. Code cleanup of BlockInfo and rename BlockInfo to
+    BlockInfoContiguous. (jing9)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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 9ef2274..1547611 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
@@ -31,7 +31,7 @@ public interface BlockCollection {
   /**
    * Get the last block of the collection.
    */
-  public BlockInfo getLastBlock();
+  public BlockInfoContiguous getLastBlock();
 
   /** 
    * Get content summary.
@@ -46,7 +46,7 @@ public interface BlockCollection {
   /**
    * Get the blocks.
    */
-  public BlockInfo[] getBlocks();
+  public BlockInfoContiguous[] getBlocks();
 
   /**
    * Get preferred block size for the collection 
@@ -73,13 +73,13 @@ public interface BlockCollection {
   /**
    * Set the block at the given index.
    */
-  public void setBlock(int index, BlockInfo blk);
+  public void setBlock(int index, BlockInfoContiguous blk);
 
   /**
    * Convert the last block of the collection to an under-construction block
    * and set the locations.
    */
-  public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
+  public BlockInfoContiguousUnderConstruction setLastBlock(BlockInfoContiguous lastBlock,
       DatanodeStorageInfo[] targets) throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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
deleted file mode 100644
index 5d0b473..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.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.util.LinkedList;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-import org.apache.hadoop.util.LightWeightGSet;
-
-/**
- * BlockInfo class maintains for a given block
- * the {@link BlockCollection} it is part of and datanodes where the replicas of 
- * the block are stored.
- */
-@InterfaceAudience.Private
-public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
-  public static final BlockInfo[] EMPTY_ARRAY = {}; 
-
-  private BlockCollection bc;
-
-  /** For implementing {@link LightWeightGSet.LinkedElement} interface */
-  private LightWeightGSet.LinkedElement nextLinkedElement;
-
-  /**
-   * This array contains triplets of references. For each i-th storage, the
-   * block belongs to triplets[3*i] is the reference to the
-   * {@link DatanodeStorageInfo} and triplets[3*i+1] and triplets[3*i+2] are
-   * references to the previous and the next blocks, respectively, in the list
-   * of blocks belonging to this storage.
-   * 
-   * Using previous and next in Object triplets is done instead of a
-   * {@link LinkedList} list to efficiently use memory. With LinkedList the cost
-   * per replica is 42 bytes (LinkedList#Entry object per replica) versus 16
-   * bytes using the triplets.
-   */
-  private Object[] triplets;
-
-  /**
-   * Construct an entry for blocksmap
-   * @param replication the block's replication factor
-   */
-  public BlockInfo(short replication) {
-    this.triplets = new Object[3*replication];
-    this.bc = null;
-  }
-  
-  public BlockInfo(Block blk, short replication) {
-    super(blk);
-    this.triplets = new Object[3*replication];
-    this.bc = null;
-  }
-
-  /**
-   * Copy construction.
-   * This is used to convert BlockInfoUnderConstruction
-   * @param from BlockInfo to copy from.
-   */
-  protected BlockInfo(BlockInfo from) {
-    this(from, from.bc.getBlockReplication());
-    this.bc = from.bc;
-  }
-
-  public BlockCollection getBlockCollection() {
-    return bc;
-  }
-
-  public void setBlockCollection(BlockCollection bc) {
-    this.bc = bc;
-  }
-
-  public DatanodeDescriptor getDatanode(int index) {
-    DatanodeStorageInfo storage = getStorageInfo(index);
-    return storage == null ? null : storage.getDatanodeDescriptor();
-  }
-
-  DatanodeStorageInfo getStorageInfo(int index) {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
-    return (DatanodeStorageInfo)triplets[index*3];
-  }
-
-  private 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];
-    assert info == null || 
-        info.getClass().getName().startsWith(BlockInfo.class.getName()) : 
-              "BlockInfo is expected at " + index*3;
-    return info;
-  }
-
-  BlockInfo getNext(int index) {
-    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];
-    assert info == null || 
-        info.getClass().getName().startsWith(BlockInfo.class.getName()) : 
-              "BlockInfo is expected at " + index*3;
-    return info;
-  }
-
-  private 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;
-  }
-
-  /**
-   * Return the previous block on the block list for the datanode at
-   * position index. Set the previous block on the list to "to".
-   *
-   * @param index - the datanode index
-   * @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) {
-	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];
-    triplets[index*3+1] = to;
-    return info;
-  }
-
-  /**
-   * Return the next block on the block list for the datanode at
-   * position index. Set the next block on the list to "to".
-   *
-   * @param index - the datanode index
-   * @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) {
-	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];
-    triplets[index*3+2] = to;
-    return info;
-  }
-
-  public int getCapacity() {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    assert triplets.length % 3 == 0 : "Malformed BlockInfo";
-    return triplets.length / 3;
-  }
-
-  /**
-   * 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;
-  }
-
-  /**
-   * Count the number of data-nodes the block belongs to.
-   */
-  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;
-  }
-
-  /**
-   * Add a {@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;
-  }
-
-  /**
-   * Remove {@link DatanodeStorageInfo} location for a block
-   */
-  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;
-  }
-
-  /**
-   * Find specified DatanodeDescriptor.
-   * @return index or -1 if not found.
-   */
-  boolean findDatanode(DatanodeDescriptor dn) {
-    int len = getCapacity();
-    for(int idx = 0; idx < len; idx++) {
-      DatanodeDescriptor cur = getDatanode(idx);
-      if(cur == dn) {
-        return true;
-      }
-      if(cur == null) {
-        break;
-      }
-    }
-    return false;
-  }
-  /**
-   * Find specified DatanodeStorageInfo.
-   * @return DatanodeStorageInfo or null if not found.
-   */
-  DatanodeStorageInfo findStorageInfo(DatanodeDescriptor dn) {
-    int len = getCapacity();
-    for(int idx = 0; idx < len; idx++) {
-      DatanodeStorageInfo cur = getStorageInfo(idx);
-      if(cur == null)
-        break;
-      if(cur.getDatanodeDescriptor() == dn)
-        return cur;
-    }
-    return null;
-  }
-  
-  /**
-   * Find specified DatanodeStorageInfo.
-   * @return index or -1 if not found.
-   */
-  int findStorageInfo(DatanodeStorageInfo storageInfo) {
-    int len = getCapacity();
-    for(int idx = 0; idx < len; idx++) {
-      DatanodeStorageInfo cur = getStorageInfo(idx);
-      if(cur == storageInfo)
-        return idx;
-      if(cur == null)
-        break;
-    }
-    return -1;
-  }
-
-  /**
-   * Insert this block into the head of the list of blocks 
-   * related to the specified DatanodeStorageInfo.
-   * If the head is null then form a new list.
-   * @return current block as the new head of the list.
-   */
-  BlockInfo listInsert(BlockInfo head, DatanodeStorageInfo storage) {
-    int dnIndex = this.findStorageInfo(storage);
-    assert dnIndex >= 0 : "Data node is not found: current";
-    assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
-            "Block is already in the list and cannot be inserted.";
-    this.setPrevious(dnIndex, null);
-    this.setNext(dnIndex, head);
-    if(head != null)
-      head.setPrevious(head.findStorageInfo(storage), this);
-    return this;
-  }
-
-  /**
-   * Remove this block from the list of blocks 
-   * related to the specified DatanodeStorageInfo.
-   * If this block is the head of the list then return the next block as 
-   * the new head.
-   * @return the new head of the list or null if the list becomes
-   * empy after deletion.
-   */
-  BlockInfo listRemove(BlockInfo head, DatanodeStorageInfo storage) {
-    if(head == null)
-      return null;
-    int dnIndex = this.findStorageInfo(storage);
-    if(dnIndex < 0) // this block is not on the data-node list
-      return head;
-
-    BlockInfo next = this.getNext(dnIndex);
-    BlockInfo prev = this.getPrevious(dnIndex);
-    this.setNext(dnIndex, null);
-    this.setPrevious(dnIndex, null);
-    if(prev != null)
-      prev.setNext(prev.findStorageInfo(storage), next);
-    if(next != null)
-      next.setPrevious(next.findStorageInfo(storage), prev);
-    if(this == head)  // removing the head
-      head = next;
-    return head;
-  }
-
-  /**
-   * Remove this block from the list of blocks related to the specified
-   * DatanodeDescriptor. Insert it into the head of the list of blocks.
-   *
-   * @return the new head of the list.
-   */
-  public BlockInfo moveBlockToHead(BlockInfo head, DatanodeStorageInfo storage,
-      int curIndex, int headIndex) {
-    if (head == this) {
-      return this;
-    }
-    BlockInfo next = this.setNext(curIndex, head);
-    BlockInfo prev = this.setPrevious(curIndex, null);
-
-    head.setPrevious(headIndex, this);
-    prev.setNext(prev.findStorageInfo(storage), next);
-    if (next != null)
-      next.setPrevious(next.findStorageInfo(storage), prev);
-    return this;
-  }
-
-  /**
-   * BlockInfo represents a block that is not being constructed.
-   * In order to start modifying the block, the BlockInfo should be converted
-   * to {@link BlockInfoUnderConstruction}.
-   * @return {@link BlockUCState#COMPLETE}
-   */
-  public BlockUCState getBlockUCState() {
-    return BlockUCState.COMPLETE;
-  }
-
-  /**
-   * Is this block complete?
-   * 
-   * @return true if the state of the block is {@link BlockUCState#COMPLETE}
-   */
-  public boolean isComplete() {
-    return getBlockUCState().equals(BlockUCState.COMPLETE);
-  }
-
-  /**
-   * Convert a complete block to an under construction block.
-   * @return BlockInfoUnderConstruction -  an under construction block.
-   */
-  public BlockInfoUnderConstruction convertToBlockUnderConstruction(
-      BlockUCState s, DatanodeStorageInfo[] targets) {
-    if(isComplete()) {
-      BlockInfoUnderConstruction ucBlock = new BlockInfoUnderConstruction(this,
-          getBlockCollection().getBlockReplication(), s, targets);
-      ucBlock.setBlockCollection(getBlockCollection());
-      return ucBlock;
-    }
-    // the block is already under construction
-    BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)this;
-    ucBlock.setBlockUCState(s);
-    ucBlock.setExpectedLocations(targets);
-    ucBlock.setBlockCollection(getBlockCollection());
-    return ucBlock;
-  }
-
-  @Override
-  public int hashCode() {
-    // Super implementation is sufficient
-    return super.hashCode();
-  }
-  
-  @Override
-  public boolean equals(Object obj) {
-    // Sufficient to rely on super's implementation
-    return (this == obj) || super.equals(obj);
-  }
-
-  @Override
-  public LightWeightGSet.LinkedElement getNext() {
-    return nextLinkedElement;
-  }
-
-  @Override
-  public void setNext(LightWeightGSet.LinkedElement next) {
-    this.nextLinkedElement = next;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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..48069c1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
@@ -0,0 +1,412 @@
+/**
+ * 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.util.LinkedList;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.util.LightWeightGSet;
+
+/**
+ * BlockInfo class maintains for a given block
+ * the {@link BlockCollection} it is part of and datanodes where the replicas of 
+ * the block are stored.
+ */
+@InterfaceAudience.Private
+public class BlockInfoContiguous extends Block
+    implements LightWeightGSet.LinkedElement {
+  public static final BlockInfoContiguous[] EMPTY_ARRAY = {};
+
+  private BlockCollection bc;
+
+  /** For implementing {@link LightWeightGSet.LinkedElement} interface */
+  private LightWeightGSet.LinkedElement nextLinkedElement;
+
+  /**
+   * This array contains triplets of references. For each i-th storage, the
+   * block belongs to triplets[3*i] is the reference to the
+   * {@link DatanodeStorageInfo} and triplets[3*i+1] and triplets[3*i+2] are
+   * references to the previous and the next blocks, respectively, in the list
+   * of blocks belonging to this storage.
+   * 
+   * Using previous and next in Object triplets is done instead of a
+   * {@link LinkedList} list to efficiently use memory. With LinkedList the cost
+   * per replica is 42 bytes (LinkedList#Entry object per replica) versus 16
+   * bytes using the triplets.
+   */
+  private Object[] triplets;
+
+  /**
+   * Construct an entry for blocksmap
+   * @param replication the block's replication factor
+   */
+  public BlockInfoContiguous(short replication) {
+    this.triplets = new Object[3*replication];
+    this.bc = null;
+  }
+  
+  public BlockInfoContiguous(Block blk, short replication) {
+    super(blk);
+    this.triplets = new Object[3*replication];
+    this.bc = null;
+  }
+
+  /**
+   * Copy construction.
+   * This is used to convert BlockInfoUnderConstruction
+   * @param from BlockInfo to copy from.
+   */
+  protected BlockInfoContiguous(BlockInfoContiguous from) {
+    this(from, from.bc.getBlockReplication());
+    this.bc = from.bc;
+  }
+
+  public BlockCollection getBlockCollection() {
+    return bc;
+  }
+
+  public void setBlockCollection(BlockCollection bc) {
+    this.bc = bc;
+  }
+
+  public DatanodeDescriptor getDatanode(int index) {
+    DatanodeStorageInfo storage = getStorageInfo(index);
+    return storage == null ? null : storage.getDatanodeDescriptor();
+  }
+
+  DatanodeStorageInfo getStorageInfo(int index) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
+    return (DatanodeStorageInfo)triplets[index*3];
+  }
+
+  private BlockInfoContiguous getPrevious(int index) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
+    BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+1];
+    assert info == null || 
+        info.getClass().getName().startsWith(BlockInfoContiguous.class.getName()) :
+              "BlockInfo is expected at " + index*3;
+    return info;
+  }
+
+  BlockInfoContiguous getNext(int index) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
+    BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+2];
+    assert info == null || info.getClass().getName().startsWith(
+        BlockInfoContiguous.class.getName()) :
+        "BlockInfo is expected at " + index*3;
+    return info;
+  }
+
+  private 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;
+  }
+
+  /**
+   * Return the previous block on the block list for the datanode at
+   * position index. Set the previous block on the list to "to".
+   *
+   * @param index - the datanode index
+   * @param to - block to be set to previous on the list of blocks
+   * @return current previous block on the list of blocks
+   */
+  private BlockInfoContiguous setPrevious(int index, BlockInfoContiguous to) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
+    BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+1];
+    triplets[index*3+1] = to;
+    return info;
+  }
+
+  /**
+   * Return the next block on the block list for the datanode at
+   * position index. Set the next block on the list to "to".
+   *
+   * @param index - the datanode index
+   * @param to - block to be set to next on the list of blocks
+   *    * @return current next block on the list of blocks
+   */
+  private BlockInfoContiguous setNext(int index, BlockInfoContiguous to) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
+    BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+2];
+    triplets[index*3+2] = to;
+    return info;
+  }
+
+  public int getCapacity() {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert triplets.length % 3 == 0 : "Malformed BlockInfo";
+    return triplets.length / 3;
+  }
+
+  /**
+   * 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;
+  }
+
+  /**
+   * Count the number of data-nodes the block belongs to.
+   */
+  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;
+  }
+
+  /**
+   * Add a {@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;
+  }
+
+  /**
+   * Remove {@link DatanodeStorageInfo} location for a block
+   */
+  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;
+  }
+
+  /**
+   * Find specified DatanodeDescriptor.
+   * @return index or -1 if not found.
+   */
+  boolean findDatanode(DatanodeDescriptor dn) {
+    int len = getCapacity();
+    for(int idx = 0; idx < len; idx++) {
+      DatanodeDescriptor cur = getDatanode(idx);
+      if(cur == dn) {
+        return true;
+      }
+      if(cur == null) {
+        break;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Find specified DatanodeStorageInfo.
+   * @return DatanodeStorageInfo or null if not found.
+   */
+  DatanodeStorageInfo findStorageInfo(DatanodeDescriptor dn) {
+    int len = getCapacity();
+    for(int idx = 0; idx < len; idx++) {
+      DatanodeStorageInfo cur = getStorageInfo(idx);
+      if(cur == null)
+        break;
+      if(cur.getDatanodeDescriptor() == dn)
+        return cur;
+    }
+    return null;
+  }
+  
+  /**
+   * Find specified DatanodeStorageInfo.
+   * @return index or -1 if not found.
+   */
+  int findStorageInfo(DatanodeStorageInfo storageInfo) {
+    int len = getCapacity();
+    for(int idx = 0; idx < len; idx++) {
+      DatanodeStorageInfo cur = getStorageInfo(idx);
+      if (cur == storageInfo) {
+        return idx;
+      }
+      if (cur == null) {
+        break;
+      }
+    }
+    return -1;
+  }
+
+  /**
+   * Insert this block into the head of the list of blocks 
+   * related to the specified DatanodeStorageInfo.
+   * If the head is null then form a new list.
+   * @return current block as the new head of the list.
+   */
+  BlockInfoContiguous listInsert(BlockInfoContiguous head,
+      DatanodeStorageInfo storage) {
+    int dnIndex = this.findStorageInfo(storage);
+    assert dnIndex >= 0 : "Data node is not found: current";
+    assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
+            "Block is already in the list and cannot be inserted.";
+    this.setPrevious(dnIndex, null);
+    this.setNext(dnIndex, head);
+    if(head != null)
+      head.setPrevious(head.findStorageInfo(storage), this);
+    return this;
+  }
+
+  /**
+   * Remove this block from the list of blocks 
+   * related to the specified DatanodeStorageInfo.
+   * If this block is the head of the list then return the next block as 
+   * the new head.
+   * @return the new head of the list or null if the list becomes
+   * empy after deletion.
+   */
+  BlockInfoContiguous listRemove(BlockInfoContiguous head,
+      DatanodeStorageInfo storage) {
+    if(head == null)
+      return null;
+    int dnIndex = this.findStorageInfo(storage);
+    if(dnIndex < 0) // this block is not on the data-node list
+      return head;
+
+    BlockInfoContiguous next = this.getNext(dnIndex);
+    BlockInfoContiguous prev = this.getPrevious(dnIndex);
+    this.setNext(dnIndex, null);
+    this.setPrevious(dnIndex, null);
+    if(prev != null)
+      prev.setNext(prev.findStorageInfo(storage), next);
+    if(next != null)
+      next.setPrevious(next.findStorageInfo(storage), prev);
+    if(this == head)  // removing the head
+      head = next;
+    return head;
+  }
+
+  /**
+   * Remove this block from the list of blocks related to the specified
+   * DatanodeDescriptor. Insert it into the head of the list of blocks.
+   *
+   * @return the new head of the list.
+   */
+  public BlockInfoContiguous moveBlockToHead(BlockInfoContiguous head,
+      DatanodeStorageInfo storage, int curIndex, int headIndex) {
+    if (head == this) {
+      return this;
+    }
+    BlockInfoContiguous next = this.setNext(curIndex, head);
+    BlockInfoContiguous prev = this.setPrevious(curIndex, null);
+
+    head.setPrevious(headIndex, this);
+    prev.setNext(prev.findStorageInfo(storage), next);
+    if (next != null) {
+      next.setPrevious(next.findStorageInfo(storage), prev);
+    }
+    return this;
+  }
+
+  /**
+   * BlockInfo represents a block that is not being constructed.
+   * In order to start modifying the block, the BlockInfo should be converted
+   * to {@link BlockInfoContiguousUnderConstruction}.
+   * @return {@link BlockUCState#COMPLETE}
+   */
+  public BlockUCState getBlockUCState() {
+    return BlockUCState.COMPLETE;
+  }
+
+  /**
+   * Is this block complete?
+   * 
+   * @return true if the state of the block is {@link BlockUCState#COMPLETE}
+   */
+  public boolean isComplete() {
+    return getBlockUCState().equals(BlockUCState.COMPLETE);
+  }
+
+  /**
+   * Convert a complete block to an under construction block.
+   * @return BlockInfoUnderConstruction -  an under construction block.
+   */
+  public BlockInfoContiguousUnderConstruction convertToBlockUnderConstruction(
+      BlockUCState s, DatanodeStorageInfo[] targets) {
+    if(isComplete()) {
+      BlockInfoContiguousUnderConstruction ucBlock =
+          new BlockInfoContiguousUnderConstruction(this,
+          getBlockCollection().getBlockReplication(), s, targets);
+      ucBlock.setBlockCollection(getBlockCollection());
+      return ucBlock;
+    }
+    // the block is already under construction
+    BlockInfoContiguousUnderConstruction ucBlock =
+        (BlockInfoContiguousUnderConstruction)this;
+    ucBlock.setBlockUCState(s);
+    ucBlock.setExpectedLocations(targets);
+    ucBlock.setBlockCollection(getBlockCollection());
+    return ucBlock;
+  }
+
+  @Override
+  public int hashCode() {
+    // Super implementation is sufficient
+    return super.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    // Sufficient to rely on super's implementation
+    return (this == obj) || super.equals(obj);
+  }
+
+  @Override
+  public LightWeightGSet.LinkedElement getNext() {
+    return nextLinkedElement;
+  }
+
+  @Override
+  public void setNext(LightWeightGSet.LinkedElement next) {
+    this.nextLinkedElement = next;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1382ae52/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
new file mode 100644
index 0000000..91b76cc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
@@ -0,0 +1,400 @@
+/**
+ * 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. 
+   */
+  BlockInfoContiguous 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().getLastUpdate(); 
+      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(", 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/1382ae52/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
deleted file mode 100644
index 335fe21..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
+++ /dev/null
@@ -1,402 +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 BlockInfoUnderConstruction extends BlockInfo {
-  /** 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 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);
-    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 BlockInfo(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().getLastUpdate(); 
-      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(", 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("]}");
-  }
-}